mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Tried to add ability to rename primary key columns but just banned this ability
This commit is contained in:
parent
70ad807511
commit
1d5a77c113
12
dbms/src/Interpreters/RenameColumnVisitor.cpp
Normal file
12
dbms/src/Interpreters/RenameColumnVisitor.cpp
Normal file
@ -0,0 +1,12 @@
|
||||
#include <Interpreters/RenameColumnVisitor.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
void RenameColumnData::visit(ASTIdentifier & identifier, ASTPtr &)
|
||||
{
|
||||
std::optional<String> identifier_column_name = IdentifierSemantic::getColumnName(identifier);
|
||||
if (identifier_column_name && identifier_column_name == column_name)
|
||||
identifier.name = rename_to;
|
||||
}
|
||||
}
|
21
dbms/src/Interpreters/RenameColumnVisitor.h
Normal file
21
dbms/src/Interpreters/RenameColumnVisitor.h
Normal file
@ -0,0 +1,21 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/// Rename ASTIdentifiers to column name
|
||||
struct RenameColumnData
|
||||
{
|
||||
using TypeToVisit = ASTIdentifier;
|
||||
|
||||
String column_name;
|
||||
String rename_to;
|
||||
|
||||
void visit(ASTIdentifier & identifier, ASTPtr & ast);
|
||||
};
|
||||
|
||||
using RenameColumnMatcher = OneTypeMatcher<RenameColumnData>;
|
||||
using RenameColumnVisitor = InDepthNodeVisitor<RenameColumnMatcher, true>;
|
||||
}
|
@ -1457,6 +1457,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
for (const auto & column : getColumns().getAllPhysical())
|
||||
old_types.emplace(column.name, column.type.get());
|
||||
|
||||
|
||||
for (const AlterCommand & command : commands)
|
||||
{
|
||||
if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned)
|
||||
@ -1471,6 +1472,15 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
"ALTER ADD INDEX is not supported for tables with the old syntax",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
if (command.type == AlterCommand::RENAME_COLUMN)
|
||||
{
|
||||
if (columns_alter_type_forbidden.count(command.column_name) || columns_alter_type_metadata_only.count(command.column_name))
|
||||
{
|
||||
throw Exception(
|
||||
"Trying to ALTER RENAME key " + backQuoteIfNeed(command.column_name) + " column which is a part of key expression",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
else if (command.isModifyingData())
|
||||
{
|
||||
if (columns_alter_type_forbidden.count(command.column_name))
|
||||
|
@ -541,10 +541,9 @@ 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.
|
||||
*/
|
||||
/** 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);
|
||||
|
||||
bool hasSortingKey() const { return !sorting_key_columns.empty(); }
|
||||
@ -864,7 +863,6 @@ protected:
|
||||
std::mutex grab_old_parts_mutex;
|
||||
/// The same for clearOldTemporaryDirectories.
|
||||
std::mutex clear_old_temporary_directories_mutex;
|
||||
/// Mutex for settings usage
|
||||
|
||||
void setProperties(const StorageInMemoryMetadata & metadata, bool only_check = false);
|
||||
|
||||
|
@ -447,7 +447,6 @@ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff)
|
||||
{
|
||||
StorageInMemoryMetadata metadata = getInMemoryMetadata();
|
||||
@ -5293,37 +5292,9 @@ bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const
|
||||
}
|
||||
|
||||
|
||||
StorageInMemoryMetadata
|
||||
StorageReplicatedMergeTree::getMetadataFromSharedZookeeper(const String & metadata_str, const String & columns_str) const
|
||||
{
|
||||
auto replicated_metadata = ReplicatedMergeTreeTableMetadata::parse(metadata_str);
|
||||
StorageInMemoryMetadata result = getInMemoryMetadata();
|
||||
result.columns = ColumnsDescription::parse(columns_str);
|
||||
result.constraints = ConstraintsDescription::parse(replicated_metadata.constraints);
|
||||
result.indices = IndicesDescription::parse(replicated_metadata.skip_indices);
|
||||
|
||||
ParserExpression expression_p;
|
||||
|
||||
/// The only thing, that can be changed is ttl expression
|
||||
if (replicated_metadata.primary_key.empty())
|
||||
throw Exception("Primary key cannot be empty" , ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!replicated_metadata.sorting_key.empty())
|
||||
{
|
||||
result.order_by_ast = parseQuery(expression_p, "(" + replicated_metadata.sorting_key + ")", 0);
|
||||
result.primary_key_ast = parseQuery(expression_p, "(" + replicated_metadata.primary_key + ")", 0);
|
||||
}
|
||||
else
|
||||
{
|
||||
result.order_by_ast = parseQuery(expression_p, "(" + replicated_metadata.primary_key + ")", 0);
|
||||
}
|
||||
return result;
|
||||
|
||||
}
|
||||
|
||||
MutationCommands StorageReplicatedMergeTree::getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const
|
||||
{
|
||||
return queue.getFirstAlterMutationCommandsForPart(part);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -526,9 +526,6 @@ private:
|
||||
void waitMutationToFinishOnReplicas(
|
||||
const Strings & replicas, const String & mutation_id) const;
|
||||
|
||||
StorageInMemoryMetadata getMetadataFromSharedZookeeper(const String & metadata_str, const String & columns_str) const;
|
||||
|
||||
|
||||
MutationCommands getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const override;
|
||||
|
||||
protected:
|
||||
|
@ -26,7 +26,6 @@ ALTER TABLE table_for_rename RENAME COLUMN value3 to value2; --{serverError 15}
|
||||
ALTER TABLE table_for_rename RENAME COLUMN value3 TO r1, RENAME COLUMN value3 TO r2; --{serverError 36}
|
||||
ALTER TABLE table_for_rename RENAME COLUMN value3 TO r1, RENAME COLUMN r1 TO value1; --{serverError 10}
|
||||
|
||||
|
||||
ALTER TABLE table_for_rename RENAME COLUMN value2 TO renamed_value2, RENAME COLUMN value3 TO renamed_value3;
|
||||
|
||||
SELECT renamed_value2, renamed_value3 FROM table_for_rename WHERE key = 7;
|
||||
|
@ -9,7 +9,7 @@ CREATE TABLE table_for_rename_replicated
|
||||
value2 String,
|
||||
value3 String
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/table_for_rename_replicaed', '1')
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/table_for_rename_replicated', '1')
|
||||
PARTITION BY date
|
||||
ORDER BY key;
|
||||
|
||||
|
@ -0,0 +1 @@
|
||||
1 1
|
@ -0,0 +1,53 @@
|
||||
DROP TABLE IF EXISTS table_for_rename_pk;
|
||||
|
||||
CREATE TABLE table_for_rename_pk
|
||||
(
|
||||
date Date,
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
key3 UInt64,
|
||||
value1 String,
|
||||
value2 String
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/table_for_rename_pk', '1')
|
||||
PARTITION BY date
|
||||
ORDER BY (key1, pow(key2, 2), key3);
|
||||
|
||||
INSERT INTO table_for_rename_pk SELECT toDate('2019-10-01') + number % 3, number, number, number, toString(number), toString(number) from numbers(9);
|
||||
|
||||
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 44}
|
||||
|
||||
ALTER TABLE table_for_rename_pk RENAME COLUMN key3 TO renamed_key3; --{serverError 44}
|
||||
|
||||
ALTER TABLE table_for_rename_pk RENAME COLUMN key2 TO renamed_key2; --{serverError 44}
|
||||
|
||||
DROP TABLE IF EXISTS table_for_rename_pk;
|
||||
|
||||
DROP TABLE IF EXISTS table_for_rename_with_primary_key;
|
||||
|
||||
CREATE TABLE table_for_rename_with_primary_key
|
||||
(
|
||||
date Date,
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
key3 UInt64,
|
||||
value1 String,
|
||||
value2 String
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/table_for_rename_pk', '1')
|
||||
PARTITION BY date
|
||||
ORDER BY (key1, key2, key3)
|
||||
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 44}
|
||||
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key2 TO renamed_key2; --{serverError 44}
|
||||
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key3 TO renamed_key3; --{serverError 44}
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS table_for_rename_with_primary_key;
|
Loading…
Reference in New Issue
Block a user