Skip to content

Commit

Permalink
Merge pull request #494 from Altinity/backports/24.3/64202_ignore_all…
Browse files Browse the repository at this point in the history
…ow_suspicious_primary_key_on_attach

24.3 Backport of ClickHouse#64202 - Ignore allow_suspicious_primary_key on ATTACH and verify on ALTER
  • Loading branch information
Enmk authored Oct 10, 2024
2 parents 98861ca + de9ab80 commit e02cf10
Show file tree
Hide file tree
Showing 6 changed files with 53 additions and 24 deletions.
13 changes: 13 additions & 0 deletions src/Storages/MergeTree/MergeTreeData.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,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>
Expand Down Expand Up @@ -189,6 +190,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)
Expand Down Expand Up @@ -8432,4 +8434,15 @@ bool MergeTreeData::initializeDiskOnConfigChange(const std::set<String> & new_ad
}
return true;
}

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());
}
}

}
2 changes: 2 additions & 0 deletions src/Storages/MergeTree/MergeTreeData.h
Original file line number Diff line number Diff line change
Expand Up @@ -737,6 +737,8 @@ class MergeTreeData : public IStorage, public WithMutableContext
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.
Expand Down
21 changes: 4 additions & 17 deletions src/Storages/MergeTree/registerStorageMergeTree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSetQuery.h>

#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>

#include <Interpreters/Context.h>
Expand All @@ -31,7 +30,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;
}


Expand Down Expand Up @@ -112,17 +110,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());
}
}


static StoragePtr create(const StorageFactory::Arguments & args)
{
/** [Replicated][|Summing|VersionedCollapsing|Collapsing|Aggregating|Replacing|Graphite]MergeTree (2 * 7 combinations) engines
Expand Down Expand Up @@ -567,8 +554,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)
Expand Down Expand Up @@ -681,8 +668,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);
Expand Down
8 changes: 6 additions & 2 deletions src/Storages/StorageMergeTree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -338,17 +338,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())
{
Expand Down Expand Up @@ -401,7 +405,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);
}

Expand Down
12 changes: 10 additions & 2 deletions src/Storages/StorageReplicatedMergeTree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -6006,6 +6006,7 @@ void StorageReplicatedMergeTree::alter(
assertNotReadonly();

auto table_id = getStorageID();
const auto & query_settings = query_context->getSettingsRef();

if (commands.isSettingsAlter())
{
Expand Down Expand Up @@ -6033,6 +6034,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
{
Expand Down Expand Up @@ -6165,7 +6173,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();
Expand Down Expand Up @@ -6288,7 +6296,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.");
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -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 }
Expand All @@ -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;

0 comments on commit e02cf10

Please sign in to comment.