2016-03-19 01:18:49 +00:00
# pragma once
2020-09-15 09:55:57 +00:00
# include <common/types.h>
2019-03-11 14:01:45 +00:00
# include <Parsers/IAST_fwd.h>
2019-05-17 14:34:25 +00:00
# include <Storages/IStorage_fwd.h>
2021-05-06 06:07:59 +00:00
# include <Interpreters/Context_fwd.h>
2019-08-27 20:43:08 +00:00
# include <Common/Exception.h>
2021-05-06 06:07:59 +00:00
# include <Core/UUID.h>
2021-04-10 23:33:54 +00:00
2019-03-11 14:01:45 +00:00
# include <ctime>
# include <functional>
# include <memory>
2021-05-06 06:07:59 +00:00
# include <mutex>
# include <vector>
# include <map>
2016-12-12 07:24:56 +00:00
2016-03-19 01:18:49 +00:00
namespace DB
{
2017-01-23 18:05:07 +00:00
struct Settings ;
2019-08-27 20:43:08 +00:00
struct ConstraintsDescription ;
struct IndicesDescription ;
2021-05-06 06:07:59 +00:00
struct StorageInMemoryMetadata ;
struct StorageID ;
2019-12-25 16:13:48 +00:00
class ASTCreateQuery ;
2020-07-15 19:25:31 +00:00
using DictionariesWithID = std : : vector < std : : pair < String , UUID > > ;
2019-08-27 20:43:08 +00:00
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED ;
2019-11-01 12:47:55 +00:00
extern const int CANNOT_GET_CREATE_TABLE_QUERY ;
2019-08-27 20:43:08 +00:00
}
2016-12-12 07:24:56 +00:00
2019-10-10 17:33:01 +00:00
class IDatabaseTablesIterator
2016-03-19 01:18:49 +00:00
{
public :
2017-04-01 07:20:54 +00:00
virtual void next ( ) = 0 ;
virtual bool isValid ( ) const = 0 ;
2016-03-19 01:18:49 +00:00
2017-04-01 07:20:54 +00:00
virtual const String & name ( ) const = 0 ;
2020-06-02 02:06:16 +00:00
/// This method can return nullptr if it's Lazy database
/// (a database with support for lazy tables loading
/// - it maintains a list of tables but tables are loaded lazily).
2019-09-26 07:16:31 +00:00
virtual const StoragePtr & table ( ) const = 0 ;
2016-03-19 01:18:49 +00:00
2019-10-10 17:33:01 +00:00
virtual ~ IDatabaseTablesIterator ( ) = default ;
2020-04-01 22:41:29 +00:00
virtual UUID uuid ( ) const { return UUIDHelpers : : Nil ; }
2020-07-07 12:11:58 +00:00
const String & databaseName ( ) const { assert ( ! database_name . empty ( ) ) ; return database_name ; }
protected :
String database_name ;
2019-10-10 17:33:01 +00:00
} ;
/// Copies list of tables and iterates through such snapshot.
2020-04-01 22:41:29 +00:00
class DatabaseTablesSnapshotIterator : public IDatabaseTablesIterator
2019-10-10 17:33:01 +00:00
{
private :
Tables tables ;
Tables : : iterator it ;
2020-04-01 22:41:29 +00:00
protected :
DatabaseTablesSnapshotIterator ( DatabaseTablesSnapshotIterator & & other )
{
size_t idx = std : : distance ( other . tables . begin ( ) , other . it ) ;
std : : swap ( tables , other . tables ) ;
other . it = other . tables . end ( ) ;
it = tables . begin ( ) ;
std : : advance ( it , idx ) ;
2020-07-07 12:11:58 +00:00
database_name = std : : move ( other . database_name ) ;
2020-04-01 22:41:29 +00:00
}
2019-10-10 17:33:01 +00:00
public :
2020-07-07 12:11:58 +00:00
DatabaseTablesSnapshotIterator ( const Tables & tables_ , const String & database_name_ )
: tables ( tables_ ) , it ( tables . begin ( ) )
{
database_name = database_name_ ;
}
2019-10-10 17:33:01 +00:00
2020-07-07 12:11:58 +00:00
DatabaseTablesSnapshotIterator ( Tables & & tables_ , String & & database_name_ )
: tables ( std : : move ( tables_ ) ) , it ( tables . begin ( ) )
{
database_name = std : : move ( database_name_ ) ;
}
2019-10-10 17:33:01 +00:00
2020-01-21 08:54:26 +00:00
void next ( ) override { + + it ; }
2019-10-10 17:33:01 +00:00
2020-01-21 08:54:26 +00:00
bool isValid ( ) const override { return it ! = tables . end ( ) ; }
2019-10-10 17:33:01 +00:00
2020-01-21 08:54:26 +00:00
const String & name ( ) const override { return it - > first ; }
2019-10-10 17:33:01 +00:00
2020-01-21 08:54:26 +00:00
const StoragePtr & table ( ) const override { return it - > second ; }
2016-03-19 01:18:49 +00:00
} ;
2019-10-10 17:33:01 +00:00
using DatabaseTablesIteratorPtr = std : : unique_ptr < IDatabaseTablesIterator > ;
2016-03-19 01:18:49 +00:00
2017-01-23 18:05:07 +00:00
/** Database engine.
* It is responsible for :
2019-10-10 17:33:01 +00:00
* - initialization of set of known tables and dictionaries ;
2017-01-23 18:05:07 +00:00
* - checking existence of a table and getting a table object ;
* - retrieving a list of all tables ;
* - creating and dropping tables ;
* - renaming tables and moving between databases with same engine .
2016-03-19 01:18:49 +00:00
*/
class IDatabase : public std : : enable_shared_from_this < IDatabase >
{
public :
2019-11-01 12:47:55 +00:00
IDatabase ( ) = delete ;
IDatabase ( String database_name_ ) : database_name ( std : : move ( database_name_ ) ) { }
2017-04-01 07:20:54 +00:00
/// Get name of database engine.
virtual String getEngineName ( ) const = 0 ;
2016-03-19 01:18:49 +00:00
2020-10-15 15:57:17 +00:00
virtual bool canContainMergeTreeTables ( ) const { return true ; }
virtual bool canContainDistributedTables ( ) const { return true ; }
2019-07-19 14:22:57 +00:00
/// Load a set of existing tables.
2017-04-17 11:56:55 +00:00
/// You can call only once, right after the object is created.
2021-05-31 14:49:02 +00:00
virtual void loadStoredObjects ( ContextMutablePtr /*context*/ , bool /*has_force_restore_data_flag*/ , bool /*force_attach*/ = false ) { }
2016-06-10 20:46:58 +00:00
2017-04-17 11:56:55 +00:00
/// Check the existence of the table.
2021-04-10 23:33:54 +00:00
virtual bool isTableExist ( const String & name , ContextPtr context ) const = 0 ;
2016-03-19 01:18:49 +00:00
2017-04-17 11:56:55 +00:00
/// Get the table for work. Return nullptr if there is no table.
2021-04-10 23:33:54 +00:00
virtual StoragePtr tryGetTable ( const String & name , ContextPtr context ) const = 0 ;
2016-03-19 01:18:49 +00:00
2020-04-13 14:09:56 +00:00
virtual UUID tryGetTableUUID ( const String & /*table_name*/ ) const { return UUIDHelpers : : Nil ; }
2019-06-02 12:11:01 +00:00
using FilterByNameFunction = std : : function < bool ( const String & ) > ;
2017-04-17 11:56:55 +00:00
/// Get an iterator that allows you to pass through all the tables.
/// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above.
2021-04-10 23:33:54 +00:00
virtual DatabaseTablesIteratorPtr getTablesIterator ( ContextPtr context , const FilterByNameFunction & filter_by_table_name = { } ) = 0 ;
2019-10-10 17:33:01 +00:00
2017-04-17 11:56:55 +00:00
/// Is the database empty.
2020-04-23 16:51:48 +00:00
virtual bool empty ( ) const = 0 ;
2016-03-19 01:18:49 +00:00
2017-04-17 11:56:55 +00:00
/// Add the table to the database. Record its presence in the metadata.
2017-04-01 07:20:54 +00:00
virtual void createTable (
2021-04-10 23:33:54 +00:00
ContextPtr /*context*/ ,
2019-11-01 12:47:55 +00:00
const String & /*name*/ ,
const StoragePtr & /*table*/ ,
const ASTPtr & /*query*/ )
{
throw Exception ( " There is no CREATE TABLE query for Database " + getEngineName ( ) , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2016-03-19 01:18:49 +00:00
2020-01-22 11:30:11 +00:00
/// Delete the table from the database, drop table and delete the metadata.
virtual void dropTable (
2021-04-10 23:33:54 +00:00
ContextPtr /*context*/ ,
2020-03-20 00:07:52 +00:00
const String & /*name*/ ,
[[maybe_unused]] bool no_delay = false )
2019-11-01 12:47:55 +00:00
{
throw Exception ( " There is no DROP TABLE query for Database " + getEngineName ( ) , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2016-03-19 01:18:49 +00:00
2017-04-17 11:56:55 +00:00
/// Add a table to the database, but do not add it to the metadata. The database may not support this method.
2020-11-30 17:52:32 +00:00
///
/// Note: ATTACH TABLE statement actually uses createTable method.
2019-11-11 11:34:03 +00:00
virtual void attachTable ( const String & /*name*/ , const StoragePtr & /*table*/ , [[maybe_unused]] const String & relative_table_path = { } )
2019-11-01 12:47:55 +00:00
{
throw Exception ( " There is no ATTACH TABLE query for Database " + getEngineName ( ) , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2016-03-19 01:18:49 +00:00
2017-04-17 11:56:55 +00:00
/// Forget about the table without deleting it, and return it. The database may not support this method.
2019-11-01 12:47:55 +00:00
virtual StoragePtr detachTable ( const String & /*name*/ )
{
throw Exception ( " There is no DETACH TABLE query for Database " + getEngineName ( ) , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2016-03-19 01:18:49 +00:00
2020-11-30 17:52:32 +00:00
/// Forget about the table without deleting it's data, but rename metadata file to prevent reloading it
/// with next restart. The database may not support this method.
2021-04-10 23:33:54 +00:00
virtual void detachTablePermanently ( ContextPtr /*context*/ , const String & /*name*/ )
2020-11-30 17:52:32 +00:00
{
throw Exception ( " There is no DETACH TABLE PERMANENTLY query for Database " + getEngineName ( ) , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2017-04-17 11:56:55 +00:00
/// Rename the table and possibly move the table to another database.
2017-04-01 07:20:54 +00:00
virtual void renameTable (
2021-04-10 23:33:54 +00:00
ContextPtr /*context*/ ,
2019-08-27 20:43:08 +00:00
const String & /*name*/ ,
IDatabase & /*to_database*/ ,
2020-03-31 20:38:05 +00:00
const String & /*to_name*/ ,
2020-07-16 21:41:26 +00:00
bool /*exchange*/ ,
bool /*dictionary*/ )
2019-08-27 20:43:08 +00:00
{
throw Exception ( getEngineName ( ) + " : renameTable() is not supported " , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2016-09-02 13:04:11 +00:00
2017-09-17 18:49:43 +00:00
using ASTModifier = std : : function < void ( IAST & ) > ;
2016-05-13 21:08:19 +00:00
2017-04-17 11:56:55 +00:00
/// Change the table structure in metadata.
2020-06-22 09:49:21 +00:00
/// You must call under the alter_lock of the corresponding table . If engine_modifier is empty, then engine does not change.
2017-04-01 07:20:54 +00:00
virtual void alterTable (
2021-04-10 23:33:54 +00:00
ContextPtr /*context*/ ,
2020-03-20 12:45:06 +00:00
const StorageID & /*table_id*/ ,
2019-12-26 18:17:05 +00:00
const StorageInMemoryMetadata & /*metadata*/ )
2019-08-27 20:43:08 +00:00
{
2019-10-23 13:46:38 +00:00
throw Exception ( getEngineName ( ) + " : alterTable() is not supported " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2019-08-27 20:43:08 +00:00
}
2016-05-13 21:08:19 +00:00
2017-09-11 12:39:01 +00:00
/// Returns time of table's metadata change, 0 if there is no corresponding metadata file.
2019-11-06 16:05:04 +00:00
virtual time_t getObjectMetadataModificationTime ( const String & /*name*/ ) const
2019-11-01 12:47:55 +00:00
{
return static_cast < time_t > ( 0 ) ;
}
2017-09-11 12:39:01 +00:00
2017-04-17 11:56:55 +00:00
/// Get the CREATE TABLE query for the table. It can also provide information for detached tables for which there is metadata.
2021-04-10 23:33:54 +00:00
ASTPtr tryGetCreateTableQuery ( const String & name , ContextPtr context ) const noexcept
2019-11-01 12:47:55 +00:00
{
2020-05-28 20:10:45 +00:00
return getCreateTableQueryImpl ( name , context , false ) ;
2019-11-01 12:47:55 +00:00
}
2016-03-19 01:18:49 +00:00
2021-04-10 23:33:54 +00:00
ASTPtr getCreateTableQuery ( const String & name , ContextPtr context ) const
2018-03-23 19:56:24 +00:00
{
2020-05-28 20:10:45 +00:00
return getCreateTableQueryImpl ( name , context , true ) ;
2018-03-23 19:56:24 +00:00
}
2018-03-14 19:56:44 +00:00
2018-03-13 13:28:32 +00:00
/// Get the CREATE DATABASE query for current database.
2020-04-23 16:51:48 +00:00
virtual ASTPtr getCreateDatabaseQuery ( ) const = 0 ;
2016-03-19 01:18:49 +00:00
2018-05-21 03:01:38 +00:00
/// Get name of database.
2020-07-07 12:11:58 +00:00
String getDatabaseName ( ) const
{
std : : lock_guard lock { mutex } ;
return database_name ;
}
2020-07-02 20:39:31 +00:00
/// Get UUID of database.
virtual UUID getUUID ( ) const { return UUIDHelpers : : Nil ; }
2020-07-07 12:11:58 +00:00
virtual void renameDatabase ( const String & /*new_name*/ )
{
throw Exception ( getEngineName ( ) + " : RENAME DATABASE is not supported " , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2017-11-03 19:53:10 +00:00
/// Returns path for persistent data storage if the database supports it, empty string otherwise
2018-02-21 19:26:59 +00:00
virtual String getDataPath ( ) const { return { } ; }
2019-12-25 21:17:49 +00:00
2019-12-25 16:13:48 +00:00
/// Returns path for persistent data storage for table if the database supports it, empty string otherwise. Table must exist
virtual String getTableDataPath ( const String & /*table_name*/ ) const { return { } ; }
2019-11-11 11:34:03 +00:00
/// Returns path for persistent data storage for CREATE/ATTACH query if the database supports it, empty string otherwise
2019-12-25 16:13:48 +00:00
virtual String getTableDataPath ( const ASTCreateQuery & /*query*/ ) const { return { } ; }
2018-02-21 19:26:59 +00:00
/// Returns metadata path if the database supports it, empty string otherwise
virtual String getMetadataPath ( ) const { return { } ; }
/// Returns metadata path of a concrete table if the database supports it, empty string otherwise
2019-10-10 17:33:01 +00:00
virtual String getObjectMetadataPath ( const String & /*table_name*/ ) const { return { } ; }
2017-11-03 19:53:10 +00:00
2020-04-12 20:50:32 +00:00
/// All tables and dictionaries should be detached before detaching the database.
virtual bool shouldBeEmptyOnDetach ( ) const { return true ; }
2020-09-14 19:25:02 +00:00
virtual void assertCanBeDetached ( bool /*cleanup*/ ) { }
2021-04-20 11:22:02 +00:00
virtual void waitDetachedTableNotInUse ( const UUID & /*uuid*/ ) { }
virtual void checkDetachedTableNotInUse ( const UUID & /*uuid*/ ) { }
2020-11-24 12:28:54 +00:00
2017-04-17 11:56:55 +00:00
/// Ask all tables to complete the background threads they are using and delete all table objects.
2017-04-01 07:20:54 +00:00
virtual void shutdown ( ) = 0 ;
2016-03-19 01:18:49 +00:00
2018-06-09 15:48:22 +00:00
/// Delete data and metadata stored inside the database, if exists.
2021-04-10 23:33:54 +00:00
virtual void drop ( ContextPtr /*context*/ ) { }
2016-03-28 11:19:14 +00:00
2021-04-10 23:33:54 +00:00
virtual ~ IDatabase ( ) = default ;
2019-11-01 12:47:55 +00:00
protected :
2021-04-10 23:33:54 +00:00
virtual ASTPtr getCreateTableQueryImpl ( const String & /*name*/ , ContextPtr /*context*/ , bool throw_on_error ) const
2019-11-01 12:47:55 +00:00
{
if ( throw_on_error )
throw Exception ( " There is no SHOW CREATE TABLE query for Database " + getEngineName ( ) , ErrorCodes : : CANNOT_GET_CREATE_TABLE_QUERY ) ;
return nullptr ;
}
2020-07-07 12:11:58 +00:00
mutable std : : mutex mutex ;
2019-11-01 12:47:55 +00:00
String database_name ;
2016-03-19 01:18:49 +00:00
} ;
using DatabasePtr = std : : shared_ptr < IDatabase > ;
using Databases = std : : map < String , DatabasePtr > ;
}