2020-02-03 12:54:36 +00:00
# include <Interpreters/DatabaseCatalog.h>
# include <Interpreters/Context.h>
# include <Interpreters/loadMetadata.h>
2023-03-26 05:15:23 +00:00
# include <Interpreters/executeQuery.h>
2023-03-27 23:27:13 +00:00
# include <Interpreters/InterpreterCreateQuery.h>
2020-03-10 19:36:17 +00:00
# include <Storages/IStorage.h>
2020-02-03 12:54:36 +00:00
# include <Databases/IDatabase.h>
# include <Databases/DatabaseMemory.h>
2020-09-14 19:25:02 +00:00
# include <Databases/DatabaseOnDisk.h>
2022-08-03 17:49:16 +00:00
# include <Disks/IDisk.h>
2020-03-10 19:36:17 +00:00
# include <Storages/StorageMemory.h>
2020-03-19 21:14:52 +00:00
# include <Core/BackgroundSchedulePool.h>
# include <Parsers/formatAST.h>
# include <IO/ReadHelpers.h>
# include <Poco/DirectoryIterator.h>
2023-04-06 17:00:51 +00:00
# include <Poco/Util/AbstractConfiguration.h>
# include <Common/quoteString.h>
2022-04-11 12:32:55 +00:00
# include <Common/atomicRename.h>
2020-10-25 17:27:21 +00:00
# include <Common/CurrentMetrics.h>
2022-04-27 15:05:45 +00:00
# include <Common/logger_useful.h>
2023-04-06 17:00:51 +00:00
# include <Common/ThreadPool.h>
2021-05-28 21:57:53 +00:00
# include <Common/filesystemHelpers.h>
2022-06-22 16:31:42 +00:00
# include <Common/noexcept_scope.h>
2022-08-25 19:26:32 +00:00
# include <Common/checkStackSize.h>
2020-02-03 12:54:36 +00:00
2022-09-28 13:29:29 +00:00
# include "config.h"
2020-09-14 19:25:02 +00:00
# if USE_MYSQL
2021-07-26 18:17:28 +00:00
# include <Databases / MySQL / MaterializedMySQLSyncThread.h>
# include <Storages / StorageMaterializedMySQL.h>
2020-09-14 19:25:02 +00:00
# endif
2021-07-01 07:33:58 +00:00
# if USE_LIBPQXX
2021-08-27 12:50:45 +00:00
# include <Databases / PostgreSQL / DatabaseMaterializedPostgreSQL.h>
2022-06-21 10:35:47 +00:00
# include <Storages / PostgreSQL / StorageMaterializedPostgreSQL.h>
2021-07-01 07:33:58 +00:00
# endif
2022-08-25 19:26:32 +00:00
2020-10-25 17:27:21 +00:00
namespace CurrentMetrics
{
extern const Metric TablesToDropQueueSize ;
2023-03-22 07:49:22 +00:00
extern const Metric DatabaseCatalogThreads ;
extern const Metric DatabaseCatalogThreadsActive ;
2020-10-25 17:27:21 +00:00
}
2020-02-03 12:54:36 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_DATABASE ;
extern const int UNKNOWN_TABLE ;
extern const int TABLE_ALREADY_EXISTS ;
extern const int DATABASE_ALREADY_EXISTS ;
2020-02-13 21:00:03 +00:00
extern const int DATABASE_NOT_EMPTY ;
2020-03-03 19:53:18 +00:00
extern const int DATABASE_ACCESS_DENIED ;
2020-03-12 12:16:16 +00:00
extern const int LOGICAL_ERROR ;
2021-11-01 18:53:07 +00:00
extern const int HAVE_DEPENDENT_OBJECTS ;
2020-02-03 12:54:36 +00:00
}
2021-04-10 23:33:54 +00:00
TemporaryTableHolder : : TemporaryTableHolder ( ContextPtr context_ , const TemporaryTableHolder : : Creator & creator , const ASTPtr & query )
: WithContext ( context_ - > getGlobalContext ( ) )
2020-03-13 15:41:36 +00:00
, temporary_tables ( DatabaseCatalog : : instance ( ) . getDatabaseForTemporaryTables ( ) . get ( ) )
2020-03-10 19:36:17 +00:00
{
ASTPtr original_create ;
ASTCreateQuery * create = dynamic_cast < ASTCreateQuery * > ( query . get ( ) ) ;
String global_name ;
2021-05-08 14:21:36 +00:00
if ( create )
2020-03-10 19:36:17 +00:00
{
original_create = create - > clone ( ) ;
if ( create - > uuid = = UUIDHelpers : : Nil )
create - > uuid = UUIDHelpers : : generateV4 ( ) ;
id = create - > uuid ;
2021-09-06 22:13:54 +00:00
create - > setTable ( " _tmp_ " + toString ( id ) ) ;
global_name = create - > getTable ( ) ;
create - > setDatabase ( DatabaseCatalog : : TEMPORARY_DATABASE ) ;
2020-03-10 19:36:17 +00:00
}
else
{
id = UUIDHelpers : : generateV4 ( ) ;
global_name = " _tmp_ " + toString ( id ) ;
}
auto table_id = StorageID ( DatabaseCatalog : : TEMPORARY_DATABASE , global_name , id ) ;
auto table = creator ( table_id ) ;
2022-06-22 16:31:42 +00:00
DatabaseCatalog : : instance ( ) . addUUIDMapping ( id ) ;
2021-04-10 23:33:54 +00:00
temporary_tables - > createTable ( getContext ( ) , global_name , table , original_create ) ;
2020-03-10 19:36:17 +00:00
table - > startup ( ) ;
}
2020-05-29 02:08:48 +00:00
TemporaryTableHolder : : TemporaryTableHolder (
2021-04-10 23:33:54 +00:00
ContextPtr context_ ,
2020-05-29 02:08:48 +00:00
const ColumnsDescription & columns ,
const ConstraintsDescription & constraints ,
2020-09-04 08:36:47 +00:00
const ASTPtr & query ,
bool create_for_global_subquery )
2021-04-23 12:18:23 +00:00
: TemporaryTableHolder (
context_ ,
2021-05-13 06:22:05 +00:00
[ & ] ( const StorageID & table_id )
{
2022-04-19 20:47:29 +00:00
auto storage = std : : make_shared < StorageMemory > ( table_id , ColumnsDescription { columns } , ConstraintsDescription { constraints } , String { } ) ;
2020-09-04 08:36:47 +00:00
2021-04-23 12:18:23 +00:00
if ( create_for_global_subquery )
storage - > delayReadForGlobalSubqueries ( ) ;
2020-09-04 08:36:47 +00:00
2021-04-23 12:18:23 +00:00
return storage ;
} ,
query )
2020-03-10 19:36:17 +00:00
{
}
2022-02-25 19:04:48 +00:00
TemporaryTableHolder : : TemporaryTableHolder ( TemporaryTableHolder & & rhs ) noexcept
2021-04-10 23:33:54 +00:00
: WithContext ( rhs . context ) , temporary_tables ( rhs . temporary_tables ) , id ( rhs . id )
2020-03-10 19:36:17 +00:00
{
rhs . id = UUIDHelpers : : Nil ;
}
2022-02-25 19:04:48 +00:00
TemporaryTableHolder & TemporaryTableHolder : : operator = ( TemporaryTableHolder & & rhs ) noexcept
2020-03-10 19:36:17 +00:00
{
id = rhs . id ;
rhs . id = UUIDHelpers : : Nil ;
return * this ;
}
TemporaryTableHolder : : ~ TemporaryTableHolder ( )
{
if ( id ! = UUIDHelpers : : Nil )
2022-05-27 04:50:36 +00:00
{
2023-02-17 07:47:23 +00:00
try
{
auto table = getTable ( ) ;
table - > flushAndShutdown ( ) ;
temporary_tables - > dropTable ( getContext ( ) , " _tmp_ " + toString ( id ) ) ;
}
catch ( . . . )
{
tryLogCurrentException ( " TemporaryTableHolder " ) ;
}
2022-05-27 04:50:36 +00:00
}
2020-03-10 19:36:17 +00:00
}
StorageID TemporaryTableHolder : : getGlobalTableID ( ) const
{
return StorageID { DatabaseCatalog : : TEMPORARY_DATABASE , " _tmp_ " + toString ( id ) , id } ;
}
StoragePtr TemporaryTableHolder : : getTable ( ) const
{
2021-04-10 23:33:54 +00:00
auto table = temporary_tables - > tryGetTable ( " _tmp_ " + toString ( id ) , getContext ( ) ) ;
2020-03-10 19:36:17 +00:00
if ( ! table )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Temporary table {} not found " , getGlobalTableID ( ) . getNameForLogs ( ) ) ;
2020-03-10 19:36:17 +00:00
return table ;
}
2021-06-03 14:05:37 +00:00
void DatabaseCatalog : : initializeAndLoadTemporaryDatabase ( )
2020-02-03 12:54:36 +00:00
{
2021-04-10 23:33:54 +00:00
drop_delay_sec = getContext ( ) - > getConfigRef ( ) . getInt ( " database_atomic_delay_before_drop_table_sec " , default_drop_delay_sec ) ;
2022-10-07 10:46:45 +00:00
unused_dir_hide_timeout_sec = getContext ( ) - > getConfigRef ( ) . getInt64 ( " database_catalog_unused_dir_hide_timeout_sec " , unused_dir_hide_timeout_sec ) ;
unused_dir_rm_timeout_sec = getContext ( ) - > getConfigRef ( ) . getInt64 ( " database_catalog_unused_dir_rm_timeout_sec " , unused_dir_rm_timeout_sec ) ;
unused_dir_cleanup_period_sec = getContext ( ) - > getConfigRef ( ) . getInt64 ( " database_catalog_unused_dir_cleanup_period_sec " , unused_dir_cleanup_period_sec ) ;
2023-02-13 23:45:28 +00:00
drop_error_cooldown_sec = getContext ( ) - > getConfigRef ( ) . getInt64 ( " database_catalog_drop_error_cooldown_sec " , drop_error_cooldown_sec ) ;
2020-02-03 12:54:36 +00:00
2021-04-10 23:33:54 +00:00
auto db_for_temporary_and_external_tables = std : : make_shared < DatabaseMemory > ( TEMPORARY_DATABASE , getContext ( ) ) ;
2020-07-08 14:28:07 +00:00
attachDatabase ( TEMPORARY_DATABASE , db_for_temporary_and_external_tables ) ;
2021-02-10 14:12:49 +00:00
}
2020-03-19 21:14:52 +00:00
2023-02-17 16:57:49 +00:00
void DatabaseCatalog : : createBackgroundTasks ( )
2021-02-10 14:12:49 +00:00
{
2023-02-17 18:18:49 +00:00
/// It has to be done before databases are loaded (to avoid a race condition on initialization)
2022-06-24 15:34:41 +00:00
if ( Context : : getGlobalContextInstance ( ) - > getApplicationType ( ) = = Context : : ApplicationType : : SERVER & & unused_dir_cleanup_period_sec )
2022-06-21 22:50:16 +00:00
{
auto cleanup_task_holder
= getContext ( ) - > getSchedulePool ( ) . createTask ( " DatabaseCatalog " , [ this ] ( ) { this - > cleanupStoreDirectoryTask ( ) ; } ) ;
cleanup_task = std : : make_unique < BackgroundSchedulePoolTaskHolder > ( std : : move ( cleanup_task_holder ) ) ;
2023-02-17 16:57:49 +00:00
}
auto task_holder = getContext ( ) - > getSchedulePool ( ) . createTask ( " DatabaseCatalog " , [ this ] ( ) { this - > dropTableDataTask ( ) ; } ) ;
drop_task = std : : make_unique < BackgroundSchedulePoolTaskHolder > ( std : : move ( task_holder ) ) ;
}
void DatabaseCatalog : : startupBackgroundCleanup ( )
{
/// And it has to be done after all databases are loaded, otherwise cleanup_task may remove something that should not be removed
if ( cleanup_task )
{
2022-06-21 22:50:16 +00:00
( * cleanup_task ) - > activate ( ) ;
/// Do not start task immediately on server startup, it's not urgent.
( * cleanup_task ) - > scheduleAfter ( unused_dir_hide_timeout_sec * 1000 ) ;
}
2022-06-21 10:35:47 +00:00
2020-11-01 17:37:54 +00:00
( * drop_task ) - > activate ( ) ;
std : : lock_guard lock { tables_marked_dropped_mutex } ;
if ( ! tables_marked_dropped . empty ( ) )
( * drop_task ) - > schedule ( ) ;
2020-02-03 12:54:36 +00:00
}
2020-04-08 23:59:39 +00:00
void DatabaseCatalog : : shutdownImpl ( )
2020-02-03 12:54:36 +00:00
{
2022-06-21 10:35:47 +00:00
if ( cleanup_task )
( * cleanup_task ) - > deactivate ( ) ;
2020-03-19 21:14:52 +00:00
if ( drop_task )
( * drop_task ) - > deactivate ( ) ;
2020-02-03 12:54:36 +00:00
/** At this point, some tables may have threads that block our mutex.
* To shutdown them correctly , we will copy the current list of tables ,
* and ask them all to finish their work .
* Then delete all objects with tables .
*/
Databases current_databases ;
{
std : : lock_guard lock ( databases_mutex ) ;
current_databases = databases ;
}
/// We still hold "databases" (instead of std::move) for Buffer tables to flush data correctly.
for ( auto & database : current_databases )
database . second - > shutdown ( ) ;
2022-06-27 20:48:27 +00:00
{
std : : lock_guard lock ( tables_marked_dropped_mutex ) ;
tables_marked_dropped . clear ( ) ;
}
2020-07-17 17:54:24 +00:00
2020-02-03 12:54:36 +00:00
std : : lock_guard lock ( databases_mutex ) ;
2022-06-22 16:31:42 +00:00
for ( const auto & db : databases )
{
UUID db_uuid = db . second - > getUUID ( ) ;
if ( db_uuid ! = UUIDHelpers : : Nil )
removeUUIDMapping ( db_uuid ) ;
}
2020-10-25 21:51:30 +00:00
assert ( std : : find_if ( uuid_map . begin ( ) , uuid_map . end ( ) , [ ] ( const auto & elem )
{
2020-11-12 23:27:18 +00:00
/// Ensure that all UUID mappings are empty (i.e. all mappings contain nullptr instead of a pointer to storage)
2020-10-27 20:52:49 +00:00
const auto & not_empty_mapping = [ ] ( const auto & mapping )
{
2022-06-22 16:31:42 +00:00
auto & db = mapping . second . first ;
2020-10-27 20:52:49 +00:00
auto & table = mapping . second . second ;
2022-06-22 16:31:42 +00:00
return db | | table ;
2020-10-27 20:52:49 +00:00
} ;
2022-06-27 20:48:27 +00:00
std : : lock_guard map_lock { elem . mutex } ;
2020-10-25 21:51:30 +00:00
auto it = std : : find_if ( elem . map . begin ( ) , elem . map . end ( ) , not_empty_mapping ) ;
return it ! = elem . map . end ( ) ;
} ) = = uuid_map . end ( ) ) ;
2020-02-12 16:54:26 +00:00
databases . clear ( ) ;
2022-12-02 14:05:46 +00:00
referential_dependencies . clear ( ) ;
2023-02-01 23:30:49 +00:00
loading_dependencies . clear ( ) ;
2020-02-12 16:54:26 +00:00
view_dependencies . clear ( ) ;
2020-02-03 12:54:36 +00:00
}
2022-07-14 16:11:35 +00:00
bool DatabaseCatalog : : isPredefinedDatabase ( std : : string_view database_name )
2022-06-09 16:19:54 +00:00
{
return database_name = = TEMPORARY_DATABASE | | database_name = = SYSTEM_DATABASE | | database_name = = INFORMATION_SCHEMA
| | database_name = = INFORMATION_SCHEMA_UPPERCASE ;
}
2022-06-28 07:59:02 +00:00
2020-02-03 12:54:36 +00:00
DatabaseAndTable DatabaseCatalog : : tryGetByUUID ( const UUID & uuid ) const
{
2020-03-16 09:16:14 +00:00
assert ( uuid ! = UUIDHelpers : : Nil & & getFirstLevelIdx ( uuid ) < uuid_map . size ( ) ) ;
2020-02-03 12:54:36 +00:00
const UUIDToStorageMapPart & map_part = uuid_map [ getFirstLevelIdx ( uuid ) ] ;
std : : lock_guard lock { map_part . mutex } ;
auto it = map_part . map . find ( uuid ) ;
if ( it = = map_part . map . end ( ) )
return { } ;
return it - > second ;
}
2020-03-23 02:12:31 +00:00
DatabaseAndTable DatabaseCatalog : : getTableImpl (
const StorageID & table_id ,
2021-04-10 23:33:54 +00:00
ContextPtr context_ ,
2020-03-23 02:12:31 +00:00
std : : optional < Exception > * exception ) const
2020-02-03 12:54:36 +00:00
{
2022-08-25 19:26:32 +00:00
checkStackSize ( ) ;
2020-02-17 13:52:59 +00:00
if ( ! table_id )
{
if ( exception )
2020-12-14 14:21:31 +00:00
exception - > emplace ( ErrorCodes : : UNKNOWN_TABLE , " Cannot find table: StorageID is empty " ) ;
2020-02-17 13:52:59 +00:00
return { } ;
}
2020-03-12 18:04:29 +00:00
if ( table_id . hasUUID ( ) )
{
2020-03-13 15:41:36 +00:00
/// Shortcut for tables which have persistent UUID
2020-03-12 18:04:29 +00:00
auto db_and_table = tryGetByUUID ( table_id . uuid ) ;
if ( ! db_and_table . first | | ! db_and_table . second )
{
assert ( ! db_and_table . first & & ! db_and_table . second ) ;
if ( exception )
2023-01-17 00:19:44 +00:00
exception - > emplace ( Exception ( ErrorCodes : : UNKNOWN_TABLE , " Table {} doesn't exist " , table_id . getNameForLogs ( ) ) ) ;
2020-03-12 18:04:29 +00:00
return { } ;
}
2020-09-14 19:25:02 +00:00
2021-07-01 07:33:58 +00:00
# if USE_LIBPQXX
if ( ! context_ - > isInternalQuery ( ) & & ( db_and_table . first - > getEngineName ( ) = = " MaterializedPostgreSQL " ) )
{
2021-08-27 12:50:45 +00:00
db_and_table . second = std : : make_shared < StorageMaterializedPostgreSQL > ( std : : move ( db_and_table . second ) , getContext ( ) ,
assert_cast < const DatabaseMaterializedPostgreSQL * > ( db_and_table . first . get ( ) ) - > getPostgreSQLDatabaseName ( ) ,
db_and_table . second - > getStorageID ( ) . table_name ) ;
2021-07-01 07:33:58 +00:00
}
# endif
2020-09-14 19:25:02 +00:00
# if USE_MYSQL
2021-07-26 18:17:28 +00:00
/// It's definitely not the best place for this logic, but behaviour must be consistent with DatabaseMaterializedMySQL::tryGetTable(...)
2021-11-10 13:13:27 +00:00
if ( ! context_ - > isInternalQuery ( ) & & db_and_table . first - > getEngineName ( ) = = " MaterializedMySQL " )
2020-09-14 19:25:02 +00:00
{
2022-05-04 08:15:25 +00:00
db_and_table . second = std : : make_shared < StorageMaterializedMySQL > ( std : : move ( db_and_table . second ) , db_and_table . first . get ( ) ) ;
2020-09-14 19:25:02 +00:00
}
# endif
2020-03-12 18:04:29 +00:00
return db_and_table ;
}
2020-02-03 12:54:36 +00:00
2021-07-01 07:33:58 +00:00
2020-03-13 15:41:36 +00:00
if ( table_id . database_name = = TEMPORARY_DATABASE )
{
/// For temporary tables UUIDs are set in Context::resolveStorageID(...).
/// If table_id has no UUID, then the name of database was specified by user and table_id was not resolved through context.
/// Do not allow access to TEMPORARY_DATABASE because it contains all temporary tables of all contexts and users.
if ( exception )
2023-01-23 13:16:14 +00:00
exception - > emplace ( Exception ( ErrorCodes : : DATABASE_ACCESS_DENIED , " Direct access to `{}` database is not allowed " , TEMPORARY_DATABASE ) ) ;
2020-03-13 15:41:36 +00:00
return { } ;
}
2020-02-11 17:25:26 +00:00
DatabasePtr database ;
2020-02-03 12:54:36 +00:00
{
2020-03-23 02:12:31 +00:00
std : : lock_guard lock { databases_mutex } ;
2020-02-11 17:25:26 +00:00
auto it = databases . find ( table_id . getDatabaseName ( ) ) ;
if ( databases . end ( ) = = it )
{
if ( exception )
2023-01-23 13:16:14 +00:00
exception - > emplace ( Exception ( ErrorCodes : : UNKNOWN_DATABASE , " Database {} doesn't exist " , backQuoteIfNeed ( table_id . getDatabaseName ( ) ) ) ) ;
2020-02-11 17:25:26 +00:00
return { } ;
}
database = it - > second ;
2020-02-03 12:54:36 +00:00
}
2023-04-25 22:13:29 +00:00
StoragePtr table = nullptr ;
try
{
table = database - > getTable ( table_id . table_name , context_ ) ;
}
catch ( const Exception & e )
{
if ( exception )
2023-05-01 18:00:26 +00:00
exception - > emplace ( e ) ;
2023-04-25 22:13:29 +00:00
}
2020-07-15 19:25:31 +00:00
if ( ! table )
database = nullptr ;
2020-02-03 12:54:36 +00:00
2020-03-04 20:29:52 +00:00
return { database , table } ;
2020-02-03 12:54:36 +00:00
}
2022-06-28 07:59:02 +00:00
bool DatabaseCatalog : : isPredefinedTable ( const StorageID & table_id ) const
{
2022-06-30 08:10:12 +00:00
static const char * information_schema_views [ ] = { " schemata " , " tables " , " views " , " columns " } ;
static const char * information_schema_views_uppercase [ ] = { " SCHEMATA " , " TABLES " , " VIEWS " , " COLUMNS " } ;
2022-06-28 07:59:02 +00:00
2022-06-30 08:10:12 +00:00
auto check_database_and_table_name = [ & ] ( const String & database_name , const String & table_name )
2022-06-28 07:59:02 +00:00
{
if ( database_name = = SYSTEM_DATABASE )
{
auto storage = getSystemDatabase ( ) - > tryGetTable ( table_name , getContext ( ) ) ;
return storage & & storage - > isSystemStorage ( ) ;
}
if ( database_name = = INFORMATION_SCHEMA )
{
2022-06-30 08:10:12 +00:00
return std : : find ( std : : begin ( information_schema_views ) , std : : end ( information_schema_views ) , table_name )
! = std : : end ( information_schema_views ) ;
2022-06-28 07:59:02 +00:00
}
if ( database_name = = INFORMATION_SCHEMA_UPPERCASE )
{
2022-06-30 08:10:12 +00:00
return std : : find ( std : : begin ( information_schema_views_uppercase ) , std : : end ( information_schema_views_uppercase ) , table_name )
! = std : : end ( information_schema_views_uppercase ) ;
2022-06-28 07:59:02 +00:00
}
return false ;
} ;
if ( table_id . hasUUID ( ) )
{
if ( auto storage = tryGetByUUID ( table_id . uuid ) . second )
{
if ( storage - > isSystemStorage ( ) )
return true ;
auto res_id = storage - > getStorageID ( ) ;
String database_name = res_id . getDatabaseName ( ) ;
2022-06-30 08:10:12 +00:00
if ( database_name ! = SYSTEM_DATABASE ) /// If (database_name == SYSTEM_DATABASE) then we have already checked it (see isSystemStorage() above).
return check_database_and_table_name ( database_name , res_id . getTableName ( ) ) ;
2022-06-28 07:59:02 +00:00
}
return false ;
}
2022-06-30 08:10:12 +00:00
return check_database_and_table_name ( table_id . getDatabaseName ( ) , table_id . getTableName ( ) ) ;
2022-06-28 07:59:02 +00:00
}
2020-02-03 12:54:36 +00:00
void DatabaseCatalog : : assertDatabaseExists ( const String & database_name ) const
{
std : : lock_guard lock { databases_mutex } ;
assertDatabaseExistsUnlocked ( database_name ) ;
}
void DatabaseCatalog : : assertDatabaseDoesntExist ( const String & database_name ) const
{
std : : lock_guard lock { databases_mutex } ;
assertDatabaseDoesntExistUnlocked ( database_name ) ;
}
void DatabaseCatalog : : assertDatabaseExistsUnlocked ( const String & database_name ) const
{
2020-02-10 18:19:35 +00:00
assert ( ! database_name . empty ( ) ) ;
2020-02-03 12:54:36 +00:00
if ( databases . end ( ) = = databases . find ( database_name ) )
2023-01-17 16:39:07 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_DATABASE , " Database {} doesn't exist " , backQuoteIfNeed ( database_name ) ) ;
2020-02-03 12:54:36 +00:00
}
void DatabaseCatalog : : assertDatabaseDoesntExistUnlocked ( const String & database_name ) const
{
2020-02-10 18:19:35 +00:00
assert ( ! database_name . empty ( ) ) ;
2020-02-03 12:54:36 +00:00
if ( databases . end ( ) ! = databases . find ( database_name ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : DATABASE_ALREADY_EXISTS , " Database {} already exists. " , backQuoteIfNeed ( database_name ) ) ;
2020-02-03 12:54:36 +00:00
}
2020-07-08 14:28:07 +00:00
void DatabaseCatalog : : attachDatabase ( const String & database_name , const DatabasePtr & database )
2020-02-03 12:54:36 +00:00
{
std : : lock_guard lock { databases_mutex } ;
assertDatabaseDoesntExistUnlocked ( database_name ) ;
2020-07-06 08:30:11 +00:00
databases . emplace ( database_name , database ) ;
2022-07-14 13:46:01 +00:00
NOEXCEPT_SCOPE ( {
UUID db_uuid = database - > getUUID ( ) ;
if ( db_uuid ! = UUIDHelpers : : Nil )
addUUIDMapping ( db_uuid , database , nullptr ) ;
} ) ;
2020-02-03 12:54:36 +00:00
}
2021-10-13 10:34:18 +00:00
DatabasePtr DatabaseCatalog : : detachDatabase ( ContextPtr local_context , const String & database_name , bool drop , bool check_empty )
2020-02-03 12:54:36 +00:00
{
2020-03-13 15:41:36 +00:00
if ( database_name = = TEMPORARY_DATABASE )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : DATABASE_ACCESS_DENIED , " Cannot detach database with temporary tables. " ) ;
2020-03-13 15:41:36 +00:00
2020-07-08 14:28:07 +00:00
DatabasePtr db ;
Fix deadlock on failed database attach at start with materialized view
This is not the problem for plain DROP DATABASE query since it first
remove tables.
(gdb) bt
0 __lll_lock_wait () at ../sysdeps/unix/sysv/linux/x86_64/lowlevellock.S:103
1 0x00007f353c262774 in __GI___pthread_mutex_lock (mutex=mutex@entry=0x11049288 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog+8>) at ../nptl/pthread_mutex_lock.c:80
2 0x0000000004fd5d1e in DB::pthread_mutex_lock (arg=arg@entry=0x11049288 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog+8>) at ../src/Common/ThreadFuzzer.cpp:253
3 0x000000000c70b3a9 in std::__1::__libcpp_mutex_lock (__m=__m@entry=0x11049288 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog+8>) at ../contrib/libcxx/include/__threading_support:322
4 std::__1::mutex::lock (this=this@entry=0x11049288 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog+8>) at ../contrib/libcxx/src/mutex.cpp:33
5 0x0000000008acd4e2 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<synthetic pointer>) at ../contrib/libcxx/include/__mutex_base:90
6 DB::DatabaseCatalog::removeDependency (this=this@entry=0x11049280 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog>, from=..., where=...) at ../src/Interpreters/DatabaseCatalog.cpp:388
7 0x000000000975044b in DB::StorageMaterializedView::shutdown (this=0x7f338d178a00) at ../src/Storages/StorageMaterializedView.cpp:362
8 0x0000000008a79602 in DB::DatabaseWithOwnTablesBase::shutdown (this=0x7f353be3cc60) at ../contrib/libcxx/include/__tree:184
9 0x0000000009546196 in DB::DatabaseWithDictionaries::shutdown (this=0x7f353be3cc60) at ../src/Databases/DatabaseWithDictionaries.cpp:265
10 0x0000000008acecdb in DB::DatabaseCatalog::detachDatabase (this=0x11049280 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog>, database_name=..., drop=drop@entry=false, check_empty=check_empty@entry=false) at ../contrib/libcxx/include/memory:3826
11 0x0000000004bf0aa5 in DB::InterpreterCreateQuery::createDatabase (this=0x7ffd6e1bab80, create=...) at ../src/Interpreters/InterpreterCreateQuery.cpp:162
12 0x0000000008b04327 in DB::InterpreterCreateQuery::execute (this=this@entry=0x7ffd6e1bab80) at ../src/Interpreters/InterpreterCreateQuery.cpp:722
13 0x0000000008d0fdaa in DB::executeCreateQuery (has_force_restore_data_flag=false, file_name=..., database=..., context=..., query=...) at ../src/Interpreters/loadMetadata.cpp:48
14 DB::loadDatabase (context=..., database=..., database_path=..., force_restore_data=<optimized out>) at ../src/Interpreters/loadMetadata.cpp:72
15 0x0000000008d103c3 in DB::loadMetadata (context=...) at ../src/Interpreters/loadMetadata.cpp:111
16 0x0000000004f4d25e in DB::Server::main (this=<optimized out>) at ../contrib/libcxx/include/memory:2582
17 0x000000000bbc8963 in Poco::Util::Application::run (this=this@entry=0x7ffd6e1bcc60) at ../contrib/poco/Util/src/Application.cpp:334
18 0x0000000004ffe1de in DB::Server::run (this=0x7ffd6e1bcc60) at ../programs/server/Server.cpp:178
19 0x0000000004ff36cc in mainEntryClickHouseServer (argc=3, argv=0x7f353be58bc0) at ../programs/server/Server.cpp:1060
20 0x0000000004f49b99 in main (argc_=<optimized out>, argv_=<optimized out>) at ../contrib/libcxx/include/vector:655
2020-04-05 11:00:11 +00:00
{
std : : lock_guard lock { databases_mutex } ;
assertDatabaseExistsUnlocked ( database_name ) ;
db = databases . find ( database_name ) - > second ;
2022-06-22 16:31:42 +00:00
UUID db_uuid = db - > getUUID ( ) ;
if ( db_uuid ! = UUIDHelpers : : Nil )
removeUUIDMapping ( db_uuid ) ;
2020-07-08 14:28:07 +00:00
databases . erase ( database_name ) ;
}
2020-02-13 21:00:03 +00:00
2020-07-08 14:28:07 +00:00
if ( check_empty )
{
try
2020-04-10 23:02:15 +00:00
{
2020-04-23 16:51:48 +00:00
if ( ! db - > empty ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : DATABASE_NOT_EMPTY , " New table appeared in database being dropped or detached. Try again. " ) ;
2020-09-14 19:25:02 +00:00
if ( ! drop )
db - > assertCanBeDetached ( false ) ;
2020-04-10 23:02:15 +00:00
}
2020-07-08 14:28:07 +00:00
catch ( . . . )
{
attachDatabase ( database_name , db ) ;
throw ;
}
Fix deadlock on failed database attach at start with materialized view
This is not the problem for plain DROP DATABASE query since it first
remove tables.
(gdb) bt
0 __lll_lock_wait () at ../sysdeps/unix/sysv/linux/x86_64/lowlevellock.S:103
1 0x00007f353c262774 in __GI___pthread_mutex_lock (mutex=mutex@entry=0x11049288 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog+8>) at ../nptl/pthread_mutex_lock.c:80
2 0x0000000004fd5d1e in DB::pthread_mutex_lock (arg=arg@entry=0x11049288 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog+8>) at ../src/Common/ThreadFuzzer.cpp:253
3 0x000000000c70b3a9 in std::__1::__libcpp_mutex_lock (__m=__m@entry=0x11049288 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog+8>) at ../contrib/libcxx/include/__threading_support:322
4 std::__1::mutex::lock (this=this@entry=0x11049288 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog+8>) at ../contrib/libcxx/src/mutex.cpp:33
5 0x0000000008acd4e2 in std::__1::lock_guard<std::__1::mutex>::lock_guard (__m=..., this=<synthetic pointer>) at ../contrib/libcxx/include/__mutex_base:90
6 DB::DatabaseCatalog::removeDependency (this=this@entry=0x11049280 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog>, from=..., where=...) at ../src/Interpreters/DatabaseCatalog.cpp:388
7 0x000000000975044b in DB::StorageMaterializedView::shutdown (this=0x7f338d178a00) at ../src/Storages/StorageMaterializedView.cpp:362
8 0x0000000008a79602 in DB::DatabaseWithOwnTablesBase::shutdown (this=0x7f353be3cc60) at ../contrib/libcxx/include/__tree:184
9 0x0000000009546196 in DB::DatabaseWithDictionaries::shutdown (this=0x7f353be3cc60) at ../src/Databases/DatabaseWithDictionaries.cpp:265
10 0x0000000008acecdb in DB::DatabaseCatalog::detachDatabase (this=0x11049280 <DB::DatabaseCatalog::init(DB::Context const*)::database_catalog>, database_name=..., drop=drop@entry=false, check_empty=check_empty@entry=false) at ../contrib/libcxx/include/memory:3826
11 0x0000000004bf0aa5 in DB::InterpreterCreateQuery::createDatabase (this=0x7ffd6e1bab80, create=...) at ../src/Interpreters/InterpreterCreateQuery.cpp:162
12 0x0000000008b04327 in DB::InterpreterCreateQuery::execute (this=this@entry=0x7ffd6e1bab80) at ../src/Interpreters/InterpreterCreateQuery.cpp:722
13 0x0000000008d0fdaa in DB::executeCreateQuery (has_force_restore_data_flag=false, file_name=..., database=..., context=..., query=...) at ../src/Interpreters/loadMetadata.cpp:48
14 DB::loadDatabase (context=..., database=..., database_path=..., force_restore_data=<optimized out>) at ../src/Interpreters/loadMetadata.cpp:72
15 0x0000000008d103c3 in DB::loadMetadata (context=...) at ../src/Interpreters/loadMetadata.cpp:111
16 0x0000000004f4d25e in DB::Server::main (this=<optimized out>) at ../contrib/libcxx/include/memory:2582
17 0x000000000bbc8963 in Poco::Util::Application::run (this=this@entry=0x7ffd6e1bcc60) at ../contrib/poco/Util/src/Application.cpp:334
18 0x0000000004ffe1de in DB::Server::run (this=0x7ffd6e1bcc60) at ../programs/server/Server.cpp:178
19 0x0000000004ff36cc in mainEntryClickHouseServer (argc=3, argv=0x7f353be58bc0) at ../programs/server/Server.cpp:1060
20 0x0000000004f49b99 in main (argc_=<optimized out>, argv_=<optimized out>) at ../contrib/libcxx/include/vector:655
2020-04-05 11:00:11 +00:00
}
2020-02-13 21:00:03 +00:00
db - > shutdown ( ) ;
if ( drop )
{
2022-06-21 10:35:47 +00:00
UUID db_uuid = db - > getUUID ( ) ;
2020-02-13 21:00:03 +00:00
/// Delete the database.
2021-10-13 10:34:18 +00:00
db - > drop ( local_context ) ;
2020-02-13 21:00:03 +00:00
/// Old ClickHouse versions did not store database.sql files
2021-10-13 10:34:18 +00:00
/// Remove metadata dir (if exists) to avoid recreation of .sql file on server startup
fs : : path database_metadata_dir = fs : : path ( getContext ( ) - > getPath ( ) ) / " metadata " / escapeForFileName ( database_name ) ;
fs : : remove ( database_metadata_dir ) ;
2021-04-28 10:42:07 +00:00
fs : : path database_metadata_file = fs : : path ( getContext ( ) - > getPath ( ) ) / " metadata " / ( escapeForFileName ( database_name ) + " .sql " ) ;
2021-10-13 10:34:18 +00:00
fs : : remove ( database_metadata_file ) ;
2022-06-21 22:50:16 +00:00
if ( db_uuid ! = UUIDHelpers : : Nil )
removeUUIDMappingFinally ( db_uuid ) ;
2020-02-13 21:00:03 +00:00
}
return db ;
2020-02-03 12:54:36 +00:00
}
2021-11-02 12:58:45 +00:00
void DatabaseCatalog : : updateDatabaseName ( const String & old_name , const String & new_name , const Strings & tables_in_database )
2020-07-07 12:11:58 +00:00
{
std : : lock_guard lock { databases_mutex } ;
2020-07-08 14:28:07 +00:00
assert ( databases . find ( new_name ) = = databases . end ( ) ) ;
2020-07-07 12:11:58 +00:00
auto it = databases . find ( old_name ) ;
2020-07-08 14:28:07 +00:00
assert ( it ! = databases . end ( ) ) ;
2020-07-07 12:11:58 +00:00
auto db = it - > second ;
databases . erase ( it ) ;
databases . emplace ( new_name , db ) ;
2021-11-02 12:58:45 +00:00
2023-02-01 23:30:49 +00:00
/// Update dependencies.
2021-11-02 12:58:45 +00:00
for ( const auto & table_name : tables_in_database )
{
2023-02-01 23:30:49 +00:00
auto removed_ref_deps = referential_dependencies . removeDependencies ( StorageID { old_name , table_name } , /* remove_isolated_tables= */ true ) ;
auto removed_loading_deps = loading_dependencies . removeDependencies ( StorageID { old_name , table_name } , /* remove_isolated_tables= */ true ) ;
referential_dependencies . addDependencies ( StorageID { new_name , table_name } , removed_ref_deps ) ;
loading_dependencies . addDependencies ( StorageID { new_name , table_name } , removed_loading_deps ) ;
2021-11-02 12:58:45 +00:00
}
2020-07-07 12:11:58 +00:00
}
2020-02-10 13:10:17 +00:00
DatabasePtr DatabaseCatalog : : getDatabase ( const String & database_name ) const
2020-02-03 12:54:36 +00:00
{
std : : lock_guard lock { databases_mutex } ;
assertDatabaseExistsUnlocked ( database_name ) ;
return databases . find ( database_name ) - > second ;
}
2020-02-10 13:10:17 +00:00
DatabasePtr DatabaseCatalog : : tryGetDatabase ( const String & database_name ) const
2020-02-03 12:54:36 +00:00
{
2020-02-10 13:10:17 +00:00
assert ( ! database_name . empty ( ) ) ;
2020-02-03 12:54:36 +00:00
std : : lock_guard lock { databases_mutex } ;
auto it = databases . find ( database_name ) ;
if ( it = = databases . end ( ) )
return { } ;
return it - > second ;
}
2020-07-06 08:30:11 +00:00
DatabasePtr DatabaseCatalog : : getDatabase ( const UUID & uuid ) const
{
2022-06-22 16:31:42 +00:00
auto db_and_table = tryGetByUUID ( uuid ) ;
if ( ! db_and_table . first | | db_and_table . second )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_DATABASE , " Database UUID {} does not exist " , uuid ) ;
2022-06-22 16:31:42 +00:00
return db_and_table . first ;
2020-07-06 08:30:11 +00:00
}
DatabasePtr DatabaseCatalog : : tryGetDatabase ( const UUID & uuid ) const
{
assert ( uuid ! = UUIDHelpers : : Nil ) ;
2022-06-22 16:31:42 +00:00
auto db_and_table = tryGetByUUID ( uuid ) ;
if ( ! db_and_table . first | | db_and_table . second )
2020-07-06 08:30:11 +00:00
return { } ;
2022-06-22 16:31:42 +00:00
return db_and_table . first ;
2020-07-06 08:30:11 +00:00
}
2020-02-03 12:54:36 +00:00
bool DatabaseCatalog : : isDatabaseExist ( const String & database_name ) const
{
2020-02-10 13:10:17 +00:00
assert ( ! database_name . empty ( ) ) ;
2020-02-03 12:54:36 +00:00
std : : lock_guard lock { databases_mutex } ;
return databases . end ( ) ! = databases . find ( database_name ) ;
}
Databases DatabaseCatalog : : getDatabases ( ) const
{
std : : lock_guard lock { databases_mutex } ;
return databases ;
}
2021-04-10 23:33:54 +00:00
bool DatabaseCatalog : : isTableExist ( const DB : : StorageID & table_id , ContextPtr context_ ) const
2020-02-03 12:54:36 +00:00
{
2020-03-12 18:04:29 +00:00
if ( table_id . hasUUID ( ) )
return tryGetByUUID ( table_id . uuid ) . second ! = nullptr ;
DatabasePtr db ;
{
std : : lock_guard lock { databases_mutex } ;
auto iter = databases . find ( table_id . database_name ) ;
if ( iter ! = databases . end ( ) )
db = iter - > second ;
}
2021-04-10 23:33:54 +00:00
return db & & db - > isTableExist ( table_id . table_name , context_ ) ;
2020-02-03 12:54:36 +00:00
}
2021-04-10 23:33:54 +00:00
void DatabaseCatalog : : assertTableDoesntExist ( const StorageID & table_id , ContextPtr context_ ) const
2020-02-03 12:54:36 +00:00
{
2021-04-10 23:33:54 +00:00
if ( isTableExist ( table_id , context_ ) )
2023-01-17 16:39:07 +00:00
throw Exception ( ErrorCodes : : TABLE_ALREADY_EXISTS , " Table {} already exists. " , table_id ) ;
2020-02-03 12:54:36 +00:00
}
DatabasePtr DatabaseCatalog : : getDatabaseForTemporaryTables ( ) const
{
2020-02-10 13:10:17 +00:00
return getDatabase ( TEMPORARY_DATABASE ) ;
}
DatabasePtr DatabaseCatalog : : getSystemDatabase ( ) const
{
return getDatabase ( SYSTEM_DATABASE ) ;
2020-02-03 12:54:36 +00:00
}
2020-10-27 20:52:49 +00:00
void DatabaseCatalog : : addUUIDMapping ( const UUID & uuid )
{
addUUIDMapping ( uuid , nullptr , nullptr ) ;
}
2020-10-25 17:27:21 +00:00
void DatabaseCatalog : : addUUIDMapping ( const UUID & uuid , const DatabasePtr & database , const StoragePtr & table )
2020-02-03 12:54:36 +00:00
{
2020-03-16 09:16:14 +00:00
assert ( uuid ! = UUIDHelpers : : Nil & & getFirstLevelIdx ( uuid ) < uuid_map . size ( ) ) ;
2022-06-22 16:31:42 +00:00
assert ( database | | ! table ) ;
2020-02-03 12:54:36 +00:00
UUIDToStorageMapPart & map_part = uuid_map [ getFirstLevelIdx ( uuid ) ] ;
std : : lock_guard lock { map_part . mutex } ;
2020-10-25 17:27:21 +00:00
auto [ it , inserted ] = map_part . map . try_emplace ( uuid , database , table ) ;
if ( inserted )
2022-06-22 16:31:42 +00:00
{
/// Mapping must be locked before actually inserting something
chassert ( ( ! database & & ! table ) ) ;
2020-10-25 17:27:21 +00:00
return ;
2022-06-22 16:31:42 +00:00
}
2020-10-25 17:27:21 +00:00
auto & prev_database = it - > second . first ;
auto & prev_table = it - > second . second ;
2022-06-22 16:31:42 +00:00
assert ( prev_database | | ! prev_table ) ;
2020-10-25 17:27:21 +00:00
2022-06-22 16:31:42 +00:00
if ( ! prev_database & & database )
2020-10-25 17:27:21 +00:00
{
/// It's empty mapping, it was created to "lock" UUID and prevent collision. Just update it.
prev_database = database ;
prev_table = table ;
return ;
}
2022-06-22 16:31:42 +00:00
/// We are trying to replace existing mapping (prev_database != nullptr), it's logical error
if ( database | | table )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Mapping for table with UUID={} already exists " , uuid ) ;
2020-10-15 16:01:23 +00:00
/// Normally this should never happen, but it's possible when the same UUIDs are explicitly specified in different CREATE queries,
/// so it's not LOGICAL_ERROR
2020-10-25 17:27:21 +00:00
throw Exception ( ErrorCodes : : TABLE_ALREADY_EXISTS , " Mapping for table with UUID={} already exists. It happened due to UUID collision, "
2023-01-23 21:13:58 +00:00
" most likely because some not random UUIDs were manually specified in CREATE queries. " , uuid ) ;
2020-02-03 12:54:36 +00:00
}
void DatabaseCatalog : : removeUUIDMapping ( const UUID & uuid )
2020-10-25 17:27:21 +00:00
{
assert ( uuid ! = UUIDHelpers : : Nil & & getFirstLevelIdx ( uuid ) < uuid_map . size ( ) ) ;
UUIDToStorageMapPart & map_part = uuid_map [ getFirstLevelIdx ( uuid ) ] ;
std : : lock_guard lock { map_part . mutex } ;
auto it = map_part . map . find ( uuid ) ;
if ( it = = map_part . map . end ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Mapping for table with UUID={} doesn't exist " , uuid ) ;
2020-10-25 17:27:21 +00:00
it - > second = { } ;
}
void DatabaseCatalog : : removeUUIDMappingFinally ( const UUID & uuid )
2020-02-03 12:54:36 +00:00
{
2020-03-16 09:16:14 +00:00
assert ( uuid ! = UUIDHelpers : : Nil & & getFirstLevelIdx ( uuid ) < uuid_map . size ( ) ) ;
2020-02-03 12:54:36 +00:00
UUIDToStorageMapPart & map_part = uuid_map [ getFirstLevelIdx ( uuid ) ] ;
std : : lock_guard lock { map_part . mutex } ;
if ( ! map_part . map . erase ( uuid ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Mapping for table with UUID={} doesn't exist " , uuid ) ;
2020-02-03 12:54:36 +00:00
}
2020-03-23 00:12:13 +00:00
void DatabaseCatalog : : updateUUIDMapping ( const UUID & uuid , DatabasePtr database , StoragePtr table )
{
assert ( uuid ! = UUIDHelpers : : Nil & & getFirstLevelIdx ( uuid ) < uuid_map . size ( ) ) ;
2020-10-25 17:27:21 +00:00
assert ( database & & table ) ;
2020-03-23 00:12:13 +00:00
UUIDToStorageMapPart & map_part = uuid_map [ getFirstLevelIdx ( uuid ) ] ;
std : : lock_guard lock { map_part . mutex } ;
auto it = map_part . map . find ( uuid ) ;
if ( it = = map_part . map . end ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Mapping for table with UUID={} doesn't exist " , uuid ) ;
2020-10-25 17:27:21 +00:00
auto & prev_database = it - > second . first ;
auto & prev_table = it - > second . second ;
assert ( prev_database & & prev_table ) ;
prev_database = std : : move ( database ) ;
prev_table = std : : move ( table ) ;
2020-03-23 00:12:13 +00:00
}
2022-06-21 10:35:47 +00:00
bool DatabaseCatalog : : hasUUIDMapping ( const UUID & uuid )
{
assert ( uuid ! = UUIDHelpers : : Nil & & getFirstLevelIdx ( uuid ) < uuid_map . size ( ) ) ;
UUIDToStorageMapPart & map_part = uuid_map [ getFirstLevelIdx ( uuid ) ] ;
std : : lock_guard lock { map_part . mutex } ;
2022-06-21 22:50:16 +00:00
return map_part . map . contains ( uuid ) ;
2022-06-21 10:35:47 +00:00
}
2020-06-03 13:34:36 +00:00
std : : unique_ptr < DatabaseCatalog > DatabaseCatalog : : database_catalog ;
2021-05-31 14:49:02 +00:00
DatabaseCatalog : : DatabaseCatalog ( ContextMutablePtr global_context_ )
2022-12-02 14:05:46 +00:00
: WithMutableContext ( global_context_ )
, referential_dependencies { " ReferentialDeps " }
2023-02-01 23:30:49 +00:00
, loading_dependencies { " LoadingDeps " }
2022-12-02 14:05:46 +00:00
, view_dependencies { " ViewDeps " }
, log ( & Poco : : Logger : : get ( " DatabaseCatalog " ) )
2020-02-10 13:10:17 +00:00
{
2020-02-13 21:00:03 +00:00
}
2021-05-31 14:49:02 +00:00
DatabaseCatalog & DatabaseCatalog : : init ( ContextMutablePtr global_context_ )
2020-02-13 21:00:03 +00:00
{
2020-06-03 12:59:11 +00:00
if ( database_catalog )
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Database catalog is initialized twice. This is a bug. " ) ;
2020-06-03 12:59:11 +00:00
}
database_catalog . reset ( new DatabaseCatalog ( global_context_ ) ) ;
return * database_catalog ;
2020-02-10 13:10:17 +00:00
}
2020-02-13 21:00:03 +00:00
DatabaseCatalog & DatabaseCatalog : : instance ( )
{
2020-06-03 12:59:11 +00:00
if ( ! database_catalog )
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Database catalog is not initialized. This is a bug. " ) ;
2020-06-03 12:59:11 +00:00
}
return * database_catalog ;
2020-02-13 21:00:03 +00:00
}
2020-04-08 23:59:39 +00:00
void DatabaseCatalog : : shutdown ( )
{
2020-06-03 12:59:11 +00:00
// The catalog might not be initialized yet by init(global_context). It can
// happen if some exception was thrown on first steps of startup.
if ( database_catalog )
2020-04-08 23:59:39 +00:00
{
2020-06-03 12:59:11 +00:00
database_catalog - > shutdownImpl ( ) ;
2020-04-08 23:59:39 +00:00
}
}
2021-04-10 23:33:54 +00:00
DatabasePtr DatabaseCatalog : : getDatabase ( const String & database_name , ContextPtr local_context ) const
2020-02-10 13:10:17 +00:00
{
2021-04-10 23:33:54 +00:00
String resolved_database = local_context - > resolveDatabase ( database_name ) ;
2020-02-10 13:10:17 +00:00
return getDatabase ( resolved_database ) ;
}
2022-12-02 14:05:46 +00:00
void DatabaseCatalog : : addViewDependency ( const StorageID & source_table_id , const StorageID & view_id )
2020-02-12 16:54:26 +00:00
{
std : : lock_guard lock { databases_mutex } ;
2022-12-02 14:05:46 +00:00
view_dependencies . addDependency ( source_table_id , view_id ) ;
2020-02-12 16:54:26 +00:00
}
2022-12-02 14:05:46 +00:00
void DatabaseCatalog : : removeViewDependency ( const StorageID & source_table_id , const StorageID & view_id )
2020-02-12 16:54:26 +00:00
{
std : : lock_guard lock { databases_mutex } ;
2022-12-02 14:05:46 +00:00
view_dependencies . removeDependency ( source_table_id , view_id , /* remove_isolated_tables= */ true ) ;
2020-02-12 16:54:26 +00:00
}
2022-12-02 14:05:46 +00:00
std : : vector < StorageID > DatabaseCatalog : : getDependentViews ( const StorageID & source_table_id ) const
2020-02-12 16:54:26 +00:00
{
std : : lock_guard lock { databases_mutex } ;
2022-12-02 14:05:46 +00:00
return view_dependencies . getDependencies ( source_table_id ) ;
2020-02-12 16:54:26 +00:00
}
2021-08-25 10:11:40 +00:00
2022-12-02 14:05:46 +00:00
void DatabaseCatalog : : updateViewDependency ( const StorageID & old_source_table_id , const StorageID & old_view_id ,
const StorageID & new_source_table_id , const StorageID & new_view_id )
2020-02-12 16:54:26 +00:00
{
std : : lock_guard lock { databases_mutex } ;
2022-12-02 14:05:46 +00:00
if ( ! old_source_table_id . empty ( ) )
view_dependencies . removeDependency ( old_source_table_id , old_view_id , /* remove_isolated_tables= */ true ) ;
if ( ! new_source_table_id . empty ( ) )
view_dependencies . addDependency ( new_source_table_id , new_view_id ) ;
2020-02-12 16:54:26 +00:00
}
2021-02-04 19:41:44 +00:00
DDLGuardPtr DatabaseCatalog : : getDDLGuard ( const String & database , const String & table )
2020-02-10 18:31:52 +00:00
{
std : : unique_lock lock ( ddl_guards_mutex ) ;
2022-06-27 20:48:27 +00:00
/// TSA does not support unique_lock
2022-06-28 14:25:29 +00:00
auto db_guard_iter = TSA_SUPPRESS_WARNING_FOR_WRITE ( ddl_guards ) . try_emplace ( database ) . first ;
2020-08-18 15:15:27 +00:00
DatabaseGuard & db_guard = db_guard_iter - > second ;
2020-10-13 15:00:36 +00:00
return std : : make_unique < DDLGuard > ( db_guard . first , db_guard . second , std : : move ( lock ) , table , database ) ;
2020-08-18 15:15:27 +00:00
}
2023-01-12 15:51:04 +00:00
std : : unique_lock < SharedMutex > DatabaseCatalog : : getExclusiveDDLGuardForDatabase ( const String & database )
2020-08-18 15:15:27 +00:00
{
DDLGuards : : iterator db_guard_iter ;
{
2022-06-27 20:48:27 +00:00
std : : lock_guard lock ( ddl_guards_mutex ) ;
2020-08-18 15:15:27 +00:00
db_guard_iter = ddl_guards . try_emplace ( database ) . first ;
2022-05-27 20:51:37 +00:00
assert ( db_guard_iter - > second . first . contains ( " " ) ) ;
2020-08-18 15:15:27 +00:00
}
DatabaseGuard & db_guard = db_guard_iter - > second ;
return std : : unique_lock { db_guard . second } ;
2020-02-10 18:31:52 +00:00
}
2020-03-13 15:41:36 +00:00
bool DatabaseCatalog : : isDictionaryExist ( const StorageID & table_id ) const
2020-02-17 13:52:59 +00:00
{
2021-04-26 10:20:57 +00:00
auto storage = tryGetTable ( table_id , getContext ( ) ) ;
2021-04-22 14:09:30 +00:00
bool storage_is_dictionary = storage & & storage - > isDictionary ( ) ;
return storage_is_dictionary ;
2020-02-17 13:52:59 +00:00
}
2021-04-10 23:33:54 +00:00
StoragePtr DatabaseCatalog : : getTable ( const StorageID & table_id , ContextPtr local_context ) const
2020-02-17 13:52:59 +00:00
{
2020-03-30 23:36:23 +00:00
std : : optional < Exception > exc ;
2021-04-10 23:33:54 +00:00
auto res = getTableImpl ( table_id , local_context , & exc ) ;
2020-03-30 23:36:23 +00:00
if ( ! res . second )
throw Exception ( * exc ) ;
return res . second ;
2020-02-17 13:52:59 +00:00
}
2021-04-10 23:33:54 +00:00
StoragePtr DatabaseCatalog : : tryGetTable ( const StorageID & table_id , ContextPtr local_context ) const
2020-02-17 13:52:59 +00:00
{
2021-04-10 23:33:54 +00:00
return getTableImpl ( table_id , local_context , nullptr ) . second ;
2020-03-04 20:29:52 +00:00
}
2021-04-10 23:33:54 +00:00
DatabaseAndTable DatabaseCatalog : : getDatabaseAndTable ( const StorageID & table_id , ContextPtr local_context ) const
2020-04-13 14:09:56 +00:00
{
std : : optional < Exception > exc ;
2021-04-10 23:33:54 +00:00
auto res = getTableImpl ( table_id , local_context , & exc ) ;
2020-04-13 14:09:56 +00:00
if ( ! res . second )
throw Exception ( * exc ) ;
return res ;
}
2021-04-10 23:33:54 +00:00
DatabaseAndTable DatabaseCatalog : : tryGetDatabaseAndTable ( const StorageID & table_id , ContextPtr local_context ) const
2020-03-04 20:29:52 +00:00
{
2021-04-10 23:33:54 +00:00
return getTableImpl ( table_id , local_context , nullptr ) ;
2020-02-17 13:52:59 +00:00
}
2020-03-19 21:14:52 +00:00
void DatabaseCatalog : : loadMarkedAsDroppedTables ( )
{
2022-06-22 16:31:42 +00:00
assert ( ! cleanup_task ) ;
2020-04-22 20:43:10 +00:00
/// /clickhouse_root/metadata_dropped/ contains files with metadata of tables,
/// which where marked as dropped by Atomic databases.
/// Data directories of such tables still exists in store/
/// and metadata still exists in ZooKeeper for ReplicatedMergeTree tables.
/// If server restarts before such tables was completely dropped,
/// we should load them and enqueue cleanup to remove data from store/ and metadata from ZooKeeper
2020-03-19 21:14:52 +00:00
std : : map < String , StorageID > dropped_metadata ;
2021-04-10 23:33:54 +00:00
String path = getContext ( ) - > getPath ( ) + " metadata_dropped/ " ;
2020-06-24 19:03:28 +00:00
if ( ! std : : filesystem : : exists ( path ) )
{
return ;
}
2020-03-19 21:14:52 +00:00
Poco : : DirectoryIterator dir_end ;
for ( Poco : : DirectoryIterator it ( path ) ; it ! = dir_end ; + + it )
{
2020-04-22 20:43:10 +00:00
/// File name has the following format:
/// database_name.table_name.uuid.sql
/// Ignore unexpected files
2020-03-19 21:14:52 +00:00
if ( ! it . name ( ) . ends_with ( " .sql " ) )
continue ;
/// Process .sql files with metadata of tables which were marked as dropped
2020-04-12 18:52:59 +00:00
StorageID dropped_id = StorageID : : createEmpty ( ) ;
size_t dot_pos = it . name ( ) . find ( ' . ' ) ;
if ( dot_pos = = std : : string : : npos )
continue ;
dropped_id . database_name = unescapeForFileName ( it . name ( ) . substr ( 0 , dot_pos ) ) ;
2020-03-19 21:14:52 +00:00
2020-04-12 18:52:59 +00:00
size_t prev_dot_pos = dot_pos ;
dot_pos = it . name ( ) . find ( ' . ' , prev_dot_pos + 1 ) ;
if ( dot_pos = = std : : string : : npos )
2020-03-19 21:14:52 +00:00
continue ;
2020-04-12 18:52:59 +00:00
dropped_id . table_name = unescapeForFileName ( it . name ( ) . substr ( prev_dot_pos + 1 , dot_pos - prev_dot_pos - 1 ) ) ;
2020-03-19 21:14:52 +00:00
2020-04-12 18:52:59 +00:00
prev_dot_pos = dot_pos ;
dot_pos = it . name ( ) . find ( ' . ' , prev_dot_pos + 1 ) ;
if ( dot_pos = = std : : string : : npos )
continue ;
dropped_id . uuid = parse < UUID > ( it . name ( ) . substr ( prev_dot_pos + 1 , dot_pos - prev_dot_pos - 1 ) ) ;
2020-03-19 21:14:52 +00:00
2020-04-12 18:52:59 +00:00
String full_path = path + it . name ( ) ;
2020-03-19 21:14:52 +00:00
dropped_metadata . emplace ( std : : move ( full_path ) , std : : move ( dropped_id ) ) ;
}
2020-10-25 17:27:21 +00:00
LOG_INFO ( log , " Found {} partially dropped tables. Will load them and retry removal. " , dropped_metadata . size ( ) ) ;
2023-03-22 07:49:22 +00:00
ThreadPool pool ( CurrentMetrics : : DatabaseCatalogThreads , CurrentMetrics : : DatabaseCatalogThreadsActive ) ;
2020-03-19 21:14:52 +00:00
for ( const auto & elem : dropped_metadata )
{
pool . scheduleOrThrowOnError ( [ & ] ( )
{
this - > enqueueDroppedTableCleanup ( elem . second , nullptr , elem . first ) ;
} ) ;
}
pool . wait ( ) ;
}
String DatabaseCatalog : : getPathForDroppedMetadata ( const StorageID & table_id ) const
{
2021-04-10 23:33:54 +00:00
return getContext ( ) - > getPath ( ) + " metadata_dropped/ " +
2020-03-19 21:14:52 +00:00
escapeForFileName ( table_id . getDatabaseName ( ) ) + " . " +
escapeForFileName ( table_id . getTableName ( ) ) + " . " +
toString ( table_id . uuid ) + " .sql " ;
}
2023-03-21 12:14:26 +00:00
String DatabaseCatalog : : getPathForMetadata ( const StorageID & table_id ) const
{
return getContext ( ) - > getPath ( ) + " metadata/ " +
escapeForFileName ( table_id . getDatabaseName ( ) ) + " / " +
escapeForFileName ( table_id . getTableName ( ) ) + " .sql " ;
}
2020-03-19 21:14:52 +00:00
void DatabaseCatalog : : enqueueDroppedTableCleanup ( StorageID table_id , StoragePtr table , String dropped_metadata_path , bool ignore_delay )
{
assert ( table_id . hasUUID ( ) ) ;
assert ( ! table | | table - > getStorageID ( ) . uuid = = table_id . uuid ) ;
assert ( dropped_metadata_path = = getPathForDroppedMetadata ( table_id ) ) ;
2020-04-22 20:43:10 +00:00
/// Table was removed from database. Enqueue removal of its data from disk.
2020-03-19 21:14:52 +00:00
time_t drop_time ;
if ( table )
2020-09-09 13:32:50 +00:00
{
2022-06-22 16:31:42 +00:00
chassert ( hasUUIDMapping ( table_id . uuid ) ) ;
2020-03-19 21:14:52 +00:00
drop_time = std : : chrono : : system_clock : : to_time_t ( std : : chrono : : system_clock : : now ( ) ) ;
2022-11-30 14:48:52 +00:00
/// Do not postpone removal of in-memory tables
ignore_delay = ignore_delay | | ! table - > storesDataOnDisk ( ) ;
2020-09-09 13:32:50 +00:00
table - > is_dropped = true ;
}
2020-03-19 21:14:52 +00:00
else
{
2020-04-22 20:43:10 +00:00
/// Try load table from metadata to drop it correctly (e.g. remove metadata from zk or remove data from all volumes)
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Trying load partially dropped table {} from {} " , table_id . getNameForLogs ( ) , dropped_metadata_path ) ;
2021-04-10 23:33:54 +00:00
ASTPtr ast = DatabaseOnDisk : : parseQueryFromMetadata (
log , getContext ( ) , dropped_metadata_path , /*throw_on_error*/ false , /*remove_empty*/ false ) ;
2020-04-23 22:08:26 +00:00
auto * create = typeid_cast < ASTCreateQuery * > ( ast . get ( ) ) ;
2020-03-19 21:14:52 +00:00
assert ( ! create | | create - > uuid = = table_id . uuid ) ;
if ( create )
{
2020-04-06 23:22:44 +00:00
String data_path = " store/ " + getPathForUUID ( table_id . uuid ) ;
2021-09-06 22:13:54 +00:00
create - > setDatabase ( table_id . database_name ) ;
create - > setTable ( table_id . table_name ) ;
2020-03-19 21:14:52 +00:00
try
{
2022-08-04 14:10:13 +00:00
table = createTableFromAST ( * create , table_id . getDatabaseName ( ) , data_path , getContext ( ) , /* force_restore */ true ) . second ;
2020-09-09 13:32:50 +00:00
table - > is_dropped = true ;
2020-03-19 21:14:52 +00:00
}
catch ( . . . )
{
tryLogCurrentException ( log , " Cannot load partially dropped table " + table_id . getNameForLogs ( ) +
" from: " + dropped_metadata_path +
" . Parsed query: " + serializeAST ( * create ) +
" . Will remove metadata and " + data_path +
" . Garbage may be left in ZooKeeper. " ) ;
}
}
else
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Cannot parse metadata of partially dropped table {} from {}. Will remove metadata file and data directory. Garbage may be left in /store directory and ZooKeeper. " , table_id . getNameForLogs ( ) , dropped_metadata_path ) ;
2020-03-19 21:14:52 +00:00
}
2020-10-27 20:52:49 +00:00
addUUIDMapping ( table_id . uuid ) ;
2021-05-28 18:17:16 +00:00
drop_time = FS : : getModificationTime ( dropped_metadata_path ) ;
2020-03-19 21:14:52 +00:00
}
2020-04-08 01:02:00 +00:00
std : : lock_guard lock ( tables_marked_dropped_mutex ) ;
2020-03-19 21:14:52 +00:00
if ( ignore_delay )
2020-11-01 17:37:54 +00:00
tables_marked_dropped . push_front ( { table_id , table , dropped_metadata_path , drop_time } ) ;
2020-03-19 21:14:52 +00:00
else
2020-11-01 17:37:54 +00:00
tables_marked_dropped . push_back ( { table_id , table , dropped_metadata_path , drop_time + drop_delay_sec } ) ;
2020-09-29 13:42:58 +00:00
tables_marked_dropped_ids . insert ( table_id . uuid ) ;
2020-10-25 17:27:21 +00:00
CurrentMetrics : : add ( CurrentMetrics : : TablesToDropQueueSize , 1 ) ;
2020-11-01 17:37:54 +00:00
/// 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 ) )
2020-04-22 20:43:10 +00:00
( * drop_task ) - > schedule ( ) ;
2020-03-19 21:14:52 +00:00
}
2023-03-21 12:14:26 +00:00
void DatabaseCatalog : : dequeueDroppedTableCleanup ( StorageID table_id )
{
String latest_metadata_dropped_path ;
TableMarkedAsDropped dropped_table ;
{
std : : lock_guard lock ( tables_marked_dropped_mutex ) ;
time_t latest_drop_time = std : : numeric_limits < time_t > : : min ( ) ;
2023-03-26 05:15:23 +00:00
auto it_dropped_table = tables_marked_dropped . end ( ) ;
2023-03-21 12:14:26 +00:00
for ( auto it = tables_marked_dropped . begin ( ) ; it ! = tables_marked_dropped . end ( ) ; + + it )
{
auto storage_ptr = it - > table ;
if ( it - > table_id . uuid = = table_id . uuid )
{
2023-03-26 05:15:23 +00:00
it_dropped_table = it ;
2023-03-21 12:14:26 +00:00
dropped_table = * it ;
break ;
}
/// If table uuid exists, only find tables with equal uuid.
if ( table_id . uuid ! = UUIDHelpers : : Nil )
continue ;
if ( it - > table_id . database_name = = table_id . database_name & &
it - > table_id . table_name = = table_id . table_name & &
it - > drop_time > = latest_drop_time )
{
latest_drop_time = it - > drop_time ;
2023-03-26 05:15:23 +00:00
it_dropped_table = it ;
2023-03-21 12:14:26 +00:00
dropped_table = * it ;
}
}
2023-03-26 05:15:23 +00:00
if ( it_dropped_table = = tables_marked_dropped . end ( ) )
2023-03-21 12:14:26 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_TABLE ,
" The drop task of table {} is in progress, has been dropped or the database engine doesn't support it " ,
table_id . getNameForLogs ( ) ) ;
2023-03-26 05:15:23 +00:00
latest_metadata_dropped_path = it_dropped_table - > metadata_path ;
String table_metadata_path = getPathForMetadata ( it_dropped_table - > table_id ) ;
2023-03-21 12:14:26 +00:00
2023-03-27 15:51:35 +00:00
/// a table is successfully marked undropped,
/// if and only if its metadata file was moved to a database.
/// This maybe throw exception.
renameNoReplace ( latest_metadata_dropped_path , table_metadata_path ) ;
2023-03-21 12:14:26 +00:00
2023-03-26 05:15:23 +00:00
tables_marked_dropped . erase ( it_dropped_table ) ;
[[maybe_unused]] auto removed = tables_marked_dropped_ids . erase ( dropped_table . table_id . uuid ) ;
assert ( removed ) ;
CurrentMetrics : : sub ( CurrentMetrics : : TablesToDropQueueSize , 1 ) ;
2023-03-21 12:14:26 +00:00
}
2023-03-28 21:47:45 +00:00
LOG_INFO ( log , " Attaching undropped table {} (metadata moved from {}) " ,
dropped_table . table_id . getNameForLogs ( ) , latest_metadata_dropped_path ) ;
/// It's unsafe to create another instance while the old one exists
/// We cannot wait on shared_ptr's refcount, so it's busy wait
while ( ! dropped_table . table . unique ( ) )
std : : this_thread : : sleep_for ( std : : chrono : : milliseconds ( 100 ) ) ;
dropped_table . table . reset ( ) ;
auto ast_attach = std : : make_shared < ASTCreateQuery > ( ) ;
ast_attach - > attach = true ;
ast_attach - > setDatabase ( dropped_table . table_id . database_name ) ;
ast_attach - > setTable ( dropped_table . table_id . table_name ) ;
auto query_context = Context : : createCopy ( getContext ( ) ) ;
/// Attach table needs to acquire ddl guard, that has already been acquired in undrop table,
/// and cannot be acquired in the attach table again.
InterpreterCreateQuery interpreter ( ast_attach , query_context ) ;
interpreter . setForceAttach ( true ) ;
interpreter . setForceRestoreData ( true ) ;
interpreter . setDontNeedDDLGuard ( ) ; /// It's already locked by caller
interpreter . execute ( ) ;
LOG_INFO ( log , " Table {} was successfully undropped. " , dropped_table . table_id . getNameForLogs ( ) ) ;
2023-03-21 12:14:26 +00:00
}
2020-03-19 21:14:52 +00:00
void DatabaseCatalog : : dropTableDataTask ( )
{
2020-04-22 20:43:10 +00:00
/// Background task that removes data of tables which were marked as dropped by Atomic databases.
/// 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 ;
2020-11-01 17:37:54 +00:00
/// Default reschedule time for the case when we are waiting for reference count to become 1.
size_t schedule_after_ms = reschedule_time_ms ;
2020-03-19 21:14:52 +00:00
TableMarkedAsDropped table ;
try
{
2020-04-08 01:02:00 +00:00
std : : lock_guard lock ( tables_marked_dropped_mutex ) ;
2023-03-21 12:14:26 +00:00
if ( tables_marked_dropped . empty ( ) )
return ;
2020-03-19 21:14:52 +00:00
time_t current_time = std : : chrono : : system_clock : : to_time_t ( std : : chrono : : system_clock : : now ( ) ) ;
2020-11-01 17:37:54 +00:00
time_t min_drop_time = std : : numeric_limits < time_t > : : max ( ) ;
size_t tables_in_use_count = 0 ;
2020-04-08 01:02:00 +00:00
auto it = std : : find_if ( tables_marked_dropped . begin ( ) , tables_marked_dropped . end ( ) , [ & ] ( const auto & elem )
2020-03-19 21:14:52 +00:00
{
2020-03-27 22:58:03 +00:00
bool not_in_use = ! elem . table | | elem . table . unique ( ) ;
2020-11-01 17:37:54 +00:00
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 ;
2023-03-27 15:51:35 +00:00
return not_in_use & & old_enough ;
2020-03-19 21:14:52 +00:00
} ) ;
2020-04-08 01:02:00 +00:00
if ( it ! = tables_marked_dropped . end ( ) )
2020-03-19 21:14:52 +00:00
{
table = std : : move ( * it ) ;
2020-11-01 17:37:54 +00:00
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 ( ) ) ;
2020-04-08 01:02:00 +00:00
tables_marked_dropped . erase ( it ) ;
2020-11-01 17:37:54 +00:00
/// Schedule the task as soon as possible, while there are suitable tables to drop.
schedule_after_ms = 0 ;
2020-03-19 21:14:52 +00:00
}
2020-11-01 17:37:54 +00:00
else if ( current_time < min_drop_time )
2020-10-25 17:27:21 +00:00
{
2020-11-01 17:37:54 +00:00
/// 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 ) ;
2020-03-19 21:14:52 +00:00
}
2020-04-22 20:43:10 +00:00
need_reschedule = ! tables_marked_dropped . empty ( ) ;
2020-03-19 21:14:52 +00:00
}
catch ( . . . )
{
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
}
if ( table . table_id )
{
try
{
dropTableFinally ( table ) ;
2020-09-29 13:42:58 +00:00
std : : lock_guard lock ( tables_marked_dropped_mutex ) ;
2020-09-29 14:15:05 +00:00
[[maybe_unused]] auto removed = tables_marked_dropped_ids . erase ( table . table_id . uuid ) ;
2020-09-29 13:42:58 +00:00
assert ( removed ) ;
2020-03-19 21:14:52 +00:00
}
catch ( . . . )
{
tryLogCurrentException ( log , " Cannot drop table " + table . table_id . getNameForLogs ( ) +
" . Will retry later. " ) ;
{
2020-11-01 17:37:54 +00:00
table . drop_time = std : : chrono : : system_clock : : to_time_t ( std : : chrono : : system_clock : : now ( ) ) + drop_error_cooldown_sec ;
2020-04-08 01:02:00 +00:00
std : : lock_guard lock ( tables_marked_dropped_mutex ) ;
tables_marked_dropped . emplace_back ( std : : move ( table ) ) ;
2020-04-22 20:43:10 +00:00
/// If list of dropped tables was empty, schedule a task to retry deletion.
if ( tables_marked_dropped . size ( ) = = 1 )
2020-11-01 17:37:54 +00:00
{
2020-04-22 20:43:10 +00:00
need_reschedule = true ;
2020-11-01 17:37:54 +00:00
schedule_after_ms = drop_error_cooldown_sec * 1000 ;
}
2020-03-19 21:14:52 +00:00
}
}
2020-09-29 13:42:58 +00:00
wait_table_finally_dropped . notify_all ( ) ;
2020-03-19 21:14:52 +00:00
}
2020-04-22 20:43:10 +00:00
/// Do not schedule a task if there is no tables to drop
if ( need_reschedule )
2020-11-01 17:37:54 +00:00
( * drop_task ) - > scheduleAfter ( schedule_after_ms ) ;
2020-03-19 21:14:52 +00:00
}
2020-10-25 17:27:21 +00:00
void DatabaseCatalog : : dropTableFinally ( const TableMarkedAsDropped & table )
2020-03-19 21:14:52 +00:00
{
if ( table . table )
{
table . table - > drop ( ) ;
}
2022-08-03 17:49:16 +00:00
/// Even if table is not loaded, try remove its data from disks.
for ( const auto & [ disk_name , disk ] : getContext ( ) - > getDisksMap ( ) )
2020-03-19 21:14:52 +00:00
{
2022-08-03 17:49:16 +00:00
String data_path = " store/ " + getPathForUUID ( table . table_id . uuid ) ;
2022-11-22 13:40:32 +00:00
if ( disk - > isReadOnly ( ) | | ! disk - > exists ( data_path ) )
2022-08-03 17:49:16 +00:00
continue ;
LOG_INFO ( log , " Removing data directory {} of dropped table {} from disk {} " , data_path , table . table_id . getNameForLogs ( ) , disk_name ) ;
disk - > removeRecursive ( data_path ) ;
2020-03-19 21:14:52 +00:00
}
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Removing metadata {} of dropped table {} " , table . metadata_path , table . table_id . getNameForLogs ( ) ) ;
2021-04-28 10:42:07 +00:00
fs : : remove ( fs : : path ( table . metadata_path ) ) ;
2020-10-25 17:27:21 +00:00
removeUUIDMappingFinally ( table . table_id . uuid ) ;
CurrentMetrics : : sub ( CurrentMetrics : : TablesToDropQueueSize , 1 ) ;
2020-03-19 21:14:52 +00:00
}
2020-04-06 23:22:44 +00:00
String DatabaseCatalog : : getPathForUUID ( const UUID & uuid )
{
const size_t uuid_prefix_len = 3 ;
return toString ( uuid ) . substr ( 0 , uuid_prefix_len ) + ' / ' + toString ( uuid ) + ' / ' ;
}
2020-09-29 13:42:58 +00:00
void DatabaseCatalog : : waitTableFinallyDropped ( const UUID & uuid )
{
if ( uuid = = UUIDHelpers : : Nil )
return ;
2020-10-25 17:27:21 +00:00
LOG_DEBUG ( log , " Waiting for table {} to be finally dropped " , toString ( uuid ) ) ;
2020-09-29 13:42:58 +00:00
std : : unique_lock lock { tables_marked_dropped_mutex } ;
2022-06-27 20:48:27 +00:00
wait_table_finally_dropped . wait ( lock , [ & ] ( ) TSA_REQUIRES ( tables_marked_dropped_mutex ) - > bool
2020-09-29 15:19:30 +00:00
{
2022-04-18 10:18:43 +00:00
return ! tables_marked_dropped_ids . contains ( uuid ) ;
2020-09-29 13:42:58 +00:00
} ) ;
}
2023-02-01 23:30:49 +00:00
void DatabaseCatalog : : addDependencies (
const StorageID & table_id ,
const std : : vector < StorageID > & new_referential_dependencies ,
const std : : vector < StorageID > & new_loading_dependencies )
2021-11-02 12:58:45 +00:00
{
2023-02-01 23:30:49 +00:00
if ( new_referential_dependencies . empty ( ) & & new_loading_dependencies . empty ( ) )
return ;
2022-12-02 14:05:46 +00:00
std : : lock_guard lock { databases_mutex } ;
2023-02-01 23:30:49 +00:00
if ( ! new_referential_dependencies . empty ( ) )
referential_dependencies . addDependencies ( table_id , new_referential_dependencies ) ;
if ( ! new_loading_dependencies . empty ( ) )
loading_dependencies . addDependencies ( table_id , new_loading_dependencies ) ;
2021-11-02 12:58:45 +00:00
}
2023-02-01 23:30:49 +00:00
void DatabaseCatalog : : addDependencies (
const QualifiedTableName & table_name ,
const TableNamesSet & new_referential_dependencies ,
const TableNamesSet & new_loading_dependencies )
2021-11-01 18:53:07 +00:00
{
2023-02-01 23:30:49 +00:00
if ( new_referential_dependencies . empty ( ) & & new_loading_dependencies . empty ( ) )
return ;
2021-11-01 18:53:07 +00:00
std : : lock_guard lock { databases_mutex } ;
2023-02-01 23:30:49 +00:00
if ( ! new_referential_dependencies . empty ( ) )
referential_dependencies . addDependencies ( table_name , new_referential_dependencies ) ;
if ( ! new_loading_dependencies . empty ( ) )
loading_dependencies . addDependencies ( table_name , new_loading_dependencies ) ;
2021-11-01 18:53:07 +00:00
}
2023-02-01 23:30:49 +00:00
void DatabaseCatalog : : addDependencies (
const TablesDependencyGraph & new_referential_dependencies , const TablesDependencyGraph & new_loading_dependencies )
2021-11-01 18:53:07 +00:00
{
std : : lock_guard lock { databases_mutex } ;
2023-02-01 23:30:49 +00:00
referential_dependencies . mergeWith ( new_referential_dependencies ) ;
loading_dependencies . mergeWith ( new_loading_dependencies ) ;
2021-11-01 18:53:07 +00:00
}
2023-02-01 23:30:49 +00:00
std : : vector < StorageID > DatabaseCatalog : : getReferentialDependencies ( const StorageID & table_id ) const
2021-11-01 18:53:07 +00:00
{
std : : lock_guard lock { databases_mutex } ;
2022-12-02 14:05:46 +00:00
return referential_dependencies . getDependencies ( table_id ) ;
2021-11-02 12:58:45 +00:00
}
2023-02-01 23:30:49 +00:00
std : : vector < StorageID > DatabaseCatalog : : getReferentialDependents ( const StorageID & table_id ) const
2021-11-02 12:58:45 +00:00
{
2022-12-02 14:05:46 +00:00
std : : lock_guard lock { databases_mutex } ;
return referential_dependencies . getDependents ( table_id ) ;
}
2021-11-01 18:53:07 +00:00
2023-02-01 23:30:49 +00:00
std : : vector < StorageID > DatabaseCatalog : : getLoadingDependencies ( const StorageID & table_id ) const
{
std : : lock_guard lock { databases_mutex } ;
return loading_dependencies . getDependencies ( table_id ) ;
}
std : : vector < StorageID > DatabaseCatalog : : getLoadingDependents ( const StorageID & table_id ) const
{
std : : lock_guard lock { databases_mutex } ;
return loading_dependencies . getDependents ( table_id ) ;
}
std : : pair < std : : vector < StorageID > , std : : vector < StorageID > > DatabaseCatalog : : removeDependencies (
const StorageID & table_id , bool check_referential_dependencies , bool check_loading_dependencies , bool is_drop_database )
{
std : : lock_guard lock { databases_mutex } ;
checkTableCanBeRemovedOrRenamedUnlocked ( table_id , check_referential_dependencies , check_loading_dependencies , is_drop_database ) ;
return { referential_dependencies . removeDependencies ( table_id , /* remove_isolated_tables= */ true ) ,
loading_dependencies . removeDependencies ( table_id , /* remove_isolated_tables= */ true ) } ;
}
void DatabaseCatalog : : updateDependencies (
const StorageID & table_id , const TableNamesSet & new_referential_dependencies , const TableNamesSet & new_loading_dependencies )
2022-12-02 14:05:46 +00:00
{
std : : lock_guard lock { databases_mutex } ;
2023-02-01 23:30:49 +00:00
referential_dependencies . removeDependencies ( table_id , /* remove_isolated_tables= */ true ) ;
loading_dependencies . removeDependencies ( table_id , /* remove_isolated_tables= */ true ) ;
if ( ! new_referential_dependencies . empty ( ) )
referential_dependencies . addDependencies ( table_id , new_referential_dependencies ) ;
if ( ! new_loading_dependencies . empty ( ) )
loading_dependencies . addDependencies ( table_id , new_loading_dependencies ) ;
2021-11-02 12:58:45 +00:00
}
2023-02-01 23:30:49 +00:00
void DatabaseCatalog : : checkTableCanBeRemovedOrRenamed (
const StorageID & table_id , bool check_referential_dependencies , bool check_loading_dependencies , bool is_drop_database ) const
2021-11-02 12:58:45 +00:00
{
2023-02-01 23:30:49 +00:00
if ( ! check_referential_dependencies & & ! check_loading_dependencies )
return ;
2021-11-02 12:58:45 +00:00
std : : lock_guard lock { databases_mutex } ;
2023-02-01 23:30:49 +00:00
return checkTableCanBeRemovedOrRenamedUnlocked ( table_id , check_referential_dependencies , check_loading_dependencies , is_drop_database ) ;
2022-10-05 20:13:16 +00:00
}
2023-02-01 23:30:49 +00:00
void DatabaseCatalog : : checkTableCanBeRemovedOrRenamedUnlocked (
const StorageID & removing_table , bool check_referential_dependencies , bool check_loading_dependencies , bool is_drop_database ) const
2022-10-05 20:13:16 +00:00
{
2023-02-01 23:30:49 +00:00
chassert ( ! check_referential_dependencies | | ! check_loading_dependencies ) ; /// These flags must not be both set.
std : : vector < StorageID > dependents ;
if ( check_referential_dependencies )
dependents = referential_dependencies . getDependents ( removing_table ) ;
else if ( check_loading_dependencies )
dependents = loading_dependencies . getDependents ( removing_table ) ;
else
return ;
2022-12-02 14:05:46 +00:00
2022-10-05 20:13:16 +00:00
if ( ! is_drop_database )
{
2022-12-02 14:05:46 +00:00
if ( ! dependents . empty ( ) )
2022-10-05 20:13:16 +00:00
throw Exception ( ErrorCodes : : HAVE_DEPENDENT_OBJECTS , " Cannot drop or rename {}, because some tables depend on it: {} " ,
2022-12-02 14:05:46 +00:00
removing_table , fmt : : join ( dependents , " , " ) ) ;
return ;
2022-10-05 20:13:16 +00:00
}
/// For DROP DATABASE we should ignore dependent tables from the same database.
/// TODO unload tables in reverse topological order and remove this code
2022-12-02 14:05:46 +00:00
std : : vector < StorageID > from_other_databases ;
for ( const auto & dependent : dependents )
if ( dependent . database_name ! = removing_table . database_name )
from_other_databases . push_back ( dependent ) ;
2022-10-05 20:13:16 +00:00
if ( ! from_other_databases . empty ( ) )
2021-11-02 12:58:45 +00:00
throw Exception ( ErrorCodes : : HAVE_DEPENDENT_OBJECTS , " Cannot drop or rename {}, because some tables depend on it: {} " ,
2022-10-05 20:13:16 +00:00
removing_table , fmt : : join ( from_other_databases , " , " ) ) ;
2021-11-01 18:53:07 +00:00
}
2022-06-21 10:35:47 +00:00
void DatabaseCatalog : : cleanupStoreDirectoryTask ( )
{
2022-08-03 17:49:16 +00:00
for ( const auto & [ disk_name , disk ] : getContext ( ) - > getDisksMap ( ) )
2022-06-21 10:35:47 +00:00
{
2022-08-03 17:49:16 +00:00
if ( ! disk - > supportsStat ( ) | | ! disk - > supportsChmod ( ) )
2022-06-21 10:35:47 +00:00
continue ;
2022-08-03 17:49:16 +00:00
size_t affected_dirs = 0 ;
2022-12-13 20:49:32 +00:00
size_t checked_dirs = 0 ;
2022-08-03 17:49:16 +00:00
for ( auto it = disk - > iterateDirectory ( " store " ) ; it - > isValid ( ) ; it - > next ( ) )
2022-06-21 10:35:47 +00:00
{
2022-08-03 17:49:16 +00:00
String prefix = it - > name ( ) ;
bool expected_prefix_dir = disk - > isDirectory ( it - > path ( ) ) & & prefix . size ( ) = = 3 & & isHexDigit ( prefix [ 0 ] ) & & isHexDigit ( prefix [ 1 ] )
& & isHexDigit ( prefix [ 2 ] ) ;
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
if ( ! expected_prefix_dir )
2022-06-21 10:35:47 +00:00
{
2022-08-03 17:49:16 +00:00
LOG_WARNING ( log , " Found invalid directory {} on disk {}, will try to remove it " , it - > path ( ) , disk_name ) ;
2022-12-13 20:49:32 +00:00
checked_dirs + = 1 ;
2022-08-03 17:49:16 +00:00
affected_dirs + = maybeRemoveDirectory ( disk_name , disk , it - > path ( ) ) ;
2022-06-21 10:35:47 +00:00
continue ;
}
2022-08-03 17:49:16 +00:00
for ( auto jt = disk - > iterateDirectory ( it - > path ( ) ) ; jt - > isValid ( ) ; jt - > next ( ) )
2022-06-21 10:35:47 +00:00
{
2022-08-03 17:49:16 +00:00
String uuid_str = jt - > name ( ) ;
UUID uuid ;
bool parsed = tryParse ( uuid , uuid_str ) ;
bool expected_dir = disk - > isDirectory ( jt - > path ( ) ) & & parsed & & uuid ! = UUIDHelpers : : Nil & & uuid_str . starts_with ( prefix ) ;
if ( ! expected_dir )
{
LOG_WARNING ( log , " Found invalid directory {} on disk {}, will try to remove it " , jt - > path ( ) , disk_name ) ;
2022-12-13 20:49:32 +00:00
checked_dirs + = 1 ;
2022-08-03 17:49:16 +00:00
affected_dirs + = maybeRemoveDirectory ( disk_name , disk , jt - > path ( ) ) ;
continue ;
}
/// Order is important
if ( ! hasUUIDMapping ( uuid ) )
{
/// We load uuids even for detached and permanently detached tables,
/// so it looks safe enough to remove directory if we don't have uuid mapping for it.
/// No table or database using this directory should concurrently appear,
/// because creation of new table would fail with "directory already exists".
2022-12-13 20:49:32 +00:00
checked_dirs + = 1 ;
2022-08-03 17:49:16 +00:00
affected_dirs + = maybeRemoveDirectory ( disk_name , disk , jt - > path ( ) ) ;
}
2022-06-21 10:35:47 +00:00
}
}
2022-08-03 17:49:16 +00:00
if ( affected_dirs )
LOG_INFO ( log , " Cleaned up {} directories from store/ on disk {} " , affected_dirs , disk_name ) ;
2022-12-13 20:49:32 +00:00
if ( checked_dirs = = 0 )
2022-11-28 17:31:22 +00:00
LOG_TEST ( log , " Nothing to clean up from store/ on disk {} " , disk_name ) ;
2022-08-03 17:49:16 +00:00
}
2022-06-21 10:35:47 +00:00
( * cleanup_task ) - > scheduleAfter ( unused_dir_cleanup_period_sec * 1000 ) ;
}
2022-08-03 17:49:16 +00:00
bool DatabaseCatalog : : maybeRemoveDirectory ( const String & disk_name , const DiskPtr & disk , const String & unused_dir )
2022-06-21 10:35:47 +00:00
{
/// "Safe" automatic removal of some directory.
/// At first we do not remove anything and only revoke all access right.
/// And remove only if nobody noticed it after, for example, one month.
2022-08-03 17:49:16 +00:00
try
2022-06-21 10:35:47 +00:00
{
2022-08-03 17:49:16 +00:00
struct stat st = disk - > stat ( unused_dir ) ;
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
if ( st . st_uid ! = geteuid ( ) )
{
/// Directory is not owned by clickhouse, it's weird, let's ignore it (chmod will likely fail anyway).
LOG_WARNING ( log , " Found directory {} with unexpected owner (uid={}) on disk {} " , unused_dir , st . st_uid , disk_name ) ;
2022-06-21 10:35:47 +00:00
return false ;
2022-08-03 17:49:16 +00:00
}
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
time_t max_modification_time = std : : max ( st . st_atime , std : : max ( st . st_mtime , st . st_ctime ) ) ;
time_t current_time = time ( nullptr ) ;
if ( st . st_mode & ( S_IRWXU | S_IRWXG | S_IRWXO ) )
{
if ( current_time < = max_modification_time + unused_dir_hide_timeout_sec )
return false ;
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
LOG_INFO ( log , " Removing access rights for unused directory {} from disk {} (will remove it when timeout exceed) " , unused_dir , disk_name ) ;
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
/// Explicitly update modification time just in case
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
disk - > setLastModified ( unused_dir , Poco : : Timestamp : : fromEpochTime ( current_time ) ) ;
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
/// Remove all access right
disk - > chmod ( unused_dir , 0 ) ;
2022-06-24 15:34:41 +00:00
2022-08-03 17:49:16 +00:00
return true ;
}
else
{
if ( ! unused_dir_rm_timeout_sec )
return false ;
if ( current_time < = max_modification_time + unused_dir_rm_timeout_sec )
return false ;
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
LOG_INFO ( log , " Removing unused directory {} from disk {} " , unused_dir , disk_name ) ;
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
/// We have to set these access rights to make recursive removal work
disk - > chmod ( unused_dir , S_IRWXU ) ;
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
disk - > removeRecursive ( unused_dir ) ;
2022-06-21 10:35:47 +00:00
2022-08-03 17:49:16 +00:00
return true ;
}
}
catch ( . . . )
{
tryLogCurrentException ( log , fmt : : format ( " Failed to remove unused directory {} from disk {} ({}) " ,
unused_dir , disk - > getName ( ) , disk - > getPath ( ) ) ) ;
return false ;
2022-06-21 10:35:47 +00:00
}
}
2022-06-22 16:31:42 +00:00
static void maybeUnlockUUID ( UUID uuid )
2022-06-21 22:50:16 +00:00
{
if ( uuid = = UUIDHelpers : : Nil )
return ;
2022-06-22 16:31:42 +00:00
chassert ( DatabaseCatalog : : instance ( ) . hasUUIDMapping ( uuid ) ) ;
auto db_and_table = DatabaseCatalog : : instance ( ) . tryGetByUUID ( uuid ) ;
if ( ! db_and_table . first & & ! db_and_table . second )
{
DatabaseCatalog : : instance ( ) . removeUUIDMappingFinally ( uuid ) ;
2022-06-21 22:50:16 +00:00
return ;
2022-06-22 16:31:42 +00:00
}
chassert ( db_and_table . first | | ! db_and_table . second ) ;
}
2022-06-21 22:50:16 +00:00
2022-06-22 16:31:42 +00:00
TemporaryLockForUUIDDirectory : : TemporaryLockForUUIDDirectory ( UUID uuid_ )
: uuid ( uuid_ )
{
if ( uuid ! = UUIDHelpers : : Nil )
DatabaseCatalog : : instance ( ) . addUUIDMapping ( uuid ) ;
2022-06-21 22:50:16 +00:00
}
2022-06-22 16:31:42 +00:00
TemporaryLockForUUIDDirectory : : ~ TemporaryLockForUUIDDirectory ( )
2022-06-21 22:50:16 +00:00
{
2022-06-22 16:31:42 +00:00
maybeUnlockUUID ( uuid ) ;
}
TemporaryLockForUUIDDirectory : : TemporaryLockForUUIDDirectory ( TemporaryLockForUUIDDirectory & & rhs ) noexcept
: uuid ( rhs . uuid )
{
rhs . uuid = UUIDHelpers : : Nil ;
2022-06-21 22:50:16 +00:00
}
2022-06-22 16:31:42 +00:00
TemporaryLockForUUIDDirectory & TemporaryLockForUUIDDirectory : : operator = ( TemporaryLockForUUIDDirectory & & rhs ) noexcept
2022-06-21 22:50:16 +00:00
{
2022-06-22 16:31:42 +00:00
maybeUnlockUUID ( uuid ) ;
uuid = rhs . uuid ;
rhs . uuid = UUIDHelpers : : Nil ;
return * this ;
2022-06-21 22:50:16 +00:00
}
2020-02-10 18:31:52 +00:00
2023-01-12 15:51:04 +00:00
DDLGuard : : DDLGuard ( Map & map_ , SharedMutex & db_mutex_ , std : : unique_lock < std : : mutex > guards_lock_ , const String & elem , const String & database_name )
2020-08-18 15:15:27 +00:00
: map ( map_ ) , db_mutex ( db_mutex_ ) , guards_lock ( std : : move ( guards_lock_ ) )
2020-02-10 18:31:52 +00:00
{
it = map . emplace ( elem , Entry { std : : make_unique < std : : mutex > ( ) , 0 } ) . first ;
+ + it - > second . counter ;
guards_lock . unlock ( ) ;
table_lock = std : : unique_lock ( * it - > second . mutex ) ;
2021-02-02 19:39:04 +00:00
is_database_guard = elem . empty ( ) ;
if ( ! is_database_guard )
2020-10-13 15:00:36 +00:00
{
bool locked_database_for_read = db_mutex . try_lock_shared ( ) ;
if ( ! locked_database_for_read )
{
2021-02-02 19:39:04 +00:00
releaseTableLock ( ) ;
2020-10-13 15:00:36 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_DATABASE , " Database {} is currently dropped or renamed " , database_name ) ;
}
}
2020-02-10 18:31:52 +00:00
}
2021-02-02 19:39:04 +00:00
void DDLGuard : : releaseTableLock ( ) noexcept
2020-02-10 18:31:52 +00:00
{
2021-02-02 19:39:04 +00:00
if ( table_lock_removed )
return ;
table_lock_removed = true ;
2020-02-10 18:31:52 +00:00
guards_lock . lock ( ) ;
2021-02-03 20:02:37 +00:00
UInt32 counter = - - it - > second . counter ;
table_lock . unlock ( ) ;
if ( counter = = 0 )
2020-02-10 18:31:52 +00:00
map . erase ( it ) ;
2021-02-02 19:39:04 +00:00
guards_lock . unlock ( ) ;
2020-02-10 18:31:52 +00:00
}
2020-10-13 15:00:36 +00:00
DDLGuard : : ~ DDLGuard ( )
{
2021-02-02 19:39:04 +00:00
if ( ! is_database_guard )
2020-10-13 15:00:36 +00:00
db_mutex . unlock_shared ( ) ;
2021-02-02 19:39:04 +00:00
releaseTableLock ( ) ;
2020-10-13 15:00:36 +00:00
}
2020-02-03 12:54:36 +00:00
}