Some intermediate solution

This commit is contained in:
alesapin 2020-07-13 20:27:52 +03:00
parent 4a53264a86
commit 1f576ee039
9 changed files with 103 additions and 12 deletions

View File

@ -62,10 +62,6 @@ BlockIO InterpreterAlterQuery::execute()
alter_commands.emplace_back(std::move(*alter_command));
else if (auto partition_command = PartitionCommand::parse(command_ast))
{
if (partition_command->type == PartitionCommand::DROP_DETACHED_PARTITION
&& !context.getSettingsRef().allow_drop_detached)
throw DB::Exception("Cannot execute query: DROP DETACHED PART is disabled "
"(see allow_drop_detached setting)", ErrorCodes::SUPPORT_IS_DISABLED);
partition_commands.emplace_back(std::move(*partition_command));
}
else if (auto mut_command = MutationCommand::parse(command_ast))
@ -90,6 +86,7 @@ BlockIO InterpreterAlterQuery::execute()
if (!partition_commands.empty())
{
table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, context.getSettingsRef());
table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context);
}

View File

@ -1003,6 +1003,11 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
}
}
if(all_columns.empty())
{
throw Exception{"Cannot DROP or CLEAR all columns", ErrorCodes::BAD_ARGUMENTS};
}
validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context);
}

View File

@ -121,6 +121,11 @@ public:
return columns.size();
}
bool empty() const
{
return columns.empty();
}
/// Keep the sequence of columns and allow to lookup by name.
using Container = boost::multi_index_container<
ColumnDescription,

View File

@ -102,9 +102,9 @@ void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settin
}
}
void IStorage::checkAlterPartitionIsPossible(const PartitionCommands & /*commands*/, const Settings & /*settings*/) const
void IStorage::checkAlterPartitionIsPossible(const PartitionCommands & /*commands*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & /*settings*/) const
{
///TODO alesap
throw Exception("Table engine " + getName() + " doesn't support partitioning", ErrorCodes::NOT_IMPLEMENTED);
}
StorageID IStorage::getStorageID() const

View File

@ -361,7 +361,7 @@ public:
}
/// Checks that partition commands can be applied to storage.
virtual void checkAlterPartitionIsPossible(const PartitionCommands & commands, const Settings & settings) const;
virtual void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const;
/** Perform any background work. For example, combining parts in a MergeTree type table.
* Returns whether any work has been done.

View File

@ -110,6 +110,7 @@ namespace ErrorCodes
extern const int UNKNOWN_DISK;
extern const int NOT_ENOUGH_SPACE;
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
extern const int SUPPORT_IS_DISABLED;
}
@ -1421,12 +1422,20 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
columns_in_keys.insert(columns_alter_type_metadata_only.begin(), columns_alter_type_metadata_only.end());
columns_in_keys.insert(columns_alter_type_check_safe_for_partition.begin(), columns_alter_type_check_safe_for_partition.end());
NameSet dropped_columns;
std::map<String, const IDataType *> old_types;
for (const auto & column : old_metadata.getColumns().getAllPhysical())
old_types.emplace(column.name, column.type.get());
for (const AlterCommand & command : commands)
{
/// Just validate partition expression
if (command.partition)
{
getPartitionIDFromQuery(command.partition, global_context);
}
if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned)
{
throw Exception(
@ -1456,6 +1465,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
"Trying to ALTER DROP key " + backQuoteIfNeed(command.column_name) + " column which is a part of key expression",
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
}
dropped_columns.emplace(command.column_name);
}
else if (command.isModifyingData(getInMemoryMetadata()))
{
@ -1530,6 +1540,27 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
checkStoragePolicy(global_context.getStoragePolicy(changed_setting.value.safeGet<String>()));
}
}
for (const auto & part : getDataPartsVector())
{
bool at_least_one_column_rest = false;
for (const auto & column : part->getColumns())
{
if (!dropped_columns.count(column.name))
{
at_least_one_column_rest = true;
break;
}
}
if (!at_least_one_column_rest)
{
std::string postfix = "";
if (dropped_columns.size() > 1)
postfix = "s";
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Cannot drop or clear column{} '{}', because all columns in part '{}' will be removed from disk. Empty parts are not allowed", postfix, boost::algorithm::join(dropped_columns, ", "), part->name);
}
}
}
MergeTreeDataPartType MergeTreeData::choosePartType(size_t bytes_uncompressed, size_t rows_count) const
@ -2525,6 +2556,21 @@ void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const StorageM
context);
}
void MergeTreeData::checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const
{
for (const auto & command : commands)
{
if (command.partition)
getPartitionIDFromQuery(command.partition, global_context);
if (command.type == PartitionCommand::DROP_DETACHED_PARTITION
&& !settings.allow_drop_detached)
throw DB::Exception("Cannot execute query: DROP DETACHED PART is disabled "
"(see allow_drop_detached setting)", ErrorCodes::SUPPORT_IS_DISABLED);
}
}
void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition)
{
const String partition_id = getPartitionIDFromQuery(partition, global_context);
@ -2639,7 +2685,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String
}
String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context & context)
String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context & context) const
{
const auto & partition_ast = ast->as<ASTPartition &>();
@ -3058,7 +3104,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector() const
}
MergeTreeData::DataPartPtr MergeTreeData::getAnyPartInPartition(
const String & partition_id, DataPartsLock & /*data_parts_lock*/)
const String & partition_id, DataPartsLock & /*data_parts_lock*/) const
{
auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndPartitionID{DataPartState::Committed, partition_id});

