2019-10-02 10:10:45 +00:00
# include <Databases/DatabaseOnDisk.h>
2019-10-03 09:31:59 +00:00
2019-10-02 10:10:45 +00:00
# include <IO/ReadBufferFromFile.h>
# include <IO/ReadHelpers.h>
# include <IO/WriteBufferFromFile.h>
# include <IO/WriteHelpers.h>
2019-10-10 17:33:01 +00:00
# include <Interpreters/Context.h>
# include <Interpreters/InterpreterCreateQuery.h>
2022-03-28 19:52:10 +00:00
# include <Interpreters/ApplyWithSubqueryVisitor.h>
2019-10-02 10:10:45 +00:00
# include <Parsers/ASTCreateQuery.h>
2021-11-26 17:21:54 +00:00
# include <Parsers/ASTFunction.h>
2019-10-03 09:31:59 +00:00
# include <Parsers/ParserCreateQuery.h>
2019-10-10 17:33:01 +00:00
# include <Parsers/formatAST.h>
# include <Parsers/parseQuery.h>
2019-10-02 10:10:45 +00:00
# include <Storages/IStorage.h>
2019-10-10 17:33:01 +00:00
# include <Storages/StorageFactory.h>
# include <TableFunctions/TableFunctionFactory.h>
# include <Common/escapeForFileName.h>
2021-10-02 07:13:14 +00:00
# include <base/logger_useful.h>
2019-11-26 19:06:19 +00:00
# include <Databases/DatabaseOrdinary.h>
# include <Databases/DatabaseAtomic.h>
2020-07-06 08:30:11 +00:00
# include <Common/assert_cast.h>
2021-04-30 17:20:53 +00:00
# include <filesystem>
2021-05-28 21:57:53 +00:00
# include <Common/filesystemHelpers.h>
2020-03-16 11:38:50 +00:00
2021-04-30 17:20:53 +00:00
namespace fs = std : : filesystem ;
2019-10-10 17:33:01 +00:00
2019-10-02 10:10:45 +00:00
namespace DB
{
2019-10-10 17:33:01 +00:00
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768 ;
2019-10-02 10:10:45 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int CANNOT_GET_CREATE_TABLE_QUERY ;
extern const int NOT_IMPLEMENTED ;
extern const int LOGICAL_ERROR ;
2019-10-03 08:27:43 +00:00
extern const int FILE_DOESNT_EXIST ;
2021-10-05 16:34:25 +00:00
extern const int CANNOT_OPEN_FILE ;
2019-10-03 08:27:43 +00:00
extern const int INCORRECT_FILE_NAME ;
2019-10-02 10:10:45 +00:00
extern const int SYNTAX_ERROR ;
2019-10-03 08:27:43 +00:00
extern const int TABLE_ALREADY_EXISTS ;
2021-04-21 14:41:46 +00:00
extern const int EMPTY_LIST_OF_COLUMNS_PASSED ;
2021-10-13 10:34:18 +00:00
extern const int DATABASE_NOT_EMPTY ;
2021-11-23 17:51:53 +00:00
extern const int INCORRECT_QUERY ;
2019-10-02 10:10:45 +00:00
}
2019-10-10 17:33:01 +00:00
std : : pair < String , StoragePtr > createTableFromAST (
ASTCreateQuery ast_create_query ,
const String & database_name ,
2019-11-01 12:47:55 +00:00
const String & table_data_path_relative ,
2021-05-31 14:49:02 +00:00
ContextMutablePtr context ,
2021-09-13 19:11:16 +00:00
bool force_restore )
2019-10-10 17:33:01 +00:00
{
ast_create_query . attach = true ;
2021-09-06 22:13:54 +00:00
ast_create_query . setDatabase ( database_name ) ;
2019-10-10 17:33:01 +00:00
2022-03-28 19:52:10 +00:00
if ( ast_create_query . select & & ast_create_query . isView ( ) )
ApplyWithSubqueryVisitor ( ) . visit ( * ast_create_query . select ) ;
2019-10-10 17:33:01 +00:00
if ( ast_create_query . as_table_function )
{
const auto & factory = TableFunctionFactory : : instance ( ) ;
2020-10-14 12:19:29 +00:00
auto table_function = factory . get ( ast_create_query . as_table_function , context ) ;
ColumnsDescription columns ;
if ( ast_create_query . columns_list & & ast_create_query . columns_list - > columns )
2021-04-16 10:15:35 +00:00
columns = InterpreterCreateQuery : : getColumnsDescription ( * ast_create_query . columns_list - > columns , context , true ) ;
2021-09-06 22:13:54 +00:00
StoragePtr storage = table_function - > execute ( ast_create_query . as_table_function , context , ast_create_query . getTable ( ) , std : : move ( columns ) ) ;
2020-04-07 14:05:51 +00:00
storage - > renameInMemory ( ast_create_query ) ;
2021-09-06 22:13:54 +00:00
return { ast_create_query . getTable ( ) , storage } ;
2019-10-10 17:33:01 +00:00
}
2021-04-21 14:41:46 +00:00
ColumnsDescription columns ;
ConstraintsDescription constraints ;
if ( ! ast_create_query . is_dictionary )
{
/// We do not directly use `InterpreterCreateQuery::execute`, because
/// - the database has not been loaded yet;
/// - the code is simpler, since the query is already brought to a suitable form.
if ( ! ast_create_query . columns_list | | ! ast_create_query . columns_list - > columns )
2021-12-15 11:30:57 +00:00
{
2022-02-10 20:35:03 +00:00
if ( ! ast_create_query . storage | | ! ast_create_query . storage - > engine )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Invalid storage definition in metadata file: "
" it's a bug or result of manual intervention in metadata files " ) ;
2021-12-15 11:30:57 +00:00
if ( ! StorageFactory : : instance ( ) . checkIfStorageSupportsSchemaInterface ( ast_create_query . storage - > engine - > name ) )
throw Exception ( " Missing definition of columns. " , ErrorCodes : : EMPTY_LIST_OF_COLUMNS_PASSED ) ;
/// Leave columns empty.
}
else
{
columns = InterpreterCreateQuery : : getColumnsDescription ( * ast_create_query . columns_list - > columns , context , true ) ;
constraints = InterpreterCreateQuery : : getConstraintsDescription ( ast_create_query . columns_list - > constraints ) ;
}
2021-04-21 14:41:46 +00:00
}
2019-10-10 17:33:01 +00:00
return
{
2021-09-06 22:13:54 +00:00
ast_create_query . getTable ( ) ,
2019-10-10 17:33:01 +00:00
StorageFactory : : instance ( ) . get (
ast_create_query ,
2019-12-04 16:06:55 +00:00
table_data_path_relative ,
context ,
2021-04-10 23:33:54 +00:00
context - > getGlobalContext ( ) ,
2019-12-04 16:06:55 +00:00
columns ,
constraints ,
2021-09-13 19:11:16 +00:00
force_restore )
2019-10-10 17:33:01 +00:00
} ;
}
String getObjectDefinitionFromCreateQuery ( const ASTPtr & query )
{
ASTPtr query_clone = query - > clone ( ) ;
2019-10-22 10:47:43 +00:00
auto * create = query_clone - > as < ASTCreateQuery > ( ) ;
2019-10-10 17:33:01 +00:00
2019-10-22 10:47:43 +00:00
if ( ! create )
{
2020-11-09 16:05:40 +00:00
WriteBufferFromOwnString query_buf ;
formatAST ( * query , query_buf , true ) ;
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Query '{}' is not CREATE query " , query_buf . str ( ) ) ;
2019-10-22 10:47:43 +00:00
}
if ( ! create - > is_dictionary )
create - > attach = true ;
2019-10-11 15:22:30 +00:00
2019-10-10 17:33:01 +00:00
/// We remove everything that is not needed for ATTACH from the query.
2020-12-25 16:28:10 +00:00
assert ( ! create - > temporary ) ;
2021-09-06 22:13:54 +00:00
create - > database . reset ( ) ;
2019-10-22 10:47:43 +00:00
create - > as_database . clear ( ) ;
create - > as_table . clear ( ) ;
create - > if_not_exists = false ;
create - > is_populate = false ;
create - > replace_view = false ;
2020-12-25 16:28:10 +00:00
create - > replace_table = false ;
create - > create_or_replace = false ;
2019-10-10 17:33:01 +00:00
/// For views it is necessary to save the SELECT query itself, for the rest - on the contrary
2020-12-16 04:07:50 +00:00
if ( ! create - > isView ( ) )
2019-10-22 10:47:43 +00:00
create - > select = nullptr ;
2019-10-10 17:33:01 +00:00
2019-10-22 10:47:43 +00:00
create - > format = nullptr ;
create - > out_file = nullptr ;
2019-10-10 17:33:01 +00:00
2020-01-17 16:07:20 +00:00
if ( create - > uuid ! = UUIDHelpers : : Nil )
2021-09-06 22:13:54 +00:00
create - > setTable ( TABLE_WITH_UUID_NAME_PLACEHOLDER ) ;
2019-12-02 19:11:18 +00:00
2020-11-09 16:05:40 +00:00
WriteBufferFromOwnString statement_buf ;
formatAST ( * create , statement_buf , false ) ;
writeChar ( ' \n ' , statement_buf ) ;
return statement_buf . str ( ) ;
2019-10-10 17:33:01 +00:00
}
2021-02-08 19:36:17 +00:00
2020-06-11 03:24:52 +00:00
DatabaseOnDisk : : DatabaseOnDisk (
const String & name ,
const String & metadata_path_ ,
const String & data_path_ ,
const String & logger ,
2021-04-10 23:33:54 +00:00
ContextPtr local_context )
: DatabaseWithOwnTablesBase ( name , logger , local_context )
2019-11-28 19:40:51 +00:00
, metadata_path ( metadata_path_ )
2020-04-06 23:22:44 +00:00
, data_path ( data_path_ )
2019-11-28 19:40:51 +00:00
{
2021-04-30 17:20:53 +00:00
fs : : create_directories ( local_context - > getPath ( ) + data_path ) ;
fs : : create_directories ( metadata_path ) ;
2019-11-28 19:40:51 +00:00
}
2019-10-03 07:59:48 +00:00
void DatabaseOnDisk : : createTable (
2021-04-10 23:33:54 +00:00
ContextPtr local_context ,
2019-10-03 07:59:48 +00:00
const String & table_name ,
const StoragePtr & table ,
const ASTPtr & query )
{
2021-04-10 23:33:54 +00:00
const auto & settings = local_context - > getSettingsRef ( ) ;
2020-03-23 00:12:13 +00:00
const auto & create = query - > as < ASTCreateQuery & > ( ) ;
2021-09-13 14:18:04 +00:00
assert ( table_name = = create . getTable ( ) ) ;
2019-10-03 07:59:48 +00:00
/// Create a file with metadata if necessary - if the query is not ATTACH.
/// Write the query of `ATTACH table` to it.
/** The code is based on the assumption that all threads share the same order of operations
* - creating the . sql . tmp file ;
* - adding a table to ` tables ` ;
* - rename . sql . tmp to . sql .
*/
/// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH.
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
2021-04-10 23:33:54 +00:00
if ( isTableExist ( table_name , getContext ( ) ) )
throw Exception (
ErrorCodes : : TABLE_ALREADY_EXISTS , " Table {}.{} already exists " , backQuote ( getDatabaseName ( ) ) , backQuote ( table_name ) ) ;
2020-11-30 17:52:32 +00:00
2020-12-15 17:06:23 +00:00
String table_metadata_path = getObjectMetadataPath ( table_name ) ;
2019-10-03 07:59:48 +00:00
2020-03-23 00:12:13 +00:00
if ( create . attach_short_syntax )
{
/// Metadata already exists, table was detached
2021-04-10 23:33:54 +00:00
removeDetachedPermanentlyFlag ( local_context , table_name , table_metadata_path , true ) ;
2021-11-18 10:45:12 +00:00
attachTable ( local_context , table_name , table , getTableDataPath ( create ) ) ;
2020-03-23 00:12:13 +00:00
return ;
}
2020-12-15 17:06:23 +00:00
if ( ! create . attach )
checkMetadataFilenameAvailability ( table_name ) ;
2021-04-30 17:20:53 +00:00
if ( create . attach & & fs : : exists ( table_metadata_path ) )
2020-12-15 17:06:23 +00:00
{
2021-04-10 23:33:54 +00:00
ASTPtr ast_detached = parseQueryFromMetadata ( log , local_context , table_metadata_path ) ;
2020-12-15 17:06:23 +00:00
auto & create_detached = ast_detached - > as < ASTCreateQuery & > ( ) ;
// either both should be Nil, either values should be equal
if ( create . uuid ! = create_detached . uuid )
throw Exception (
ErrorCodes : : TABLE_ALREADY_EXISTS ,
" Table {}.{} already exist (detached permanently). To attach it back "
" you need to use short ATTACH syntax or a full statement with the same UUID " ,
backQuote ( getDatabaseName ( ) ) , backQuote ( table_name ) ) ;
}
2020-01-22 11:30:11 +00:00
String table_metadata_tmp_path = table_metadata_path + create_suffix ;
2019-10-03 07:59:48 +00:00
String statement ;
{
2019-10-10 17:33:01 +00:00
statement = getObjectDefinitionFromCreateQuery ( query ) ;
2019-10-03 07:59:48 +00:00
/// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown.
WriteBufferFromFile out ( table_metadata_tmp_path , statement . size ( ) , O_WRONLY | O_CREAT | O_EXCL ) ;
writeString ( statement , out ) ;
out . next ( ) ;
if ( settings . fsync_metadata )
out . sync ( ) ;
out . close ( ) ;
}
2021-04-10 23:33:54 +00:00
commitCreateTable ( create , table , table_metadata_tmp_path , table_metadata_path , local_context ) ;
2020-12-15 17:06:23 +00:00
2021-04-10 23:33:54 +00:00
removeDetachedPermanentlyFlag ( local_context , table_name , table_metadata_path , false ) ;
2020-12-15 17:06:23 +00:00
}
/// If the table was detached permanently we will have a flag file with
/// .sql.detached extension, is not needed anymore since we attached the table back
2021-04-10 23:33:54 +00:00
void DatabaseOnDisk : : removeDetachedPermanentlyFlag ( ContextPtr , const String & table_name , const String & table_metadata_path , bool ) const
2020-12-15 17:06:23 +00:00
{
try
{
2021-04-30 17:20:53 +00:00
fs : : path detached_permanently_flag ( table_metadata_path + detached_suffix ) ;
2020-12-15 17:06:23 +00:00
2021-04-30 17:20:53 +00:00
if ( fs : : exists ( detached_permanently_flag ) )
fs : : remove ( detached_permanently_flag ) ;
2020-12-15 17:06:23 +00:00
}
catch ( Exception & e )
{
2020-12-24 10:11:07 +00:00
e . addMessage ( " while trying to remove permanently detached flag. Table {}.{} may still be marked as permanently detached, and will not be reattached during server restart. " , backQuote ( getDatabaseName ( ) ) , backQuote ( table_name ) ) ;
2020-12-15 17:06:23 +00:00
throw ;
}
2020-03-23 00:12:13 +00:00
}
void DatabaseOnDisk : : commitCreateTable ( const ASTCreateQuery & query , const StoragePtr & table ,
2020-11-20 16:06:27 +00:00
const String & table_metadata_tmp_path , const String & table_metadata_path ,
2021-11-10 13:13:27 +00:00
ContextPtr query_context )
2020-03-23 00:12:13 +00:00
{
2019-10-03 07:59:48 +00:00
try
{
/// Add a table to the map of known tables.
2021-11-10 13:13:27 +00:00
attachTable ( query_context , query . getTable ( ) , table , getTableDataPath ( query ) ) ;
2019-10-03 07:59:48 +00:00
/// If it was ATTACH query and file with table metadata already exist
/// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one.
2021-04-30 17:20:53 +00:00
fs : : rename ( table_metadata_tmp_path , table_metadata_path ) ;
2019-10-03 07:59:48 +00:00
}
catch ( . . . )
{
2021-04-30 17:20:53 +00:00
fs : : remove ( table_metadata_tmp_path ) ;
2019-10-03 07:59:48 +00:00
throw ;
}
}
2021-11-10 13:13:27 +00:00
void DatabaseOnDisk : : detachTablePermanently ( ContextPtr query_context , const String & table_name )
2020-11-30 17:52:32 +00:00
{
2021-11-10 13:13:27 +00:00
auto table = detachTable ( query_context , table_name ) ;
2020-11-30 17:52:32 +00:00
2021-05-07 21:53:44 +00:00
fs : : path detached_permanently_flag ( getObjectMetadataPath ( table_name ) + detached_suffix ) ;
2020-11-30 17:52:32 +00:00
try
{
2021-05-18 10:11:12 +00:00
FS : : createFile ( detached_permanently_flag ) ;
2020-11-30 17:52:32 +00:00
}
catch ( Exception & e )
{
2020-12-24 10:11:07 +00:00
e . addMessage ( " while trying to set permanently detached flag. Table {}.{} may be reattached during server restart. " , backQuote ( getDatabaseName ( ) ) , backQuote ( table_name ) ) ;
2020-11-30 17:52:32 +00:00
throw ;
}
2019-10-03 07:59:48 +00:00
}
2021-04-10 23:33:54 +00:00
void DatabaseOnDisk : : dropTable ( ContextPtr local_context , const String & table_name , bool /*no_delay*/ )
2019-10-03 07:59:48 +00:00
{
2019-11-06 16:05:04 +00:00
String table_metadata_path = getObjectMetadataPath ( table_name ) ;
2020-01-22 11:30:11 +00:00
String table_metadata_path_drop = table_metadata_path + drop_suffix ;
String table_data_path_relative = getTableDataPath ( table_name ) ;
2020-07-16 21:41:26 +00:00
if ( table_data_path_relative . empty ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Path is empty " ) ;
2019-10-03 07:59:48 +00:00
2021-11-10 13:13:27 +00:00
StoragePtr table = detachTable ( local_context , table_name ) ;
2020-09-03 01:06:12 +00:00
/// This is possible for Lazy database.
if ( ! table )
return ;
2020-01-22 11:30:11 +00:00
bool renamed = false ;
2019-10-03 07:59:48 +00:00
try
{
2021-04-30 17:20:53 +00:00
fs : : rename ( table_metadata_path , table_metadata_path_drop ) ;
2020-01-22 11:30:11 +00:00
renamed = true ;
table - > drop ( ) ;
table - > is_dropped = true ;
2021-04-30 17:20:53 +00:00
fs : : path table_data_dir ( local_context - > getPath ( ) + table_data_path_relative ) ;
if ( fs : : exists ( table_data_dir ) )
fs : : remove_all ( table_data_dir ) ;
2019-10-03 07:59:48 +00:00
}
catch ( . . . )
{
Use fmt::runtime() for LOG_* for non constexpr
Here is oneliner:
$ gg 'LOG_\(DEBUG\|TRACE\|INFO\|TEST\|WARNING\|ERROR\|FATAL\)([^,]*, [a-zA-Z]' -- :*.cpp :*.h | cut -d: -f1 | sort -u | xargs -r sed -E -i 's#(LOG_[A-Z]*)\(([^,]*), ([A-Za-z][^,)]*)#\1(\2, fmt::runtime(\3)#'
Note, that I tried to do this with coccinelle (tool for semantic
patchin), but it cannot parse C++:
$ cat fmt.cocci
@@
expression log;
expression var;
@@
-LOG_DEBUG(log, var)
+LOG_DEBUG(log, fmt::runtime(var))
I've also tried to use some macros/templates magic to do this implicitly
in logger_useful.h, but I failed to do so, and apparently it is not
possible for now.
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
v2: manual fixes
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-02-01 09:10:27 +00:00
LOG_WARNING ( log , fmt : : runtime ( getCurrentExceptionMessage ( __PRETTY_FUNCTION__ ) ) ) ;
2021-11-10 13:13:27 +00:00
attachTable ( local_context , table_name , table , table_data_path_relative ) ;
2020-01-22 11:30:11 +00:00
if ( renamed )
2021-04-30 17:20:53 +00:00
fs : : rename ( table_metadata_path_drop , table_metadata_path ) ;
2019-10-03 07:59:48 +00:00
throw ;
}
2020-01-22 11:30:11 +00:00
2021-04-30 17:20:53 +00:00
fs : : remove ( table_metadata_path_drop ) ;
2019-10-03 07:59:48 +00:00
}
2020-12-15 14:46:13 +00:00
void DatabaseOnDisk : : checkMetadataFilenameAvailability ( const String & to_table_name ) const
2020-12-08 10:00:32 +00:00
{
2020-12-15 14:46:13 +00:00
std : : unique_lock lock ( mutex ) ;
2020-12-15 17:06:23 +00:00
checkMetadataFilenameAvailabilityUnlocked ( to_table_name , lock ) ;
2020-12-15 14:46:13 +00:00
}
2020-12-08 10:00:32 +00:00
2020-12-15 14:46:13 +00:00
void DatabaseOnDisk : : checkMetadataFilenameAvailabilityUnlocked ( const String & to_table_name , std : : unique_lock < std : : mutex > & ) const
{
2020-12-14 14:21:31 +00:00
String table_metadata_path = getObjectMetadataPath ( to_table_name ) ;
2020-12-08 10:00:32 +00:00
2021-04-30 17:20:53 +00:00
if ( fs : : exists ( table_metadata_path ) )
2020-12-08 10:00:32 +00:00
{
2021-04-30 17:20:53 +00:00
fs : : path detached_permanently_flag ( table_metadata_path + detached_suffix ) ;
2020-12-08 10:00:32 +00:00
2021-04-30 17:20:53 +00:00
if ( fs : : exists ( detached_permanently_flag ) )
2020-12-15 14:46:13 +00:00
throw Exception ( ErrorCodes : : TABLE_ALREADY_EXISTS , " Table {}.{} already exists (detached permanently) " , backQuote(database_name), backQuote(to_table_name)) ;
else
throw Exception ( ErrorCodes : : TABLE_ALREADY_EXISTS , " Table {}.{} already exists (detached) " , backQuote(database_name), backQuote(to_table_name)) ;
2020-12-08 10:00:32 +00:00
}
}
2019-11-06 16:05:04 +00:00
void DatabaseOnDisk : : renameTable (
2021-04-10 23:33:54 +00:00
ContextPtr local_context ,
2019-11-06 16:05:04 +00:00
const String & table_name ,
IDatabase & to_database ,
2020-03-31 20:38:05 +00:00
const String & to_table_name ,
2020-07-16 21:41:26 +00:00
bool exchange ,
bool dictionary )
2019-10-10 17:33:01 +00:00
{
2021-11-23 17:51:53 +00:00
if ( exchange )
throw Exception ( " Tables can be exchanged only in Atomic databases " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2019-11-26 19:06:19 +00:00
bool from_ordinary_to_atomic = false ;
2019-12-02 19:11:18 +00:00
bool from_atomic_to_ordinary = false ;
2019-11-06 16:05:04 +00:00
if ( typeid ( * this ) ! = typeid ( to_database ) )
2019-11-26 19:06:19 +00:00
{
if ( typeid_cast < DatabaseOrdinary * > ( this ) & & typeid_cast < DatabaseAtomic * > ( & to_database ) )
from_ordinary_to_atomic = true ;
2019-12-02 19:11:18 +00:00
else if ( typeid_cast < DatabaseAtomic * > ( this ) & & typeid_cast < DatabaseOrdinary * > ( & to_database ) )
from_atomic_to_ordinary = true ;
2021-02-11 22:23:40 +00:00
else if ( dynamic_cast < DatabaseAtomic * > ( this ) & & typeid_cast < DatabaseOrdinary * > ( & to_database ) & & getEngineName ( ) = = " Replicated " )
from_atomic_to_ordinary = true ;
2019-11-26 19:06:19 +00:00
else
throw Exception ( " Moving tables between databases of different engines is not supported " , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2019-11-06 16:05:04 +00:00
2020-01-22 11:30:11 +00:00
auto table_data_relative_path = getTableDataPath ( table_name ) ;
2020-06-18 16:10:47 +00:00
TableExclusiveLockHolder table_lock ;
2020-01-22 11:30:11 +00:00
String table_metadata_path ;
ASTPtr attach_query ;
2020-03-18 17:38:52 +00:00
/// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case.
2021-11-18 10:45:12 +00:00
StoragePtr table = tryGetTable ( table_name , local_context ) ;
2021-11-24 09:06:19 +00:00
if ( dictionary & & table & & ! table - > isDictionary ( ) )
2021-11-23 17:51:53 +00:00
throw Exception ( " Use RENAME/EXCHANGE TABLE (instead of RENAME/EXCHANGE DICTIONARY) for tables " , ErrorCodes::INCORRECT_QUERY) ;
2021-11-22 15:48:22 +00:00
2021-11-10 13:13:27 +00:00
detachTable ( local_context , table_name ) ;
2021-11-22 15:41:35 +00:00
2021-02-11 22:23:40 +00:00
UUID prev_uuid = UUIDHelpers : : Nil ;
2019-10-10 17:33:01 +00:00
try
{
2021-04-10 23:33:54 +00:00
table_lock = table - > lockExclusively (
local_context - > getCurrentQueryId ( ) , local_context - > getSettingsRef ( ) . lock_acquire_timeout ) ;
2020-01-22 11:30:11 +00:00
table_metadata_path = getObjectMetadataPath ( table_name ) ;
2021-04-10 23:33:54 +00:00
attach_query = parseQueryFromMetadata ( log , local_context , table_metadata_path ) ;
2020-01-22 11:30:11 +00:00
auto & create = attach_query - > as < ASTCreateQuery & > ( ) ;
2021-09-06 22:13:54 +00:00
create . setDatabase ( to_database . getDatabaseName ( ) ) ;
create . setTable ( to_table_name ) ;
2020-01-22 11:30:11 +00:00
if ( from_ordinary_to_atomic )
create . uuid = UUIDHelpers : : generateV4 ( ) ;
if ( from_atomic_to_ordinary )
2021-02-11 22:23:40 +00:00
std : : swap ( create . uuid , prev_uuid ) ;
2020-01-22 11:30:11 +00:00
2020-12-15 14:46:13 +00:00
if ( auto * target_db = dynamic_cast < DatabaseOnDisk * > ( & to_database ) )
target_db - > checkMetadataFilenameAvailability ( to_table_name ) ;
2020-12-08 10:00:32 +00:00
2020-01-22 11:30:11 +00:00
/// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID
2020-04-07 14:05:51 +00:00
table - > rename ( to_database . getTableDataPath ( create ) , StorageID ( create ) ) ;
2019-10-10 17:33:01 +00:00
}
2019-11-06 16:05:04 +00:00
catch ( const Exception & )
2019-10-10 17:33:01 +00:00
{
2022-04-13 14:51:59 +00:00
setDetachedTableNotInUseForce ( prev_uuid ) ;
2021-11-10 13:13:27 +00:00
attachTable ( local_context , table_name , table , table_data_relative_path ) ;
2019-10-10 17:33:01 +00:00
throw ;
}
2019-11-06 16:05:04 +00:00
catch ( const Poco : : Exception & e )
{
2022-04-13 14:51:59 +00:00
setDetachedTableNotInUseForce ( prev_uuid ) ;
2021-11-10 13:13:27 +00:00
attachTable ( local_context , table_name , table , table_data_relative_path ) ;
2019-11-06 16:05:04 +00:00
/// Better diagnostics.
2020-06-04 17:49:14 +00:00
throw Exception { Exception : : CreateFromPocoTag { } , e } ;
2019-11-06 16:05:04 +00:00
}
2019-10-10 17:33:01 +00:00
2020-01-22 11:30:11 +00:00
/// Now table data are moved to new database, so we must add metadata and attach table to new database
2021-04-10 23:33:54 +00:00
to_database . createTable ( local_context , to_table_name , table , attach_query ) ;
2020-01-22 11:30:11 +00:00
2021-04-30 17:20:53 +00:00
fs : : remove ( table_metadata_path ) ;
2020-07-06 08:30:11 +00:00
2021-02-11 22:23:40 +00:00
if ( from_atomic_to_ordinary )
2020-07-06 08:30:11 +00:00
{
2021-02-16 14:05:58 +00:00
auto & atomic_db = dynamic_cast < DatabaseAtomic & > ( * this ) ;
2021-02-11 22:23:40 +00:00
/// Special case: usually no actions with symlinks are required when detaching/attaching table,
/// but not when moving from Atomic database to Ordinary
if ( table - > storesDataOnDisk ( ) )
atomic_db . tryRemoveSymlink ( table_name ) ;
/// Forget about UUID, now it's possible to reuse it for new table
DatabaseCatalog : : instance ( ) . removeUUIDMappingFinally ( prev_uuid ) ;
atomic_db . setDetachedTableNotInUseForce ( prev_uuid ) ;
2020-07-06 08:30:11 +00:00
}
2019-11-06 16:05:04 +00:00
}
2020-11-30 17:52:32 +00:00
2020-12-15 14:46:13 +00:00
/// It returns create table statement (even if table is detached)
2021-04-10 23:33:54 +00:00
ASTPtr DatabaseOnDisk : : getCreateTableQueryImpl ( const String & table_name , ContextPtr , bool throw_on_error ) const
2019-10-03 07:59:48 +00:00
{
ASTPtr ast ;
2021-11-19 10:25:55 +00:00
StoragePtr storage = tryGetTable ( table_name , getContext ( ) ) ;
bool has_table = storage ! = nullptr ;
bool is_system_storage = false ;
if ( has_table )
is_system_storage = storage - > isSystemStorage ( ) ;
2019-11-06 16:05:04 +00:00
auto table_metadata_path = getObjectMetadataPath ( table_name ) ;
2020-01-29 18:45:03 +00:00
try
2019-10-03 07:59:48 +00:00
{
2020-04-23 16:51:48 +00:00
ast = getCreateQueryFromMetadata ( table_metadata_path , throw_on_error ) ;
2020-01-29 18:45:03 +00:00
}
catch ( const Exception & e )
{
if ( ! has_table & & e . code ( ) = = ErrorCodes : : FILE_DOESNT_EXIST & & throw_on_error )
throw Exception { " Table " + backQuote ( table_name ) + " doesn't exist " ,
ErrorCodes : : CANNOT_GET_CREATE_TABLE_QUERY } ;
2021-11-19 10:25:55 +00:00
else if ( is_system_storage )
ast = getCreateQueryFromStorage ( table_name , storage , throw_on_error ) ;
2020-01-29 18:45:03 +00:00
else if ( throw_on_error )
throw ;
2019-10-03 07:59:48 +00:00
}
return ast ;
}
2020-04-23 16:51:48 +00:00
ASTPtr DatabaseOnDisk : : getCreateDatabaseQuery ( ) const
2019-10-03 07:59:48 +00:00
{
ASTPtr ast ;
2021-04-10 23:33:54 +00:00
auto settings = getContext ( ) - > getSettingsRef ( ) ;
2020-07-07 12:11:58 +00:00
{
std : : lock_guard lock ( mutex ) ;
2021-04-10 23:33:54 +00:00
auto database_metadata_path = getContext ( ) - > getPath ( ) + " metadata/ " + escapeForFileName ( database_name ) + " .sql " ;
ast = parseQueryFromMetadata ( log , getContext ( ) , database_metadata_path , true ) ;
2020-07-07 12:11:58 +00:00
auto & ast_create_query = ast - > as < ASTCreateQuery & > ( ) ;
ast_create_query . attach = false ;
2021-09-06 22:13:54 +00:00
ast_create_query . setDatabase ( database_name ) ;
2020-07-07 12:11:58 +00:00
}
2019-10-03 07:59:48 +00:00
if ( ! ast )
{
/// Handle databases (such as default) for which there are no database.sql files.
2019-12-02 19:11:18 +00:00
/// If database.sql doesn't exist, then engine is Ordinary
String query = " CREATE DATABASE " + backQuoteIfNeed ( getDatabaseName ( ) ) + " ENGINE = Ordinary " ;
2019-10-03 07:59:48 +00:00
ParserCreateQuery parser ;
2020-01-14 11:11:01 +00:00
ast = parseQuery ( parser , query . data ( ) , query . data ( ) + query . size ( ) , " " , 0 , settings . max_parser_depth ) ;
2019-10-03 07:59:48 +00:00
}
2021-09-27 11:23:18 +00:00
if ( const auto database_comment = getDatabaseComment ( ) ; ! database_comment . empty ( ) )
{
auto & ast_create_query = ast - > as < ASTCreateQuery & > ( ) ;
2021-10-08 10:07:11 +00:00
ast_create_query . set ( ast_create_query . comment , std : : make_shared < ASTLiteral > ( database_comment ) ) ;
2021-09-27 11:23:18 +00:00
}
2019-10-03 07:59:48 +00:00
return ast ;
}
2021-04-10 23:33:54 +00:00
void DatabaseOnDisk : : drop ( ContextPtr local_context )
2019-10-03 07:59:48 +00:00
{
2020-08-18 15:15:27 +00:00
assert ( tables . empty ( ) ) ;
2021-10-13 10:34:18 +00:00
if ( local_context - > getSettingsRef ( ) . force_remove_data_recursively_on_drop )
{
fs : : remove_all ( local_context - > getPath ( ) + getDataPath ( ) ) ;
fs : : remove_all ( getMetadataPath ( ) ) ;
}
else
{
try
{
fs : : remove ( local_context - > getPath ( ) + getDataPath ( ) ) ;
fs : : remove ( getMetadataPath ( ) ) ;
}
catch ( const fs : : filesystem_error & e )
{
if ( e . code ( ) ! = std : : errc : : directory_not_empty )
throw Exception ( Exception : : CreateFromSTDTag { } , e ) ;
throw Exception ( ErrorCodes : : DATABASE_NOT_EMPTY , " Cannot drop: {}. "
" Probably database contain some detached tables or metadata leftovers from Ordinary engine. "
" If you want to remove all data anyway, try to attach database back and drop it again "
" with enabled force_remove_data_recursively_on_drop setting " , e . what ( ) ) ;
}
}
2019-10-03 07:59:48 +00:00
}
2020-03-08 22:38:12 +00:00
String DatabaseOnDisk : : getObjectMetadataPath ( const String & object_name ) const
2019-10-03 07:59:48 +00:00
{
2020-03-08 22:38:12 +00:00
return getMetadataPath ( ) + escapeForFileName ( object_name ) + " .sql " ;
2019-10-02 10:10:45 +00:00
}
2020-03-08 22:38:12 +00:00
time_t DatabaseOnDisk : : getObjectMetadataModificationTime ( const String & object_name ) const
2019-10-03 08:35:58 +00:00
{
2020-03-08 22:38:12 +00:00
String table_metadata_path = getObjectMetadataPath ( object_name ) ;
2019-10-03 08:35:58 +00:00
2021-05-12 06:19:11 +00:00
if ( fs : : exists ( table_metadata_path ) )
2021-05-28 18:17:16 +00:00
return FS : : getModificationTime ( table_metadata_path ) ;
2019-10-03 08:35:58 +00:00
else
return static_cast < time_t > ( 0 ) ;
}
2021-04-10 23:33:54 +00:00
void DatabaseOnDisk : : iterateMetadataFiles ( ContextPtr local_context , const IteratingFunction & process_metadata_file ) const
2020-01-23 19:10:09 +00:00
{
2020-04-06 23:22:44 +00:00
auto process_tmp_drop_metadata_file = [ & ] ( const String & file_name )
2020-01-23 19:10:09 +00:00
{
2020-09-14 19:25:02 +00:00
assert ( getUUID ( ) = = UUIDHelpers : : Nil ) ;
2020-01-23 19:10:09 +00:00
static const char * tmp_drop_ext = " .sql.tmp_drop " ;
const std : : string object_name = file_name . substr ( 0 , file_name . size ( ) - strlen ( tmp_drop_ext ) ) ;
2021-04-30 17:20:53 +00:00
if ( fs : : exists ( local_context - > getPath ( ) + getDataPath ( ) + ' / ' + object_name ) )
2020-01-23 19:10:09 +00:00
{
2021-04-30 17:20:53 +00:00
fs : : rename ( getMetadataPath ( ) + file_name , getMetadataPath ( ) + object_name + " .sql " ) ;
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Object {} was not dropped previously and will be restored " , backQuote ( object_name ) ) ;
2020-01-23 19:10:09 +00:00
process_metadata_file ( object_name + " .sql " ) ;
}
else
{
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Removing file {} " , getMetadataPath ( ) + file_name ) ;
2021-04-30 17:20:53 +00:00
fs : : remove ( getMetadataPath ( ) + file_name ) ;
2020-01-23 19:10:09 +00:00
}
} ;
2020-06-29 23:28:25 +00:00
/// Metadata files to load: name and flag for .tmp_drop files
std : : set < std : : pair < String , bool > > metadata_files ;
2021-05-07 21:53:44 +00:00
fs : : directory_iterator dir_end ;
for ( fs : : directory_iterator dir_it ( getMetadataPath ( ) ) ; dir_it ! = dir_end ; + + dir_it )
2019-10-03 08:27:43 +00:00
{
2021-05-07 21:53:44 +00:00
String file_name = dir_it - > path ( ) . filename ( ) ;
2019-10-03 08:27:43 +00:00
/// For '.svn', '.gitignore' directory and similar.
2021-05-07 21:53:44 +00:00
if ( file_name . at ( 0 ) = = ' . ' )
2019-10-03 08:27:43 +00:00
continue ;
/// There are .sql.bak files - skip them.
2021-05-07 21:53:44 +00:00
if ( endsWith ( file_name , " .sql.bak " ) )
2019-10-03 08:27:43 +00:00
continue ;
2020-12-15 14:46:13 +00:00
/// Permanently detached table flag
2021-05-07 21:53:44 +00:00
if ( endsWith ( file_name , " .sql.detached " ) )
2020-11-30 17:52:32 +00:00
continue ;
2021-05-07 21:53:44 +00:00
if ( endsWith ( file_name , " .sql.tmp_drop " ) )
2019-10-03 08:27:43 +00:00
{
2020-01-23 19:10:09 +00:00
/// There are files that we tried to delete previously
2021-05-07 21:53:44 +00:00
metadata_files . emplace ( file_name , false ) ;
2019-10-03 08:27:43 +00:00
}
2021-05-07 21:53:44 +00:00
else if ( endsWith ( file_name , " .sql.tmp " ) )
2019-10-03 08:27:43 +00:00
{
2020-01-23 19:10:09 +00:00
/// There are files .sql.tmp - delete
2021-05-07 21:53:44 +00:00
LOG_INFO ( log , " Removing file {} " , dir_it - > path ( ) . string ( ) ) ;
2021-04-30 17:20:53 +00:00
fs : : remove ( dir_it - > path ( ) ) ;
2019-10-03 08:27:43 +00:00
}
2021-05-07 21:53:44 +00:00
else if ( endsWith ( file_name , " .sql " ) )
2019-10-03 08:27:43 +00:00
{
2020-01-23 19:10:09 +00:00
/// The required files have names like `table_name.sql`
2021-05-07 21:53:44 +00:00
metadata_files . emplace ( file_name , true ) ;
2019-10-03 08:27:43 +00:00
}
else
2021-05-07 21:53:44 +00:00
throw Exception ( ErrorCodes : : INCORRECT_FILE_NAME , " Incorrect file extension: {} in metadata directory {} " , file_name , getMetadataPath ( ) ) ;
2019-10-03 08:27:43 +00:00
}
2020-06-29 23:28:25 +00:00
/// Read and parse metadata in parallel
2020-07-16 23:12:47 +00:00
ThreadPool pool ;
2020-06-29 23:28:25 +00:00
for ( const auto & file : metadata_files )
{
pool . scheduleOrThrowOnError ( [ & ] ( )
{
if ( file . second )
process_metadata_file ( file . first ) ;
else
process_tmp_drop_metadata_file ( file . first ) ;
} ) ;
}
pool . wait ( ) ;
2019-10-03 08:27:43 +00:00
}
2021-04-10 23:33:54 +00:00
ASTPtr DatabaseOnDisk : : parseQueryFromMetadata (
Poco : : Logger * logger ,
ContextPtr local_context ,
const String & metadata_file_path ,
bool throw_on_error /*= true*/ ,
bool remove_empty /*= false*/ )
2019-11-06 16:05:04 +00:00
{
String query ;
2021-10-05 16:34:25 +00:00
int metadata_file_fd = : : open ( metadata_file_path . c_str ( ) , O_RDONLY | O_CLOEXEC ) ;
if ( metadata_file_fd = = - 1 )
2019-11-06 16:05:04 +00:00
{
2021-10-05 16:34:25 +00:00
if ( errno = = ENOENT & & ! throw_on_error )
2019-11-06 16:05:04 +00:00
return nullptr ;
2021-10-05 16:34:25 +00:00
throwFromErrnoWithPath ( " Cannot open file " + metadata_file_path , metadata_file_path ,
errno = = ENOENT ? ErrorCodes : : FILE_DOESNT_EXIST : ErrorCodes : : CANNOT_OPEN_FILE ) ;
2019-11-06 16:05:04 +00:00
}
2021-10-05 16:34:25 +00:00
ReadBufferFromFile in ( metadata_file_fd , metadata_file_path , METADATA_FILE_BUFFER_SIZE ) ;
readStringUntilEOF ( query , in ) ;
2019-12-02 19:11:18 +00:00
/** Empty files with metadata are generated after a rough restart of the server.
* Remove these files to slightly reduce the work of the admins on startup .
*/
if ( remove_empty & & query . empty ( ) )
{
2020-07-06 08:30:11 +00:00
if ( logger )
LOG_ERROR ( logger , " File {} is empty. Removing. " , metadata_file_path ) ;
2021-04-30 17:20:53 +00:00
fs : : remove ( metadata_file_path ) ;
2019-12-02 19:11:18 +00:00
return nullptr ;
}
2021-04-10 23:33:54 +00:00
auto settings = local_context - > getSettingsRef ( ) ;
2019-11-06 16:05:04 +00:00
ParserCreateQuery parser ;
const char * pos = query . data ( ) ;
std : : string error_message ;
auto ast = tryParseQuery ( parser , pos , pos + query . size ( ) , error_message , /* hilite = */ false ,
2020-03-19 21:14:52 +00:00
" in file " + metadata_file_path , /* allow_multi_statements = */ false , 0 , settings . max_parser_depth ) ;
2019-11-06 16:05:04 +00:00
if ( ! ast & & throw_on_error )
throw Exception ( error_message , ErrorCodes : : SYNTAX_ERROR ) ;
2019-12-02 19:11:18 +00:00
else if ( ! ast )
return nullptr ;
auto & create = ast - > as < ASTCreateQuery & > ( ) ;
2021-11-11 13:28:18 +00:00
if ( create . table & & create . uuid ! = UUIDHelpers : : Nil )
2019-12-02 19:11:18 +00:00
{
2021-05-09 11:59:49 +00:00
String table_name = unescapeForFileName ( fs : : path ( metadata_file_path ) . stem ( ) ) ;
2020-01-23 19:10:09 +00:00
2021-09-06 22:13:54 +00:00
if ( create . getTable ( ) ! = TABLE_WITH_UUID_NAME_PLACEHOLDER & & logger )
2021-04-10 23:33:54 +00:00
LOG_WARNING (
logger ,
" File {} contains both UUID and table name. Will use name `{}` instead of `{}` " ,
metadata_file_path ,
table_name ,
2021-09-13 14:18:04 +00:00
create . getTable ( ) ) ;
2021-09-06 22:13:54 +00:00
create . setTable ( table_name ) ;
2019-12-02 19:11:18 +00:00
}
2019-11-06 16:05:04 +00:00
return ast ;
}
2020-04-23 16:51:48 +00:00
ASTPtr DatabaseOnDisk : : getCreateQueryFromMetadata ( const String & database_metadata_path , bool throw_on_error ) const
2019-11-06 16:05:04 +00:00
{
2021-04-10 23:33:54 +00:00
ASTPtr ast = parseQueryFromMetadata ( log , getContext ( ) , database_metadata_path , throw_on_error ) ;
2019-11-06 16:05:04 +00:00
if ( ast )
{
auto & ast_create_query = ast - > as < ASTCreateQuery & > ( ) ;
ast_create_query . attach = false ;
2021-09-06 22:13:54 +00:00
ast_create_query . setDatabase ( getDatabaseName ( ) ) ;
2019-11-06 16:05:04 +00:00
}
return ast ;
}
2021-11-19 10:25:55 +00:00
ASTPtr DatabaseOnDisk : : getCreateQueryFromStorage ( const String & table_name , const StoragePtr & storage , bool throw_on_error ) const
{
auto metadata_ptr = storage - > getInMemoryMetadataPtr ( ) ;
if ( metadata_ptr = = nullptr )
{
if ( throw_on_error )
throw Exception ( ErrorCodes : : CANNOT_GET_CREATE_TABLE_QUERY , " Cannot get metadata of {}.{} " , backQuote ( getDatabaseName ( ) ) , backQuote ( table_name ) ) ;
else
return nullptr ;
}
/// setup create table query storage info.
auto ast_engine = std : : make_shared < ASTFunction > ( ) ;
ast_engine - > name = storage - > getName ( ) ;
auto ast_storage = std : : make_shared < ASTStorage > ( ) ;
ast_storage - > set ( ast_storage - > engine , ast_engine ) ;
auto create_table_query = DB : : getCreateQueryFromStorage ( storage , ast_storage , false ,
getContext ( ) - > getSettingsRef ( ) . max_parser_depth , throw_on_error ) ;
create_table_query - > set ( create_table_query - > as < ASTCreateQuery > ( ) - > comment ,
std : : make_shared < ASTLiteral > ( " SYSTEM TABLE is built on the fly. " ) ) ;
return create_table_query ;
}
2021-09-08 22:25:08 +00:00
void DatabaseOnDisk : : modifySettingsMetadata ( const SettingsChanges & settings_changes , ContextPtr query_context )
2021-09-04 22:59:44 +00:00
{
2021-09-08 22:25:08 +00:00
std : : lock_guard lock ( modify_settings_mutex ) ;
2021-09-04 22:59:44 +00:00
auto create_query = getCreateDatabaseQuery ( ) - > clone ( ) ;
auto * create = create_query - > as < ASTCreateQuery > ( ) ;
auto * settings = create - > storage - > settings ;
if ( settings )
{
auto & storage_settings = settings - > changes ;
for ( const auto & change : settings_changes )
{
auto it = std : : find_if ( storage_settings . begin ( ) , storage_settings . end ( ) ,
[ & ] ( const auto & prev ) { return prev . name = = change . name ; } ) ;
if ( it ! = storage_settings . end ( ) )
it - > value = change . value ;
else
storage_settings . push_back ( change ) ;
}
}
else
{
auto storage_settings = std : : make_shared < ASTSetQuery > ( ) ;
storage_settings - > is_standalone = false ;
storage_settings - > changes = settings_changes ;
create - > storage - > set ( create - > storage - > settings , storage_settings - > clone ( ) ) ;
}
create - > attach = true ;
create - > if_not_exists = false ;
WriteBufferFromOwnString statement_buf ;
formatAST ( * create , statement_buf , false ) ;
writeChar ( ' \n ' , statement_buf ) ;
String statement = statement_buf . str ( ) ;
String database_name_escaped = escapeForFileName ( database_name ) ;
2021-09-08 22:25:08 +00:00
fs : : path metadata_root_path = fs : : canonical ( query_context - > getGlobalContext ( ) - > getPath ( ) ) ;
2021-09-04 22:59:44 +00:00
fs : : path metadata_file_tmp_path = fs : : path ( metadata_root_path ) / " metadata " / ( database_name_escaped + " .sql.tmp " ) ;
fs : : path metadata_file_path = fs : : path ( metadata_root_path ) / " metadata " / ( database_name_escaped + " .sql " ) ;
WriteBufferFromFile out ( metadata_file_tmp_path , statement . size ( ) , O_WRONLY | O_CREAT | O_EXCL ) ;
writeString ( statement , out ) ;
out . next ( ) ;
if ( getContext ( ) - > getSettingsRef ( ) . fsync_metadata )
out . sync ( ) ;
out . close ( ) ;
fs : : rename ( metadata_file_tmp_path , metadata_file_path ) ;
}
2019-10-05 10:21:47 +00:00
}