mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Better checks around metadata
This commit is contained in:
parent
ec933d9d03
commit
8be957ecb5
@ -1,5 +1,6 @@
|
||||
#include "Internals.h"
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/extractKeyExpressionList.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -184,9 +185,9 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast)
|
||||
const auto sorting_key_ast = extractOrderBy(storage_ast);
|
||||
const auto primary_key_ast = extractPrimaryKey(storage_ast);
|
||||
|
||||
const auto sorting_key_expr_list = MergeTreeData::extractKeyExpressionList(sorting_key_ast);
|
||||
const auto sorting_key_expr_list = extractKeyExpressionList(sorting_key_ast);
|
||||
const auto primary_key_expr_list = primary_key_ast
|
||||
? MergeTreeData::extractKeyExpressionList(primary_key_ast) : sorting_key_expr_list->clone();
|
||||
? extractKeyExpressionList(primary_key_ast) : sorting_key_expr_list->clone();
|
||||
|
||||
/// Maybe we have to handle VersionedCollapsing engine separately. But in our case in looks pointless.
|
||||
|
||||
|
@ -324,7 +324,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con
|
||||
metadata.primary_key = metadata.sorting_key;
|
||||
}
|
||||
|
||||
metadata.sorting_key = KeyDescription::getKeyFromAST(order_by, metadata.columns, context);
|
||||
metadata.sorting_key = KeyDescription::getKeyFromAST(order_by, metadata.columns, context, metadata.sorting_key.additional_key_column);
|
||||
}
|
||||
else if (type == COMMENT_COLUMN)
|
||||
{
|
||||
@ -713,7 +713,11 @@ 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 = KeyDescription::getKeyFromAST(
|
||||
metadata_copy.sorting_key.definition_ast,
|
||||
metadata_copy.columns,
|
||||
context,
|
||||
metadata_copy.sorting_key.additional_key_column);
|
||||
|
||||
if (metadata_copy.primary_key.definition_ast != nullptr)
|
||||
{
|
||||
@ -721,7 +725,7 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata, const Context & co
|
||||
}
|
||||
else
|
||||
{
|
||||
metadata_copy.primary_key = metadata_copy.sorting_key;
|
||||
metadata_copy.primary_key = KeyDescription::getKeyFromAST(metadata_copy.sorting_key.definition_ast, metadata_copy.columns, context);
|
||||
metadata_copy.primary_key.definition_ast = nullptr;
|
||||
}
|
||||
|
||||
|
@ -387,7 +387,7 @@ void IStorage::alter(
|
||||
}
|
||||
|
||||
|
||||
void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */)
|
||||
void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
|
@ -359,7 +359,7 @@ public:
|
||||
/** Checks that alter commands can be applied to storage. For example, columns can be modified,
|
||||
* or primary key can be changes, etc.
|
||||
*/
|
||||
virtual void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings);
|
||||
virtual void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const;
|
||||
|
||||
/** ALTER tables with regard to its partitions.
|
||||
* Should handle locks for each command on its own.
|
||||
|
@ -17,6 +17,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)
|
||||
{
|
||||
}
|
||||
|
||||
@ -36,18 +37,23 @@ 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();
|
||||
return *this;
|
||||
}
|
||||
|
||||
|
||||
KeyDescription KeyDescription::getKeyFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context, ASTPtr additional_key_expression)
|
||||
KeyDescription KeyDescription::getKeyFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context, const ASTPtr & additional_key_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_expression)
|
||||
result.expression_list_ast->children.push_back(additional_key_expression);
|
||||
if (additional_key_column != nullptr)
|
||||
result.expression_list_ast->children.push_back(additional_key_column);
|
||||
|
||||
const auto & children = result.expression_list_ast->children;
|
||||
for (const auto & child : children)
|
||||
|
@ -30,9 +30,16 @@ struct KeyDescription
|
||||
/// Types from sample block ordered in columns order.
|
||||
DataTypes data_types;
|
||||
|
||||
/// Additional key column added by storage
|
||||
ASTPtr additional_key_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, ASTPtr additional_key_expression = nullptr);
|
||||
static KeyDescription getKeyFromAST(
|
||||
const ASTPtr & definition_ast,
|
||||
const ColumnsDescription & columns,
|
||||
const Context & context,
|
||||
const ASTPtr & additional_key_expression = nullptr);
|
||||
|
||||
KeyDescription() = default;
|
||||
|
||||
|
@ -144,7 +144,7 @@ MergeTreeData::MergeTreeData(
|
||||
|
||||
setSettingsChanges(metadata_.settings_changes);
|
||||
const auto settings = getSettings();
|
||||
setProperties(metadata_, /*only_check*/ false, attach);
|
||||
setProperties(metadata_, attach);
|
||||
|
||||
/// NOTE: using the same columns list as is read when performing actual merges.
|
||||
merging_params.check(getColumns().getAllPhysical());
|
||||
@ -275,33 +275,13 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata, bool only_check, bool attach)
|
||||
void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata, bool attach) const
|
||||
{
|
||||
KeyDescription new_primary_key = new_metadata.primary_key;
|
||||
|
||||
if (!new_metadata.sorting_key.definition_ast)
|
||||
throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
KeyDescription new_sorting_key;
|
||||
if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing)
|
||||
new_sorting_key = KeyDescription::getKeyFromAST(
|
||||
new_metadata.sorting_key.definition_ast,
|
||||
new_metadata.columns,
|
||||
global_context,
|
||||
std::make_shared<ASTIdentifier>(merging_params.version_column));
|
||||
else
|
||||
new_sorting_key = new_metadata.sorting_key;
|
||||
|
||||
/// Primary key not defined at all
|
||||
if (new_primary_key.definition_ast == nullptr)
|
||||
{
|
||||
/// We copy sorting key, and restore definition_ast to empty value,
|
||||
/// because in merge tree code we chech, that our primary key is fake
|
||||
/// (copied from sorting key, i.e. isPrimaryKeyDefined() == false, but
|
||||
/// hasSortingKey() == true)
|
||||
new_primary_key = new_metadata.sorting_key;
|
||||
new_primary_key.definition_ast = nullptr;
|
||||
}
|
||||
KeyDescription new_sorting_key = new_metadata.sorting_key;
|
||||
KeyDescription new_primary_key = new_metadata.primary_key;
|
||||
|
||||
size_t sorting_key_size = new_sorting_key.column_names.size();
|
||||
size_t primary_key_size = new_primary_key.column_names.size();
|
||||
@ -333,7 +313,7 @@ void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata,
|
||||
auto all_columns = new_metadata.columns.getAllPhysical();
|
||||
|
||||
/// Order by check AST
|
||||
if (hasSortingKey() && only_check)
|
||||
if (hasSortingKey())
|
||||
{
|
||||
/// This is ALTER, not CREATE/ATTACH TABLE. Let us check that all new columns used in the sorting key
|
||||
/// expression have just been added (so that the sorting order is guaranteed to be valid with the new key).
|
||||
@ -400,15 +380,18 @@ void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata,
|
||||
|
||||
checkKeyExpression(*new_sorting_key.expression, new_sorting_key.sample_block, "Sorting");
|
||||
|
||||
if (!only_check)
|
||||
{
|
||||
/// Other parts of metadata initialized is separate methods
|
||||
setColumns(std::move(new_metadata.columns));
|
||||
setSecondaryIndices(std::move(new_metadata.secondary_indices));
|
||||
setConstraints(std::move(new_metadata.constraints));
|
||||
setSortingKey(std::move(new_sorting_key));
|
||||
setPrimaryKey(std::move(new_primary_key));
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata, bool attach)
|
||||
{
|
||||
checkProperties(new_metadata, attach);
|
||||
|
||||
/// Other parts of metadata initialized is separate methods
|
||||
setColumns(std::move(new_metadata.columns));
|
||||
setSecondaryIndices(std::move(new_metadata.secondary_indices));
|
||||
setConstraints(std::move(new_metadata.constraints));
|
||||
setSortingKey(std::move(new_metadata.sorting_key));
|
||||
setPrimaryKey(std::move(new_metadata.primary_key));
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -442,27 +425,6 @@ ExpressionActionsPtr MergeTreeData::getSortingKeyAndSkipIndicesExpression() cons
|
||||
return getCombinedIndicesExpression(getSortingKey(), getSecondaryIndices(), getColumns(), global_context);
|
||||
}
|
||||
|
||||
ASTPtr MergeTreeData::extractKeyExpressionList(const ASTPtr & node)
|
||||
{
|
||||
if (!node)
|
||||
return std::make_shared<ASTExpressionList>();
|
||||
|
||||
const auto * expr_func = node->as<ASTFunction>();
|
||||
|
||||
if (expr_func && expr_func->name == "tuple")
|
||||
{
|
||||
/// Primary key is specified in tuple, extract its arguments.
|
||||
return expr_func->arguments->clone();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Primary key consists of one column.
|
||||
auto res = std::make_shared<ASTExpressionList>();
|
||||
res->children.push_back(node);
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::initPartitionKey(const KeyDescription & new_partition_key)
|
||||
{
|
||||
@ -521,8 +483,7 @@ void MergeTreeData::initPartitionKey(const KeyDescription & new_partition_key)
|
||||
}
|
||||
|
||||
|
||||
/// Todo replace columns with TTL for columns
|
||||
void MergeTreeData::setTTLExpressions(const StorageInMemoryMetadata & new_metadata, bool only_check)
|
||||
void MergeTreeData::checkTTLExpressios(const StorageInMemoryMetadata & new_metadata) const
|
||||
{
|
||||
auto new_column_ttls = new_metadata.column_ttls_by_name;
|
||||
|
||||
@ -543,11 +504,7 @@ void MergeTreeData::setTTLExpressions(const StorageInMemoryMetadata & new_metada
|
||||
if (columns_ttl_forbidden.count(name))
|
||||
throw Exception("Trying to set TTL for key column " + name, ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
if (!only_check)
|
||||
setColumnTTLs(new_column_ttls);
|
||||
}
|
||||
|
||||
auto new_table_ttl = new_metadata.table_ttl;
|
||||
|
||||
if (new_table_ttl.definition_ast)
|
||||
@ -564,15 +521,18 @@ void MergeTreeData::setTTLExpressions(const StorageInMemoryMetadata & new_metada
|
||||
throw Exception(message, ErrorCodes::BAD_TTL_EXPRESSION);
|
||||
}
|
||||
}
|
||||
|
||||
if (!only_check)
|
||||
{
|
||||
auto move_ttl_entries_lock = std::lock_guard<std::mutex>(move_ttl_entries_mutex);
|
||||
setTableTTLs(new_table_ttl);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Todo replace columns with TTL for columns
|
||||
void MergeTreeData::setTTLExpressions(const StorageInMemoryMetadata & new_metadata)
|
||||
{
|
||||
checkTTLExpressios(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);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const
|
||||
{
|
||||
@ -1264,7 +1224,7 @@ bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to)
|
||||
|
||||
}
|
||||
|
||||
void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings)
|
||||
void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const
|
||||
{
|
||||
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
|
||||
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
|
||||
@ -1359,9 +1319,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
}
|
||||
}
|
||||
|
||||
setProperties(new_metadata, /* only_check = */ true);
|
||||
checkProperties(new_metadata);
|
||||
|
||||
setTTLExpressions(new_metadata, /* only_check = */ true);
|
||||
checkTTLExpressios(new_metadata);
|
||||
|
||||
if (hasSettingsChanges())
|
||||
{
|
||||
|
@ -492,7 +492,7 @@ public:
|
||||
/// - all type conversions can be done.
|
||||
/// - columns corresponding to primary key, indices, sign, sampling expression and date are not affected.
|
||||
/// If something is wrong, throws an exception.
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
|
||||
|
||||
/// Change MergeTreeSettings
|
||||
void changeSettings(
|
||||
@ -508,12 +508,6 @@ public:
|
||||
broken_part_callback(name);
|
||||
}
|
||||
|
||||
/** Get the key expression AST as an ASTExpressionList. It can be specified
|
||||
* in the tuple: (CounterID, Date), or as one column: CounterID.
|
||||
*/
|
||||
static ASTPtr extractKeyExpressionList(const ASTPtr & node);
|
||||
|
||||
|
||||
/// Check that the part is not broken and calculate the checksums for it if they are not present.
|
||||
MutableDataPartPtr loadPartAndFixMetadata(const VolumePtr & volume, const String & relative_path) const;
|
||||
|
||||
@ -780,11 +774,14 @@ protected:
|
||||
/// The same for clearOldTemporaryDirectories.
|
||||
std::mutex clear_old_temporary_directories_mutex;
|
||||
|
||||
void setProperties(const StorageInMemoryMetadata & new_metadata, bool only_check = false, bool attach = false);
|
||||
void checkProperties(const StorageInMemoryMetadata & new_metadata, bool attach = false) const;
|
||||
|
||||
void setProperties(const StorageInMemoryMetadata & new_metadata, bool attach = false);
|
||||
|
||||
void initPartitionKey(const KeyDescription & new_partition_key);
|
||||
|
||||
void setTTLExpressions(const StorageInMemoryMetadata & new_metadata, bool only_check = false);
|
||||
void checkTTLExpressios(const StorageInMemoryMetadata & new_metadata) const;
|
||||
void setTTLExpressions(const StorageInMemoryMetadata & new_metadata);
|
||||
|
||||
void checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const;
|
||||
|
||||
|
@ -417,6 +417,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
++arg_num;
|
||||
}
|
||||
|
||||
ASTPtr merging_param_key_arg = nullptr;
|
||||
|
||||
if (merging_params.mode == MergeTreeData::MergingParams::Collapsing)
|
||||
{
|
||||
if (!tryGetIdentifierNameInto(engine_args[arg_cnt - 1], merging_params.sign_column))
|
||||
@ -480,6 +482,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
--arg_cnt;
|
||||
merging_param_key_arg = std::make_shared<ASTIdentifier>(merging_params.version_column);
|
||||
}
|
||||
|
||||
String date_column_name;
|
||||
@ -502,13 +505,15 @@ 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);
|
||||
metadata.sorting_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.context, merging_param_key_arg);
|
||||
|
||||
if (args.storage_def->primary_key)
|
||||
{
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context);
|
||||
}
|
||||
else
|
||||
{
|
||||
metadata.primary_key = metadata.sorting_key;
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.context);
|
||||
metadata.primary_key.definition_ast = nullptr;
|
||||
}
|
||||
|
||||
@ -560,7 +565,11 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
}
|
||||
|
||||
/// Now only two parameters remain - primary_key, index_granularity.
|
||||
metadata.sorting_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, args.context);
|
||||
metadata.sorting_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, args.context, merging_param_key_arg);
|
||||
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, args.context);
|
||||
metadata.primary_key.definition_ast = nullptr;
|
||||
|
||||
++arg_num;
|
||||
|
||||
const auto * ast = engine_args[arg_num]->as<ASTLiteral>();
|
||||
|
@ -723,7 +723,7 @@ void StorageBuffer::reschedule()
|
||||
flush_handle->scheduleAfter(std::min(min, max) * 1000);
|
||||
}
|
||||
|
||||
void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */)
|
||||
void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
|
@ -85,7 +85,7 @@ public:
|
||||
|
||||
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override;
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
|
||||
|
||||
/// The structure of the subordinate table is not checked and does not change.
|
||||
void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
|
||||
|
@ -538,7 +538,7 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const Context & c
|
||||
}
|
||||
|
||||
|
||||
void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */)
|
||||
void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
|
@ -84,7 +84,7 @@ public:
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
void renameOnDisk(const String & new_path_to_table_data);
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
|
||||
|
||||
/// in the sub-tables, you need to manually add and delete columns
|
||||
/// the structure of the sub-table is not checked
|
||||
|
@ -213,7 +213,7 @@ void StorageMaterializedView::alter(
|
||||
}
|
||||
|
||||
|
||||
void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings)
|
||||
void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const
|
||||
{
|
||||
if (settings.allow_experimental_alter_materialized_view_structure)
|
||||
{
|
||||
|
@ -41,7 +41,7 @@ public:
|
||||
|
||||
void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
|
||||
|
||||
void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) override;
|
||||
|
||||
|
@ -372,7 +372,7 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const Context & cont
|
||||
}
|
||||
|
||||
|
||||
void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */)
|
||||
void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
|
@ -37,7 +37,7 @@ public:
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
|
||||
|
||||
/// you need to add and remove columns in the sub-tables manually
|
||||
/// the structure of sub-tables is not checked
|
||||
|
@ -32,7 +32,7 @@ void registerStorageNull(StorageFactory & factory)
|
||||
});
|
||||
}
|
||||
|
||||
void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */)
|
||||
void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
|
@ -40,7 +40,7 @@ public:
|
||||
return std::make_shared<NullBlockOutputStream>(getSampleBlock());
|
||||
}
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) override;
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
|
||||
|
||||
void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
|
||||
|
||||
|
@ -492,7 +492,7 @@ 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 = KeyDescription::getKeyFromAST(order_by_ast, new_metadata.columns, global_context, new_metadata.sorting_key.additional_key_column);
|
||||
|
||||
if (!isPrimaryKeyDefined())
|
||||
{
|
||||
|
@ -17,11 +17,11 @@ INSERT INTO table_for_rename_pk SELECT toDate('2019-10-01') + number % 3, number
|
||||
|
||||
SELECT key1, value1 FROM table_for_rename_pk WHERE key1 = 1 AND key2 = 1 AND key3 = 1;
|
||||
|
||||
ALTER TABLE table_for_rename_pk RENAME COLUMN key1 TO renamed_key1; --{serverError 524}
|
||||
ALTER TABLE table_for_rename_pk RENAME COLUMN key1 TO renamed_key1; --{serverError 47}
|
||||
|
||||
ALTER TABLE table_for_rename_pk RENAME COLUMN key3 TO renamed_key3; --{serverError 524}
|
||||
ALTER TABLE table_for_rename_pk RENAME COLUMN key3 TO renamed_key3; --{serverError 47}
|
||||
|
||||
ALTER TABLE table_for_rename_pk RENAME COLUMN key2 TO renamed_key2; --{serverError 524}
|
||||
ALTER TABLE table_for_rename_pk RENAME COLUMN key2 TO renamed_key2; --{serverError 47}
|
||||
|
||||
DROP TABLE IF EXISTS table_for_rename_pk NO DELAY;
|
||||
SELECT sleep(1) FORMAT Null;
|
||||
@ -45,11 +45,11 @@ PRIMARY KEY (key1, key2);
|
||||
|
||||
INSERT INTO table_for_rename_with_primary_key SELECT toDate('2019-10-01') + number % 3, number, number, number, toString(number), toString(number) from numbers(9);
|
||||
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key1 TO renamed_key1; --{serverError 524}
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key1 TO renamed_key1; --{serverError 47}
|
||||
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key2 TO renamed_key2; --{serverError 524}
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key2 TO renamed_key2; --{serverError 47}
|
||||
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key3 TO renamed_key3; --{serverError 524}
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key3 TO renamed_key3; --{serverError 47}
|
||||
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN value1 TO renamed_value1; --{serverError 524}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user