mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #11786 from ClickHouse/fix_alter_with_materialized
Fix alter key when materialized column passed
This commit is contained in:
commit
ef89a6f728
@ -728,6 +728,10 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata, const Context & co
|
||||
metadata_copy.primary_key.definition_ast = nullptr;
|
||||
}
|
||||
|
||||
/// Changes in columns may lead to changes in secondary indices
|
||||
for (auto & index : metadata_copy.secondary_indices)
|
||||
index.recalculateWithNewColumns(metadata_copy.columns, context);
|
||||
|
||||
/// Changes in columns may lead to changes in TTL expressions.
|
||||
auto column_ttl_asts = metadata_copy.columns.getColumnTTLs();
|
||||
for (const auto & [name, ast] : column_ttl_asts)
|
||||
|
@ -289,7 +289,7 @@ void IStorage::check(const Block & block, bool need_all) const
|
||||
|
||||
void IStorage::setColumns(ColumnsDescription columns_)
|
||||
{
|
||||
if (columns_.getOrdinary().empty())
|
||||
if (columns_.getAllPhysical().empty())
|
||||
throw Exception("Empty list of columns passed", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
||||
metadata.columns = std::move(columns_);
|
||||
}
|
||||
|
@ -117,6 +117,10 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast
|
||||
return result;
|
||||
}
|
||||
|
||||
void IndexDescription::recalculateWithNewColumns(const ColumnsDescription & new_columns, const Context & context)
|
||||
{
|
||||
*this = getIndexFromAST(definition_ast, new_columns, context);
|
||||
}
|
||||
|
||||
bool IndicesDescription::has(const String & name) const
|
||||
{
|
||||
@ -154,6 +158,7 @@ IndicesDescription IndicesDescription::parse(const String & str, const ColumnsDe
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
ExpressionActionsPtr IndicesDescription::getSingleExpressionForIndices(const ColumnsDescription & columns, const Context & context) const
|
||||
{
|
||||
ASTPtr combined_expr_list = std::make_shared<ASTExpressionList>();
|
||||
|
@ -55,6 +55,10 @@ struct IndexDescription
|
||||
/// unintentionaly share AST variables and modify them.
|
||||
IndexDescription(const IndexDescription & other);
|
||||
IndexDescription & operator=(const IndexDescription & other);
|
||||
|
||||
/// Recalculate index with new columns because index expression may change
|
||||
/// if something change in columns.
|
||||
void recalculateWithNewColumns(const ColumnsDescription & new_columns, const Context & context);
|
||||
};
|
||||
|
||||
/// All secondary indices in storage
|
||||
|
@ -7,15 +7,11 @@ cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node1 = cluster.add_instance('node1',
|
||||
config_dir='configs',
|
||||
main_configs=['configs/logs_config.xml'],
|
||||
with_zookeeper=True,
|
||||
macros={"shard": 0, "replica": 1} )
|
||||
main_configs=['configs/logs_config.xml'])
|
||||
|
||||
node2 = cluster.add_instance('node2',
|
||||
config_dir='configs',
|
||||
main_configs=['configs/logs_config.xml'],
|
||||
with_zookeeper=True,
|
||||
macros={"shard": 0, "replica": 2} )
|
||||
main_configs=['configs/logs_config.xml'])
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
@ -32,23 +28,34 @@ def test_alter_codec_pk(started_cluster):
|
||||
try:
|
||||
name = "test_alter_codec_pk"
|
||||
node1.query("""
|
||||
CREATE TABLE {name} (id UInt64) Engine=MergeTree() ORDER BY id
|
||||
CREATE TABLE {name} (id UInt64, value UInt64) Engine=MergeTree() ORDER BY id
|
||||
""".format(name=name))
|
||||
|
||||
node1.query("INSERT INTO {name} SELECT number, number * number from numbers(100)".format(name=name))
|
||||
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(NONE)".format(name=name))
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(Delta, LZ4)".format(name=name))
|
||||
|
||||
assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "4950\n"
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt32 CODEC(Delta, LZ4)".format(name=name))
|
||||
|
||||
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 DEFAULT 3 CODEC(Delta, LZ4)".format(name=name))
|
||||
|
||||
node1.query("INSERT INTO {name} (value) VALUES (1)".format(name=name))
|
||||
|
||||
assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "4953\n"
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 ALIAS 3 CODEC(Delta, LZ4)".format(name=name))
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name))
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name))
|
||||
|
||||
node1.query("INSERT INTO {name} (value) VALUES (1)".format(name=name))
|
||||
|
||||
assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "4956\n"
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64".format(name=name))
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
@ -62,9 +69,11 @@ def test_alter_codec_index(started_cluster):
|
||||
try:
|
||||
name = "test_alter_codec_index"
|
||||
node1.query("""
|
||||
CREATE TABLE {name} (`id` UInt64, INDEX id_index id TYPE minmax GRANULARITY 1) Engine=MergeTree() ORDER BY tuple()
|
||||
CREATE TABLE {name} (`id` UInt64, value UInt64, INDEX id_index id TYPE minmax GRANULARITY 1) Engine=MergeTree() ORDER BY tuple()
|
||||
""".format(name=name))
|
||||
|
||||
node1.query("INSERT INTO {name} SELECT number, number * number from numbers(100)".format(name=name))
|
||||
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(NONE)".format(name=name))
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 CODEC(Delta, LZ4)".format(name=name))
|
||||
|
||||
@ -73,11 +82,18 @@ def test_alter_codec_index(started_cluster):
|
||||
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 DEFAULT 3 CODEC(Delta, LZ4)".format(name=name))
|
||||
|
||||
node1.query("INSERT INTO {name} (value) VALUES (1)".format(name=name))
|
||||
|
||||
assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "4953\n"
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 ALIAS 3 CODEC(Delta, LZ4)".format(name=name))
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name))
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64 MATERIALIZED 3 CODEC(Delta, LZ4)".format(name=name))
|
||||
|
||||
node1.query("INSERT INTO {name} (value) VALUES (1)".format(name=name))
|
||||
|
||||
assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "4956\n"
|
||||
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id UInt64".format(name=name))
|
||||
|
||||
|
@ -51,6 +51,6 @@ ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key2 TO renamed_key2
|
||||
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key3 TO renamed_key3; --{serverError 47}
|
||||
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN value1 TO renamed_value1; --{serverError 524}
|
||||
ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN value1 TO renamed_value1; --{serverError 47}
|
||||
|
||||
DROP TABLE IF EXISTS table_for_rename_with_primary_key;
|
||||
|
Loading…
Reference in New Issue
Block a user