mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Slightly better interfaces and comments
This commit is contained in:
parent
4eb8a1cfef
commit
465c4b65b7
@ -317,14 +317,18 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con
|
||||
}
|
||||
else if (type == MODIFY_ORDER_BY)
|
||||
{
|
||||
if (metadata.primary_key.definition_ast == nullptr && metadata.sorting_key.definition_ast != nullptr)
|
||||
auto & sorting_key = metadata.sorting_key;
|
||||
auto & primary_key = metadata.primary_key;
|
||||
if (primary_key.definition_ast == nullptr && sorting_key.definition_ast != nullptr)
|
||||
{
|
||||
/// Primary and sorting key become independent after this ALTER so we have to
|
||||
/// save the old ORDER BY expression as the new primary key.
|
||||
metadata.primary_key = metadata.sorting_key;
|
||||
/// Primary and sorting key become independent after this ALTER so
|
||||
/// we have to save the old ORDER BY expression as the new primary
|
||||
/// key.
|
||||
primary_key = KeyDescription::getKeyFromAST(sorting_key.definition_ast, metadata.columns, context);
|
||||
}
|
||||
|
||||
metadata.sorting_key = KeyDescription::getKeyFromAST(order_by, metadata.columns, context, metadata.sorting_key.additional_key_column);
|
||||
/// Recalculate key with new order_by expression
|
||||
sorting_key.recalculateWithNewAST(order_by, metadata.columns, context);
|
||||
}
|
||||
else if (type == COMMENT_COLUMN)
|
||||
{
|
||||
@ -713,15 +717,10 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata, const Context & co
|
||||
command.apply(metadata_copy, context);
|
||||
|
||||
/// Changes in columns may lead to changes in keys expression
|
||||
metadata_copy.sorting_key = KeyDescription::getKeyFromAST(
|
||||
metadata_copy.sorting_key.definition_ast,
|
||||
metadata_copy.columns,
|
||||
context,
|
||||
metadata_copy.sorting_key.additional_key_column);
|
||||
|
||||
metadata_copy.sorting_key.recalculateWithNewColumns(metadata_copy.columns, context);
|
||||
if (metadata_copy.primary_key.definition_ast != nullptr)
|
||||
{
|
||||
metadata_copy.primary_key = KeyDescription::getKeyFromAST(metadata_copy.primary_key.definition_ast, metadata_copy.columns, context);
|
||||
metadata_copy.primary_key.recalculateWithNewColumns(metadata_copy.columns, context);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/KeyDescription.h>
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
@ -10,6 +11,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
KeyDescription::KeyDescription(const KeyDescription & other)
|
||||
: definition_ast(other.definition_ast ? other.definition_ast->clone() : nullptr)
|
||||
, expression_list_ast(other.expression_list_ast ? other.expression_list_ast->clone() : nullptr)
|
||||
@ -17,7 +23,7 @@ KeyDescription::KeyDescription(const KeyDescription & other)
|
||||
, sample_block(other.sample_block)
|
||||
, column_names(other.column_names)
|
||||
, data_types(other.data_types)
|
||||
, additional_key_column(other.additional_key_column ? other.additional_key_column->clone() : nullptr)
|
||||
, additional_column(other.additional_column)
|
||||
{
|
||||
}
|
||||
|
||||
@ -37,27 +43,54 @@ KeyDescription & KeyDescription::operator=(const KeyDescription & other)
|
||||
sample_block = other.sample_block;
|
||||
column_names = other.column_names;
|
||||
data_types = other.data_types;
|
||||
if (other.additional_key_column)
|
||||
additional_key_column = other.additional_key_column->clone();
|
||||
else
|
||||
additional_key_column.reset();
|
||||
|
||||
/// additional_column is constant property It should never be lost.
|
||||
if (additional_column.has_value() && !other.additional_column.has_value())
|
||||
throw Exception("Wrong key assignment, loosing additional_column", ErrorCodes::LOGICAL_ERROR);
|
||||
additional_column = other.additional_column;
|
||||
return *this;
|
||||
}
|
||||
|
||||
|
||||
void KeyDescription::recalculateWithNewAST(
|
||||
const ASTPtr & new_ast,
|
||||
const ColumnsDescription & columns,
|
||||
const Context & context)
|
||||
{
|
||||
*this = getSortingKeyFromAST(new_ast, columns, context, additional_column);
|
||||
}
|
||||
|
||||
void KeyDescription::recalculateWithNewColumns(
|
||||
const ColumnsDescription & new_columns,
|
||||
const Context & context)
|
||||
{
|
||||
*this = getSortingKeyFromAST(definition_ast, new_columns, context, additional_column);
|
||||
}
|
||||
|
||||
KeyDescription KeyDescription::getKeyFromAST(
|
||||
const ASTPtr & definition_ast,
|
||||
const ColumnsDescription & columns,
|
||||
const Context & context)
|
||||
{
|
||||
return getSortingKeyFromAST(definition_ast, columns, context, {});
|
||||
}
|
||||
|
||||
KeyDescription KeyDescription::getSortingKeyFromAST(
|
||||
const ASTPtr & definition_ast,
|
||||
const ColumnsDescription & columns,
|
||||
const Context & context,
|
||||
const ASTPtr & additional_key_column)
|
||||
const std::optional<String> & additional_column)
|
||||
{
|
||||
KeyDescription result;
|
||||
result.definition_ast = definition_ast;
|
||||
result.additional_key_column = additional_key_column;
|
||||
result.expression_list_ast = extractKeyExpressionList(definition_ast);
|
||||
|
||||
if (additional_key_column != nullptr)
|
||||
result.expression_list_ast->children.push_back(additional_key_column);
|
||||
if (additional_column)
|
||||
{
|
||||
result.additional_column = additional_column;
|
||||
ASTPtr column_identifier = std::make_shared<ASTIdentifier>(*additional_column);
|
||||
result.expression_list_ast->children.push_back(column_identifier);
|
||||
}
|
||||
|
||||
const auto & children = result.expression_list_ast->children;
|
||||
for (const auto & child : children)
|
||||
|
@ -30,16 +30,38 @@ struct KeyDescription
|
||||
/// Types from sample block ordered in columns order.
|
||||
DataTypes data_types;
|
||||
|
||||
/// Additional key column added by storage type
|
||||
ASTPtr additional_key_column;
|
||||
/// Additional key column added by storage type. Never changes after
|
||||
/// initialization with non empty value. Doesn't stored in definition_ast,
|
||||
/// but added to expression_list_ast and all its derivatives.
|
||||
std::optional<String> additional_column;
|
||||
|
||||
/// Parse key structure from key definition. Requires all columns, available
|
||||
/// in storage.
|
||||
static KeyDescription getKeyFromAST(
|
||||
const ASTPtr & definition_ast,
|
||||
const ColumnsDescription & columns,
|
||||
const Context & context);
|
||||
|
||||
/// Sorting key can contain additional column defined by storage type (like
|
||||
/// Version column in VersionedCollapsingMergeTree).
|
||||
static KeyDescription getSortingKeyFromAST(
|
||||
const ASTPtr & definition_ast,
|
||||
const ColumnsDescription & columns,
|
||||
const Context & context,
|
||||
const ASTPtr & additional_key_column = nullptr);
|
||||
const std::optional<String> & additional_column);
|
||||
|
||||
/// Recalculate all expressions and fields for key with new columns without
|
||||
/// changes in constant fields. Just wrapper for static methods.
|
||||
void recalculateWithNewColumns(
|
||||
const ColumnsDescription & new_columns,
|
||||
const Context & context);
|
||||
|
||||
/// Recalculate all expressions and fields for key with new ast without
|
||||
/// changes in constant fields. Just wrapper for static methods.
|
||||
void recalculateWithNewAST(
|
||||
const ASTPtr & new_ast,
|
||||
const ColumnsDescription & columns,
|
||||
const Context & context);
|
||||
|
||||
KeyDescription() = default;
|
||||
|
||||
|
@ -483,7 +483,7 @@ void MergeTreeData::initPartitionKey(const KeyDescription & new_partition_key)
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::checkTTLExpressios(const StorageInMemoryMetadata & new_metadata) const
|
||||
void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_metadata) const
|
||||
{
|
||||
auto new_column_ttls = new_metadata.column_ttls_by_name;
|
||||
|
||||
@ -527,7 +527,7 @@ void MergeTreeData::checkTTLExpressios(const StorageInMemoryMetadata & new_metad
|
||||
/// Todo replace columns with TTL for columns
|
||||
void MergeTreeData::setTTLExpressions(const StorageInMemoryMetadata & new_metadata)
|
||||
{
|
||||
checkTTLExpressios(new_metadata);
|
||||
checkTTLExpressions(new_metadata);
|
||||
setColumnTTLs(new_metadata.column_ttls_by_name);
|
||||
auto move_ttl_entries_lock = std::lock_guard<std::mutex>(move_ttl_entries_mutex);
|
||||
setTableTTLs(new_metadata.table_ttl);
|
||||
@ -1339,7 +1339,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
|
||||
checkProperties(new_metadata);
|
||||
|
||||
checkTTLExpressios(new_metadata);
|
||||
checkTTLExpressions(new_metadata);
|
||||
|
||||
if (hasSettingsChanges())
|
||||
{
|
||||
|
@ -783,7 +783,7 @@ protected:
|
||||
|
||||
void initPartitionKey(const KeyDescription & new_partition_key);
|
||||
|
||||
void checkTTLExpressios(const StorageInMemoryMetadata & new_metadata) const;
|
||||
void checkTTLExpressions(const StorageInMemoryMetadata & new_metadata) const;
|
||||
void setTTLExpressions(const StorageInMemoryMetadata & new_metadata);
|
||||
|
||||
void checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const;
|
||||
|
@ -417,7 +417,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
++arg_num;
|
||||
}
|
||||
|
||||
ASTPtr merging_param_key_arg = nullptr;
|
||||
/// This merging param maybe used as part of sorting key
|
||||
std::optional<String> merging_param_key_arg;
|
||||
|
||||
if (merging_params.mode == MergeTreeData::MergingParams::Collapsing)
|
||||
{
|
||||
@ -482,7 +483,9 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
--arg_cnt;
|
||||
merging_param_key_arg = std::make_shared<ASTIdentifier>(merging_params.version_column);
|
||||
/// Version collapsing is the only engine which add additional column to
|
||||
/// sorting key.
|
||||
merging_param_key_arg = merging_params.version_column;
|
||||
}
|
||||
|
||||
String date_column_name;
|
||||
@ -498,6 +501,9 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
if (args.storage_def->partition_by)
|
||||
partition_by_key = args.storage_def->partition_by->ptr();
|
||||
|
||||
/// Partition key may be undefined, but despite this we store it's empty
|
||||
/// value in partition_key structure. MergeTree checks this case and use
|
||||
/// single default partition with name "all".
|
||||
metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_key, metadata.columns, args.context);
|
||||
|
||||
if (!args.storage_def->order_by)
|
||||
@ -505,15 +511,23 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
"If you don't want this table to be sorted, use ORDER BY tuple()",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
metadata.sorting_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.context, merging_param_key_arg);
|
||||
/// Get sorting key from engine arguments.
|
||||
///
|
||||
/// NOTE: store merging_param_key_arg as additional key column. We do it
|
||||
/// before storage creation. After that storage will just copy this
|
||||
/// column if sorting key will be changed.
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.context, merging_param_key_arg);
|
||||
|
||||
/// If primary key explicitely defined, than get it from AST
|
||||
if (args.storage_def->primary_key)
|
||||
{
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context);
|
||||
}
|
||||
else
|
||||
else /// Otherwise we copy it from primary key definition
|
||||
{
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.context);
|
||||
/// and set it's definition_ast to nullptr (so isPrimaryKeyDefined()
|
||||
/// will return false but hasPrimaryKey() will return true.
|
||||
metadata.primary_key.definition_ast = nullptr;
|
||||
}
|
||||
|
||||
@ -564,10 +578,17 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
++arg_num;
|
||||
}
|
||||
|
||||
/// Now only two parameters remain - primary_key, index_granularity.
|
||||
metadata.sorting_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, args.context, merging_param_key_arg);
|
||||
/// Get sorting key from engine arguments.
|
||||
///
|
||||
/// NOTE: store merging_param_key_arg as additional key column. We do it
|
||||
/// before storage creation. After that storage will just copy this
|
||||
/// column if sorting key will be changed.
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(engine_args[arg_num], metadata.columns, args.context, merging_param_key_arg);
|
||||
|
||||
/// In old syntax primary_key always equals to sorting key.
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, args.context);
|
||||
/// But it's not explicitely defined, so we evaluate definition to
|
||||
/// nullptr
|
||||
metadata.primary_key.definition_ast = nullptr;
|
||||
|
||||
++arg_num;
|
||||
|
@ -50,4 +50,5 @@ StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemo
|
||||
return *this;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -775,13 +775,18 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
tuple->arguments->children = new_sorting_key_expr_list->children;
|
||||
order_by_ast = tuple;
|
||||
}
|
||||
new_metadata.sorting_key = KeyDescription::getKeyFromAST(order_by_ast, new_metadata.columns, global_context, new_metadata.sorting_key.additional_key_column);
|
||||
auto & sorting_key = new_metadata.sorting_key;
|
||||
auto & primary_key = new_metadata.primary_key;
|
||||
|
||||
if (!isPrimaryKeyDefined())
|
||||
sorting_key.recalculateWithNewAST(order_by_ast, new_metadata.columns, global_context);
|
||||
|
||||
if (primary_key.definition_ast == nullptr)
|
||||
{
|
||||
/// Primary and sorting key become independent after this ALTER so we have to
|
||||
/// save the old ORDER BY expression as the new primary key.
|
||||
new_metadata.primary_key = getSortingKey();
|
||||
auto old_sorting_key_ast = getSortingKey().definition_ast;
|
||||
primary_key = KeyDescription::getKeyFromAST(
|
||||
old_sorting_key_ast, new_metadata.columns, global_context);
|
||||
}
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user