mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Allowed to alter column in non-modifying data mode when the same type is specified.
This commit is contained in:
parent
61d33a8d9a
commit
82985ff594
@ -472,11 +472,22 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
|
||||
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
bool AlterCommand::isModifyingData() const
|
||||
bool AlterCommand::isModifyingData(const StorageInMemoryMetadata & metadata) const
|
||||
{
|
||||
/// Possible change data representation on disk
|
||||
if (type == MODIFY_COLUMN)
|
||||
return data_type != nullptr;
|
||||
{
|
||||
if (data_type == nullptr)
|
||||
return false;
|
||||
|
||||
for (const auto & column : metadata.columns.getAllPhysical())
|
||||
{
|
||||
if (column.name == column_name)
|
||||
return column.type->getName() != data_type->getName();
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return type == ADD_COLUMN /// We need to change columns.txt in each part for MergeTree
|
||||
|| type == DROP_COLUMN /// We need to change columns.txt in each part for MergeTree
|
||||
@ -888,11 +899,11 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
|
||||
validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context);
|
||||
}
|
||||
|
||||
bool AlterCommands::isModifyingData() const
|
||||
bool AlterCommands::isModifyingData(const StorageInMemoryMetadata & metadata) const
|
||||
{
|
||||
for (const auto & param : *this)
|
||||
{
|
||||
if (param.isModifyingData())
|
||||
if (param.isModifyingData(metadata))
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -108,7 +108,7 @@ struct AlterCommand
|
||||
/// * column files (data and marks)
|
||||
/// * each part meta (columns.txt)
|
||||
/// in each part on disk (it's not lightweight alter).
|
||||
bool isModifyingData() const;
|
||||
bool isModifyingData(const StorageInMemoryMetadata & metadata) const;
|
||||
|
||||
bool isRequireMutationStage(const StorageInMemoryMetadata & metadata) const;
|
||||
|
||||
@ -151,7 +151,7 @@ public:
|
||||
void apply(StorageInMemoryMetadata & metadata) const;
|
||||
|
||||
/// At least one command modify data on disk.
|
||||
bool isModifyingData() const;
|
||||
bool isModifyingData(const StorageInMemoryMetadata & metadata) const;
|
||||
|
||||
/// At least one command modify settings.
|
||||
bool isSettingsAlter() const;
|
||||
|
@ -1484,7 +1484,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
else if (command.isModifyingData())
|
||||
else if (command.isModifyingData(getInMemoryMetadata()))
|
||||
{
|
||||
if (columns_alter_type_forbidden.count(command.column_name))
|
||||
throw Exception("Trying to ALTER key column " + command.column_name, ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
0
tests/integration/test_alter_codec/__init__.py
Normal file
0
tests/integration/test_alter_codec/__init__.py
Normal file
@ -0,0 +1,16 @@
|
||||
<yandex>
|
||||
<remote_servers>
|
||||
<test_cluster>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>node1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster>
|
||||
</remote_servers>
|
||||
</yandex>
|
17
tests/integration/test_alter_codec/configs/logs_config.xml
Normal file
17
tests/integration/test_alter_codec/configs/logs_config.xml
Normal file
@ -0,0 +1,17 @@
|
||||
<yandex>
|
||||
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/log.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||
</logger>
|
||||
<part_log>
|
||||
<database>system</database>
|
||||
<table>part_log</table>
|
||||
<flush_interval_milliseconds>500</flush_interval_milliseconds>
|
||||
</part_log>
|
||||
</yandex>
|
88
tests/integration/test_alter_codec/test.py
Normal file
88
tests/integration/test_alter_codec/test.py
Normal file
@ -0,0 +1,88 @@
|
||||
import pytest
|
||||
from helpers.client import QueryRuntimeException
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
|
||||
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} )
|
||||
|
||||
node2 = cluster.add_instance('node2',
|
||||
config_dir='configs',
|
||||
main_configs=['configs/logs_config.xml'],
|
||||
with_zookeeper=True,
|
||||
macros={"shard": 0, "replica": 2} )
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
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
|
||||
""".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))
|
||||
|
||||
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))
|
||||
|
||||
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".format(name=name))
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id Int64".format(name=name))
|
||||
|
||||
finally:
|
||||
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
|
||||
|
||||
|
||||
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()
|
||||
""".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))
|
||||
|
||||
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))
|
||||
|
||||
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".format(name=name))
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node1.query("ALTER TABLE {name} MODIFY COLUMN id Int64".format(name=name))
|
||||
|
||||
finally:
|
||||
node1.query("DROP TABLE IF EXISTS {name}".format(name=name))
|
Loading…
Reference in New Issue
Block a user