mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Ignore allow_suspicious_primary_key on ATTACH and verify on ALTER
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com> Co-authored-by: Alexander Tokmakov <tavplubix@clickhouse.com>
This commit is contained in:
parent
26c9c156c4
commit
8dd52a2625
@ -25,6 +25,7 @@
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
@ -188,6 +189,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_SCHEDULE_TASK;
|
||||
extern const int LIMIT_EXCEEDED;
|
||||
extern const int CANNOT_FORGET_PARTITION;
|
||||
extern const int DATA_TYPE_CANNOT_BE_USED_IN_KEY;
|
||||
}
|
||||
|
||||
static void checkSuspiciousIndices(const ASTFunction * index_function)
|
||||
@ -8538,6 +8540,16 @@ void MergeTreeData::unloadPrimaryKeys()
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key)
|
||||
{
|
||||
/// Aggregate functions already forbidden, but SimpleAggregateFunction are not
|
||||
for (const auto & data_type : sorting_key.data_types)
|
||||
{
|
||||
if (dynamic_cast<const DataTypeCustomSimpleAggregateFunction *>(data_type->getCustomName()))
|
||||
throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_USED_IN_KEY, "Column with type {} is not allowed in key expression", data_type->getCustomName()->getName());
|
||||
}
|
||||
}
|
||||
|
||||
bool updateAlterConversionsMutations(const MutationCommands & commands, std::atomic<ssize_t> & alter_conversions_mutations, bool remove)
|
||||
{
|
||||
for (const auto & command : commands)
|
||||
|
@ -736,6 +736,8 @@ public:
|
||||
const ASTPtr & new_settings,
|
||||
AlterLockHolder & table_lock_holder);
|
||||
|
||||
static void verifySortingKey(const KeyDescription & sorting_key);
|
||||
|
||||
/// Should be called if part data is suspected to be corrupted.
|
||||
/// Has the ability to check all other parts
|
||||
/// which reside on the same disk of the suspicious part.
|
||||
|
@ -14,7 +14,6 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
|
||||
#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
@ -32,7 +31,6 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_STORAGE;
|
||||
extern const int NO_REPLICA_NAME_GIVEN;
|
||||
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
|
||||
extern const int DATA_TYPE_CANNOT_BE_USED_IN_KEY;
|
||||
}
|
||||
|
||||
|
||||
@ -113,16 +111,6 @@ static ColumnsDescription getColumnsDescriptionFromZookeeper(const String & raw_
|
||||
return ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_path) / "columns", &columns_stat));
|
||||
}
|
||||
|
||||
static void verifySortingKey(const KeyDescription & sorting_key)
|
||||
{
|
||||
/// Aggregate functions already forbidden, but SimpleAggregateFunction are not
|
||||
for (const auto & data_type : sorting_key.data_types)
|
||||
{
|
||||
if (dynamic_cast<const DataTypeCustomSimpleAggregateFunction *>(data_type->getCustomName()))
|
||||
throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_USED_IN_KEY, "Column with type {} is not allowed in key expression", data_type->getCustomName()->getName());
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns whether a new syntax is used to define a table engine, i.e. MergeTree() PRIMARY KEY ... PARTITION BY ... SETTINGS ...
|
||||
/// instead of MergeTree(MergeTree(date, [sample_key], primary_key).
|
||||
static bool isExtendedStorageDef(const ASTCreateQuery & query)
|
||||
@ -678,8 +666,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
/// column if sorting key will be changed.
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(
|
||||
args.storage_def->order_by->ptr(), metadata.columns, context, merging_param_key_arg);
|
||||
if (!local_settings.allow_suspicious_primary_key)
|
||||
verifySortingKey(metadata.sorting_key);
|
||||
if (!local_settings.allow_suspicious_primary_key && args.mode <= LoadingStrictnessLevel::CREATE)
|
||||
MergeTreeData::verifySortingKey(metadata.sorting_key);
|
||||
|
||||
/// If primary key explicitly defined, than get it from AST
|
||||
if (args.storage_def->primary_key)
|
||||
@ -792,8 +780,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
/// column if sorting key will be changed.
|
||||
metadata.sorting_key
|
||||
= KeyDescription::getSortingKeyFromAST(engine_args[arg_num], metadata.columns, context, merging_param_key_arg);
|
||||
if (!local_settings.allow_suspicious_primary_key)
|
||||
verifySortingKey(metadata.sorting_key);
|
||||
if (!local_settings.allow_suspicious_primary_key && args.mode <= LoadingStrictnessLevel::CREATE)
|
||||
MergeTreeData::verifySortingKey(metadata.sorting_key);
|
||||
|
||||
/// In old syntax primary_key always equals to sorting key.
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, context);
|
||||
|
@ -333,17 +333,21 @@ void StorageMergeTree::alter(
|
||||
|
||||
auto table_id = getStorageID();
|
||||
auto old_storage_settings = getSettings();
|
||||
const auto & query_settings = local_context->getSettingsRef();
|
||||
|
||||
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
|
||||
StorageInMemoryMetadata old_metadata = getInMemoryMetadata();
|
||||
|
||||
auto maybe_mutation_commands = commands.getMutationCommands(new_metadata, local_context->getSettingsRef().materialize_ttl_after_modify, local_context);
|
||||
auto maybe_mutation_commands = commands.getMutationCommands(new_metadata, query_settings.materialize_ttl_after_modify, local_context);
|
||||
if (!maybe_mutation_commands.empty())
|
||||
delayMutationOrThrowIfNeeded(nullptr, local_context);
|
||||
|
||||
Int64 mutation_version = -1;
|
||||
commands.apply(new_metadata, local_context);
|
||||
|
||||
if (!query_settings.allow_suspicious_primary_key)
|
||||
MergeTreeData::verifySortingKey(new_metadata.sorting_key);
|
||||
|
||||
/// This alter can be performed at new_metadata level only
|
||||
if (commands.isSettingsAlter())
|
||||
{
|
||||
@ -396,7 +400,7 @@ void StorageMergeTree::alter(
|
||||
resetObjectColumnsFromActiveParts(parts_lock);
|
||||
}
|
||||
|
||||
if (!maybe_mutation_commands.empty() && local_context->getSettingsRef().alter_sync > 0)
|
||||
if (!maybe_mutation_commands.empty() && query_settings.alter_sync > 0)
|
||||
waitForMutation(mutation_version, false);
|
||||
}
|
||||
|
||||
|
@ -6027,6 +6027,7 @@ void StorageReplicatedMergeTree::alter(
|
||||
assertNotReadonly();
|
||||
|
||||
auto table_id = getStorageID();
|
||||
const auto & query_settings = query_context->getSettingsRef();
|
||||
|
||||
if (commands.isSettingsAlter())
|
||||
{
|
||||
@ -6054,6 +6055,13 @@ void StorageReplicatedMergeTree::alter(
|
||||
return;
|
||||
}
|
||||
|
||||
if (!query_settings.allow_suspicious_primary_key)
|
||||
{
|
||||
StorageInMemoryMetadata future_metadata = getInMemoryMetadata();
|
||||
commands.apply(future_metadata, query_context);
|
||||
|
||||
MergeTreeData::verifySortingKey(future_metadata.sorting_key);
|
||||
}
|
||||
|
||||
auto ast_to_str = [](ASTPtr query) -> String
|
||||
{
|
||||
@ -6186,7 +6194,7 @@ void StorageReplicatedMergeTree::alter(
|
||||
|
||||
auto maybe_mutation_commands = commands.getMutationCommands(
|
||||
*current_metadata,
|
||||
query_context->getSettingsRef().materialize_ttl_after_modify,
|
||||
query_settings.materialize_ttl_after_modify,
|
||||
query_context);
|
||||
|
||||
bool have_mutation = !maybe_mutation_commands.empty();
|
||||
@ -6309,7 +6317,7 @@ void StorageReplicatedMergeTree::alter(
|
||||
{
|
||||
LOG_DEBUG(log, "Metadata changes applied. Will wait for data changes.");
|
||||
merge_selecting_task->schedule();
|
||||
waitMutation(*mutation_znode, query_context->getSettingsRef().alter_sync);
|
||||
waitMutation(*mutation_znode, query_settings.alter_sync);
|
||||
LOG_DEBUG(log, "Data changes applied.");
|
||||
}
|
||||
}
|
||||
|
@ -1,6 +1,6 @@
|
||||
set allow_suspicious_primary_key = 0;
|
||||
|
||||
DROP TABLE IF EXISTS data;
|
||||
drop table if exists data;
|
||||
|
||||
create table data (key Int, value AggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() order by (key, value); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
|
||||
create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() order by (key, value); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
|
||||
@ -12,7 +12,22 @@ create table data (key Int, value AggregateFunction(sum, UInt64)) engine=Aggrega
|
||||
create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value order by (value, key); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
|
||||
|
||||
set allow_suspicious_primary_key = 1;
|
||||
|
||||
create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value order by (value, key);
|
||||
|
||||
DROP TABLE data;
|
||||
-- ATTACH should work regardless allow_suspicious_primary_key
|
||||
set allow_suspicious_primary_key = 0;
|
||||
detach table data;
|
||||
attach table data;
|
||||
drop table data;
|
||||
|
||||
-- ALTER AggregatingMergeTree
|
||||
create table data (key Int) engine=AggregatingMergeTree() order by (key);
|
||||
alter table data add column value SimpleAggregateFunction(sum, UInt64), modify order by (key, value); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
|
||||
alter table data add column value SimpleAggregateFunction(sum, UInt64), modify order by (key, value) settings allow_suspicious_primary_key=1;
|
||||
drop table data;
|
||||
|
||||
-- ALTER ReplicatedAggregatingMergeTree
|
||||
create table data_rep (key Int) engine=ReplicatedAggregatingMergeTree('/tables/{database}', 'r1') order by (key);
|
||||
alter table data_rep add column value SimpleAggregateFunction(sum, UInt64), modify order by (key, value); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
|
||||
alter table data_rep add column value SimpleAggregateFunction(sum, UInt64), modify order by (key, value) settings allow_suspicious_primary_key=1;
|
||||
drop table data_rep;
|
||||
|
Loading…
Reference in New Issue
Block a user