View File

@ -24,6 +24,7 @@
#include <Disks/StoragePolicy.h>
#include <Interpreters/Aggregator.h>
#include <Storages/extractKeyExpressionList.h>
#include <Storages/PartitionCommands.h>
#include <boost/multi_index_container.hpp>
#include <boost/multi_index/ordered_index.hpp>
@ -505,6 +506,8 @@ public:
/// If something is wrong, throws an exception.
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override;
/// Change MergeTreeSettings
void changeSettings(
const ASTPtr & new_settings,
@ -563,7 +566,7 @@ public:
}
/// For ATTACH/DETACH/DROP PARTITION.
String getPartitionIDFromQuery(const ASTPtr & ast, const Context & context);
String getPartitionIDFromQuery(const ASTPtr & ast, const Context & context) const;
/// Extracts MergeTreeData of other *MergeTree* storage
/// and checks that their structure suitable for ALTER TABLE ATTACH PARTITION FROM
@ -817,7 +820,7 @@ protected:
void removePartContributionToColumnSizes(const DataPartPtr & part);
/// If there is no part in the partition with ID `partition_id`, returns empty ptr. Should be called under the lock.
DataPartPtr getAnyPartInPartition(const String & partition_id, DataPartsLock & data_parts_lock);
DataPartPtr getAnyPartInPartition(const String & partition_id, DataPartsLock & data_parts_lock) const;
/// Return parts in the Committed set that are covered by the new_part_info or the part that covers it.
/// Will check that the new part doesn't already exist and that it doesn't intersect existing part.

View File

@ -0,0 +1,6 @@
0 1 Hello
0 1 Hello
0 2 Hello
0 2 Hello
0 3 Hello
0 3 Hello

View File

@ -1,5 +1,34 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test (x UInt8) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO test (x) VALUES (1), (2), (3);
ALTER TABLE test CLEAR COLUMN x;
ALTER TABLE test CLEAR COLUMN x; --{serverError 36}
DROP TABLE test;
DROP TABLE IF EXISTS test;
CREATE TABLE test (x UInt8, y UInt8) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO test (x, y) VALUES (1, 1), (2, 2), (3, 3);
ALTER TABLE test CLEAR COLUMN x;
ALTER TABLE test CLEAR COLUMN x IN PARTITION ''; --{serverError 248}
ALTER TABLE test CLEAR COLUMN x IN PARTITION 'asdasd'; --{serverError 248}
ALTER TABLE test CLEAR COLUMN x IN PARTITION '123'; --{serverError 248}
ALTER TABLE test CLEAR COLUMN y; --{serverError 36}
ALTER TABLE test ADD COLUMN z String DEFAULT 'Hello';
-- y is only real column in table
ALTER TABLE test CLEAR COLUMN y; --{serverError 36}
ALTER TABLE test CLEAR COLUMN x;
ALTER TABLE test CLEAR COLUMN z;
INSERT INTO test (x, y, z) VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c');
ALTER TABLE test CLEAR COLUMN z;
ALTER TABLE test CLEAR COLUMN x;
SELECT * FROM test ORDER BY y;
DROP TABLE IF EXISTS test;