ClickHouse/src/Interpreters/DatabaseCatalog.h

300 lines
12 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/TablesLoader.h>
2020-03-10 19:36:17 +00:00
#include <Parsers/IAST_fwd.h>
#include <Storages/IStorage_fwd.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>
#include <shared_mutex>
2020-02-03 12:54:36 +00:00
#include <unordered_map>
#include <unordered_set>
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;
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>>;
2020-02-03 12:54:36 +00:00
/// Table -> set of table-views that make SELECT from it.
using ViewDependencies = std::map<StorageID, std::set<StorageID>>;
using Dependencies = std::vector<StorageID>;
2020-02-10 18:31:52 +00:00
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_,
std::shared_mutex & 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;
2020-08-18 15:15:27 +00:00
std::shared_mutex & 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>;
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);
TemporaryTableHolder & operator = (TemporaryTableHolder && rhs);
~TemporaryTableHolder();
StorageID getGlobalTableID() const;
StoragePtr getTable() const;
2020-03-13 15:41:36 +00:00
operator bool () const { return id != UUIDHelpers::Nil; }
IDatabase * temporary_tables = nullptr;
UUID id = UUIDHelpers::Nil;
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:
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
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
void initializeAndLoadTemporaryDatabase();
2020-02-03 12:54:36 +00:00
void loadDatabases();
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
std::unique_lock<std::shared_mutex> getExclusiveDDLGuardForDatabase(const String & database);
2020-02-03 12:54:36 +00:00
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
void addDependency(const StorageID & from, const StorageID & where);
void removeDependency(const StorageID & from, const StorageID & where);
Dependencies getDependencies(const StorageID & from) const;
/// For Materialized and Live View
2020-03-11 19:10:55 +00:00
void updateDependency(const StorageID & old_from, const StorageID & old_where,const StorageID & new_from, const StorageID & new_where);
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
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;
void enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay = false);
void waitTableFinallyDropped(const UUID & uuid);
2021-11-02 12:58:45 +00:00
void addLoadingDependencies(const QualifiedTableName & table, TableNamesSet && dependencies);
void addLoadingDependencies(const DependenciesInfos & new_infos);
DependenciesInfo getLoadingDependenciesInfo(const StorageID & table_id) const;
2021-11-02 12:58:45 +00:00
TableNamesSet tryRemoveLoadingDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database = false);
2021-11-03 15:06:19 +00:00
TableNamesSet tryRemoveLoadingDependenciesUnlocked(const QualifiedTableName & removing_table, bool check_dependencies, bool is_drop_database = false);
2021-11-02 12:58:45 +00:00
void checkTableCanBeRemovedOrRenamed(const StorageID & table_id) const;
2021-11-02 14:30:22 +00:00
void updateLoadingDependencies(const StorageID & table_id, TableNamesSet && new_dependencies);
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_);
2020-02-03 12:54:36 +00:00
void assertDatabaseExistsUnlocked(const String & database_name) const;
void assertDatabaseDoesntExistUnlocked(const String & database_name) const;
2020-04-08 23:59:39 +00:00
void shutdownImpl();
2020-03-19 21:14:52 +00:00
2020-02-03 12:54:36 +00:00
struct UUIDToStorageMapPart
{
std::unordered_map<UUID, DatabaseAndTable> map;
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
{
2021-01-27 00:54:57 +00:00
return uuid.toUnderType().items[0] >> (64 - bits_for_first_level);
2020-02-03 12:54:36 +00:00
}
2020-03-19 21:14:52 +00:00
struct TableMarkedAsDropped
{
StorageID table_id = StorageID::createEmpty();
StoragePtr table;
String metadata_path;
2021-05-08 15:20:40 +00:00
time_t drop_time{};
2020-03-19 21:14:52 +00:00
};
using TablesMarkedAsDropped = std::list<TableMarkedAsDropped>;
void dropTableDataTask();
2020-10-25 17:27:21 +00:00
void dropTableFinally(const TableMarkedAsDropped & table);
2020-03-19 21:14:52 +00:00
static constexpr size_t reschedule_time_ms = 100;
2020-11-01 17:37:54 +00:00
static constexpr time_t drop_error_cooldown_sec = 5;
2020-03-19 21:14:52 +00:00
using UUIDToDatabaseMap = std::unordered_map<UUID, DatabasePtr>;
2020-02-17 13:52:59 +00:00
mutable std::mutex databases_mutex;
2020-03-13 15:41:36 +00:00
ViewDependencies view_dependencies;
2020-02-03 12:54:36 +00:00
Databases databases;
UUIDToDatabaseMap db_uuid_map;
2020-02-03 12:54:36 +00:00
UUIDToStorageMap uuid_map;
DependenciesInfos loading_dependencies;
Poco::Logger * log;
2020-02-03 12:54:36 +00:00
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.
2020-08-18 15:15:27 +00:00
using DatabaseGuard = std::pair<DDLGuard::Map, std::shared_mutex>;
using DDLGuards = std::map<String, DatabaseGuard>;
2020-02-10 18:31:52 +00:00
DDLGuards ddl_guards;
/// 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
2020-04-08 01:02:00 +00:00
TablesMarkedAsDropped tables_marked_dropped;
std::unordered_set<UUID> tables_marked_dropped_ids;
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;
2020-02-03 12:54:36 +00:00
};
}