mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #3887 from yandex/remove-alter-primary-key
Remove ALTER MODIFY PRIMARY KEY command
This commit is contained in:
commit
b1602a5e62
@ -204,7 +204,6 @@ static bool isSupportedAlterType(int type)
|
|||||||
ASTAlterCommand::ADD_COLUMN,
|
ASTAlterCommand::ADD_COLUMN,
|
||||||
ASTAlterCommand::DROP_COLUMN,
|
ASTAlterCommand::DROP_COLUMN,
|
||||||
ASTAlterCommand::MODIFY_COLUMN,
|
ASTAlterCommand::MODIFY_COLUMN,
|
||||||
ASTAlterCommand::MODIFY_PRIMARY_KEY,
|
|
||||||
ASTAlterCommand::DROP_PARTITION,
|
ASTAlterCommand::DROP_PARTITION,
|
||||||
ASTAlterCommand::DELETE,
|
ASTAlterCommand::DELETE,
|
||||||
ASTAlterCommand::UPDATE,
|
ASTAlterCommand::UPDATE,
|
||||||
|
@ -25,11 +25,6 @@ ASTPtr ASTAlterCommand::clone() const
|
|||||||
res->column = column->clone();
|
res->column = column->clone();
|
||||||
res->children.push_back(res->column);
|
res->children.push_back(res->column);
|
||||||
}
|
}
|
||||||
if (primary_key)
|
|
||||||
{
|
|
||||||
res->primary_key = primary_key->clone();
|
|
||||||
res->children.push_back(res->primary_key);
|
|
||||||
}
|
|
||||||
if (order_by)
|
if (order_by)
|
||||||
{
|
{
|
||||||
res->order_by = order_by->clone();
|
res->order_by = order_by->clone();
|
||||||
@ -82,11 +77,6 @@ void ASTAlterCommand::formatImpl(
|
|||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (settings.hilite ? hilite_none : "");
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (settings.hilite ? hilite_none : "");
|
||||||
col_decl->formatImpl(settings, state, frame);
|
col_decl->formatImpl(settings, state, frame);
|
||||||
}
|
}
|
||||||
else if (type == ASTAlterCommand::MODIFY_PRIMARY_KEY)
|
|
||||||
{
|
|
||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY PRIMARY KEY " << (settings.hilite ? hilite_none : "");
|
|
||||||
primary_key->formatImpl(settings, state, frame);
|
|
||||||
}
|
|
||||||
else if (type == ASTAlterCommand::MODIFY_ORDER_BY)
|
else if (type == ASTAlterCommand::MODIFY_ORDER_BY)
|
||||||
{
|
{
|
||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : "");
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : "");
|
||||||
|
@ -26,7 +26,6 @@ public:
|
|||||||
DROP_COLUMN,
|
DROP_COLUMN,
|
||||||
MODIFY_COLUMN,
|
MODIFY_COLUMN,
|
||||||
COMMENT_COLUMN,
|
COMMENT_COLUMN,
|
||||||
MODIFY_PRIMARY_KEY,
|
|
||||||
MODIFY_ORDER_BY,
|
MODIFY_ORDER_BY,
|
||||||
|
|
||||||
DROP_PARTITION,
|
DROP_PARTITION,
|
||||||
@ -55,10 +54,6 @@ public:
|
|||||||
*/
|
*/
|
||||||
ASTPtr column;
|
ASTPtr column;
|
||||||
|
|
||||||
/** For MODIFY PRIMARY KEY
|
|
||||||
*/
|
|
||||||
ASTPtr primary_key;
|
|
||||||
|
|
||||||
/** For MODIFY ORDER BY
|
/** For MODIFY ORDER BY
|
||||||
*/
|
*/
|
||||||
ASTPtr order_by;
|
ASTPtr order_by;
|
||||||
|
@ -24,7 +24,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
|||||||
ParserKeyword s_clear_column("CLEAR COLUMN");
|
ParserKeyword s_clear_column("CLEAR COLUMN");
|
||||||
ParserKeyword s_modify_column("MODIFY COLUMN");
|
ParserKeyword s_modify_column("MODIFY COLUMN");
|
||||||
ParserKeyword s_comment_column("COMMENT COLUMN");
|
ParserKeyword s_comment_column("COMMENT COLUMN");
|
||||||
ParserKeyword s_modify_primary_key("MODIFY PRIMARY KEY");
|
|
||||||
ParserKeyword s_modify_order_by("MODIFY ORDER BY");
|
ParserKeyword s_modify_order_by("MODIFY ORDER BY");
|
||||||
|
|
||||||
ParserKeyword s_attach_partition("ATTACH PARTITION");
|
ParserKeyword s_attach_partition("ATTACH PARTITION");
|
||||||
@ -196,13 +195,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
|||||||
|
|
||||||
command->type = ASTAlterCommand::MODIFY_COLUMN;
|
command->type = ASTAlterCommand::MODIFY_COLUMN;
|
||||||
}
|
}
|
||||||
else if (s_modify_primary_key.ignore(pos, expected))
|
|
||||||
{
|
|
||||||
if (!parser_exp_elem.parse(pos, command->primary_key, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
command->type = ASTAlterCommand::MODIFY_PRIMARY_KEY;
|
|
||||||
}
|
|
||||||
else if (s_modify_order_by.ignore(pos, expected))
|
else if (s_modify_order_by.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
if (!parser_exp_elem.parse(pos, command->order_by, expected))
|
if (!parser_exp_elem.parse(pos, command->order_by, expected))
|
||||||
@ -247,14 +239,16 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
|||||||
command->children.push_back(command->col_decl);
|
command->children.push_back(command->col_decl);
|
||||||
if (command->column)
|
if (command->column)
|
||||||
command->children.push_back(command->column);
|
command->children.push_back(command->column);
|
||||||
if (command->primary_key)
|
|
||||||
command->children.push_back(command->primary_key);
|
|
||||||
if (command->partition)
|
if (command->partition)
|
||||||
command->children.push_back(command->partition);
|
command->children.push_back(command->partition);
|
||||||
|
if (command->order_by)
|
||||||
|
command->children.push_back(command->order_by);
|
||||||
if (command->predicate)
|
if (command->predicate)
|
||||||
command->children.push_back(command->predicate);
|
command->children.push_back(command->predicate);
|
||||||
if (command->update_assignments)
|
if (command->update_assignments)
|
||||||
command->children.push_back(command->update_assignments);
|
command->children.push_back(command->update_assignments);
|
||||||
|
if (command->comment)
|
||||||
|
command->children.push_back(command->comment);
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -101,13 +101,6 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
|||||||
command.comment = ast_comment.value.get<String>();
|
command.comment = ast_comment.value.get<String>();
|
||||||
return command;
|
return command;
|
||||||
}
|
}
|
||||||
else if (command_ast->type == ASTAlterCommand::MODIFY_PRIMARY_KEY)
|
|
||||||
{
|
|
||||||
AlterCommand command;
|
|
||||||
command.type = AlterCommand::MODIFY_PRIMARY_KEY;
|
|
||||||
command.primary_key = command_ast->primary_key;
|
|
||||||
return command;
|
|
||||||
}
|
|
||||||
else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY)
|
else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY)
|
||||||
{
|
{
|
||||||
AlterCommand command;
|
AlterCommand command;
|
||||||
@ -271,13 +264,6 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & orde
|
|||||||
/// both old and new columns have default expression, update it
|
/// both old and new columns have default expression, update it
|
||||||
columns_description.defaults[column_name].expression = default_expression;
|
columns_description.defaults[column_name].expression = default_expression;
|
||||||
}
|
}
|
||||||
else if (type == MODIFY_PRIMARY_KEY)
|
|
||||||
{
|
|
||||||
if (!primary_key_ast)
|
|
||||||
order_by_ast = primary_key;
|
|
||||||
else
|
|
||||||
primary_key_ast = primary_key;
|
|
||||||
}
|
|
||||||
else if (type == MODIFY_ORDER_BY)
|
else if (type == MODIFY_ORDER_BY)
|
||||||
{
|
{
|
||||||
if (!primary_key_ast)
|
if (!primary_key_ast)
|
||||||
|
@ -22,7 +22,6 @@ struct AlterCommand
|
|||||||
DROP_COLUMN,
|
DROP_COLUMN,
|
||||||
MODIFY_COLUMN,
|
MODIFY_COLUMN,
|
||||||
COMMENT_COLUMN,
|
COMMENT_COLUMN,
|
||||||
MODIFY_PRIMARY_KEY,
|
|
||||||
MODIFY_ORDER_BY,
|
MODIFY_ORDER_BY,
|
||||||
UKNOWN_TYPE,
|
UKNOWN_TYPE,
|
||||||
};
|
};
|
||||||
@ -44,9 +43,6 @@ struct AlterCommand
|
|||||||
/// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible.
|
/// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible.
|
||||||
String after_column;
|
String after_column;
|
||||||
|
|
||||||
/// For MODIFY_PRIMARY_KEY
|
|
||||||
ASTPtr primary_key;
|
|
||||||
|
|
||||||
/// For MODIFY_ORDER_BY
|
/// For MODIFY_ORDER_BY
|
||||||
ASTPtr order_by;
|
ASTPtr order_by;
|
||||||
|
|
||||||
@ -73,7 +69,7 @@ class AlterCommands : public std::vector<AlterCommand>
|
|||||||
public:
|
public:
|
||||||
void apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const;
|
void apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const;
|
||||||
|
|
||||||
/// For storages that don't support MODIFY_PRIMARY_KEY or MODIFY_ORDER_BY.
|
/// For storages that don't support MODIFY_ORDER_BY.
|
||||||
void apply(ColumnsDescription & columns_description) const;
|
void apply(ColumnsDescription & columns_description) const;
|
||||||
|
|
||||||
void validate(const IStorage & table, const Context & context);
|
void validate(const IStorage & table, const Context & context);
|
||||||
|
@ -1229,7 +1229,6 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name
|
|||||||
MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
|
MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
|
||||||
const DataPartPtr & part,
|
const DataPartPtr & part,
|
||||||
const NamesAndTypesList & new_columns,
|
const NamesAndTypesList & new_columns,
|
||||||
const ASTPtr & new_primary_key_expr_list,
|
|
||||||
bool skip_sanity_checks)
|
bool skip_sanity_checks)
|
||||||
{
|
{
|
||||||
ExpressionActionsPtr expression;
|
ExpressionActionsPtr expression;
|
||||||
@ -1290,63 +1289,6 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
|
|||||||
|
|
||||||
DataPart::Checksums add_checksums;
|
DataPart::Checksums add_checksums;
|
||||||
|
|
||||||
/// Update primary key if needed.
|
|
||||||
size_t new_primary_key_file_size{};
|
|
||||||
MergeTreeDataPartChecksum::uint128 new_primary_key_hash{};
|
|
||||||
|
|
||||||
if (new_primary_key_expr_list)
|
|
||||||
{
|
|
||||||
ASTPtr query = new_primary_key_expr_list;
|
|
||||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, new_columns);
|
|
||||||
ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer(query, syntax_result, context).getActions(true);
|
|
||||||
Block new_primary_key_sample = new_primary_expr->getSampleBlock();
|
|
||||||
size_t new_key_size = new_primary_key_sample.columns();
|
|
||||||
|
|
||||||
Columns new_index(new_key_size);
|
|
||||||
|
|
||||||
/// Copy the existing primary key columns. Fill new columns with default values.
|
|
||||||
/// NOTE default expressions are not supported.
|
|
||||||
|
|
||||||
ssize_t prev_position_of_existing_column = -1;
|
|
||||||
for (size_t i = 0; i < new_key_size; ++i)
|
|
||||||
{
|
|
||||||
const String & column_name = new_primary_key_sample.safeGetByPosition(i).name;
|
|
||||||
|
|
||||||
if (primary_key_sample.has(column_name))
|
|
||||||
{
|
|
||||||
ssize_t position_of_existing_column = primary_key_sample.getPositionByName(column_name);
|
|
||||||
|
|
||||||
if (position_of_existing_column < prev_position_of_existing_column)
|
|
||||||
throw Exception("Permuting of columns of primary key is not supported", ErrorCodes::BAD_ARGUMENTS);
|
|
||||||
|
|
||||||
new_index[i] = part->index.at(position_of_existing_column);
|
|
||||||
prev_position_of_existing_column = position_of_existing_column;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
const IDataType & type = *new_primary_key_sample.safeGetByPosition(i).type;
|
|
||||||
new_index[i] = type.createColumnConstWithDefaultValue(part->marks_count)->convertToFullColumnIfConst();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (prev_position_of_existing_column == -1)
|
|
||||||
throw Exception("No common columns while modifying primary key", ErrorCodes::BAD_ARGUMENTS);
|
|
||||||
|
|
||||||
String index_tmp_path = full_path + part->name + "/primary.idx.tmp";
|
|
||||||
WriteBufferFromFile index_file(index_tmp_path);
|
|
||||||
HashingWriteBuffer index_stream(index_file);
|
|
||||||
|
|
||||||
for (size_t i = 0, marks_count = part->marks_count; i < marks_count; ++i)
|
|
||||||
for (size_t j = 0; j < new_key_size; ++j)
|
|
||||||
new_primary_key_sample.getByPosition(j).type->serializeBinary(*new_index[j].get(), i, index_stream);
|
|
||||||
|
|
||||||
transaction->rename_map["primary.idx.tmp"] = "primary.idx";
|
|
||||||
|
|
||||||
index_stream.next();
|
|
||||||
new_primary_key_file_size = index_stream.count();
|
|
||||||
new_primary_key_hash = index_stream.getHash();
|
|
||||||
}
|
|
||||||
|
|
||||||
if (transaction->rename_map.empty() && !force_update_metadata)
|
if (transaction->rename_map.empty() && !force_update_metadata)
|
||||||
{
|
{
|
||||||
transaction->clear();
|
transaction->clear();
|
||||||
@ -1395,12 +1337,6 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
|
|||||||
new_checksums.files[it.second] = add_checksums.files[it.first];
|
new_checksums.files[it.second] = add_checksums.files[it.first];
|
||||||
}
|
}
|
||||||
|
|
||||||
if (new_primary_key_file_size)
|
|
||||||
{
|
|
||||||
new_checksums.files["primary.idx"].file_size = new_primary_key_file_size;
|
|
||||||
new_checksums.files["primary.idx"].file_hash = new_primary_key_hash;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Write the checksums to the temporary file.
|
/// Write the checksums to the temporary file.
|
||||||
if (!part->checksums.empty())
|
if (!part->checksums.empty())
|
||||||
{
|
{
|
||||||
|
@ -479,13 +479,11 @@ public:
|
|||||||
|
|
||||||
/// Performs ALTER of the data part, writes the result to temporary files.
|
/// Performs ALTER of the data part, writes the result to temporary files.
|
||||||
/// Returns an object allowing to rename temporary files to permanent files.
|
/// Returns an object allowing to rename temporary files to permanent files.
|
||||||
/// If new_primary_key_expr_list is not nullptr, will prepare the new primary.idx file.
|
|
||||||
/// If the number of affected columns is suspiciously high and skip_sanity_checks is false, throws an exception.
|
/// If the number of affected columns is suspiciously high and skip_sanity_checks is false, throws an exception.
|
||||||
/// If no data transformations are necessary, returns nullptr.
|
/// If no data transformations are necessary, returns nullptr.
|
||||||
AlterDataPartTransactionPtr alterDataPart(
|
AlterDataPartTransactionPtr alterDataPart(
|
||||||
const DataPartPtr & part,
|
const DataPartPtr & part,
|
||||||
const NamesAndTypesList & new_columns,
|
const NamesAndTypesList & new_columns,
|
||||||
const ASTPtr & new_primary_key_expr_list,
|
|
||||||
bool skip_sanity_checks);
|
bool skip_sanity_checks);
|
||||||
|
|
||||||
/// Freezes all parts.
|
/// Freezes all parts.
|
||||||
|
@ -150,7 +150,7 @@ void ReplicatedMergeTreeAlterThread::run()
|
|||||||
/// Update the part and write result to temporary files.
|
/// Update the part and write result to temporary files.
|
||||||
/// TODO: You can skip checking for too large changes if ZooKeeper has, for example,
|
/// TODO: You can skip checking for too large changes if ZooKeeper has, for example,
|
||||||
/// node /flags/force_alter.
|
/// node /flags/force_alter.
|
||||||
auto transaction = storage.data.alterDataPart(part, columns_for_parts, nullptr, false);
|
auto transaction = storage.data.alterDataPart(part, columns_for_parts, false);
|
||||||
|
|
||||||
if (!transaction)
|
if (!transaction)
|
||||||
continue;
|
continue;
|
||||||
|
@ -210,24 +210,12 @@ void StorageMergeTree::alter(
|
|||||||
ASTPtr new_primary_key_ast = data.primary_key_ast;
|
ASTPtr new_primary_key_ast = data.primary_key_ast;
|
||||||
params.apply(new_columns, new_order_by_ast, new_primary_key_ast);
|
params.apply(new_columns, new_order_by_ast, new_primary_key_ast);
|
||||||
|
|
||||||
ASTPtr primary_expr_list_for_altering_parts;
|
|
||||||
for (const AlterCommand & param : params)
|
|
||||||
{
|
|
||||||
if (param.type == AlterCommand::MODIFY_PRIMARY_KEY)
|
|
||||||
{
|
|
||||||
if (supportsSampling())
|
|
||||||
throw Exception("MODIFY PRIMARY KEY only supported for tables without sampling key", ErrorCodes::BAD_ARGUMENTS);
|
|
||||||
|
|
||||||
primary_expr_list_for_altering_parts = MergeTreeData::extractKeyExpressionList(param.primary_key);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
|
auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
|
||||||
auto columns_for_parts = new_columns.getAllPhysical();
|
auto columns_for_parts = new_columns.getAllPhysical();
|
||||||
std::vector<MergeTreeData::AlterDataPartTransactionPtr> transactions;
|
std::vector<MergeTreeData::AlterDataPartTransactionPtr> transactions;
|
||||||
for (const MergeTreeData::DataPartPtr & part : parts)
|
for (const MergeTreeData::DataPartPtr & part : parts)
|
||||||
{
|
{
|
||||||
if (auto transaction = data.alterDataPart(part, columns_for_parts, primary_expr_list_for_altering_parts, false))
|
if (auto transaction = data.alterDataPart(part, columns_for_parts, false))
|
||||||
transactions.push_back(std::move(transaction));
|
transactions.push_back(std::move(transaction));
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -238,19 +226,7 @@ void StorageMergeTree::alter(
|
|||||||
auto & storage_ast = typeid_cast<ASTStorage &>(ast);
|
auto & storage_ast = typeid_cast<ASTStorage &>(ast);
|
||||||
|
|
||||||
if (new_order_by_ast.get() != data.order_by_ast.get())
|
if (new_order_by_ast.get() != data.order_by_ast.get())
|
||||||
{
|
storage_ast.set(storage_ast.order_by, new_order_by_ast);
|
||||||
if (storage_ast.order_by)
|
|
||||||
{
|
|
||||||
/// The table was created using the "new" syntax (with key expressions in separate clauses).
|
|
||||||
storage_ast.set(storage_ast.order_by, new_order_by_ast);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/// Primary key is in the second place in table engine description and can be represented as a tuple.
|
|
||||||
/// TODO: Not always in second place. If there is a sampling key, then the third one. Fix it.
|
|
||||||
storage_ast.engine->arguments->children.at(1) = new_order_by_ast;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (new_primary_key_ast.get() != data.primary_key_ast.get())
|
if (new_primary_key_ast.get() != data.primary_key_ast.get())
|
||||||
storage_ast.set(storage_ast.primary_key, new_primary_key_ast);
|
storage_ast.set(storage_ast.primary_key, new_primary_key_ast);
|
||||||
@ -266,9 +242,6 @@ void StorageMergeTree::alter(
|
|||||||
|
|
||||||
/// Columns sizes could be changed
|
/// Columns sizes could be changed
|
||||||
data.recalculateColumnSizes();
|
data.recalculateColumnSizes();
|
||||||
|
|
||||||
if (primary_expr_list_for_altering_parts)
|
|
||||||
data.loadDataParts(false);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -725,7 +698,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi
|
|||||||
if (part->info.partition_id != partition_id)
|
if (part->info.partition_id != partition_id)
|
||||||
throw Exception("Unexpected partition ID " + part->info.partition_id + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Unexpected partition ID " + part->info.partition_id + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
if (auto transaction = data.alterDataPart(part, columns_for_parts, nullptr, false))
|
if (auto transaction = data.alterDataPart(part, columns_for_parts, false))
|
||||||
transactions.push_back(std::move(transaction));
|
transactions.push_back(std::move(transaction));
|
||||||
|
|
||||||
LOG_DEBUG(log, "Removing column " << get<String>(column_name) << " from part " << part->name);
|
LOG_DEBUG(log, "Removing column " << get<String>(column_name) << " from part " << part->name);
|
||||||
|
@ -1504,7 +1504,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry &
|
|||||||
|
|
||||||
LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name);
|
LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name);
|
||||||
|
|
||||||
auto transaction = data.alterDataPart(part, columns_for_parts, nullptr, false);
|
auto transaction = data.alterDataPart(part, columns_for_parts, false);
|
||||||
if (!transaction)
|
if (!transaction)
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
@ -3059,12 +3059,6 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params,
|
|||||||
|
|
||||||
data.checkAlter(params);
|
data.checkAlter(params);
|
||||||
|
|
||||||
for (const AlterCommand & param : params)
|
|
||||||
{
|
|
||||||
if (param.type == AlterCommand::MODIFY_PRIMARY_KEY)
|
|
||||||
throw Exception("Modification of primary key is not supported for replicated tables", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
|
|
||||||
ColumnsDescription new_columns = data.getColumns();
|
ColumnsDescription new_columns = data.getColumns();
|
||||||
ASTPtr new_order_by_ast = data.order_by_ast;
|
ASTPtr new_order_by_ast = data.order_by_ast;
|
||||||
ASTPtr new_primary_key_ast = data.primary_key_ast;
|
ASTPtr new_primary_key_ast = data.primary_key_ast;
|
||||||
|
@ -1,130 +0,0 @@
|
|||||||
1
|
|
||||||
2
|
|
||||||
3
|
|
||||||
2
|
|
||||||
3
|
|
||||||
1
|
|
||||||
2
|
|
||||||
3
|
|
||||||
2
|
|
||||||
3
|
|
||||||
1
|
|
||||||
2
|
|
||||||
3
|
|
||||||
2
|
|
||||||
3
|
|
||||||
2
|
|
||||||
3
|
|
||||||
1
|
|
||||||
1 Hello
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2 World
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
1
|
|
||||||
2
|
|
||||||
3
|
|
||||||
1
|
|
||||||
1 Hello
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2 World
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
1
|
|
||||||
2
|
|
||||||
3
|
|
||||||
1
|
|
||||||
1 Hello
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
1
|
|
||||||
1 Hello
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
1
|
|
||||||
1 Hello
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2 World
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
1
|
|
||||||
2
|
|
||||||
3
|
|
||||||
1
|
|
||||||
1 Hello
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2 World
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
2
|
|
||||||
2 World
|
|
||||||
3
|
|
||||||
3 abc
|
|
||||||
4 def
|
|
||||||
1
|
|
||||||
2
|
|
||||||
3
|
|
||||||
*** Check table creation statement ***
|
|
||||||
CREATE TABLE test.pk2 ( x UInt32, y UInt32, z UInt32) ENGINE = MergeTree PRIMARY KEY (x, y) ORDER BY (x, y, z) SETTINGS index_granularity = 8192
|
|
||||||
*** Check that the inserted values were correctly sorted ***
|
|
||||||
100 20 1
|
|
||||||
100 20 2
|
|
||||||
100 30 1
|
|
||||||
100 30 2
|
|
@ -1,83 +0,0 @@
|
|||||||
SET send_logs_level = 'none';
|
|
||||||
|
|
||||||
DROP TABLE IF EXISTS test.pk;
|
|
||||||
CREATE TABLE test.pk (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 1);
|
|
||||||
|
|
||||||
INSERT INTO test.pk (x) VALUES (1), (2), (3);
|
|
||||||
|
|
||||||
SELECT x FROM test.pk ORDER BY x;
|
|
||||||
SELECT x FROM test.pk WHERE x >= 2 ORDER BY x;
|
|
||||||
|
|
||||||
ALTER TABLE test.pk MODIFY PRIMARY KEY (x);
|
|
||||||
|
|
||||||
SELECT x FROM test.pk ORDER BY x;
|
|
||||||
SELECT x FROM test.pk WHERE x >= 2 ORDER BY x;
|
|
||||||
|
|
||||||
ALTER TABLE test.pk ADD COLUMN y String, MODIFY PRIMARY KEY (x, y);
|
|
||||||
|
|
||||||
SELECT x, y FROM test.pk ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 AND y = '' ORDER BY x, y;
|
|
||||||
|
|
||||||
INSERT INTO test.pk (x, y) VALUES (1, 'Hello'), (2, 'World'), (3, 'abc'), (4, 'def');
|
|
||||||
|
|
||||||
SELECT x, y FROM test.pk ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 AND y > '' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 AND y >= '' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE y < 'A' ORDER BY x, y;
|
|
||||||
|
|
||||||
DETACH TABLE test.pk;
|
|
||||||
ATTACH TABLE test.pk (d Date DEFAULT '2000-01-01', x UInt64, y String) ENGINE = MergeTree(d, (x, y), 1);
|
|
||||||
|
|
||||||
SELECT x, y FROM test.pk ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 AND y > '' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 AND y >= '' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE y < 'A' ORDER BY x, y;
|
|
||||||
|
|
||||||
SET max_rows_to_read = 3;
|
|
||||||
SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y;
|
|
||||||
SET max_rows_to_read = 0;
|
|
||||||
|
|
||||||
OPTIMIZE TABLE test.pk;
|
|
||||||
SELECT x, y FROM test.pk;
|
|
||||||
SELECT x, y FROM test.pk ORDER BY x, y;
|
|
||||||
|
|
||||||
ALTER TABLE test.pk MODIFY PRIMARY KEY (x);
|
|
||||||
|
|
||||||
SELECT x, y FROM test.pk ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 AND y > '' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 AND y >= '' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE y < 'A' ORDER BY x, y;
|
|
||||||
|
|
||||||
DETACH TABLE test.pk;
|
|
||||||
ATTACH TABLE test.pk (d Date DEFAULT '2000-01-01', x UInt64, y String) ENGINE = MergeTree(d, (x), 1);
|
|
||||||
|
|
||||||
SELECT x, y FROM test.pk ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 AND y > '' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x >= 2 AND y >= '' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y;
|
|
||||||
SELECT x, y FROM test.pk WHERE y < 'A' ORDER BY x, y;
|
|
||||||
|
|
||||||
DROP TABLE test.pk;
|
|
||||||
|
|
||||||
DROP TABLE IF EXISTS test.pk2;
|
|
||||||
CREATE TABLE test.pk2 (x UInt32) ENGINE MergeTree ORDER BY x;
|
|
||||||
|
|
||||||
ALTER TABLE test.pk2 ADD COLUMN y UInt32, ADD COLUMN z UInt32, MODIFY ORDER BY (x, y, z);
|
|
||||||
ALTER TABLE test.pk2 MODIFY PRIMARY KEY (y); -- { serverError 36 }
|
|
||||||
ALTER TABLE test.pk2 MODIFY PRIMARY KEY (x, y);
|
|
||||||
SELECT '*** Check table creation statement ***';
|
|
||||||
SHOW CREATE TABLE test.pk2;
|
|
||||||
|
|
||||||
INSERT INTO test.pk2 VALUES (100, 30, 2), (100, 30, 1), (100, 20, 2), (100, 20, 1);
|
|
||||||
SELECT '*** Check that the inserted values were correctly sorted ***';
|
|
||||||
SELECT * FROM test.pk2;
|
|
||||||
|
|
||||||
DROP TABLE test.pk2;
|
|
Loading…
Reference in New Issue
Block a user