mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add setting allow_non_metadata_alters
This commit is contained in:
parent
18eef29217
commit
daaa5cc382
@ -22,8 +22,9 @@ namespace ErrorCodes
|
||||
class TestHint
|
||||
{
|
||||
public:
|
||||
TestHint(bool enabled_, const String & query)
|
||||
: enabled(enabled_)
|
||||
TestHint(bool enabled_, const String & query_)
|
||||
: enabled(enabled_)
|
||||
, query(query_)
|
||||
{
|
||||
if (!enabled_)
|
||||
return;
|
||||
@ -71,7 +72,7 @@ public:
|
||||
|
||||
if (lostExpectedError(actual_server_error, actual_client_error))
|
||||
{
|
||||
std::cerr << "Success when error expected. It expects server error "
|
||||
std::cerr << "Success when error expected in query: " << query << "It expects server error "
|
||||
<< server_error << ", client error " << client_error << "." << std::endl;
|
||||
got_exception = true;
|
||||
last_exception = std::make_unique<Exception>("Success when error expected", ErrorCodes::UNEXPECTED_ERROR_CODE); /// return error to OS
|
||||
@ -86,6 +87,7 @@ public:
|
||||
|
||||
private:
|
||||
bool enabled = false;
|
||||
const String & query;
|
||||
int server_error = 0;
|
||||
int client_error = 0;
|
||||
|
||||
|
@ -469,7 +469,8 @@ struct Settings : public SettingsCollection<Settings>
|
||||
M(SettingBool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \
|
||||
\
|
||||
M(SettingBool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \
|
||||
M(SettingBool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0)
|
||||
M(SettingBool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \
|
||||
M(SettingBool, allow_non_metadata_alters, true, "Allow to execute alters which not only tables metadata, but also data on disk", 0) \
|
||||
|
||||
#define LIST_OF_SETTINGS(M) \
|
||||
COMMON_SETTINGS(M) \
|
||||
|
@ -354,8 +354,12 @@ bool DataTypeEnum<Type>::contains(const IDataType & rhs) const
|
||||
auto check = [&](const auto & value)
|
||||
{
|
||||
auto it = name_to_value_map.find(value.first);
|
||||
/// If we don't have this name, than we have to be sure,
|
||||
/// that this value exists in enum
|
||||
if (it == name_to_value_map.end())
|
||||
return false;
|
||||
return value_to_name_map.count(value.second) > 0;
|
||||
|
||||
/// If we have this name, than it should have the same value
|
||||
return it->value.second == value.second;
|
||||
};
|
||||
|
||||
|
@ -120,7 +120,10 @@ public:
|
||||
bool textCanContainOnlyValidUTF8() const override;
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(FieldType); }
|
||||
|
||||
/// Check current Enum type extends another Enum type (contains all fields with same values).
|
||||
/// Check current Enum type extends another Enum type (contains all the same values and doesn't override name's with other values)
|
||||
/// Example:
|
||||
/// Enum('a' = 1, 'b' = 2) -> Enum('c' = 1, 'b' = 2, 'd' = 3) OK
|
||||
/// Enum('a' = 1, 'b' = 2) -> Enum('a' = 2, 'b' = 1) NOT OK
|
||||
bool contains(const IDataType & rhs) const;
|
||||
};
|
||||
|
||||
|
@ -501,32 +501,6 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con
|
||||
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
bool AlterCommand::isModifyingData(const StorageInMemoryMetadata & metadata) const
|
||||
{
|
||||
/// Possible change data representation on disk
|
||||
if (type == MODIFY_COLUMN)
|
||||
{
|
||||
if (data_type == nullptr)
|
||||
return false;
|
||||
|
||||
/// It is allowed to ALTER data type to the same type as before.
|
||||
for (const auto & column : metadata.columns.getAllPhysical())
|
||||
if (column.name == column_name)
|
||||
return !column.type->equals(*data_type);
|
||||
|
||||
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
|
||||
|| type == DROP_INDEX; /// We need to remove file from filesystem for MergeTree
|
||||
}
|
||||
|
||||
bool AlterCommand::isSettingsAlter() const
|
||||
{
|
||||
return type == MODIFY_SETTING;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
@ -538,11 +512,21 @@ bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to)
|
||||
if (from->equals(*to))
|
||||
return true;
|
||||
|
||||
if (const auto * from_enum8 = typeid_cast<const DataTypeEnum8 *>(from))
|
||||
{
|
||||
if (const auto * to_enum8 = typeid_cast<const DataTypeEnum8 *>(to))
|
||||
return to_enum8->contains(*from_enum8);
|
||||
}
|
||||
|
||||
if (const auto * from_enum16 = typeid_cast<const DataTypeEnum16 *>(from))
|
||||
{
|
||||
if (const auto * to_enum16 = typeid_cast<const DataTypeEnum16 *>(to))
|
||||
return to_enum16->contains(*from_enum16);
|
||||
}
|
||||
|
||||
static const std::unordered_multimap<std::type_index, const std::type_info &> ALLOWED_CONVERSIONS =
|
||||
{
|
||||
{ typeid(DataTypeEnum8), typeid(DataTypeEnum8) },
|
||||
{ typeid(DataTypeEnum8), typeid(DataTypeInt8) },
|
||||
{ typeid(DataTypeEnum16), typeid(DataTypeEnum16) },
|
||||
{ typeid(DataTypeEnum16), typeid(DataTypeInt16) },
|
||||
{ typeid(DataTypeDateTime), typeid(DataTypeUInt32) },
|
||||
{ typeid(DataTypeUInt32), typeid(DataTypeDateTime) },
|
||||
@ -583,6 +567,10 @@ bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to)
|
||||
|
||||
}
|
||||
|
||||
bool AlterCommand::isSettingsAlter() const
|
||||
{
|
||||
return type == MODIFY_SETTING;
|
||||
}
|
||||
|
||||
bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metadata) const
|
||||
{
|
||||
@ -1009,17 +997,6 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
|
||||
validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context);
|
||||
}
|
||||
|
||||
bool AlterCommands::isModifyingData(const StorageInMemoryMetadata & metadata) const
|
||||
{
|
||||
for (const auto & param : *this)
|
||||
{
|
||||
if (param.isModifyingData(metadata))
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
bool AlterCommands::isSettingsAlter() const
|
||||
{
|
||||
return std::all_of(begin(), end(), [](const AlterCommand & c) { return c.isSettingsAlter(); });
|
||||
|
@ -1272,60 +1272,7 @@ void MergeTreeData::dropIfEmpty()
|
||||
}
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// If true, then in order to ALTER the type of the column from the type from to the type to
|
||||
/// we don't need to rewrite the data, we only need to update metadata and columns.txt in part directories.
|
||||
/// The function works for Arrays and Nullables of the same structure.
|
||||
bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to)
|
||||
{
|
||||
if (from->getName() == to->getName())
|
||||
return true;
|
||||
|
||||
static const std::unordered_multimap<std::type_index, const std::type_info &> ALLOWED_CONVERSIONS =
|
||||
{
|
||||
{ typeid(DataTypeEnum8), typeid(DataTypeEnum8) },
|
||||
{ typeid(DataTypeEnum8), typeid(DataTypeInt8) },
|
||||
{ typeid(DataTypeEnum16), typeid(DataTypeEnum16) },
|
||||
{ typeid(DataTypeEnum16), typeid(DataTypeInt16) },
|
||||
{ typeid(DataTypeDateTime), typeid(DataTypeUInt32) },
|
||||
{ typeid(DataTypeUInt32), typeid(DataTypeDateTime) },
|
||||
{ typeid(DataTypeDate), typeid(DataTypeUInt16) },
|
||||
{ typeid(DataTypeUInt16), typeid(DataTypeDate) },
|
||||
};
|
||||
|
||||
while (true)
|
||||
{
|
||||
auto it_range = ALLOWED_CONVERSIONS.equal_range(typeid(*from));
|
||||
for (auto it = it_range.first; it != it_range.second; ++it)
|
||||
{
|
||||
if (it->second == typeid(*to))
|
||||
return true;
|
||||
}
|
||||
|
||||
const auto * arr_from = typeid_cast<const DataTypeArray *>(from);
|
||||
const auto * arr_to = typeid_cast<const DataTypeArray *>(to);
|
||||
if (arr_from && arr_to)
|
||||
{
|
||||
from = arr_from->getNestedType().get();
|
||||
to = arr_to->getNestedType().get();
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto * nullable_from = typeid_cast<const DataTypeNullable *>(from);
|
||||
const auto * nullable_to = typeid_cast<const DataTypeNullable *>(to);
|
||||
if (nullable_from && nullable_to)
|
||||
{
|
||||
from = nullable_from->getNestedType().get();
|
||||
to = nullable_to->getNestedType().get();
|
||||
continue;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
namespace {
|
||||
/// Conversion that is allowed for partition key.
|
||||
/// Partition key should be serialized in the same way after conversion.
|
||||
/// NOTE: The list is not complete.
|
||||
@ -1361,11 +1308,20 @@ bool isSafeForPartitionKeyConversion(const IDataType * from, const IDataType * t
|
||||
|
||||
}
|
||||
|
||||
void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Settings &) const
|
||||
void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const
|
||||
{
|
||||
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
|
||||
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
|
||||
StorageInMemoryMetadata old_metadata = getInMemoryMetadata();
|
||||
|
||||
if (!settings.allow_non_metadata_alters)
|
||||
{
|
||||
|
||||
auto mutation_commands = commands.getMutationCommands(new_metadata, settings.materialize_ttl_after_modify, global_context);
|
||||
|
||||
if (!mutation_commands.empty())
|
||||
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, "The following alter commands: '{}' will modify data on disk, but setting `allow_non_metadata_alters` is disabled", queryToString(mutation_commands.ast()));
|
||||
}
|
||||
commands.apply(new_metadata, global_context);
|
||||
|
||||
/// Set of columns that shouldn't be altered.
|
||||
@ -1467,8 +1423,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
}
|
||||
dropped_columns.emplace(command.column_name);
|
||||
}
|
||||
else if (command.isModifyingData(getInMemoryMetadata()))
|
||||
else if (command.isRequireMutationStage(getInMemoryMetadata()))
|
||||
{
|
||||
/// This alter will override data on disk. Let's check that it doesn't
|
||||
/// modify immutable column.
|
||||
if (columns_alter_type_forbidden.count(command.column_name))
|
||||
throw Exception("ALTER of key column " + backQuoteIfNeed(command.column_name) + " is forbidden",
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
@ -1478,7 +1436,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
if (command.type == AlterCommand::MODIFY_COLUMN)
|
||||
{
|
||||
auto it = old_types.find(command.column_name);
|
||||
if (it == old_types.end() || !isSafeForPartitionKeyConversion(it->second, command.data_type.get()))
|
||||
|
||||
assert(it != old_types.end());
|
||||
if (!isSafeForPartitionKeyConversion(it->second, command.data_type.get()))
|
||||
throw Exception("ALTER of partition key column " + backQuoteIfNeed(command.column_name) + " from type "
|
||||
+ it->second->getName() + " to type " + command.data_type->getName()
|
||||
+ " is not safe because it can change the representation of partition key",
|
||||
@ -1491,10 +1451,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
if (command.type == AlterCommand::MODIFY_COLUMN)
|
||||
{
|
||||
auto it = old_types.find(command.column_name);
|
||||
if (it == old_types.end() || !isMetadataOnlyConversion(it->second, command.data_type.get()))
|
||||
throw Exception("ALTER of key column " + backQuoteIfNeed(command.column_name) + " from type "
|
||||
+ it->second->getName() + " to type " + command.data_type->getName() + " must be metadata-only",
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
assert(it != old_types.end());
|
||||
throw Exception("ALTER of key column " + backQuoteIfNeed(command.column_name) + " from type "
|
||||
+ it->second->getName() + " to type " + command.data_type->getName() + " must be metadata-only",
|
||||
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -28,7 +28,7 @@ struct MutationCommand
|
||||
DELETE,
|
||||
UPDATE,
|
||||
MATERIALIZE_INDEX,
|
||||
READ_COLUMN,
|
||||
READ_COLUMN, /// Read column and apply conversions (MODIFY COLUMN alter query).
|
||||
DROP_COLUMN,
|
||||
DROP_INDEX,
|
||||
MATERIALIZE_TTL,
|
||||
|
@ -12,7 +12,7 @@ SELECT * FROM test ORDER BY x;
|
||||
SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition;
|
||||
|
||||
ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2); -- { serverError 524 }
|
||||
ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'test' = 3); -- { serverError 524 }
|
||||
ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'test' = 3);
|
||||
ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'goodbye' = 4); -- { serverError 524 }
|
||||
|
||||
ALTER TABLE test MODIFY COLUMN x Int8;
|
||||
|
@ -21,7 +21,8 @@ SELECT name, partition, partition_id FROM system.parts WHERE database = currentD
|
||||
SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition;
|
||||
|
||||
ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2); -- { serverError 524 }
|
||||
ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'test' = 3); -- { serverError 524 }
|
||||
ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'test' = 3);
|
||||
|
||||
ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'goodbye' = 4); -- { serverError 524 }
|
||||
|
||||
ALTER TABLE test MODIFY COLUMN x Int8;
|
||||
|
Loading…
Reference in New Issue
Block a user