ClickHouse/src/Interpreters/DatabaseCatalog.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

455 lines
20 KiB
C++
Raw Normal View History

2020-02-03 12:54:36 +00:00
#pragma once
#include <Core/UUID.h>
#include <Interpreters/Context_fwd.h>
2020-03-13 10:30:55 +00:00
#include <Interpreters/StorageID.h>
#include <Databases/TablesDependencyGraph.h>
2020-03-10 19:36:17 +00:00
#include <Parsers/IAST_fwd.h>
#include <Storages/IStorage_fwd.h>
Added suggestions for mistyped names for db and tables with different… (#49801) * Added suggestions for mistyped names for db and tables with different scenarios commented * fixed bugs * fixed style check * fixed errors * fixed errors * fixed error with exceptions * fixed exceptions * fixed exceptions * fixed exceptions * added test and fixed bugs * fixed style check * fixed style check * fixed style check * fixed check black * Update test.py * Fixed server crash * Fixed server crash and style check * Fixed style check * Fixed style check * Fixed style check * Fixed bugs with drop_db * fixed fast test * added tests * fixed style check * fixed style check * fixed bug with lock_db * fixed bug with lock_db and fixed reviews * fixed bug with lock_db and fixed reviews * fixed style check * fixed fast test * fixed fast test * revert to fd582a2 * revert to fd582a2 * Removed unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Remove unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * resolved arguments issue in assertDatabaseExists * fixing fast test * fixed fast test * fixed stateless test for default db * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Fixing tests. * resolved problem with mutex * Fixed mutex in assertDatabaseExists * changes about assertDatabaseExists * fixed bugs with file types * fixed string types * fixed fast test * fixed mutex * fixed mutex * fixed style check * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * fixed build * added -unlocked versions of functions * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * changed usage of assertDatabaseExistsUnlocked() * fixed style check * style check * style check * Revert "style check" This reverts commit 28a9ee85a0a801438f05ca8c6604d67b8c132e69. * Merge branch 'master' into hints-for-wrong-db-or-table-name * Changed AssertDatabaseExists and unified exception output * resolved proposed changes and modified tests * Revert "resolved proposed changes and modified tests" This reverts commit d45337d65c16bd55b6bbb5c29023a1d0e5e1a38e. * resolved requested changes * fixed tests * fixed tests * fixed check black * Update include brackets Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update suggested changes * Update suggested changes * Fixed style check * Added test to analyzer_integration_broken * Update DatabaseCatalog.cpp * Update test.py * fixed test * Revert "fixed test" This reverts commit ca6d4c17c85b2bebba56ebc24dceac0e425afa0e. * fixed test * Revert "fixed test" This reverts commit fe6d0d9c865fcfe030ea63f370d2930a3bb3b53a, reversing changes made to 22f44967041fbd32f5f8884daa6f8736247f6fd8. * Update test.py * fixed black check * Update test.py * fixed long_log_tinylog_deadlock_race * Update DatabaseCatalog.cpp * Update test.py * style * Update DatabaseCatalog.cpp * Fixed test * implemented for IDatabase * Style check * removed const_cast * Update DatabaseCatalog.h * Update DatabaseCatalog.h * Update DatabaseCatalog.cpp * Update DatabaseCatalog.cpp * Added db name to hints * Update 00002_log_and_exception_messages_formatting.sql * Update 00002_log_and_exception_messages_formatting.sql --------- Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com>
2023-08-11 10:24:16 +00:00
#include "Common/NamePrompter.h"
2023-01-12 15:51:04 +00:00
#include <Common/SharedMutex.h>
Added suggestions for mistyped names for db and tables with different… (#49801) * Added suggestions for mistyped names for db and tables with different scenarios commented * fixed bugs * fixed style check * fixed errors * fixed errors * fixed error with exceptions * fixed exceptions * fixed exceptions * fixed exceptions * added test and fixed bugs * fixed style check * fixed style check * fixed style check * fixed check black * Update test.py * Fixed server crash * Fixed server crash and style check * Fixed style check * Fixed style check * Fixed style check * Fixed bugs with drop_db * fixed fast test * added tests * fixed style check * fixed style check * fixed bug with lock_db * fixed bug with lock_db and fixed reviews * fixed bug with lock_db and fixed reviews * fixed style check * fixed fast test * fixed fast test * revert to fd582a2 * revert to fd582a2 * Removed unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Remove unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * resolved arguments issue in assertDatabaseExists * fixing fast test * fixed fast test * fixed stateless test for default db * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Fixing tests. * resolved problem with mutex * Fixed mutex in assertDatabaseExists * changes about assertDatabaseExists * fixed bugs with file types * fixed string types * fixed fast test * fixed mutex * fixed mutex * fixed style check * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * fixed build * added -unlocked versions of functions * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * changed usage of assertDatabaseExistsUnlocked() * fixed style check * style check * style check * Revert "style check" This reverts commit 28a9ee85a0a801438f05ca8c6604d67b8c132e69. * Merge branch 'master' into hints-for-wrong-db-or-table-name * Changed AssertDatabaseExists and unified exception output * resolved proposed changes and modified tests * Revert "resolved proposed changes and modified tests" This reverts commit d45337d65c16bd55b6bbb5c29023a1d0e5e1a38e. * resolved requested changes * fixed tests * fixed tests * fixed check black * Update include brackets Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update suggested changes * Update suggested changes * Fixed style check * Added test to analyzer_integration_broken * Update DatabaseCatalog.cpp * Update test.py * fixed test * Revert "fixed test" This reverts commit ca6d4c17c85b2bebba56ebc24dceac0e425afa0e. * fixed test * Revert "fixed test" This reverts commit fe6d0d9c865fcfe030ea63f370d2930a3bb3b53a, reversing changes made to 22f44967041fbd32f5f8884daa6f8736247f6fd8. * Update test.py * fixed black check * Update test.py * fixed long_log_tinylog_deadlock_race * Update DatabaseCatalog.cpp * Update test.py * style * Update DatabaseCatalog.cpp * Fixed test * implemented for IDatabase * Style check * removed const_cast * Update DatabaseCatalog.h * Update DatabaseCatalog.h * Update DatabaseCatalog.cpp * Update DatabaseCatalog.cpp * Added db name to hints * Update 00002_log_and_exception_messages_formatting.sql * Update 00002_log_and_exception_messages_formatting.sql --------- Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com>
2023-08-11 10:24:16 +00:00
#include "Storages/IStorage.h"
#include "Databases/IDatabase.h"
2020-02-03 12:54:36 +00:00
#include <boost/noncopyable.hpp>
#include <Poco/Logger.h>
#include <array>
#include <condition_variable>
#include <list>
2020-02-03 12:54:36 +00:00
#include <map>
#include <memory>
#include <mutex>
#include <set>
2020-02-03 12:54:36 +00:00
#include <unordered_map>
#include <unordered_set>
#include <filesystem>
2020-02-03 12:54:36 +00:00
namespace fs = std::filesystem;
2020-02-03 12:54:36 +00:00
namespace DB
{
class IDatabase;
class Exception;
2020-03-13 15:41:36 +00:00
class ColumnsDescription;
struct ConstraintsDescription;
class IDisk;
2020-02-03 12:54:36 +00:00
using DatabasePtr = std::shared_ptr<IDatabase>;
using DatabaseAndTable = std::pair<DatabasePtr, StoragePtr>;
2020-02-10 18:19:35 +00:00
using Databases = std::map<String, std::shared_ptr<IDatabase>>;
using DiskPtr = std::shared_ptr<IDisk>;
using TableNamesSet = std::unordered_set<QualifiedTableName>;
2020-03-13 15:41:36 +00:00
2020-02-10 18:31:52 +00:00
/// Allows executing DDL query only in one thread.
/// Puts an element into the map, locks tables's mutex, counts how much threads run parallel query on the table,
/// when counter is 0 erases element in the destructor.
/// If the element already exists in the map, waits when ddl query will be finished in other thread.
2020-02-10 18:31:52 +00:00
class DDLGuard
{
public:
struct Entry
{
std::unique_ptr<std::mutex> mutex;
UInt32 counter;
};
/// Element name -> (mutex, counter).
/// NOTE: using std::map here (and not std::unordered_map) to avoid iterator invalidation on insertion.
using Map = std::map<String, Entry>;
DDLGuard(
Map & map_,
2023-01-12 15:51:04 +00:00
SharedMutex & db_mutex_,
std::unique_lock<std::mutex> guards_lock_,
const String & elem,
const String & database_name);
2020-02-10 18:31:52 +00:00
~DDLGuard();
2021-02-02 19:39:04 +00:00
/// Unlocks table name, keeps holding read lock for database name
void releaseTableLock() noexcept;
2020-02-10 18:31:52 +00:00
private:
Map & map;
2023-01-12 15:51:04 +00:00
SharedMutex & db_mutex;
2020-02-10 18:31:52 +00:00
Map::iterator it;
std::unique_lock<std::mutex> guards_lock;
std::unique_lock<std::mutex> table_lock;
2021-02-02 19:39:04 +00:00
bool table_lock_removed = false;
bool is_database_guard = false;
2020-02-10 18:31:52 +00:00
};
2021-02-04 19:41:44 +00:00
using DDLGuardPtr = std::unique_ptr<DDLGuard>;
2023-05-25 19:18:11 +00:00
class FutureSet;
using FutureSetPtr = std::shared_ptr<FutureSet>;
2020-03-10 19:36:17 +00:00
2020-03-13 15:41:36 +00:00
/// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID.
/// Such table can be accessed from everywhere by its ID.
/// Removes the table from database on destruction.
/// TemporaryTableHolder object can be attached to a query or session Context, so table will be accessible through the context.
struct TemporaryTableHolder : boost::noncopyable, WithContext
2020-03-10 19:36:17 +00:00
{
using Creator = std::function<StoragePtr (const StorageID &)>;
2020-03-10 19:36:17 +00:00
TemporaryTableHolder(ContextPtr context, const Creator & creator, const ASTPtr & query = {});
2020-03-10 19:36:17 +00:00
/// Creates temporary table with Engine=Memory
TemporaryTableHolder(
ContextPtr context,
const ColumnsDescription & columns,
const ConstraintsDescription & constraints,
2020-09-04 08:36:47 +00:00
const ASTPtr & query = {},
bool create_for_global_subquery = false);
2020-03-10 19:36:17 +00:00
TemporaryTableHolder(TemporaryTableHolder && rhs) noexcept;
TemporaryTableHolder & operator=(TemporaryTableHolder && rhs) noexcept;
2020-03-10 19:36:17 +00:00
~TemporaryTableHolder();
StorageID getGlobalTableID() const;
StoragePtr getTable() const;
operator bool () const { return id != UUIDHelpers::Nil; } /// NOLINT
2020-03-13 15:41:36 +00:00
IDatabase * temporary_tables = nullptr;
UUID id = UUIDHelpers::Nil;
2023-05-25 19:18:11 +00:00
FutureSetPtr future_set;
2020-03-10 19:36:17 +00:00
};
2020-03-13 15:41:36 +00:00
///TODO maybe remove shared_ptr from here?
2020-03-10 19:36:17 +00:00
using TemporaryTablesMapping = std::map<String, std::shared_ptr<TemporaryTableHolder>>;
2020-02-10 18:31:52 +00:00
2020-03-19 21:14:52 +00:00
class BackgroundSchedulePoolTaskHolder;
2020-03-13 15:41:36 +00:00
/// For some reason Context is required to get Storage from Database object
2021-05-31 14:49:02 +00:00
class DatabaseCatalog : boost::noncopyable, WithMutableContext
2020-02-03 12:54:36 +00:00
{
public:
/// Names of predefined databases.
2020-02-03 12:54:36 +00:00
static constexpr const char * TEMPORARY_DATABASE = "_temporary_and_external_tables";
static constexpr const char * SYSTEM_DATABASE = "system";
static constexpr const char * INFORMATION_SCHEMA = "information_schema";
static constexpr const char * INFORMATION_SCHEMA_UPPERCASE = "INFORMATION_SCHEMA";
2020-02-03 12:54:36 +00:00
/// Returns true if a passed name is one of the predefined databases' names.
static bool isPredefinedDatabase(std::string_view database_name);
2021-05-31 14:49:02 +00:00
static DatabaseCatalog & init(ContextMutablePtr global_context_);
static DatabaseCatalog & instance();
2020-04-08 23:59:39 +00:00
static void shutdown();
2020-02-03 12:54:36 +00:00
2023-02-17 16:57:49 +00:00
void createBackgroundTasks();
void initializeAndLoadTemporaryDatabase();
void startupBackgroundTasks();
Fix UUID overlap in DROP TABLE for internal DDL from MaterializeMySQL This will fix race with DatabaseCatalog::loadMarkedAsDroppedTables(), since MaterializeMySQL, and MaterializedMySQLSyncThread in background, will be started earlier then DatabaseCatalog::loadMarkedAsDroppedTables() and will move those tables to metadata_dropped, and after loadMarkedAsDroppedTables() will start and try to load partially dropped tables and will hit UUID overlap: 12:02:51.536783 [ 3026034 ] {} <Information> Application: starting up 12:02:53.019282 [ 3026034 ] {} <Information> DatabaseMaterializeMySQL<Atomic> (mysql): Total 9 tables and 0 dictionaries. 12:02:53.041699 [ 3026200 ] {} <Debug> mysql.data (7143b65f-6982-4600-b143-b65f6982e600): Loading data parts 12:02:53.041740 [ 3026200 ] {} <Debug> mysql.data (7143b65f-6982-4600-b143-b65f6982e600): There are no data parts 12:02:53.620382 [ 3026034 ] {} <Information> DatabaseMaterializeMySQL<Atomic> (mysql): Starting up tables. 12:03:00.669730 [ 3026183 ] {} <Debug> executeQuery: (internal) /*Materialize MySQL step 1: execute MySQL DDL for dump data*/ DROP TABLE mysql.data 12:03:00.741894 [ 3026269 ] {} <Information> DatabaseCatalog: Trying load partially dropped table mysql.data (7143b65f-6982-4600-b143-b65f6982e600) from /var/lib/clickhouse/metadata_dropped/mysql.data.7143b65f-6982-4600-b143-b65f6982e600.sql 12:03:00.742582 [ 3026269 ] {} <Debug> mysql.data (7143b65f-6982-4600-b143-b65f6982e600): Loading data parts 12:03:00.742650 [ 3026269 ] {} <Debug> mysql.data (7143b65f-6982-4600-b143-b65f6982e600): There are no data parts 12:03:00.773137 [ 3026034 ] {} <Error> Application: Caught exception while loading metadata: Code: 57, e.displayText() = DB::Exception: Mapping for table with UUID=7143b65f-6982-4600-b143-b65f6982e600 already exists. It happened due to UUID collision, most likely because some not random UUIDs were manually specified in CREATE queries., Stack trace (when copying this message, always include the lines below): 12:03:01.224557 [ 3026034 ] {} <Error> Application: DB::Exception: Mapping for table with UUID=7143b65f-6982-4600-b143-b65f6982e600 already exists. It happened due to UUID collision, most likely because some not random UUIDs were manually specified in CREATE queries. Cc: @zhang2014
2021-09-02 18:05:13 +00:00
void loadMarkedAsDroppedTables();
2020-02-03 12:54:36 +00:00
2020-02-10 18:31:52 +00:00
/// Get an object that protects the table from concurrently executing multiple DDL operations.
2021-02-04 19:41:44 +00:00
DDLGuardPtr getDDLGuard(const String & database, const String & table);
2020-08-18 15:15:27 +00:00
/// Get an object that protects the database from concurrent DDL queries all tables in the database
2023-01-12 15:51:04 +00:00
std::unique_lock<SharedMutex> getExclusiveDDLGuardForDatabase(const String & database);
2020-02-03 12:54:36 +00:00
/// We need special synchronization between DROP/DETACH DATABASE and SYSTEM RESTART REPLICA
/// because IStorage::flushAndPrepareForShutdown cannot be protected by DDLGuard (and a race with IStorage::startup is possible)
std::unique_lock<SharedMutex> getLockForDropDatabase(const String & database);
std::optional<std::shared_lock<SharedMutex>> tryGetLockForRestartReplica(const String & database);
2020-03-13 15:41:36 +00:00
2020-02-03 12:54:36 +00:00
void assertDatabaseExists(const String & database_name) const;
void assertDatabaseDoesntExist(const String & database_name) const;
DatabasePtr getDatabaseForTemporaryTables() const;
DatabasePtr getSystemDatabase() const;
2020-02-03 12:54:36 +00:00
2020-07-08 14:28:07 +00:00
void attachDatabase(const String & database_name, const DatabasePtr & database);
DatabasePtr detachDatabase(ContextPtr local_context, const String & database_name, bool drop = false, bool check_empty = true);
2021-11-02 12:58:45 +00:00
void updateDatabaseName(const String & old_name, const String & new_name, const Strings & tables_in_database);
2020-02-03 12:54:36 +00:00
2020-03-13 15:41:36 +00:00
/// database_name must be not empty
2020-02-17 13:52:59 +00:00
DatabasePtr getDatabase(const String & database_name) const;
DatabasePtr tryGetDatabase(const String & database_name) const;
DatabasePtr getDatabase(const UUID & uuid) const;
DatabasePtr tryGetDatabase(const UUID & uuid) const;
2020-02-17 13:52:59 +00:00
bool isDatabaseExist(const String & database_name) const;
2020-02-03 12:54:36 +00:00
Databases getDatabases() const;
2020-03-13 15:41:36 +00:00
/// Same as getDatabase(const String & database_name), but if database_name is empty, current database of local_context is used
DatabasePtr getDatabase(const String & database_name, ContextPtr local_context) const;
2020-02-03 12:54:36 +00:00
2020-03-13 15:41:36 +00:00
/// For all of the following methods database_name in table_id must be not empty (even for temporary tables).
void assertTableDoesntExist(const StorageID & table_id, ContextPtr context) const;
bool isTableExist(const StorageID & table_id, ContextPtr context) const;
2020-03-13 15:41:36 +00:00
bool isDictionaryExist(const StorageID & table_id) const;
2020-02-03 12:54:36 +00:00
StoragePtr getTable(const StorageID & table_id, ContextPtr context) const;
StoragePtr tryGetTable(const StorageID & table_id, ContextPtr context) const;
DatabaseAndTable getDatabaseAndTable(const StorageID & table_id, ContextPtr context) const;
DatabaseAndTable tryGetDatabaseAndTable(const StorageID & table_id, ContextPtr context) const;
2020-05-28 23:01:18 +00:00
DatabaseAndTable getTableImpl(const StorageID & table_id,
ContextPtr context,
2020-05-28 23:01:18 +00:00
std::optional<Exception> * exception = nullptr) const;
2020-02-03 12:54:36 +00:00
/// Returns true if a passed table_id refers to one of the predefined tables' names.
/// All tables in the "system" database with System* table engine are predefined.
/// Four views (tables, views, columns, schemata) in the "information_schema" database are predefined too.
bool isPredefinedTable(const StorageID & table_id) const;
/// View dependencies between a source table and its view.
void addViewDependency(const StorageID & source_table_id, const StorageID & view_id);
void removeViewDependency(const StorageID & source_table_id, const StorageID & view_id);
std::vector<StorageID> getDependentViews(const StorageID & source_table_id) const;
void updateViewDependency(const StorageID & old_source_table_id, const StorageID & old_view_id, const StorageID & new_source_table_id, const StorageID & new_view_id);
2020-03-13 15:41:36 +00:00
/// If table has UUID, addUUIDMapping(...) must be called when table attached to some database
2020-10-25 17:27:21 +00:00
/// removeUUIDMapping(...) must be called when it detached,
/// and removeUUIDMappingFinally(...) must be called when table is dropped and its data removed from disk.
2020-03-13 15:41:36 +00:00
/// Such tables can be accessed by persistent UUID instead of database and table name.
2020-10-25 17:27:21 +00:00
void addUUIDMapping(const UUID & uuid, const DatabasePtr & database, const StoragePtr & table);
2020-03-13 15:41:36 +00:00
void removeUUIDMapping(const UUID & uuid);
2020-10-25 17:27:21 +00:00
void removeUUIDMappingFinally(const UUID & uuid);
2020-03-23 00:12:13 +00:00
/// For moving table between databases
void updateUUIDMapping(const UUID & uuid, DatabasePtr database, StoragePtr table);
2020-10-27 20:52:49 +00:00
/// This method adds empty mapping (with database and storage equal to nullptr).
/// It's required to "lock" some UUIDs and protect us from collision.
/// Collisions of random 122-bit integers are very unlikely to happen,
/// but we allow to explicitly specify UUID in CREATE query (in particular for testing).
/// If some UUID was already added and we are trying to add it again,
/// this method will throw an exception.
void addUUIDMapping(const UUID & uuid);
2020-03-23 00:12:13 +00:00
bool hasUUIDMapping(const UUID & uuid);
2020-04-06 23:22:44 +00:00
static String getPathForUUID(const UUID & uuid);
2020-03-13 15:41:36 +00:00
DatabaseAndTable tryGetByUUID(const UUID & uuid) const;
2020-03-19 21:14:52 +00:00
String getPathForDroppedMetadata(const StorageID & table_id) const;
2023-03-21 12:14:26 +00:00
String getPathForMetadata(const StorageID & table_id) const;
2020-03-19 21:14:52 +00:00
void enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay = false);
2023-03-21 12:14:26 +00:00
void dequeueDroppedTableCleanup(StorageID table_id);
2020-03-19 21:14:52 +00:00
void waitTableFinallyDropped(const UUID & uuid);
/// Referential dependencies between tables: table "A" depends on table "B"
/// if "B" is referenced in the definition of "A".
/// Loading dependencies were used to check whether a table can be removed before we had those referential dependencies.
/// Now we support this mode (see `check_table_referential_dependencies` in Setting.h) for compatibility.
void addDependencies(const StorageID & table_id, const std::vector<StorageID> & new_referential_dependencies, const std::vector<StorageID> & new_loading_dependencies);
void addDependencies(const QualifiedTableName & table_name, const TableNamesSet & new_referential_dependencies, const TableNamesSet & new_loading_dependencies);
void addDependencies(const TablesDependencyGraph & new_referential_dependencies, const TablesDependencyGraph & new_loading_dependencies);
std::pair<std::vector<StorageID>, std::vector<StorageID>> removeDependencies(const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database = false);
std::vector<StorageID> getReferentialDependencies(const StorageID & table_id) const;
std::vector<StorageID> getReferentialDependents(const StorageID & table_id) const;
std::vector<StorageID> getLoadingDependencies(const StorageID & table_id) const;
std::vector<StorageID> getLoadingDependents(const StorageID & table_id) const;
void updateDependencies(const StorageID & table_id, const TableNamesSet & new_referential_dependencies, const TableNamesSet & new_loading_dependencies);
void checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database = false) const;
2021-11-02 14:30:22 +00:00
struct TableMarkedAsDropped
{
StorageID table_id = StorageID::createEmpty();
StoragePtr table;
String metadata_path;
time_t drop_time{};
};
using TablesMarkedAsDropped = std::list<TableMarkedAsDropped>;
TablesMarkedAsDropped getTablesMarkedDropped()
{
std::lock_guard lock(tables_marked_dropped_mutex);
return tables_marked_dropped;
}
void triggerReloadDisksTask(const Strings & new_added_disks);
2020-02-03 12:54:36 +00:00
private:
// The global instance of database catalog. unique_ptr is to allow
// deferred initialization. Thought I'd use std::optional, but I can't
// make emplace(global_context_) compile with private constructor ¯\_(ツ)_/¯.
static std::unique_ptr<DatabaseCatalog> database_catalog;
2021-05-31 14:49:02 +00:00
explicit DatabaseCatalog(ContextMutablePtr global_context_);
2022-06-28 14:25:29 +00:00
void assertDatabaseDoesntExistUnlocked(const String & database_name) const TSA_REQUIRES(databases_mutex);
2020-02-03 12:54:36 +00:00
2020-04-08 23:59:39 +00:00
void shutdownImpl();
void checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database) const TSA_REQUIRES(databases_mutex);
2020-03-19 21:14:52 +00:00
2020-02-03 12:54:36 +00:00
struct UUIDToStorageMapPart
{
2022-06-27 20:48:27 +00:00
std::unordered_map<UUID, DatabaseAndTable> map TSA_GUARDED_BY(mutex);
2020-02-03 12:54:36 +00:00
mutable std::mutex mutex;
};
2020-03-12 18:04:29 +00:00
static constexpr UInt64 bits_for_first_level = 4;
2020-02-03 12:54:36 +00:00
using UUIDToStorageMap = std::array<UUIDToStorageMapPart, 1ull << bits_for_first_level>;
static inline size_t getFirstLevelIdx(const UUID & uuid)
2020-02-03 12:54:36 +00:00
{
2023-08-18 20:50:31 +00:00
return UUIDHelpers::getHighBytes(uuid) >> (64 - bits_for_first_level);
2020-02-03 12:54:36 +00:00
}
2020-03-19 21:14:52 +00:00
void dropTableDataTask();
2020-10-25 17:27:21 +00:00
void dropTableFinally(const TableMarkedAsDropped & table);
2020-03-19 21:14:52 +00:00
void cleanupStoreDirectoryTask();
bool maybeRemoveDirectory(const String & disk_name, const DiskPtr & disk, const String & unused_dir);
void reloadDisksTask();
2020-03-19 21:14:52 +00:00
static constexpr size_t reschedule_time_ms = 100;
2020-02-17 13:52:59 +00:00
mutable std::mutex databases_mutex;
2022-06-27 20:48:27 +00:00
Databases databases TSA_GUARDED_BY(databases_mutex);
2020-02-03 12:54:36 +00:00
UUIDToStorageMap uuid_map;
/// Referential dependencies between tables: table "A" depends on table "B"
/// if the table "B" is referenced in the definition of the table "A".
TablesDependencyGraph referential_dependencies TSA_GUARDED_BY(databases_mutex);
/// Loading dependencies were used to check whether a table can be removed before we had referential dependencies.
TablesDependencyGraph loading_dependencies TSA_GUARDED_BY(databases_mutex);
/// View dependencies between a source table and its view.
TablesDependencyGraph view_dependencies TSA_GUARDED_BY(databases_mutex);
Poco::Logger * log;
2020-02-03 12:54:36 +00:00
std::atomic_bool is_shutting_down = false;
2020-02-10 18:31:52 +00:00
/// Do not allow simultaneous execution of DDL requests on the same table.
2020-08-18 15:15:27 +00:00
/// database name -> database guard -> (table name mutex, counter),
/// counter: how many threads are running a query on the table at the same time
2020-02-10 18:31:52 +00:00
/// For the duration of the operation, an element is placed here, and an object is returned,
/// which deletes the element in the destructor when counter becomes zero.
/// In case the element already exists, waits when query will be executed in other thread. See class DDLGuard below.
struct DatabaseGuard
{
SharedMutex database_ddl_mutex;
SharedMutex restart_replica_mutex;
DDLGuard::Map table_guards;
};
DatabaseGuard & getDatabaseGuard(const String & database);
2020-08-18 15:15:27 +00:00
using DDLGuards = std::map<String, DatabaseGuard>;
2022-06-27 20:48:27 +00:00
DDLGuards ddl_guards TSA_GUARDED_BY(ddl_guards_mutex);
2020-02-10 18:31:52 +00:00
/// If you capture mutex and ddl_guards_mutex, then you need to grab them strictly in this order.
mutable std::mutex ddl_guards_mutex;
2020-03-19 21:14:52 +00:00
2022-06-27 20:48:27 +00:00
TablesMarkedAsDropped tables_marked_dropped TSA_GUARDED_BY(tables_marked_dropped_mutex);
TablesMarkedAsDropped::iterator first_async_drop_in_queue TSA_GUARDED_BY(tables_marked_dropped_mutex);
2022-06-27 20:48:27 +00:00
std::unordered_set<UUID> tables_marked_dropped_ids TSA_GUARDED_BY(tables_marked_dropped_mutex);
2020-04-08 01:02:00 +00:00
mutable std::mutex tables_marked_dropped_mutex;
2020-03-19 21:14:52 +00:00
std::unique_ptr<BackgroundSchedulePoolTaskHolder> drop_task;
static constexpr time_t default_drop_delay_sec = 8 * 60;
time_t drop_delay_sec = default_drop_delay_sec;
std::condition_variable wait_table_finally_dropped;
std::unique_ptr<BackgroundSchedulePoolTaskHolder> cleanup_task;
static constexpr time_t default_unused_dir_hide_timeout_sec = 60 * 60; /// 1 hour
time_t unused_dir_hide_timeout_sec = default_unused_dir_hide_timeout_sec;
static constexpr time_t default_unused_dir_rm_timeout_sec = 30 * 24 * 60 * 60; /// 30 days
time_t unused_dir_rm_timeout_sec = default_unused_dir_rm_timeout_sec;
static constexpr time_t default_unused_dir_cleanup_period_sec = 24 * 60 * 60; /// 1 day
time_t unused_dir_cleanup_period_sec = default_unused_dir_cleanup_period_sec;
2023-02-13 23:45:28 +00:00
static constexpr time_t default_drop_error_cooldown_sec = 5;
time_t drop_error_cooldown_sec = default_drop_error_cooldown_sec;
std::unique_ptr<BackgroundSchedulePoolTaskHolder> reload_disks_task;
std::mutex reload_disks_mutex;
std::set<String> disks_to_reload;
static constexpr time_t DBMS_DEFAULT_DISK_RELOAD_PERIOD_SEC = 5;
2020-02-03 12:54:36 +00:00
};
2023-12-02 03:48:59 +00:00
class TableNameHints : public IHints<>
{
public:
TableNameHints(ConstDatabasePtr database_, ContextPtr context_)
: context(context_),
database(database_)
{
}
/// getHintForTable tries to get a hint for the provided table_name in the provided
/// database. If the results are empty, it goes for extended hints for the table
/// with getExtendedHintForTable which looks for the table name in every database that's
/// available in the database catalog. It finally returns a single hint which is the database
/// name and table_name pair which is similar to the table_name provided. Perhaps something to
/// consider is should we return more than one pair of hint?
std::pair<String, String> getHintForTable(const String & table_name) const
{
auto results = this->getHints(table_name, getAllRegisteredNames());
if (results.empty())
return getExtendedHintForTable(table_name);
return std::make_pair(database->getDatabaseName(), results[0]);
}
/// getExtendedHintsForTable tries to get hint for the given table_name across all
/// the databases that are available in the database catalog.
std::pair<String, String> getExtendedHintForTable(const String & table_name) const
{
/// load all available databases from the DatabaseCatalog instance
auto & database_catalog = DatabaseCatalog::instance();
auto all_databases = database_catalog.getDatabases();
for (const auto & [db_name, db] : all_databases)
{
/// this case should be covered already by getHintForTable
if (db_name == database->getDatabaseName())
continue;
TableNameHints hints(db, context);
auto results = hints.getHints(table_name);
2023-12-14 19:20:52 +00:00
/// if the results are not empty, return the first instance of the table_name
/// and the corresponding database_name that was found.
if (!results.empty())
return std::make_pair(db_name, results[0]);
2023-12-02 03:48:59 +00:00
}
2023-12-14 19:20:52 +00:00
return {};
2023-12-02 03:48:59 +00:00
}
Names getAllRegisteredNames() const override
{
Names result;
if (database)
for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next())
result.emplace_back(table_it->name());
return result;
}
private:
ContextPtr context;
ConstDatabasePtr database;
};
2022-06-22 16:31:42 +00:00
/// This class is useful when creating a table or database.
/// Usually we create IStorage/IDatabase object first and then add it to IDatabase/DatabaseCatalog.
/// But such object may start using a directory in store/ since its creation.
/// To avoid race with cleanupStoreDirectoryTask() we have to mark UUID as used first.
/// Then we can either add DatabasePtr/StoragePtr to the created UUID mapping
/// or remove the lock if creation failed.
/// See also addUUIDMapping(...)
class TemporaryLockForUUIDDirectory : private boost::noncopyable
2022-06-21 22:50:16 +00:00
{
2022-06-22 16:31:42 +00:00
UUID uuid = UUIDHelpers::Nil;
2022-06-21 22:50:16 +00:00
public:
2022-06-22 16:31:42 +00:00
TemporaryLockForUUIDDirectory() = default;
TemporaryLockForUUIDDirectory(UUID uuid_);
~TemporaryLockForUUIDDirectory();
TemporaryLockForUUIDDirectory(TemporaryLockForUUIDDirectory && rhs) noexcept;
TemporaryLockForUUIDDirectory & operator = (TemporaryLockForUUIDDirectory && rhs) noexcept;
2022-06-21 22:50:16 +00:00
};
2020-02-03 12:54:36 +00:00
}