2011-08-15 01:12:57 +00:00
# pragma once
2010-03-04 19:20:28 +00:00
2018-11-19 14:31:16 +00:00
# include <Core/Names.h>
2017-04-01 09:19:00 +00:00
# include <Core/QueryProcessingStage.h>
2018-10-14 15:30:06 +00:00
# include <Databases/IDatabase.h>
2019-02-04 13:04:02 +00:00
# include <Interpreters/CancellationCode.h>
2021-04-10 23:33:54 +00:00
# include <Interpreters/Context_fwd.h>
2020-03-13 10:30:55 +00:00
# include <Interpreters/StorageID.h>
2019-07-03 13:17:19 +00:00
# include <Storages/CheckResults.h>
2020-02-17 20:39:24 +00:00
# include <Storages/ColumnDependency.h>
2021-04-10 23:33:54 +00:00
# include <Storages/IStorage_fwd.h>
2020-06-05 11:54:54 +00:00
# include <Storages/SelectQueryDescription.h>
2021-04-10 23:33:54 +00:00
# include <Storages/StorageInMemoryMetadata.h>
# include <Storages/TableLockHolder.h>
2021-07-09 03:15:41 +00:00
# include <Storages/StorageSnapshot.h>
2018-05-21 13:49:54 +00:00
# include <Common/ActionLock.h>
2019-05-17 14:34:25 +00:00
# include <Common/Exception.h>
# include <Common/RWLock.h>
2019-10-17 11:12:35 +00:00
# include <Common/TypePromotion.h>
2019-05-17 14:34:25 +00:00
# include <optional>
2021-12-09 10:39:28 +00:00
# include <compare>
2013-01-23 17:38:03 +00:00
2010-03-01 16:59:51 +00:00
namespace DB
{
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
2018-11-22 21:19:58 +00:00
extern const int NOT_IMPLEMENTED ;
2016-01-11 21:46:36 +00:00
}
2018-05-21 13:49:54 +00:00
using StorageActionBlockType = size_t ;
2017-09-17 18:49:43 +00:00
class ASTCreateQuery ;
2021-04-04 10:27:45 +00:00
class ASTInsertQuery ;
2010-03-01 16:59:51 +00:00
2017-01-21 04:24:28 +00:00
struct Settings ;
class AlterCommands ;
2018-06-13 20:02:27 +00:00
class MutationCommands ;
2020-03-17 13:49:50 +00:00
struct PartitionCommand ;
using PartitionCommands = std : : vector < PartitionCommand > ;
2017-01-21 04:24:28 +00:00
2019-09-13 12:59:48 +00:00
class IProcessor ;
using ProcessorPtr = std : : shared_ptr < IProcessor > ;
using Processors = std : : vector < ProcessorPtr > ;
2019-10-20 09:12:42 +00:00
class Pipe ;
2020-09-14 14:13:58 +00:00
class QueryPlan ;
2020-10-01 17:34:22 +00:00
using QueryPlanPtr = std : : unique_ptr < QueryPlan > ;
2019-09-13 12:59:48 +00:00
2021-07-23 14:25:35 +00:00
class SinkToStorage ;
using SinkToStoragePtr = std : : shared_ptr < SinkToStorage > ;
2022-05-20 19:49:31 +00:00
class QueryPipeline ;
2021-04-04 10:27:45 +00:00
2021-01-23 15:20:15 +00:00
class IStoragePolicy ;
using StoragePolicyPtr = std : : shared_ptr < const IStoragePolicy > ;
2020-02-10 15:50:12 +00:00
2020-09-15 17:23:49 +00:00
struct StreamLocalLimits ;
class EnabledQuota ;
2020-09-10 19:55:36 +00:00
struct SelectQueryInfo ;
2020-09-15 17:23:49 +00:00
2021-02-28 05:24:39 +00:00
using NameDependencies = std : : unordered_map < String , std : : vector < String > > ;
2021-08-18 22:19:14 +00:00
using DatabaseAndTableName = std : : pair < String , String > ;
2022-05-31 09:33:23 +00:00
class BackupEntriesCollector ;
class RestorerFromBackup ;
2021-02-28 05:24:39 +00:00
2019-07-17 01:24:37 +00:00
struct ColumnSize
{
size_t marks = 0 ;
size_t data_compressed = 0 ;
size_t data_uncompressed = 0 ;
void add ( const ColumnSize & other )
{
marks + = other . marks ;
data_compressed + = other . data_compressed ;
data_uncompressed + = other . data_uncompressed ;
}
} ;
2017-01-21 04:24:28 +00:00
2021-10-08 13:13:56 +00:00
using IndexSize = ColumnSize ;
2019-05-17 14:34:25 +00:00
/** Storage. Describes the table. Responsible for
2017-04-16 15:00:33 +00:00
* - storage of the table data ;
2017-06-06 17:06:14 +00:00
* - the definition in which files ( or not in files ) the data is stored ;
* - data lookups and appends ;
2017-04-16 15:00:33 +00:00
* - data storage structure ( compression , etc . )
* - concurrent access to data ( locks , etc . )
2010-03-01 16:59:51 +00:00
*/
2020-12-07 09:30:47 +00:00
class IStorage : public std : : enable_shared_from_this < IStorage > , public TypePromotion < IStorage > , public IHints < 1 , IStorage >
2010-03-01 16:59:51 +00:00
{
public :
2019-12-03 16:25:32 +00:00
IStorage ( ) = delete ;
2020-06-19 15:39:41 +00:00
/// Storage metadata can be set separately in setInMemoryMetadata method
2020-06-19 17:17:13 +00:00
explicit IStorage ( StorageID storage_id_ )
: storage_id ( std : : move ( storage_id_ ) )
2023-02-19 22:15:09 +00:00
, metadata ( std : : make_unique < StorageInMemoryMetadata > ( ) ) { }
2019-05-17 14:34:25 +00:00
IStorage ( const IStorage & ) = delete ;
IStorage & operator = ( const IStorage & ) = delete ;
2017-04-16 15:00:33 +00:00
/// The main name of the table type (for example, StorageMergeTree).
2010-03-04 19:20:28 +00:00
virtual std : : string getName ( ) const = 0 ;
2017-04-01 07:20:54 +00:00
2019-03-29 20:31:06 +00:00
/// The name of the table.
2019-12-27 19:30:22 +00:00
StorageID getStorageID ( ) const ;
2018-03-06 20:18:34 +00:00
2023-01-10 15:31:01 +00:00
virtual bool isMergeTree ( ) const { return false ; }
2019-05-17 14:34:25 +00:00
/// Returns true if the storage receives data from a remote server or servers.
2012-10-30 20:15:07 +00:00
virtual bool isRemote ( ) const { return false ; }
2017-04-01 07:20:54 +00:00
2020-01-24 16:20:36 +00:00
/// Returns true if the storage is a view of a table or another view.
virtual bool isView ( ) const { return false ; }
2021-04-21 13:45:13 +00:00
/// Returns true if the storage is dictionary
virtual bool isDictionary ( ) const { return false ; }
2019-05-17 14:34:25 +00:00
/// Returns true if the storage supports queries with the SAMPLE section.
2020-06-17 12:07:09 +00:00
virtual bool supportsSampling ( ) const { return getInMemoryMetadataPtr ( ) - > hasSamplingKey ( ) ; }
2017-04-01 07:20:54 +00:00
2019-05-17 14:34:25 +00:00
/// Returns true if the storage supports queries with the FINAL section.
2013-04-23 11:08:41 +00:00
virtual bool supportsFinal ( ) const { return false ; }
2017-04-01 07:20:54 +00:00
2021-07-14 08:49:05 +00:00
/// Returns true if the storage supports insert queries with the PARTITION BY section.
virtual bool supportsPartitionBy ( ) const { return false ; }
2022-01-11 01:58:53 +00:00
/// Returns true if the storage supports queries with the TTL section.
virtual bool supportsTTL ( ) const { return false ; }
2019-05-17 14:34:25 +00:00
/// Returns true if the storage supports queries with the PREWHERE section.
2013-11-28 13:16:46 +00:00
virtual bool supportsPrewhere ( ) const { return false ; }
2017-04-01 07:20:54 +00:00
Fix PREWHERE for Merge with different default types
In case of underlying table has an ALIAS for this column, while in Merge
table it is not marked as an alias, there will NOT_FOUND_COLUMN_IN_BLOCK
error.
Further more, when underlying tables has different default type for the
column, i.e. one has ALIAS and another has real column, then you will
also get NOT_FOUND_COLUMN_IN_BLOCK, because Merge engine should take
care of this.
Also this patch reworks how PREWHERE is handled for Merge table, and now
if you use PREWHERE on the column that has the same type and default
type (ALIAS, ...) then it will be possible, and only if the type
differs, it will be prohibited and throw ILLEGAL_PREWHERE error.
And last, but not least, also respect this restrictions for
optimize_move_to_prewhere.
v2: introduce IStorage::supportedPrewhereColumns()
v3: Remove excessive condition for PREWHERE in StorageMerge::read()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-02-15 19:03:40 +00:00
/// Returns which columns supports PREWHERE, or empty std::nullopt if all columns is supported.
/// This is needed for engines whose aggregates data from multiple tables, like Merge.
virtual std : : optional < NameSet > supportedPrewhereColumns ( ) const { return std : : nullopt ; }
2021-12-29 01:33:39 +00:00
/// Returns true if the storage supports optimization of moving conditions to PREWHERE section.
virtual bool canMoveConditionsToPrewhere ( ) const { return supportsPrewhere ( ) ; }
2019-05-17 14:34:25 +00:00
/// Returns true if the storage replicates SELECT, INSERT and ALTER commands among replicas.
2017-04-25 15:21:03 +00:00
virtual bool supportsReplication ( ) const { return false ; }
2019-12-12 10:49:15 +00:00
/// Returns true if the storage supports parallel insert.
virtual bool supportsParallelInsert ( ) const { return false ; }
2019-05-17 14:34:25 +00:00
/// Returns true if the storage supports deduplication of inserted data blocks.
2018-05-21 23:17:57 +00:00
virtual bool supportsDeduplication ( ) const { return false ; }
2017-04-25 15:21:03 +00:00
2019-10-22 10:31:28 +00:00
/// Returns true if the blocks shouldn't be pushed to associated views on insert.
virtual bool noPushingToViews ( ) const { return false ; }
2020-01-24 15:10:24 +00:00
/// Read query returns streams which automatically distribute data between themselves.
/// So, it's impossible for one stream run out of data when there is data in other streams.
/// Example is StorageSystemNumbers.
2020-01-20 07:24:28 +00:00
virtual bool hasEvenlyDistributedRead ( ) const { return false ; }
2020-12-22 16:40:53 +00:00
/// Returns true if the storage supports reading of subcolumns of complex types.
virtual bool supportsSubcolumns ( ) const { return false ; }
2022-01-31 22:27:55 +00:00
/// Returns true if the storage supports transactions for SELECT, INSERT and ALTER queries.
/// Storage may throw an exception later if some query kind is not fully supported.
2022-02-14 19:47:17 +00:00
/// This method can return true for readonly engines that return the same rows for reading (such as SystemNumbers)
2022-01-31 22:27:55 +00:00
virtual bool supportsTransactions ( ) const { return false ; }
2022-03-01 16:32:55 +00:00
/// Returns true if the storage supports storing of dynamic subcolumns.
/// For now it makes sense only for data type Object.
2022-02-15 23:14:09 +00:00
virtual bool supportsDynamicSubcolumns ( ) const { return false ; }
2021-02-07 01:49:52 +00:00
/// Requires squashing small blocks to large for optimal storage.
/// This is true for most storages that store data on disk.
virtual bool prefersLargeBlocks ( ) const { return true ; }
2021-11-19 10:25:55 +00:00
/// Returns true if the storage is for system, which cannot be target of SHOW CREATE TABLE.
virtual bool isSystemStorage ( ) const { return false ; }
2020-02-18 19:03:40 +00:00
2019-07-17 01:24:37 +00:00
/// Optional size information of each physical column.
/// Currently it's only used by the MergeTree family for query optimizations.
2019-07-16 17:13:12 +00:00
using ColumnSizeByName = std : : unordered_map < std : : string , ColumnSize > ;
virtual ColumnSizeByName getColumnSizes ( ) const { return { } ; }
2019-03-05 10:12:20 +00:00
2021-10-08 13:13:56 +00:00
/// Optional size information of each secondary index.
/// Valid only for MergeTree family.
using IndexSizeByName = std : : unordered_map < std : : string , IndexSize > ;
virtual IndexSizeByName getSecondaryIndexSizes ( ) const { return { } ; }
2020-06-22 09:49:21 +00:00
/// Get mutable version (snapshot) of storage metadata. Metadata object is
2020-08-08 00:47:03 +00:00
/// multiversion, so it can be concurrently changed, but returned copy can be
2020-06-22 09:49:21 +00:00
/// used without any locks.
2020-06-18 11:02:31 +00:00
StorageInMemoryMetadata getInMemoryMetadata ( ) const { return * metadata . get ( ) ; }
2020-06-19 15:14:08 +00:00
2020-06-22 09:49:21 +00:00
/// Get immutable version (snapshot) of storage metadata. Metadata object is
2020-08-08 00:47:03 +00:00
/// multiversion, so it can be concurrently changed, but returned copy can be
2020-06-22 09:49:21 +00:00
/// used without any locks.
2020-06-18 11:02:31 +00:00
StorageMetadataPtr getInMemoryMetadataPtr ( ) const { return metadata . get ( ) ; }
2020-06-19 15:14:08 +00:00
2020-06-22 09:49:21 +00:00
/// Update storage metadata. Used in ALTER or initialization of Storage.
/// Metadata object is multiversion, so this method can be called without
/// any locks.
2020-06-18 11:02:31 +00:00
void setInMemoryMetadata ( const StorageInMemoryMetadata & metadata_ )
{
metadata . set ( std : : make_unique < StorageInMemoryMetadata > ( metadata_ ) ) ;
}
2019-12-26 18:17:05 +00:00
2019-05-17 14:34:25 +00:00
2020-04-27 17:46:51 +00:00
/// Return list of virtual columns (like _part, _table, etc). In the vast
/// majority of cases virtual columns are static constant part of Storage
/// class and don't depend on Storage object. But sometimes we have fake
/// storages, like Merge, which works as proxy for other storages and it's
/// virtual columns must contain virtual columns from underlying table.
///
/// User can create columns with the same name as virtual column. After that
2020-08-08 00:47:03 +00:00
/// virtual column will be overridden and inaccessible.
2020-04-27 17:46:51 +00:00
///
/// By default return empty list of columns.
2020-04-28 10:38:57 +00:00
virtual NamesAndTypesList getVirtuals ( ) const ;
2020-05-20 15:16:39 +00:00
2020-12-07 09:30:47 +00:00
Names getAllRegisteredNames ( ) const override ;
2019-05-21 11:24:32 +00:00
2021-04-10 23:33:54 +00:00
NameDependencies getDependentViewsByColumn ( ContextPtr context ) const ;
2021-02-28 05:24:39 +00:00
2022-05-31 09:33:23 +00:00
/// Returns whether the column is virtual - by default all columns are real.
/// Initially reserved virtual column name may be shadowed by real column.
bool isVirtualColumn ( const String & column_name , const StorageMetadataPtr & metadata_snapshot ) const ;
2022-05-29 19:53:56 +00:00
2022-06-22 22:56:41 +00:00
/// Modify a CREATE TABLE query to make a variant which must be written to a backup.
2022-06-23 10:17:54 +00:00
virtual void adjustCreateQueryForBackup ( ASTPtr & create_query ) const ;
2021-08-18 22:19:14 +00:00
2022-05-31 09:33:23 +00:00
/// Makes backup entries to backup the data of this storage.
virtual void backupData ( BackupEntriesCollector & backup_entries_collector , const String & data_path_in_backup , const std : : optional < ASTs > & partitions ) ;
/// Extracts data from the backup and put it to the storage.
virtual void restoreDataFromBackup ( RestorerFromBackup & restorer , const String & data_path_in_backup , const std : : optional < ASTs > & partitions ) ;
2019-08-27 09:34:53 +00:00
2022-06-29 12:42:23 +00:00
/// Returns true if the storage supports backup/restore for specific partitions.
virtual bool supportsBackupPartition ( ) const { return false ; }
2022-07-21 19:50:19 +00:00
/// Return true if there is at least one part containing lightweight deleted mask.
2022-07-06 10:29:29 +00:00
virtual bool hasLightweightDeletedMask ( ) const { return false ; }
2022-06-21 03:07:25 +00:00
2022-07-23 11:55:43 +00:00
/// Return true if storage can execute lightweight delete mutations.
virtual bool supportsLightweightDelete ( ) const { return false ; }
2022-12-29 11:24:48 +00:00
/// Return true if storage can execute 'DELETE FROM' mutations. This is different from lightweight delete
/// because those are internally translated into 'ALTER UDPATE' mutations.
virtual bool supportsDelete ( ) const { return false ; }
2019-05-17 14:34:25 +00:00
private :
2021-06-20 08:24:43 +00:00
2019-12-12 12:30:31 +00:00
StorageID storage_id ;
2021-06-20 08:24:43 +00:00
2019-12-03 16:25:32 +00:00
mutable std : : mutex id_mutex ;
2020-05-20 12:16:55 +00:00
2020-06-22 09:49:21 +00:00
/// Multiversion storage metadata. Allows to read/write storage metadata
/// without locks.
2020-06-18 11:02:31 +00:00
MultiVersionStorageMetadataPtr metadata ;
2020-08-03 11:33:11 +00:00
2021-09-29 17:30:07 +00:00
protected :
2020-04-27 15:21:37 +00:00
RWLockImpl : : LockHolder tryLockTimed (
2022-05-29 19:53:56 +00:00
const RWLock & rwlock , RWLockImpl : : Type type , const String & query_id , const std : : chrono : : milliseconds & acquire_timeout ) const ;
2019-05-17 14:34:25 +00:00
public :
2022-09-29 06:44:10 +00:00
/// Lock table for share. This lock must be acquired if you want to be sure,
2020-06-22 09:49:21 +00:00
/// that table will be not dropped while you holding this lock. It's used in
/// variety of cases starting from SELECT queries to background merges in
/// MergeTree.
2020-07-16 13:38:58 +00:00
TableLockHolder lockForShare ( const String & query_id , const std : : chrono : : milliseconds & acquire_timeout ) ;
2017-04-01 07:20:54 +00:00
2022-09-29 06:44:10 +00:00
/// Similar to lockForShare, but returns a nullptr if the table is dropped while
/// acquiring the lock instead of raising a TABLE_IS_DROPPED exception
TableLockHolder tryLockForShare ( const String & query_id , const std : : chrono : : milliseconds & acquire_timeout ) ;
2023-04-08 15:39:04 +00:00
/// Lock table for alter. This lock must be acquired in ALTER queries to be
2020-06-22 09:49:21 +00:00
/// sure, that we execute only one simultaneous alter. Doesn't affect share lock.
2021-10-25 17:49:49 +00:00
using AlterLockHolder = std : : unique_lock < std : : timed_mutex > ;
AlterLockHolder lockForAlter ( const std : : chrono : : milliseconds & acquire_timeout ) ;
2019-03-05 10:12:20 +00:00
2020-08-08 00:47:03 +00:00
/// Lock table exclusively. This lock must be acquired if you want to be
2020-06-22 09:49:21 +00:00
/// sure, that no other thread (SELECT, merge, ALTER, etc.) doing something
/// with table. For example it allows to wait all threads before DROP or
/// truncate query.
///
/// NOTE: You have to be 100% sure that you need this lock. It's extremely
/// heavyweight and makes table irresponsive.
2020-07-16 13:38:58 +00:00
TableExclusiveLockHolder lockExclusively ( const String & query_id , const std : : chrono : : milliseconds & acquire_timeout ) ;
2017-04-01 07:20:54 +00:00
2018-04-19 14:47:09 +00:00
/** Returns stage to which query is going to be processed in read() function.
* ( Normally , the function only reads the columns from the list , but in other cases ,
2021-02-10 14:12:49 +00:00
* for example , the request can be partially processed on a remote server , or an aggregate projection . )
2020-03-22 17:43:01 +00:00
*
* SelectQueryInfo is required since the stage can depends on the query
2021-02-10 14:12:49 +00:00
* ( see Distributed ( ) engine and optimize_skip_unused_shards ,
2022-07-30 21:40:21 +00:00
* see also MergeTree engine and projection optimization ) .
2020-09-10 19:55:36 +00:00
* And to store optimized cluster ( after optimize_skip_unused_shards ) .
2021-02-10 14:12:49 +00:00
* It will also store needed stuff for projection query pipeline .
2020-04-01 18:38:01 +00:00
*
* QueryProcessingStage : : Enum required for Distributed over Distributed ,
* since it cannot return Complete for intermediate queries never .
2018-04-19 14:47:09 +00:00
*/
2021-07-09 03:15:41 +00:00
virtual QueryProcessingStage : : Enum getQueryProcessingStage ( ContextPtr , QueryProcessingStage : : Enum , const StorageSnapshotPtr & , SelectQueryInfo & ) const
2020-04-01 18:38:01 +00:00
{
return QueryProcessingStage : : FetchColumns ;
}
2017-04-01 07:20:54 +00:00
2019-05-28 21:17:48 +00:00
/** Watch live changes to the table.
* Accepts a list of columns to read , as well as a description of the query ,
* from which information can be extracted about how to retrieve data
* ( indexes , locks , etc . )
* Returns a stream with which you can read data sequentially
* or multiple streams for parallel data reading .
* The ` processed_stage ` info is also written to what stage the request was processed .
* ( Normally , the function only reads the columns from the list , but in other cases ,
* for example , the request can be partially processed on a remote server . )
*
* context contains settings for one query .
* Usually Storage does not care about these settings , since they are used in the interpreter .
* But , for example , for distributed query processing , the settings are passed to the remote server .
*
* num_streams - a recommendation , how many streams to return ,
* if the storage can return a different number of streams .
*
* It is guaranteed that the structure of the table will not change over the lifetime of the returned streams ( that is , there will not be ALTER , RENAME and DROP ) .
*/
2021-08-11 17:28:54 +00:00
virtual Pipe watch (
2019-05-28 21:17:48 +00:00
const Names & /*column_names*/ ,
const SelectQueryInfo & /*query_info*/ ,
2021-04-10 23:33:54 +00:00
ContextPtr /*context*/ ,
2019-05-28 21:17:48 +00:00
QueryProcessingStage : : Enum & /*processed_stage*/ ,
size_t /*max_block_size*/ ,
2022-10-07 10:46:45 +00:00
size_t /*num_streams*/ ) ;
2019-05-28 21:17:48 +00:00
2021-06-28 10:35:55 +00:00
/// Returns true if FINAL modifier must be added to SELECT query depending on required columns.
/// It's needed for ReplacingMergeTree wrappers such as MaterializedMySQL and MaterializedPostrgeSQL
virtual bool needRewriteQueryWithFinal ( const Names & /*column_names*/ ) const { return false ; }
2022-05-23 19:47:32 +00:00
private :
2017-04-16 15:00:33 +00:00
/** Read a set of columns from the table.
* Accepts a list of columns to read , as well as a description of the query ,
* from which information can be extracted about how to retrieve data
* ( indexes , locks , etc . )
* Returns a stream with which you can read data sequentially
* or multiple streams for parallel data reading .
2018-04-19 14:47:09 +00:00
* The ` processed_stage ` must be the result of getQueryProcessingStage ( ) function .
2012-01-09 19:20:48 +00:00
*
2017-05-24 21:06:29 +00:00
* context contains settings for one query .
2017-04-16 15:00:33 +00:00
* Usually Storage does not care about these settings , since they are used in the interpreter .
* But , for example , for distributed query processing , the settings are passed to the remote server .
2013-02-01 19:02:04 +00:00
*
2017-06-02 15:54:39 +00:00
* num_streams - a recommendation , how many streams to return ,
* if the storage can return a different number of streams .
2014-03-19 10:45:13 +00:00
*
2020-06-22 09:49:21 +00:00
* metadata_snapshot is consistent snapshot of table metadata , it should be
* passed in all parts of the returned pipeline . Storage metadata can be
* changed during lifetime of the returned pipeline , but the snapshot is
* guaranteed to be immutable .
2010-03-01 16:59:51 +00:00
*/
2020-08-03 11:33:11 +00:00
virtual Pipe read (
2019-09-13 12:59:48 +00:00
const Names & /*column_names*/ ,
2021-07-09 03:15:41 +00:00
const StorageSnapshotPtr & /*storage_snapshot*/ ,
2020-09-20 17:52:17 +00:00
SelectQueryInfo & /*query_info*/ ,
2021-04-10 23:33:54 +00:00
ContextPtr /*context*/ ,
2018-04-19 14:47:09 +00:00
QueryProcessingStage : : Enum /*processed_stage*/ ,
2019-02-18 17:31:18 +00:00
size_t /*max_block_size*/ ,
2022-10-07 10:46:45 +00:00
size_t /*num_streams*/ ) ;
2017-04-01 07:20:54 +00:00
2022-05-23 19:47:32 +00:00
public :
2020-09-15 09:22:45 +00:00
/// Other version of read which adds reading step to query plan.
/// Default implementation creates ReadFromStorageStep and uses usual read.
2020-09-14 14:13:58 +00:00
virtual void read (
QueryPlan & query_plan ,
2020-09-18 11:39:07 +00:00
const Names & /*column_names*/ ,
2021-07-09 03:15:41 +00:00
const StorageSnapshotPtr & /*storage_snapshot*/ ,
2020-11-10 12:02:22 +00:00
SelectQueryInfo & /*query_info*/ ,
2021-04-10 23:33:54 +00:00
ContextPtr /*context*/ ,
2020-09-18 11:39:07 +00:00
QueryProcessingStage : : Enum /*processed_stage*/ ,
size_t /*max_block_size*/ ,
2022-10-07 10:46:45 +00:00
size_t /*num_streams*/ ) ;
2020-09-14 14:13:58 +00:00
2017-04-16 15:00:33 +00:00
/** Writes the data to a table.
* Receives a description of the query , which can contain information about the data write method .
* Returns an object by which you can write data sequentially .
2014-03-19 10:45:13 +00:00
*
2020-06-22 09:49:21 +00:00
* metadata_snapshot is consistent snapshot of table metadata , it should be
* passed in all parts of the returned streams . Storage metadata can be
* changed during lifetime of the returned streams , but the snapshot is
* guaranteed to be immutable .
2010-03-18 19:32:14 +00:00
*/
2021-07-23 14:25:35 +00:00
virtual SinkToStoragePtr write (
2017-12-01 19:34:51 +00:00
const ASTPtr & /*query*/ ,
2020-06-15 19:08:58 +00:00
const StorageMetadataPtr & /*metadata_snapshot*/ ,
2021-04-10 23:33:54 +00:00
ContextPtr /*context*/ )
2010-03-18 19:32:14 +00:00
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Method write is not supported by storage {} " , getName ( ) ) ;
2011-08-19 18:31:14 +00:00
}
2017-04-01 07:20:54 +00:00
2021-04-04 10:27:45 +00:00
/** Writes the data to a table in distributed manner.
* It is supposed that implementation looks into SELECT part of the query and executes distributed
* INSERT SELECT if it is possible with current storage as a receiver and query SELECT part as a producer .
*
* Returns query pipeline if distributed writing is possible , and nullptr otherwise .
*/
2022-05-20 19:49:31 +00:00
virtual std : : optional < QueryPipeline > distributedWrite (
2021-04-04 10:27:45 +00:00
const ASTInsertQuery & /*query*/ ,
2022-05-20 19:49:31 +00:00
ContextPtr /*context*/ ) ;
2021-04-04 10:27:45 +00:00
2017-04-16 15:00:33 +00:00
/** Delete the table data. Called before deleting the directory with the data.
2018-06-09 15:48:22 +00:00
* The method can be called only after detaching table from Context ( when no queries are performed with table ) .
* The table is not usable during and after call to this method .
2020-01-22 11:30:11 +00:00
* If some queries may still use the table , then it must be called under exclusive lock .
2017-04-16 15:00:33 +00:00
* If you do not need any action other than deleting the directory with data , you can leave this method blank .
2013-01-23 17:38:03 +00:00
*/
2020-01-22 11:30:11 +00:00
virtual void drop ( ) { }
2017-04-01 07:20:54 +00:00
2022-06-23 07:59:13 +00:00
virtual void dropInnerTableIfAny ( bool /* sync */ , ContextPtr /* context */ ) { }
2021-04-10 17:58:09 +00:00
2018-06-09 15:48:22 +00:00
/** Clear the table data and leave it empty.
2020-06-22 09:49:21 +00:00
* Must be called under exclusive lock ( lockExclusively ) .
2018-04-21 00:35:20 +00:00
*/
2020-06-18 10:29:13 +00:00
virtual void truncate (
const ASTPtr & /*query*/ ,
const StorageMetadataPtr & /* metadata_snapshot */ ,
2021-04-10 23:33:54 +00:00
ContextPtr /* context */ ,
2020-06-18 16:10:47 +00:00
TableExclusiveLockHolder & )
2018-04-21 00:35:20 +00:00
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Truncate is not supported by storage {} " , getName ( ) ) ;
2018-04-21 00:35:20 +00:00
}
2022-04-13 14:51:59 +00:00
virtual void checkTableCanBeRenamed ( const StorageID & /*new_name*/ ) const { }
2020-09-26 19:18:28 +00:00
2017-04-16 15:00:33 +00:00
/** Rename the table.
* Renaming a name in a file with metadata , the name in the list of tables in the RAM , is done separately .
* In this function , you need to rename the directory with the data , if any .
* Called when the table structure is locked for write .
2020-04-07 14:05:51 +00:00
* Table UUID must remain unchanged , unless table moved between Ordinary and Atomic databases .
2012-06-18 06:19:13 +00:00
*/
2020-04-07 14:05:51 +00:00
virtual void rename ( const String & /*new_path_to_table_data*/ , const StorageID & new_table_id )
2012-06-18 06:19:13 +00:00
{
2020-04-07 14:05:51 +00:00
renameInMemory ( new_table_id ) ;
2012-06-18 06:19:13 +00:00
}
2017-04-01 07:20:54 +00:00
2019-12-03 16:25:32 +00:00
/**
* Just updates names of database and table without moving any data on disk
2019-12-12 12:30:31 +00:00
* Can be called directly only from DatabaseAtomic .
2019-12-03 16:25:32 +00:00
*/
2020-04-07 14:05:51 +00:00
virtual void renameInMemory ( const StorageID & new_table_id ) ;
2019-12-03 16:25:32 +00:00
2020-06-22 09:49:21 +00:00
/** ALTER tables in the form of column changes that do not affect the change
* to Storage or its parameters . Executes under alter lock ( lockForAlter ) .
2011-08-19 18:31:14 +00:00
*/
2021-10-25 17:49:49 +00:00
virtual void alter ( const AlterCommands & params , ContextPtr context , AlterLockHolder & alter_lock_holder ) ;
2018-05-21 13:49:54 +00:00
2019-12-27 15:07:53 +00:00
/** Checks that alter commands can be applied to storage. For example, columns can be modified,
* or primary key can be changes , etc .
*/
2021-04-10 23:33:54 +00:00
virtual void checkAlterIsPossible ( const AlterCommands & commands , ContextPtr context ) const ;
2019-12-26 18:17:05 +00:00
2021-02-25 10:07:48 +00:00
/**
* Checks that mutation commands can be applied to storage .
*/
virtual void checkMutationIsPossible ( const MutationCommands & commands , const Settings & settings ) const ;
2018-11-13 13:48:53 +00:00
/** ALTER tables with regard to its partitions.
* Should handle locks for each command on its own .
2014-08-06 09:24:30 +00:00
*/
2020-08-03 11:33:11 +00:00
virtual Pipe alterPartition (
const StorageMetadataPtr & /* metadata_snapshot */ ,
const PartitionCommands & /* commands */ ,
2021-04-10 23:33:54 +00:00
ContextPtr /* context */ ) ;
2017-04-01 07:20:54 +00:00
2020-07-13 16:19:08 +00:00
/// Checks that partition commands can be applied to storage.
2020-07-13 17:27:52 +00:00
virtual void checkAlterPartitionIsPossible ( const PartitionCommands & commands , const StorageMetadataPtr & metadata_snapshot , const Settings & settings ) const ;
2020-07-13 16:19:08 +00:00
2017-04-16 15:00:33 +00:00
/** Perform any background work. For example, combining parts in a MergeTree type table.
* Returns whether any work has been done .
2012-07-16 20:25:19 +00:00
*/
2020-06-17 13:39:26 +00:00
virtual bool optimize (
const ASTPtr & /*query*/ ,
const StorageMetadataPtr & /*metadata_snapshot*/ ,
const ASTPtr & /*partition*/ ,
bool /*final*/ ,
bool /*deduplicate*/ ,
2020-12-01 09:10:12 +00:00
const Names & /* deduplicate_by_columns */ ,
2023-02-16 13:03:16 +00:00
bool /*cleanup*/ ,
2021-04-10 23:33:54 +00:00
ContextPtr /*context*/ )
2012-07-16 20:25:19 +00:00
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Method optimize is not supported by storage {} " , getName ( ) ) ;
2012-07-16 20:25:19 +00:00
}
2017-04-01 07:20:54 +00:00
2018-02-02 16:02:43 +00:00
/// Mutate the table contents
2023-01-30 17:38:28 +00:00
virtual void mutate ( const MutationCommands & , ContextPtr )
2018-02-02 16:02:43 +00:00
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Mutations are not supported by storage {} " , getName ( ) ) ;
2018-02-02 16:02:43 +00:00
}
2019-01-10 18:19:29 +00:00
/// Cancel a mutation.
2019-02-04 13:04:02 +00:00
virtual CancellationCode killMutation ( const String & /*mutation_id*/ )
2019-01-10 18:19:29 +00:00
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Mutations are not supported by storage {} " , getName ( ) ) ;
2019-01-10 18:19:29 +00:00
}
2021-12-14 20:06:34 +00:00
virtual void waitForMutation ( const String & /*mutation_id*/ )
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Mutations are not supported by storage {} " , getName ( ) ) ;
2021-12-14 20:06:34 +00:00
}
2022-03-17 17:26:18 +00:00
virtual void setMutationCSN ( const String & /*mutation_id*/ , UInt64 /*csn*/ )
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Mutations are not supported by storage {} " , getName ( ) ) ;
2022-03-17 17:26:18 +00:00
}
2021-09-16 16:03:31 +00:00
/// Cancel a part move to shard.
virtual CancellationCode killPartMoveToShard ( const UUID & /*task_uuid*/ )
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Part moves between shards are not supported by storage {} " , getName ( ) ) ;
2021-09-16 16:03:31 +00:00
}
2017-06-06 17:06:14 +00:00
/** If the table have to do some complicated work on startup,
* that must be postponed after creation of table object
* ( like launching some background threads ) ,
* do it in this method .
* You should call this method after creation of object .
* By default , does nothing .
* Cannot be called simultaneously by multiple threads .
*/
virtual void startup ( ) { }
2021-05-13 07:03:00 +00:00
/**
* If the storage requires some complicated work on destroying ,
* then you have two virtual methods :
* - flush ( )
* - shutdown ( )
*
* @ see shutdown ( )
* @ see flush ( )
*/
void flushAndShutdown ( )
{
flush ( ) ;
shutdown ( ) ;
}
2017-06-06 17:06:14 +00:00
/** If the table have to do some complicated work when destroying an object - do it in advance.
2017-04-16 15:00:33 +00:00
* For example , if the table contains any threads for background work - ask them to complete and wait for completion .
2017-06-06 17:06:14 +00:00
* By default , does nothing .
2017-04-16 15:00:33 +00:00
* Can be called simultaneously from different threads , even after a call to drop ( ) .
2013-09-30 01:29:19 +00:00
*/
virtual void shutdown ( ) { }
2017-04-01 07:20:54 +00:00
2021-05-12 18:14:00 +00:00
/// Called before shutdown() to flush data to underlying storage
2021-12-14 08:31:17 +00:00
/// Data in memory need to be persistent
2021-05-12 18:14:00 +00:00
virtual void flush ( ) { }
2018-05-21 13:49:54 +00:00
/// Asks table to stop executing some action identified by action_type
/// If table does not support such type of lock, and empty lock is returned
2018-05-28 15:37:30 +00:00
virtual ActionLock getActionLock ( StorageActionBlockType /* action_type */ )
2018-05-21 13:49:54 +00:00
{
return { } ;
}
2020-10-16 10:12:31 +00:00
/// Call when lock from previous method removed
2020-10-15 16:10:22 +00:00
virtual void onActionLockRemove ( StorageActionBlockType /* action_type */ ) { }
2019-06-12 13:11:44 +00:00
std : : atomic < bool > is_dropped { false } ;
2023-01-24 13:15:51 +00:00
std : : atomic < bool > is_detached { false } ;
2017-04-01 07:20:54 +00:00
2017-01-23 19:18:25 +00:00
/// Does table support index for IN sections
2016-05-16 18:43:38 +00:00
virtual bool supportsIndexForIn ( ) const { return false ; }
2017-04-01 07:20:54 +00:00
2018-01-23 08:18:12 +00:00
/// Provides a hint that the storage engine may evaluate the IN-condition by using an index.
2021-04-10 23:33:54 +00:00
virtual bool mayBenefitFromIndexForIn ( const ASTPtr & /* left_in_operand */ , ContextPtr /* query_context */ , const StorageMetadataPtr & /* metadata_snapshot */ ) const { return false ; }
2018-01-21 07:30:07 +00:00
2017-01-23 19:18:25 +00:00
/// Checks validity of the data
2023-01-23 21:13:58 +00:00
virtual CheckResults checkData ( const ASTPtr & /* query */ , ContextPtr /* context */ ) { throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Check query is not supported for {} storage " , getName ( ) ) ; }
2017-04-01 07:20:54 +00:00
2017-01-23 19:18:25 +00:00
/// Checks that table could be dropped right now
2018-08-03 09:54:46 +00:00
/// Otherwise - throws an exception with detailed information.
2018-08-03 13:19:53 +00:00
/// We do not use mutex because it is not very important that the size could change during the operation.
2018-08-03 09:39:01 +00:00
virtual void checkTableCanBeDropped ( ) const { }
2020-11-11 14:34:58 +00:00
/// Similar to above but checks for DETACH. It's only used for DICTIONARIES.
virtual void checkTableCanBeDetached ( ) const { }
2017-01-23 19:18:25 +00:00
2020-11-01 17:38:43 +00:00
/// Returns true if Storage may store some data on disk.
/// NOTE: may not be equivalent to !getDataPaths().empty()
virtual bool storesDataOnDisk ( ) const { return false ; }
2019-08-01 10:29:14 +00:00
/// Returns data paths if storage supports it, empty vector otherwise.
2019-04-04 13:13:59 +00:00
virtual Strings getDataPaths ( ) const { return { } ; }
2018-02-21 19:26:59 +00:00
2020-05-25 17:57:08 +00:00
/// Returns storage policy if storage supports it.
2019-11-27 09:39:44 +00:00
virtual StoragePolicyPtr getStoragePolicy ( ) const { return { } ; }
2014-03-09 17:36:01 +00:00
2022-11-17 18:26:40 +00:00
/// Returns true if all disks of storage are read-only or write-once.
/// NOTE: write-once also does not support INSERTs/merges/... for MergeTree
2021-09-04 09:02:07 +00:00
virtual bool isStaticStorage ( ) const ;
2021-08-23 11:26:54 +00:00
2022-05-13 18:39:19 +00:00
virtual bool supportsSubsetOfColumns ( ) const { return false ; }
2022-02-28 13:29:05 +00:00
2020-03-29 08:38:38 +00:00
/// If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it.
/// Used for:
2021-05-12 20:37:18 +00:00
/// - Simple count() optimization
2020-03-29 08:38:38 +00:00
/// - For total_rows column in system.tables
///
/// Does takes underlying Storage (if any) into account.
2020-11-25 13:47:32 +00:00
virtual std : : optional < UInt64 > totalRows ( const Settings & ) const { return { } ; }
2019-10-28 17:27:43 +00:00
2020-09-21 10:13:01 +00:00
/// Same as above but also take partition predicate into account.
2021-04-10 23:33:54 +00:00
virtual std : : optional < UInt64 > totalRowsByPartitionPredicate ( const SelectQueryInfo & , ContextPtr ) const { return { } ; }
2020-09-21 10:13:01 +00:00
2020-03-29 08:38:38 +00:00
/// If it is possible to quickly determine exact number of bytes for the table on storage:
2020-06-24 21:27:53 +00:00
/// - memory (approximated, resident)
2020-03-29 08:38:38 +00:00
/// - disk (compressed)
///
/// Used for:
/// - For total_bytes column in system.tables
//
/// Does not takes underlying Storage (if any) into account
/// (since for Buffer we still need to know how much bytes it uses).
2020-06-24 21:27:53 +00:00
///
/// Memory part should be estimated as a resident memory size.
/// In particular, alloctedBytes() is preferable over bytes()
/// when considering in-memory blocks.
2020-11-25 13:47:32 +00:00
virtual std : : optional < UInt64 > totalBytes ( const Settings & ) const { return { } ; }
2020-03-29 08:38:38 +00:00
2020-07-11 12:19:06 +00:00
/// Number of rows INSERTed since server start.
///
2022-12-18 00:57:06 +00:00
/// Does not take the underlying Storage (if any) into account.
2020-07-11 12:19:06 +00:00
virtual std : : optional < UInt64 > lifetimeRows ( ) const { return { } ; }
/// Number of bytes INSERTed since server start.
///
2022-12-18 00:57:06 +00:00
/// Does not take the underlying Storage (if any) into account.
2020-07-11 12:19:06 +00:00
virtual std : : optional < UInt64 > lifetimeBytes ( ) const { return { } ; }
2022-03-01 16:32:55 +00:00
/// Creates a storage snapshot from given metadata.
2022-03-17 17:26:18 +00:00
virtual StorageSnapshotPtr getStorageSnapshot ( const StorageMetadataPtr & metadata_snapshot , ContextPtr /*query_context*/ ) const
2022-02-15 23:14:09 +00:00
{
return std : : make_shared < StorageSnapshot > ( * this , metadata_snapshot ) ;
}
2022-03-01 16:32:55 +00:00
/// Creates a storage snapshot from given metadata and columns, which are used in query.
2022-03-17 17:26:18 +00:00
virtual StorageSnapshotPtr getStorageSnapshotForQuery ( const StorageMetadataPtr & metadata_snapshot , const ASTPtr & /*query*/ , ContextPtr query_context ) const
2022-02-15 23:14:09 +00:00
{
2022-03-17 17:26:18 +00:00
return getStorageSnapshot ( metadata_snapshot , query_context ) ;
2022-02-15 23:14:09 +00:00
}
2021-10-06 10:37:58 +00:00
2022-05-20 19:49:31 +00:00
/// A helper to implement read()
static void readFromPipe (
QueryPlan & query_plan ,
Pipe pipe ,
const Names & column_names ,
const StorageSnapshotPtr & storage_snapshot ,
SelectQueryInfo & query_info ,
ContextPtr context ,
std : : string storage_name ) ;
2013-01-23 17:38:03 +00:00
private :
2021-10-25 17:49:49 +00:00
/// Lock required for alter queries (lockForAlter).
/// Allows to execute only one simultaneous alter query.
mutable std : : timed_mutex alter_lock ;
2020-06-18 16:10:47 +00:00
2020-06-22 09:49:21 +00:00
/// Lock required for drop queries. Every thread that want to ensure, that
2020-08-08 00:47:03 +00:00
/// table is not dropped have to table this lock for read (lockForShare).
2020-06-22 09:49:21 +00:00
/// DROP-like queries take this lock for write (lockExclusively), to be sure
/// that all table threads finished.
2020-06-18 16:10:47 +00:00
mutable RWLock drop_lock = RWLockImpl : : create ( ) ;
2013-01-23 17:38:03 +00:00
} ;
2010-03-01 16:59:51 +00:00
2011-08-15 01:12:57 +00:00
}