mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-18 05:32:52 +00:00
Preparation for rename
This commit is contained in:
parent
df19d6aac1
commit
4159fa0382
@ -28,6 +28,7 @@ enum class AccessType
|
|||||||
ADD_COLUMN,
|
ADD_COLUMN,
|
||||||
DROP_COLUMN,
|
DROP_COLUMN,
|
||||||
MODIFY_COLUMN,
|
MODIFY_COLUMN,
|
||||||
|
RENAME_COLUMN,
|
||||||
COMMENT_COLUMN,
|
COMMENT_COLUMN,
|
||||||
CLEAR_COLUMN,
|
CLEAR_COLUMN,
|
||||||
ALTER_COLUMN, /// allow to execute ALTER {ADD|DROP|MODIFY...} COLUMN
|
ALTER_COLUMN, /// allow to execute ALTER {ADD|DROP|MODIFY...} COLUMN
|
||||||
@ -195,6 +196,7 @@ namespace impl
|
|||||||
ACCESS_TYPE_TO_KEYWORD_CASE(ADD_COLUMN);
|
ACCESS_TYPE_TO_KEYWORD_CASE(ADD_COLUMN);
|
||||||
ACCESS_TYPE_TO_KEYWORD_CASE(DROP_COLUMN);
|
ACCESS_TYPE_TO_KEYWORD_CASE(DROP_COLUMN);
|
||||||
ACCESS_TYPE_TO_KEYWORD_CASE(MODIFY_COLUMN);
|
ACCESS_TYPE_TO_KEYWORD_CASE(MODIFY_COLUMN);
|
||||||
|
ACCESS_TYPE_TO_KEYWORD_CASE(RENAME_COLUMN);
|
||||||
ACCESS_TYPE_TO_KEYWORD_CASE(COMMENT_COLUMN);
|
ACCESS_TYPE_TO_KEYWORD_CASE(COMMENT_COLUMN);
|
||||||
ACCESS_TYPE_TO_KEYWORD_CASE(CLEAR_COLUMN);
|
ACCESS_TYPE_TO_KEYWORD_CASE(CLEAR_COLUMN);
|
||||||
ACCESS_TYPE_TO_KEYWORD_CASE(ALTER_COLUMN);
|
ACCESS_TYPE_TO_KEYWORD_CASE(ALTER_COLUMN);
|
||||||
|
@ -280,6 +280,11 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS
|
|||||||
required_access.emplace_back(AccessType::REFRESH_VIEW, database, table);
|
required_access.emplace_back(AccessType::REFRESH_VIEW, database, table);
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
case ASTAlterCommand::RENAME_COLUMN:
|
||||||
|
{
|
||||||
|
required_access.emplace_back(AccessType::RENAME_COLUMN, database, table);
|
||||||
|
break;
|
||||||
|
}
|
||||||
case ASTAlterCommand::NO_TYPE: break;
|
case ASTAlterCommand::NO_TYPE: break;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -56,6 +56,11 @@ ASTPtr ASTAlterCommand::clone() const
|
|||||||
res->values = values->clone();
|
res->values = values->clone();
|
||||||
res->children.push_back(res->values);
|
res->children.push_back(res->values);
|
||||||
}
|
}
|
||||||
|
if (rename_to)
|
||||||
|
{
|
||||||
|
res->rename_to = rename_to->clone();
|
||||||
|
res->children.push_back(res->rename_to);
|
||||||
|
}
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
@ -285,6 +290,15 @@ void ASTAlterCommand::formatImpl(
|
|||||||
{
|
{
|
||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : "");
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : "");
|
||||||
}
|
}
|
||||||
|
else if (type == ASTAlterCommand::RENAME_COLUMN)
|
||||||
|
{
|
||||||
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "RENAME COLUMN " << (if_exists ? "IF EXISTS " : "")
|
||||||
|
<< (settings.hilite ? hilite_none : "");
|
||||||
|
column->formatImpl(settings, state, frame);
|
||||||
|
|
||||||
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO ";
|
||||||
|
rename_to->formatImpl(settings, state, frame);
|
||||||
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||||
}
|
}
|
||||||
|
@ -29,6 +29,7 @@ public:
|
|||||||
DROP_COLUMN,
|
DROP_COLUMN,
|
||||||
MODIFY_COLUMN,
|
MODIFY_COLUMN,
|
||||||
COMMENT_COLUMN,
|
COMMENT_COLUMN,
|
||||||
|
RENAME_COLUMN,
|
||||||
MODIFY_ORDER_BY,
|
MODIFY_ORDER_BY,
|
||||||
MODIFY_TTL,
|
MODIFY_TTL,
|
||||||
MATERIALIZE_TTL,
|
MATERIALIZE_TTL,
|
||||||
@ -69,6 +70,7 @@ public:
|
|||||||
|
|
||||||
/** The ADD COLUMN query here optionally stores the name of the column following AFTER
|
/** The ADD COLUMN query here optionally stores the name of the column following AFTER
|
||||||
* The DROP query stores the column name for deletion here
|
* The DROP query stores the column name for deletion here
|
||||||
|
* Also used for RENAME COLUMN.
|
||||||
*/
|
*/
|
||||||
ASTPtr column;
|
ASTPtr column;
|
||||||
|
|
||||||
@ -155,6 +157,9 @@ public:
|
|||||||
String to_database;
|
String to_database;
|
||||||
String to_table;
|
String to_table;
|
||||||
|
|
||||||
|
/// Target column name
|
||||||
|
ASTPtr rename_to;
|
||||||
|
|
||||||
String getID(char delim) const override { return "AlterCommand" + (delim + std::to_string(static_cast<int>(type))); }
|
String getID(char delim) const override { return "AlterCommand" + (delim + std::to_string(static_cast<int>(type))); }
|
||||||
|
|
||||||
ASTPtr clone() const override;
|
ASTPtr clone() const override;
|
||||||
|
@ -231,10 +231,20 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
|||||||
else if (command_ast->type == ASTAlterCommand::MODIFY_QUERY)
|
else if (command_ast->type == ASTAlterCommand::MODIFY_QUERY)
|
||||||
{
|
{
|
||||||
AlterCommand command;
|
AlterCommand command;
|
||||||
|
command.ast = command_ast->clone();
|
||||||
command.type = AlterCommand::MODIFY_QUERY;
|
command.type = AlterCommand::MODIFY_QUERY;
|
||||||
command.select = command_ast->select;
|
command.select = command_ast->select;
|
||||||
return command;
|
return command;
|
||||||
}
|
}
|
||||||
|
else if (command_ast->type == ASTAlterCommand::RENAME_COLUMN)
|
||||||
|
{
|
||||||
|
AlterCommand command;
|
||||||
|
command.ast = command_ast->clone();
|
||||||
|
command.type = AlterCommand::RENAME_COLUMN;
|
||||||
|
command.column_name = command_ast->column->as<ASTIdentifier &>().name;
|
||||||
|
command.rename_to = command_ast->rename_to->as<ASTIdentifier &>().name;
|
||||||
|
return command;
|
||||||
|
}
|
||||||
else
|
else
|
||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
@ -437,6 +447,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
|
|||||||
settings_from_storage.push_back(change);
|
settings_from_storage.push_back(change);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
else if (type == RENAME_COLUMN)
|
||||||
|
{
|
||||||
|
metadata.columns.rename(column_name, rename_to);
|
||||||
|
}
|
||||||
else
|
else
|
||||||
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
@ -519,7 +533,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada
|
|||||||
if (ignore)
|
if (ignore)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
if (type == DROP_COLUMN || type == DROP_INDEX)
|
if (type == DROP_COLUMN || type == DROP_INDEX || type == RENAME_COLUMN)
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
if (type != MODIFY_COLUMN || data_type == nullptr)
|
if (type != MODIFY_COLUMN || data_type == nullptr)
|
||||||
@ -619,6 +633,8 @@ String alterTypeToString(const AlterCommand::Type type)
|
|||||||
return "MODIFY SETTING";
|
return "MODIFY SETTING";
|
||||||
case AlterCommand::Type::MODIFY_QUERY:
|
case AlterCommand::Type::MODIFY_QUERY:
|
||||||
return "MODIFY QUERY";
|
return "MODIFY QUERY";
|
||||||
|
case AlterCommand::Type::RENAME_COLUMN:
|
||||||
|
return "RENAME COLUMN";
|
||||||
}
|
}
|
||||||
__builtin_unreachable();
|
__builtin_unreachable();
|
||||||
}
|
}
|
||||||
|
@ -35,6 +35,7 @@ struct AlterCommand
|
|||||||
MODIFY_TTL,
|
MODIFY_TTL,
|
||||||
MODIFY_SETTING,
|
MODIFY_SETTING,
|
||||||
MODIFY_QUERY,
|
MODIFY_QUERY,
|
||||||
|
RENAME_COLUMN,
|
||||||
};
|
};
|
||||||
|
|
||||||
Type type;
|
Type type;
|
||||||
@ -96,6 +97,9 @@ struct AlterCommand
|
|||||||
/// For MODIFY_QUERY
|
/// For MODIFY_QUERY
|
||||||
ASTPtr select = nullptr;
|
ASTPtr select = nullptr;
|
||||||
|
|
||||||
|
/// Target column name
|
||||||
|
String rename_to;
|
||||||
|
|
||||||
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
|
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
|
||||||
|
|
||||||
void apply(StorageInMemoryMetadata & metadata) const;
|
void apply(StorageInMemoryMetadata & metadata) const;
|
||||||
|
@ -1,4 +1,6 @@
|
|||||||
#include <Storages/ColumnsDescription.h>
|
#include <Storages/ColumnsDescription.h>
|
||||||
|
|
||||||
|
#include <boost/algorithm/string/replace.hpp>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
@ -195,6 +197,27 @@ void ColumnsDescription::remove(const String & column_name)
|
|||||||
list_it = columns.get<0>().erase(list_it);
|
list_it = columns.get<0>().erase(list_it);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void ColumnsDescription::rename(const String & column_from, const String & column_to)
|
||||||
|
{
|
||||||
|
auto range = getNameRange(columns, column_from);
|
||||||
|
|
||||||
|
if (range.first == range.second)
|
||||||
|
throw Exception("There is no column " + column_from + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||||
|
|
||||||
|
std::vector<ColumnDescription> iterators;
|
||||||
|
for (auto list_it = range.first; list_it != range.second;)
|
||||||
|
{
|
||||||
|
iterators.push_back(*list_it);
|
||||||
|
list_it = columns.get<0>().erase(list_it);
|
||||||
|
}
|
||||||
|
|
||||||
|
for (auto & col_desc : iterators)
|
||||||
|
{
|
||||||
|
boost::replace_all(col_desc.name, column_from, column_to);
|
||||||
|
add(col_desc);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void ColumnsDescription::flattenNested()
|
void ColumnsDescription::flattenNested()
|
||||||
{
|
{
|
||||||
|
@ -57,6 +57,9 @@ public:
|
|||||||
/// `column_name` can be a Nested column name;
|
/// `column_name` can be a Nested column name;
|
||||||
void remove(const String & column_name);
|
void remove(const String & column_name);
|
||||||
|
|
||||||
|
/// TODO(alesap)
|
||||||
|
void rename(const String & column_from, const String & column_to);
|
||||||
|
|
||||||
void flattenNested(); /// TODO: remove, insert already flattened Nested columns.
|
void flattenNested(); /// TODO: remove, insert already flattened Nested columns.
|
||||||
|
|
||||||
bool operator==(const ColumnsDescription & other) const { return columns == other.columns; }
|
bool operator==(const ColumnsDescription & other) const { return columns == other.columns; }
|
||||||
|
@ -3589,4 +3589,8 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
MergeTreeData::AlterConversions MergeTreeData::getAlterConversionsForPart(const MergeTreeDataPartPtr /*part*/) const
|
||||||
|
{
|
||||||
|
return AlterConversions{};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -33,6 +33,7 @@ namespace DB
|
|||||||
class MergeListEntry;
|
class MergeListEntry;
|
||||||
class AlterCommands;
|
class AlterCommands;
|
||||||
class MergeTreePartsMover;
|
class MergeTreePartsMover;
|
||||||
|
class MutationCommands;
|
||||||
|
|
||||||
class ExpressionActions;
|
class ExpressionActions;
|
||||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||||
@ -124,6 +125,11 @@ public:
|
|||||||
|
|
||||||
STRONG_TYPEDEF(String, PartitionID)
|
STRONG_TYPEDEF(String, PartitionID)
|
||||||
|
|
||||||
|
struct AlterConversions
|
||||||
|
{
|
||||||
|
std::unordered_map<String, String> rename_map;
|
||||||
|
};
|
||||||
|
|
||||||
struct LessDataPart
|
struct LessDataPart
|
||||||
{
|
{
|
||||||
using is_transparent = void;
|
using is_transparent = void;
|
||||||
@ -647,6 +653,8 @@ public:
|
|||||||
/// Reserves 0 bytes
|
/// Reserves 0 bytes
|
||||||
ReservationPtr makeEmptyReservationOnLargestDisk() { return getStoragePolicy()->makeEmptyReservationOnLargestDisk(); }
|
ReservationPtr makeEmptyReservationOnLargestDisk() { return getStoragePolicy()->makeEmptyReservationOnLargestDisk(); }
|
||||||
|
|
||||||
|
AlterConversions getAlterConversionsForPart(const MergeTreeDataPartPtr part) const;
|
||||||
|
|
||||||
MergeTreeDataFormatVersion format_version;
|
MergeTreeDataFormatVersion format_version;
|
||||||
|
|
||||||
Context & global_context;
|
Context & global_context;
|
||||||
@ -908,6 +916,7 @@ protected:
|
|||||||
/// mechanisms for parts locking
|
/// mechanisms for parts locking
|
||||||
virtual bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const = 0;
|
virtual bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const = 0;
|
||||||
|
|
||||||
|
virtual MutationCommands getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const = 0;
|
||||||
/// Moves part to specified space, used in ALTER ... MOVE ... queries
|
/// Moves part to specified space, used in ALTER ... MOVE ... queries
|
||||||
bool movePartsToSpace(const DataPartsVector & parts, SpacePtr space);
|
bool movePartsToSpace(const DataPartsVector & parts, SpacePtr space);
|
||||||
|
|
||||||
|
@ -1309,6 +1309,21 @@ ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zk
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
MutationCommands ReplicatedMergeTreeQueue::getFirstAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const
|
||||||
|
{
|
||||||
|
std::lock_guard lock(state_mutex);
|
||||||
|
auto in_partition = mutations_by_partition.find(part->info.partition_id);
|
||||||
|
if (in_partition == mutations_by_partition.end())
|
||||||
|
return MutationCommands{};
|
||||||
|
|
||||||
|
Int64 part_version = part->info.getDataVersion();
|
||||||
|
for (auto [mutation_version, mutation_status] : in_partition->second)
|
||||||
|
if (mutation_version > part_version && mutation_status->entry->alter_version != -1)
|
||||||
|
return mutation_status->entry->commands;
|
||||||
|
|
||||||
|
return MutationCommands{};
|
||||||
|
}
|
||||||
|
|
||||||
MutationCommands ReplicatedMergeTreeQueue::getMutationCommands(
|
MutationCommands ReplicatedMergeTreeQueue::getMutationCommands(
|
||||||
const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version) const
|
const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version) const
|
||||||
{
|
{
|
||||||
|
@ -331,6 +331,9 @@ public:
|
|||||||
|
|
||||||
MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version) const;
|
MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version) const;
|
||||||
|
|
||||||
|
/// TODO(alesap)
|
||||||
|
MutationCommands getFirstAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const;
|
||||||
|
|
||||||
/// Mark finished mutations as done. If the function needs to be called again at some later time
|
/// Mark finished mutations as done. If the function needs to be called again at some later time
|
||||||
/// (because some mutations are probably done but we are not sure yet), returns true.
|
/// (because some mutations are probably done but we are not sure yet), returns true.
|
||||||
bool tryFinalizeMutations(zkutil::ZooKeeperPtr zookeeper);
|
bool tryFinalizeMutations(zkutil::ZooKeeperPtr zookeeper);
|
||||||
|
@ -164,6 +164,8 @@ protected:
|
|||||||
const MergingParams & merging_params_,
|
const MergingParams & merging_params_,
|
||||||
std::unique_ptr<MergeTreeSettings> settings_,
|
std::unique_ptr<MergeTreeSettings> settings_,
|
||||||
bool has_force_restore_data_flag);
|
bool has_force_restore_data_flag);
|
||||||
|
|
||||||
|
MutationCommands getFirtsAlterMutationCommandsForPart(const DataPartPtr & /* part */) const override { return {}; }
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -5297,4 +5297,9 @@ StorageReplicatedMergeTree::getMetadataFromSharedZookeeper(const String & metada
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
MutationCommands StorageReplicatedMergeTree::getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const
|
||||||
|
{
|
||||||
|
return queue.getFirstAlterMutationCommandsForPart(part);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -527,6 +527,9 @@ private:
|
|||||||
|
|
||||||
StorageInMemoryMetadata getMetadataFromSharedZookeeper(const String & metadata_str, const String & columns_str) const;
|
StorageInMemoryMetadata getMetadataFromSharedZookeeper(const String & metadata_str, const String & columns_str) const;
|
||||||
|
|
||||||
|
|
||||||
|
MutationCommands getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const override;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
/** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table.
|
/** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table.
|
||||||
*/
|
*/
|
||||||
@ -542,6 +545,7 @@ protected:
|
|||||||
const MergingParams & merging_params_,
|
const MergingParams & merging_params_,
|
||||||
std::unique_ptr<MergeTreeSettings> settings_,
|
std::unique_ptr<MergeTreeSettings> settings_,
|
||||||
bool has_force_restore_data_flag);
|
bool has_force_restore_data_flag);
|
||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user