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>
2019-10-02 10:10:45 +00:00
# include <Parsers/ASTCreateQuery.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>
2019-10-02 10:10:45 +00:00
# include <common/logger_useful.h>
2019-10-03 09:31:59 +00:00
# include <Poco/DirectoryIterator.h>
2019-10-02 10:10:45 +00:00
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>
2020-03-16 11:38:50 +00:00
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 ;
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 ;
2019-10-10 17:33:01 +00:00
extern const int DICTIONARY_ALREADY_EXISTS ;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED ;
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 ,
2019-10-10 17:33:01 +00:00
Context & context ,
bool has_force_restore_data_flag )
{
ast_create_query . attach = true ;
ast_create_query . database = database_name ;
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 )
columns = InterpreterCreateQuery : : getColumnsDescription ( * ast_create_query . columns_list - > columns , context , false ) ;
StoragePtr storage = table_function - > execute ( ast_create_query . as_table_function , context , ast_create_query . table , std : : move ( columns ) ) ;
2020-04-07 14:05:51 +00:00
storage - > renameInMemory ( ast_create_query ) ;
2019-10-10 17:33:01 +00:00
return { ast_create_query . table , storage } ;
}
/// We do not directly use `InterpreterCreateQuery::execute`, because
2019-12-02 19:11:18 +00:00
/// - the database has not been loaded yet;
2019-10-10 17:33:01 +00:00
/// - 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 )
throw Exception ( " Missing definition of columns. " , ErrorCodes : : EMPTY_LIST_OF_COLUMNS_PASSED ) ;
2020-05-04 00:11:49 +00:00
ColumnsDescription columns = InterpreterCreateQuery : : getColumnsDescription ( * ast_create_query . columns_list - > columns , context , false ) ;
2019-10-10 17:33:01 +00:00
ConstraintsDescription constraints = InterpreterCreateQuery : : getConstraintsDescription ( ast_create_query . columns_list - > constraints ) ;
return
{
ast_create_query . table ,
StorageFactory : : instance ( ) . get (
ast_create_query ,
2019-12-04 16:06:55 +00:00
table_data_path_relative ,
context ,
context . getGlobalContext ( ) ,
columns ,
constraints ,
has_force_restore_data_flag )
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 ) ;
2019-10-22 10:47:43 +00:00
create - > database . clear ( ) ;
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 )
2019-12-02 19:11:18 +00:00
create - > table = TABLE_WITH_UUID_NAME_PLACEHOLDER ;
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
void applyMetadataChangesToCreateQuery ( const ASTPtr & query , const StorageInMemoryMetadata & metadata )
{
auto & ast_create_query = query - > as < ASTCreateQuery & > ( ) ;
bool has_structure = ast_create_query . columns_list & & ast_create_query . columns_list - > columns ;
if ( ast_create_query . as_table_function & & ! has_structure )
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Cannot alter table {} because it was created AS table function "
" and doesn't have structure in metadata " , backQuote ( ast_create_query . table ) ) ;
assert ( has_structure ) ;
ASTPtr new_columns = InterpreterCreateQuery : : formatColumns ( metadata . columns ) ;
ASTPtr new_indices = InterpreterCreateQuery : : formatIndices ( metadata . secondary_indices ) ;
ASTPtr new_constraints = InterpreterCreateQuery : : formatConstraints ( metadata . constraints ) ;
ast_create_query . columns_list - > replace ( ast_create_query . columns_list - > columns , new_columns ) ;
ast_create_query . columns_list - > setOrReplace ( ast_create_query . columns_list - > indices , new_indices ) ;
ast_create_query . columns_list - > setOrReplace ( ast_create_query . columns_list - > constraints , new_constraints ) ;
if ( metadata . select . select_query )
{
query - > replace ( ast_create_query . select , metadata . select . select_query ) ;
}
/// MaterializedView is one type of CREATE query without storage.
if ( ast_create_query . storage )
{
ASTStorage & storage_ast = * ast_create_query . storage ;
bool is_extended_storage_def
= storage_ast . partition_by | | storage_ast . primary_key | | storage_ast . order_by | | storage_ast . sample_by | | storage_ast . settings ;
if ( is_extended_storage_def )
{
if ( metadata . sorting_key . definition_ast )
storage_ast . set ( storage_ast . order_by , metadata . sorting_key . definition_ast ) ;
if ( metadata . primary_key . definition_ast )
storage_ast . set ( storage_ast . primary_key , metadata . primary_key . definition_ast ) ;
if ( metadata . sampling_key . definition_ast )
storage_ast . set ( storage_ast . sample_by , metadata . sampling_key . definition_ast ) ;
if ( metadata . table_ttl . definition_ast )
storage_ast . set ( storage_ast . ttl_table , metadata . table_ttl . definition_ast ) ;
else if ( storage_ast . ttl_table ! = nullptr ) /// TTL was removed
storage_ast . ttl_table = nullptr ;
if ( metadata . settings_changes )
storage_ast . set ( storage_ast . settings , metadata . settings_changes ) ;
}
}
}
2020-06-11 03:24:52 +00:00
DatabaseOnDisk : : DatabaseOnDisk (
const String & name ,
const String & metadata_path_ ,
const String & data_path_ ,
const String & logger ,
2020-11-27 14:04:03 +00:00
const Context & context )
2020-05-28 20:10:45 +00:00
: DatabaseWithOwnTablesBase ( name , logger , 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
{
2020-04-06 23:22:44 +00:00
Poco : : File ( context . getPath ( ) + data_path ) . createDirectories ( ) ;
Poco : : File ( metadata_path ) . createDirectories ( ) ;
2019-11-28 19:40:51 +00:00
}
2019-10-03 07:59:48 +00:00
void DatabaseOnDisk : : createTable (
const Context & context ,
const String & table_name ,
const StoragePtr & table ,
const ASTPtr & query )
{
const auto & settings = context . getSettingsRef ( ) ;
2020-03-23 00:12:13 +00:00
const auto & create = query - > as < ASTCreateQuery & > ( ) ;
2020-07-07 12:11:58 +00:00
assert ( table_name = = create . table ) ;
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.
2020-04-23 16:51:48 +00:00
if ( isDictionaryExist ( table_name ) )
2020-12-15 14:46:13 +00:00
throw Exception ( ErrorCodes : : DICTIONARY_ALREADY_EXISTS , " Dictionary {}.{} already exists " , backQuote ( getDatabaseName ( ) ) , backQuote ( table_name ) ) ;
2019-10-10 17:33:01 +00:00
2020-05-28 20:10:45 +00:00
if ( isTableExist ( table_name , global_context ) )
2020-12-15 14:46:13 +00:00
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
attachTable ( table_name , table , getTableDataPath ( create ) ) ;
2020-12-15 17:06:23 +00:00
removeDetachedPermanentlyFlag ( table_name , table_metadata_path ) ;
2020-03-23 00:12:13 +00:00
return ;
}
2020-12-15 17:06:23 +00:00
if ( ! create . attach )
checkMetadataFilenameAvailability ( table_name ) ;
if ( create . attach & & Poco : : File ( table_metadata_path ) . exists ( ) )
{
ASTPtr ast_detached = parseQueryFromMetadata ( log , context , table_metadata_path ) ;
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 ( ) ;
}
2020-11-20 16:06:27 +00:00
commitCreateTable ( create , table , table_metadata_tmp_path , table_metadata_path , context ) ;
2020-12-15 17:06:23 +00:00
removeDetachedPermanentlyFlag ( table_name , table_metadata_path ) ;
}
/// 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
void DatabaseOnDisk : : removeDetachedPermanentlyFlag ( const String & table_name , const String & table_metadata_path ) const
{
try
{
auto detached_permanently_flag = Poco : : File ( table_metadata_path + detached_suffix ) ;
if ( detached_permanently_flag . exists ( ) )
detached_permanently_flag . remove ( ) ;
}
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 ,
const Context & /*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.
2020-03-23 00:12:13 +00:00
attachTable ( query . table , 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.
Poco : : File ( table_metadata_tmp_path ) . renameTo ( table_metadata_path ) ;
}
catch ( . . . )
{
Poco : : File ( table_metadata_tmp_path ) . remove ( ) ;
throw ;
}
}
2021-02-10 20:30:40 +00:00
void DatabaseOnDisk : : detachTablePermanently ( const Context & , const String & table_name )
2020-11-30 17:52:32 +00:00
{
2020-12-14 14:21:31 +00:00
auto table = detachTable ( table_name ) ;
2020-11-30 17:52:32 +00:00
2020-12-15 14:46:13 +00:00
Poco : : File detached_permanently_flag ( getObjectMetadataPath ( table_name ) + detached_suffix ) ;
2020-11-30 17:52:32 +00:00
try
{
2020-12-15 14:46:13 +00:00
detached_permanently_flag . createFile ( ) ;
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
}
2020-03-20 00:07:52 +00:00
void DatabaseOnDisk : : dropTable ( const Context & 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
2020-01-22 11:30:11 +00:00
StoragePtr table = detachTable ( 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
{
2020-01-22 11:30:11 +00:00
Poco : : File ( table_metadata_path ) . renameTo ( table_metadata_path_drop ) ;
renamed = true ;
table - > drop ( ) ;
table - > is_dropped = true ;
Poco : : File table_data_dir { context . getPath ( ) + table_data_path_relative } ;
if ( table_data_dir . exists ( ) )
table_data_dir . remove ( true ) ;
2019-10-03 07:59:48 +00:00
}
catch ( . . . )
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , getCurrentExceptionMessage ( __PRETTY_FUNCTION__ ) ) ;
2020-01-22 11:30:11 +00:00
attachTable ( table_name , table , table_data_path_relative ) ;
if ( renamed )
Poco : : File ( table_metadata_path_drop ) . renameTo ( table_metadata_path ) ;
2019-10-03 07:59:48 +00:00
throw ;
}
2020-01-22 11:30:11 +00:00
Poco : : File ( table_metadata_path_drop ) . remove ( ) ;
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
2020-12-15 14:46:13 +00:00
if ( Poco : : File ( table_metadata_path ) . exists ( ) )
2020-12-08 10:00:32 +00:00
{
2020-12-15 14:46:13 +00:00
auto detached_permanently_flag = Poco : : File ( table_metadata_path + detached_suffix ) ;
2020-12-08 10:00:32 +00:00
2020-12-15 14:46:13 +00:00
if ( detached_permanently_flag . exists ( ) )
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 (
const Context & context ,
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
{
2020-03-31 20:38:05 +00:00
if ( exchange )
throw Exception ( " Tables can be exchanged only in Atomic databases " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2020-07-16 21:41:26 +00:00
if ( dictionary )
throw Exception ( " Dictionaries can be renamed only in Atomic databases " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2020-03-31 20:38:05 +00:00
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 ;
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.
2020-05-28 20:10:45 +00:00
StoragePtr table = tryGetTable ( table_name , global_context ) ;
2020-03-18 17:38:52 +00:00
detachTable ( table_name ) ;
2019-10-10 17:33:01 +00:00
try
{
2020-04-12 13:35:44 +00:00
table_lock = table - > lockExclusively ( context . getCurrentQueryId ( ) , context . getSettingsRef ( ) . lock_acquire_timeout ) ;
2020-01-22 11:30:11 +00:00
table_metadata_path = getObjectMetadataPath ( table_name ) ;
2020-03-19 21:14:52 +00:00
attach_query = parseQueryFromMetadata ( log , context , table_metadata_path ) ;
2020-01-22 11:30:11 +00:00
auto & create = attach_query - > as < ASTCreateQuery & > ( ) ;
2020-03-23 22:40:40 +00:00
create . database = to_database . getDatabaseName ( ) ;
2020-01-22 11:30:11 +00:00
create . table = to_table_name ;
if ( from_ordinary_to_atomic )
create . uuid = UUIDHelpers : : generateV4 ( ) ;
if ( from_atomic_to_ordinary )
create . uuid = UUIDHelpers : : Nil ;
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
{
2020-01-22 11:30:11 +00:00
attachTable ( 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 )
{
2020-01-22 11:30:11 +00:00
attachTable ( 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
to_database . createTable ( context , to_table_name , table , attach_query ) ;
Poco : : File ( table_metadata_path ) . remove ( ) ;
2020-07-06 08:30:11 +00:00
/// Special case: usually no actions with symlinks are required when detaching/attaching table,
/// but not when moving from Atomic database to Ordinary
2020-11-01 17:38:43 +00:00
if ( from_atomic_to_ordinary & & table - > storesDataOnDisk ( ) )
2020-07-06 08:30:11 +00:00
{
auto & atomic_db = assert_cast < DatabaseAtomic & > ( * this ) ;
atomic_db . tryRemoveSymlink ( table_name ) ;
}
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)
2020-05-28 20:10:45 +00:00
ASTPtr DatabaseOnDisk : : getCreateTableQueryImpl ( const String & table_name , const Context & , bool throw_on_error ) const
2019-10-03 07:59:48 +00:00
{
ASTPtr ast ;
2020-05-28 20:10:45 +00:00
bool has_table = tryGetTable ( table_name , global_context ) ! = nullptr ;
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 } ;
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 ;
2020-04-23 16:51:48 +00:00
auto settings = global_context . getSettingsRef ( ) ;
2020-07-07 12:11:58 +00:00
{
std : : lock_guard lock ( mutex ) ;
auto database_metadata_path = global_context . getPath ( ) + " metadata/ " + escapeForFileName ( database_name ) + " .sql " ;
ast = parseQueryFromMetadata ( log , global_context , database_metadata_path , true ) ;
auto & ast_create_query = ast - > as < ASTCreateQuery & > ( ) ;
ast_create_query . attach = false ;
ast_create_query . database = database_name ;
}
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
}
return ast ;
}
2019-11-06 16:05:04 +00:00
void DatabaseOnDisk : : drop ( const Context & context )
2019-10-03 07:59:48 +00:00
{
2020-08-18 15:15:27 +00:00
assert ( tables . empty ( ) ) ;
2019-11-06 16:05:04 +00:00
Poco : : File ( context . getPath ( ) + getDataPath ( ) ) . remove ( false ) ;
Poco : : File ( getMetadataPath ( ) ) . remove ( false ) ;
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
Poco : : File meta_file ( table_metadata_path ) ;
if ( meta_file . exists ( ) )
return meta_file . getLastModified ( ) . epochTime ( ) ;
else
return static_cast < time_t > ( 0 ) ;
}
2020-04-06 23:22:44 +00:00
void DatabaseOnDisk : : iterateMetadataFiles ( const Context & 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 ) ) ;
if ( Poco : : File ( context . getPath ( ) + getDataPath ( ) + ' / ' + object_name ) . exists ( ) )
{
2020-01-28 19:39:52 +00:00
Poco : : File ( getMetadataPath ( ) + file_name ) . renameTo ( 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 ) ;
2020-01-23 19:10:09 +00:00
Poco : : File ( getMetadataPath ( ) + file_name ) . remove ( ) ;
}
} ;
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 ;
2019-10-03 08:27:43 +00:00
Poco : : DirectoryIterator dir_end ;
2019-11-06 16:05:04 +00:00
for ( Poco : : DirectoryIterator dir_it ( getMetadataPath ( ) ) ; dir_it ! = dir_end ; + + dir_it )
2019-10-03 08:27:43 +00:00
{
/// For '.svn', '.gitignore' directory and similar.
if ( dir_it . name ( ) . at ( 0 ) = = ' . ' )
continue ;
/// There are .sql.bak files - skip them.
if ( endsWith ( dir_it . name ( ) , " .sql.bak " ) )
continue ;
2020-12-15 14:46:13 +00:00
/// Permanently detached table flag
if ( endsWith ( dir_it . name ( ) , " .sql.detached " ) )
2020-11-30 17:52:32 +00:00
continue ;
2020-12-15 14:46:13 +00:00
if ( endsWith ( dir_it . 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
2020-06-29 23:28:25 +00:00
metadata_files . emplace ( dir_it . name ( ) , false ) ;
2019-10-03 08:27:43 +00:00
}
2020-01-23 19:10:09 +00:00
else if ( endsWith ( dir_it . 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
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Removing file {} " , dir_it - > path ( ) ) ;
2019-10-03 08:27:43 +00:00
Poco : : File ( dir_it - > path ( ) ) . remove ( ) ;
}
2020-01-23 19:10:09 +00:00
else if ( endsWith ( dir_it . 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`
2020-06-29 23:28:25 +00:00
metadata_files . emplace ( dir_it . name ( ) , true ) ;
2019-10-03 08:27:43 +00:00
}
else
2019-11-06 16:05:04 +00:00
throw Exception ( " Incorrect file extension: " + dir_it . name ( ) + " in metadata directory " + getMetadataPath ( ) ,
2019-10-03 08:27:43 +00:00
ErrorCodes : : INCORRECT_FILE_NAME ) ;
}
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
}
2020-06-27 19:05:00 +00:00
ASTPtr DatabaseOnDisk : : parseQueryFromMetadata ( Poco : : Logger * logger , const Context & 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 ;
try
{
2019-12-02 19:11:18 +00:00
ReadBufferFromFile in ( metadata_file_path , METADATA_FILE_BUFFER_SIZE ) ;
2019-11-06 16:05:04 +00:00
readStringUntilEOF ( query , in ) ;
}
catch ( const Exception & e )
{
if ( ! throw_on_error & & e . code ( ) = = ErrorCodes : : FILE_DOESNT_EXIST )
return nullptr ;
else
throw ;
}
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 ) ;
2019-12-02 19:11:18 +00:00
Poco : : File ( metadata_file_path ) . remove ( ) ;
return nullptr ;
}
2020-01-14 11:11:01 +00:00
auto settings = 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 & > ( ) ;
2020-07-02 20:39:31 +00:00
if ( ! create . table . empty ( ) & & create . uuid ! = UUIDHelpers : : Nil )
2019-12-02 19:11:18 +00:00
{
String table_name = Poco : : Path ( metadata_file_path ) . makeFile ( ) . getBaseName ( ) ;
2020-03-18 17:38:52 +00:00
table_name = unescapeForFileName ( table_name ) ;
2020-01-23 19:10:09 +00:00
2020-07-06 08:30:11 +00:00
if ( create . table ! = TABLE_WITH_UUID_NAME_PLACEHOLDER & & logger )
2020-06-27 19:05:00 +00:00
LOG_WARNING ( logger , " File {} contains both UUID and table name. Will use name `{}` instead of `{}` " , metadata_file_path , table_name , create . table ) ;
2019-12-02 19:11:18 +00:00
create . table = table_name ;
}
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
{
2020-04-23 16:51:48 +00:00
ASTPtr ast = parseQueryFromMetadata ( log , global_context , 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 ;
2020-07-07 12:11:58 +00:00
ast_create_query . database = getDatabaseName ( ) ;
2019-11-06 16:05:04 +00:00
}
return ast ;
}
2019-10-05 10:21:47 +00:00
}