Merge pull request #3887 from yandex/remove-alter-primary-key

Remove ALTER MODIFY PRIMARY KEY command
This commit is contained in:
alexey-milovidov 2018-12-20 22:21:08 +03:00 committed by GitHub
commit b1602a5e62
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 10 additions and 362 deletions

View File

@ -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,

View File

@ -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 : "");

View File

@ -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;

View File

@ -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;
} }

View File

@ -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)

View File

@ -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);

View File

@ -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())
{ {

View File

@ -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.

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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

View File

@ -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;