2019-12-26 18:17:05 +00:00
|
|
|
#pragma once
|
|
|
|
|
2020-06-05 17:29:40 +00:00
|
|
|
#include <Parsers/IAST_fwd.h>
|
2020-06-16 12:03:27 +00:00
|
|
|
#include <Storages/ColumnDependency.h>
|
2019-12-26 18:17:05 +00:00
|
|
|
#include <Storages/ColumnsDescription.h>
|
|
|
|
#include <Storages/ConstraintsDescription.h>
|
2020-06-05 17:29:40 +00:00
|
|
|
#include <Storages/IndicesDescription.h>
|
|
|
|
#include <Storages/KeyDescription.h>
|
|
|
|
#include <Storages/SelectQueryDescription.h>
|
2020-06-16 12:03:27 +00:00
|
|
|
#include <Storages/TTLDescription.h>
|
2020-06-05 17:29:40 +00:00
|
|
|
|
2020-06-15 16:55:33 +00:00
|
|
|
#include <Common/MultiVersion.h>
|
2019-12-26 18:17:05 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Common metadata for all storages. Contains all possible parts of CREATE
|
|
|
|
/// query from all storages, but only some subset used.
|
2019-12-26 18:17:05 +00:00
|
|
|
struct StorageInMemoryMetadata
|
|
|
|
{
|
2019-12-27 15:01:41 +00:00
|
|
|
/// Columns of table with their names, types,
|
|
|
|
/// defaults, comments, etc. All table engines have columns.
|
2019-12-26 18:17:05 +00:00
|
|
|
ColumnsDescription columns;
|
2019-12-27 15:01:41 +00:00
|
|
|
/// Table indices. Currently supported for MergeTree only.
|
2020-06-01 12:11:23 +00:00
|
|
|
IndicesDescription secondary_indices;
|
2019-12-27 15:01:41 +00:00
|
|
|
/// Table constraints. Currently supported for MergeTree only.
|
2019-12-26 18:17:05 +00:00
|
|
|
ConstraintsDescription constraints;
|
2019-12-27 15:01:41 +00:00
|
|
|
/// PARTITION BY expression. Currently supported for MergeTree only.
|
2020-06-05 17:29:40 +00:00
|
|
|
KeyDescription partition_key;
|
|
|
|
/// PRIMARY KEY expression. If absent, than equal to order_by_ast.
|
|
|
|
KeyDescription primary_key;
|
2019-12-27 15:01:41 +00:00
|
|
|
/// ORDER BY expression. Required field for all MergeTree tables
|
|
|
|
/// even in old syntax MergeTree(partition_key, order_by, ...)
|
2020-06-05 17:29:40 +00:00
|
|
|
KeyDescription sorting_key;
|
2019-12-27 15:01:41 +00:00
|
|
|
/// SAMPLE BY expression. Supported for MergeTree only.
|
2020-06-05 17:29:40 +00:00
|
|
|
KeyDescription sampling_key;
|
|
|
|
/// Separate ttl expressions for columns
|
|
|
|
TTLColumnsDescription column_ttls_by_name;
|
|
|
|
/// TTL expressions for table (Move and Rows)
|
|
|
|
TTLTableDescription table_ttl;
|
2019-12-27 15:01:41 +00:00
|
|
|
/// SETTINGS expression. Supported for MergeTree, Buffer and Kafka.
|
2020-06-05 17:29:40 +00:00
|
|
|
ASTPtr settings_changes;
|
|
|
|
/// SELECT QUERY. Supported for MaterializedView and View (have to support LiveView).
|
|
|
|
SelectQueryDescription select;
|
2020-02-14 13:17:50 +00:00
|
|
|
|
|
|
|
StorageInMemoryMetadata() = default;
|
2020-06-08 14:18:38 +00:00
|
|
|
|
|
|
|
StorageInMemoryMetadata(const StorageInMemoryMetadata & other);
|
|
|
|
StorageInMemoryMetadata & operator=(const StorageInMemoryMetadata & other);
|
2020-06-15 16:55:33 +00:00
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// NOTE: Thread unsafe part. You should modify same StorageInMemoryMetadata
|
|
|
|
/// structure from different threads. It should be used as MultiVersion
|
|
|
|
/// object. See example in IStorage.
|
2020-06-15 16:55:33 +00:00
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Sets only real columns, possibly overwrites virtual ones.
|
|
|
|
void setColumns(ColumnsDescription columns_);
|
2020-06-15 16:55:33 +00:00
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Sets secondary indices
|
2020-06-15 16:55:33 +00:00
|
|
|
void setSecondaryIndices(IndicesDescription secondary_indices_);
|
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Sets constraints
|
2020-06-15 16:55:33 +00:00
|
|
|
void setConstraints(ConstraintsDescription constraints_);
|
|
|
|
|
2020-12-24 10:11:07 +00:00
|
|
|
/// Set partition key for storage (methods below, are just wrappers for this struct).
|
2020-06-15 17:17:06 +00:00
|
|
|
void setPartitionKey(const KeyDescription & partition_key_);
|
2020-12-24 10:11:07 +00:00
|
|
|
/// Set sorting key for storage (methods below, are just wrappers for this struct).
|
2020-06-15 17:17:06 +00:00
|
|
|
void setSortingKey(const KeyDescription & sorting_key_);
|
2020-12-24 10:11:07 +00:00
|
|
|
/// Set primary key for storage (methods below, are just wrappers for this struct).
|
2020-06-15 17:17:06 +00:00
|
|
|
void setPrimaryKey(const KeyDescription & primary_key_);
|
2020-12-24 10:11:07 +00:00
|
|
|
/// Set sampling key for storage (methods below, are just wrappers for this struct).
|
2020-06-15 17:17:06 +00:00
|
|
|
void setSamplingKey(const KeyDescription & sampling_key_);
|
2020-06-15 17:50:53 +00:00
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Set common table TTLs
|
2020-06-15 17:50:53 +00:00
|
|
|
void setTableTTLs(const TTLTableDescription & table_ttl_);
|
|
|
|
|
2020-08-08 00:47:03 +00:00
|
|
|
/// TTLs for separate columns
|
2020-06-15 17:50:53 +00:00
|
|
|
void setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_);
|
2020-06-15 18:08:05 +00:00
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Set settings changes in metadata (some settings exlicetely specified in
|
|
|
|
/// CREATE query)
|
2020-06-15 18:08:05 +00:00
|
|
|
void setSettingsChanges(const ASTPtr & settings_changes_);
|
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Set SELECT query for (Materialized)View
|
2020-06-15 18:08:05 +00:00
|
|
|
void setSelectQuery(const SelectQueryDescription & select_);
|
2020-06-16 12:03:27 +00:00
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Returns combined set of columns
|
|
|
|
const ColumnsDescription & getColumns() const;
|
2021-02-25 11:07:16 +00:00
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Returns secondary indices
|
2020-06-16 12:03:27 +00:00
|
|
|
const IndicesDescription & getSecondaryIndices() const;
|
2021-03-31 14:21:19 +00:00
|
|
|
|
2020-06-16 12:03:27 +00:00
|
|
|
/// Has at least one non primary index
|
|
|
|
bool hasSecondaryIndices() const;
|
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Return table constraints
|
2020-06-16 12:03:27 +00:00
|
|
|
const ConstraintsDescription & getConstraints() const;
|
|
|
|
|
2020-06-17 13:39:26 +00:00
|
|
|
/// Returns true if there is set table TTL, any column TTL or any move TTL.
|
|
|
|
bool hasAnyTTL() const { return hasAnyColumnTTL() || hasAnyTableTTL(); }
|
|
|
|
|
2020-06-16 12:03:27 +00:00
|
|
|
/// Common tables TTLs (for rows and moves).
|
|
|
|
TTLTableDescription getTableTTLs() const;
|
|
|
|
bool hasAnyTableTTL() const;
|
|
|
|
|
|
|
|
/// Separate TTLs for columns.
|
|
|
|
TTLColumnsDescription getColumnTTLs() const;
|
|
|
|
bool hasAnyColumnTTL() const;
|
|
|
|
|
|
|
|
/// Just wrapper for table TTLs, return rows part of table TTLs.
|
|
|
|
TTLDescription getRowsTTL() const;
|
|
|
|
bool hasRowsTTL() const;
|
|
|
|
|
2021-01-13 14:04:27 +00:00
|
|
|
TTLDescriptions getRowsWhereTTLs() const;
|
|
|
|
bool hasAnyRowsWhereTTL() const;
|
2021-01-11 23:07:21 +00:00
|
|
|
|
2020-06-16 12:03:27 +00:00
|
|
|
/// Just wrapper for table TTLs, return moves (to disks or volumes) parts of
|
|
|
|
/// table TTL.
|
|
|
|
TTLDescriptions getMoveTTLs() const;
|
|
|
|
bool hasAnyMoveTTL() const;
|
|
|
|
|
2020-08-31 12:12:51 +00:00
|
|
|
// Just wrapper for table TTLs, return info about recompression ttl
|
|
|
|
TTLDescriptions getRecompressionTTLs() const;
|
|
|
|
bool hasAnyRecompressionTTL() const;
|
|
|
|
|
2020-12-25 14:52:46 +00:00
|
|
|
// Just wrapper for table TTLs, return info about recompression ttl
|
|
|
|
TTLDescriptions getGroupByTTLs() const;
|
|
|
|
bool hasAnyGroupByTTL() const;
|
|
|
|
|
2020-06-16 12:03:27 +00:00
|
|
|
/// Returns columns, which will be needed to calculate dependencies (skip
|
|
|
|
/// indices, TTL expressions) if we update @updated_columns set of columns.
|
|
|
|
ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const;
|
2020-06-16 12:48:10 +00:00
|
|
|
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Block with ordinary + materialized columns.
|
|
|
|
Block getSampleBlock() const;
|
|
|
|
|
|
|
|
/// Block with ordinary columns.
|
|
|
|
Block getSampleBlockNonMaterialized() const;
|
|
|
|
|
|
|
|
/// Block with ordinary + materialized + virtuals. Virtuals have to be
|
2020-08-08 00:47:03 +00:00
|
|
|
/// explicitly specified, because they are part of Storage type, not
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Storage metadata.
|
|
|
|
Block getSampleBlockWithVirtuals(const NamesAndTypesList & virtuals) const;
|
|
|
|
|
|
|
|
|
|
|
|
/// Block with ordinary + materialized + aliases + virtuals. Virtuals have
|
2020-08-08 00:47:03 +00:00
|
|
|
/// to be explicitly specified, because they are part of Storage type, not
|
2020-06-22 09:49:21 +00:00
|
|
|
/// Storage metadata. StorageID required only for more clear exception
|
|
|
|
/// message.
|
2020-06-16 14:25:08 +00:00
|
|
|
Block getSampleBlockForColumns(
|
2021-03-30 21:25:37 +00:00
|
|
|
const Names & column_names, const NamesAndTypesList & virtuals = {}, const StorageID & storage_id = StorageID::createEmpty()) const;
|
2020-06-17 10:34:23 +00:00
|
|
|
/// Returns structure with partition key.
|
|
|
|
const KeyDescription & getPartitionKey() const;
|
|
|
|
/// Returns ASTExpressionList of partition key expression for storage or nullptr if there is none.
|
|
|
|
ASTPtr getPartitionKeyAST() const { return partition_key.definition_ast; }
|
|
|
|
/// Storage has user-defined (in CREATE query) partition key.
|
|
|
|
bool isPartitionKeyDefined() const;
|
|
|
|
/// Storage has partition key.
|
|
|
|
bool hasPartitionKey() const;
|
|
|
|
/// Returns column names that need to be read to calculate partition key.
|
|
|
|
Names getColumnsRequiredForPartitionKey() const;
|
2020-06-17 11:05:11 +00:00
|
|
|
|
|
|
|
/// Returns structure with sorting key.
|
|
|
|
const KeyDescription & getSortingKey() const;
|
|
|
|
/// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none.
|
|
|
|
ASTPtr getSortingKeyAST() const { return sorting_key.definition_ast; }
|
|
|
|
/// Storage has user-defined (in CREATE query) sorting key.
|
|
|
|
bool isSortingKeyDefined() const;
|
|
|
|
/// Storage has sorting key. It means, that it contains at least one column.
|
|
|
|
bool hasSortingKey() const;
|
|
|
|
/// Returns column names that need to be read to calculate sorting key.
|
|
|
|
Names getColumnsRequiredForSortingKey() const;
|
|
|
|
/// Returns columns names in sorting key specified by user in ORDER BY
|
|
|
|
/// expression. For example: 'a', 'x * y', 'toStartOfMonth(date)', etc.
|
|
|
|
Names getSortingKeyColumns() const;
|
|
|
|
|
|
|
|
/// Returns column names that need to be read for FINAL to work.
|
|
|
|
Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); }
|
2020-06-17 12:07:09 +00:00
|
|
|
|
|
|
|
/// Returns structure with sampling key.
|
|
|
|
const KeyDescription & getSamplingKey() const;
|
|
|
|
/// Returns sampling expression AST for storage or nullptr if there is none.
|
|
|
|
ASTPtr getSamplingKeyAST() const { return sampling_key.definition_ast; }
|
|
|
|
/// Storage has user-defined (in CREATE query) sampling key.
|
|
|
|
bool isSamplingKeyDefined() const;
|
|
|
|
/// Storage has sampling key.
|
|
|
|
bool hasSamplingKey() const;
|
|
|
|
/// Returns column names that need to be read to calculate sampling key.
|
|
|
|
Names getColumnsRequiredForSampling() const;
|
2020-06-17 12:39:20 +00:00
|
|
|
|
|
|
|
/// Returns structure with primary key.
|
|
|
|
const KeyDescription & getPrimaryKey() const;
|
|
|
|
/// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none.
|
|
|
|
ASTPtr getPrimaryKeyAST() const { return primary_key.definition_ast; }
|
|
|
|
/// Storage has user-defined (in CREATE query) sorting key.
|
|
|
|
bool isPrimaryKeyDefined() const;
|
|
|
|
/// Storage has primary key (maybe part of some other key). It means, that
|
|
|
|
/// it contains at least one column.
|
|
|
|
bool hasPrimaryKey() const;
|
|
|
|
/// Returns column names that need to be read to calculate primary key.
|
|
|
|
Names getColumnsRequiredForPrimaryKey() const;
|
|
|
|
/// Returns columns names in sorting key specified by. For example: 'a', 'x
|
|
|
|
/// * y', 'toStartOfMonth(date)', etc.
|
|
|
|
Names getPrimaryKeyColumns() const;
|
2020-06-17 13:46:01 +00:00
|
|
|
|
|
|
|
/// Storage settings
|
|
|
|
ASTPtr getSettingsChanges() const;
|
|
|
|
bool hasSettingsChanges() const { return settings_changes != nullptr; }
|
2020-06-17 14:06:22 +00:00
|
|
|
|
|
|
|
/// Select query for *View storages.
|
|
|
|
const SelectQueryDescription & getSelectQuery() const;
|
|
|
|
bool hasSelectQuery() const;
|
2020-06-17 14:32:25 +00:00
|
|
|
|
|
|
|
/// Verify that all the requested names are in the table and are set correctly:
|
|
|
|
/// list of names is not empty and the names do not repeat.
|
2020-06-19 17:17:13 +00:00
|
|
|
void check(const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const;
|
2020-06-17 14:32:25 +00:00
|
|
|
|
|
|
|
/// Check that all the requested names are in the table and have the correct types.
|
|
|
|
void check(const NamesAndTypesList & columns) const;
|
|
|
|
|
|
|
|
/// Check that all names from the intersection of `names` and `columns` are in the table and have the same types.
|
|
|
|
void check(const NamesAndTypesList & columns, const Names & column_names) const;
|
|
|
|
|
|
|
|
/// Check that the data block contains all the columns of the table with the correct types,
|
|
|
|
/// contains only the columns of the table, and all the columns are different.
|
|
|
|
/// If |need_all| is set, then checks that all the columns of the table are in the block.
|
|
|
|
void check(const Block & block, bool need_all = false) const;
|
2019-12-26 18:17:05 +00:00
|
|
|
};
|
|
|
|
|
2020-06-18 11:02:31 +00:00
|
|
|
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
2020-06-15 16:55:33 +00:00
|
|
|
using MultiVersionStorageMetadataPtr = MultiVersion<StorageInMemoryMetadata>;
|
|
|
|
|
2019-12-26 18:17:05 +00:00
|
|
|
}
|