mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Better names
This commit is contained in:
parent
254d55f2ab
commit
b49be4c7f2
@ -246,7 +246,7 @@ void DatabaseOrdinary::alterTable(
|
||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||
|
||||
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns);
|
||||
ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.indices);
|
||||
ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices);
|
||||
ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints);
|
||||
|
||||
ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns);
|
||||
|
@ -332,8 +332,8 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con
|
||||
else if (type == ADD_INDEX)
|
||||
{
|
||||
if (std::any_of(
|
||||
metadata.indices.cbegin(),
|
||||
metadata.indices.cend(),
|
||||
metadata.secondary_indices.cbegin(),
|
||||
metadata.secondary_indices.cend(),
|
||||
[this](const auto & index)
|
||||
{
|
||||
return index.name == index_name;
|
||||
@ -346,47 +346,47 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con
|
||||
ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
auto insert_it = metadata.indices.end();
|
||||
auto insert_it = metadata.secondary_indices.end();
|
||||
|
||||
if (!after_index_name.empty())
|
||||
{
|
||||
insert_it = std::find_if(
|
||||
metadata.indices.begin(),
|
||||
metadata.indices.end(),
|
||||
metadata.secondary_indices.begin(),
|
||||
metadata.secondary_indices.end(),
|
||||
[this](const auto & index)
|
||||
{
|
||||
return index.name == after_index_name;
|
||||
});
|
||||
|
||||
if (insert_it == metadata.indices.end())
|
||||
if (insert_it == metadata.secondary_indices.end())
|
||||
throw Exception("Wrong index name. Cannot find index " + backQuote(after_index_name) + " to insert after.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
++insert_it;
|
||||
}
|
||||
|
||||
metadata.indices.emplace(insert_it, IndexDescription::getIndexFromAST(index_decl, metadata.columns, context));
|
||||
metadata.secondary_indices.emplace(insert_it, IndexDescription::getIndexFromAST(index_decl, metadata.columns, context));
|
||||
}
|
||||
else if (type == DROP_INDEX)
|
||||
{
|
||||
if (!partition && !clear)
|
||||
{
|
||||
auto erase_it = std::find_if(
|
||||
metadata.indices.begin(),
|
||||
metadata.indices.end(),
|
||||
metadata.secondary_indices.begin(),
|
||||
metadata.secondary_indices.end(),
|
||||
[this](const auto & index)
|
||||
{
|
||||
return index.name == index_name;
|
||||
});
|
||||
|
||||
if (erase_it == metadata.indices.end())
|
||||
if (erase_it == metadata.secondary_indices.end())
|
||||
{
|
||||
if (if_exists)
|
||||
return;
|
||||
throw Exception("Wrong index name. Cannot find index " + backQuote(index_name) + " to drop.", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
metadata.indices.erase(erase_it);
|
||||
metadata.secondary_indices.erase(erase_it);
|
||||
}
|
||||
}
|
||||
else if (type == ADD_CONSTRAINT)
|
||||
|
@ -39,13 +39,13 @@ const ColumnsDescription & IStorage::getColumns() const
|
||||
|
||||
const IndicesDescription & IStorage::getSecondaryIndices() const
|
||||
{
|
||||
return indices;
|
||||
return secondary_indices;
|
||||
}
|
||||
|
||||
|
||||
bool IStorage::hasSecondaryIndices() const
|
||||
{
|
||||
return !indices.empty();
|
||||
return !secondary_indices.empty();
|
||||
}
|
||||
|
||||
const ConstraintsDescription & IStorage::getConstraints() const
|
||||
@ -295,9 +295,9 @@ void IStorage::setColumns(ColumnsDescription columns_)
|
||||
columns = std::move(columns_);
|
||||
}
|
||||
|
||||
void IStorage::setSecondaryIndices(IndicesDescription indices_)
|
||||
void IStorage::setSecondaryIndices(IndicesDescription secondary_indices_)
|
||||
{
|
||||
indices = std::move(indices_);
|
||||
secondary_indices = std::move(secondary_indices_);
|
||||
}
|
||||
|
||||
void IStorage::setConstraints(ConstraintsDescription constraints_)
|
||||
|
@ -144,7 +144,7 @@ public: /// thread-unsafe part. lockStructure must be acquired
|
||||
const ColumnsDescription & getColumns() const; /// returns combined set of columns
|
||||
void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones.
|
||||
|
||||
void setSecondaryIndices(IndicesDescription indices_);
|
||||
void setSecondaryIndices(IndicesDescription secondary_indices_);
|
||||
const IndicesDescription & getSecondaryIndices() const;
|
||||
/// Has at least one non primary index
|
||||
bool hasSecondaryIndices() const;
|
||||
@ -200,7 +200,7 @@ private:
|
||||
mutable std::mutex id_mutex;
|
||||
|
||||
ColumnsDescription columns;
|
||||
IndicesDescription indices;
|
||||
IndicesDescription secondary_indices;
|
||||
ConstraintsDescription constraints;
|
||||
|
||||
StorageMetadataKeyField partition_key;
|
||||
|
@ -425,11 +425,11 @@ void MergeTreeData::setProperties(const StorageInMemoryMetadata & metadata, bool
|
||||
ASTPtr skip_indices_with_primary_key_expr_list = new_primary_key_expr_list->clone();
|
||||
ASTPtr skip_indices_with_sorting_key_expr_list = new_sorting_key_expr_list->clone();
|
||||
|
||||
if (!metadata.indices.empty())
|
||||
if (!metadata.secondary_indices.empty())
|
||||
{
|
||||
std::set<String> indices_names;
|
||||
|
||||
for (const auto & index : metadata.indices)
|
||||
for (const auto & index : metadata.secondary_indices)
|
||||
{
|
||||
|
||||
MergeTreeIndexFactory::instance().validate(index, attach);
|
||||
@ -480,7 +480,7 @@ void MergeTreeData::setProperties(const StorageInMemoryMetadata & metadata, bool
|
||||
new_primary_key.data_types = std::move(new_primary_key_data_types);
|
||||
setPrimaryKey(new_primary_key);
|
||||
|
||||
setSecondaryIndices(metadata.indices);
|
||||
setSecondaryIndices(metadata.secondary_indices);
|
||||
|
||||
setConstraints(metadata.constraints);
|
||||
|
||||
@ -1357,7 +1357,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
|
||||
StorageInMemoryMetadata metadata = getInMemoryMetadata();
|
||||
commands.apply(metadata, global_context);
|
||||
if (getSecondaryIndices().empty() && !metadata.indices.empty() &&
|
||||
if (getSecondaryIndices().empty() && !metadata.secondary_indices.empty() &&
|
||||
!settings.allow_experimental_data_skipping_indices)
|
||||
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
|
||||
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
@ -13,17 +13,17 @@ namespace DB
|
||||
|
||||
StorageInMemoryMetadata::StorageInMemoryMetadata(
|
||||
const ColumnsDescription & columns_,
|
||||
const IndicesDescription & indices_,
|
||||
const IndicesDescription & secondary_indices_,
|
||||
const ConstraintsDescription & constraints_)
|
||||
: columns(columns_)
|
||||
, indices(indices_)
|
||||
, secondary_indices(secondary_indices_)
|
||||
, constraints(constraints_)
|
||||
{
|
||||
}
|
||||
|
||||
StorageInMemoryMetadata::StorageInMemoryMetadata(const StorageInMemoryMetadata & other)
|
||||
: columns(other.columns)
|
||||
, indices(other.indices)
|
||||
, secondary_indices(other.secondary_indices)
|
||||
, constraints(other.constraints)
|
||||
{
|
||||
if (other.partition_by_ast)
|
||||
@ -48,7 +48,7 @@ StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemo
|
||||
return *this;
|
||||
|
||||
columns = other.columns;
|
||||
indices = other.indices;
|
||||
secondary_indices = other.secondary_indices;
|
||||
constraints = other.constraints;
|
||||
|
||||
if (other.partition_by_ast)
|
||||
|
@ -18,7 +18,7 @@ struct StorageInMemoryMetadata
|
||||
/// defaults, comments, etc. All table engines have columns.
|
||||
ColumnsDescription columns;
|
||||
/// Table indices. Currently supported for MergeTree only.
|
||||
IndicesDescription indices;
|
||||
IndicesDescription secondary_indices;
|
||||
/// Table constraints. Currently supported for MergeTree only.
|
||||
ConstraintsDescription constraints;
|
||||
/// PARTITION BY expression. Currently supported for MergeTree only.
|
||||
@ -39,7 +39,7 @@ struct StorageInMemoryMetadata
|
||||
|
||||
StorageInMemoryMetadata(const StorageInMemoryMetadata & other);
|
||||
StorageInMemoryMetadata() = default;
|
||||
StorageInMemoryMetadata(const ColumnsDescription & columns_, const IndicesDescription & indices_, const ConstraintsDescription & constraints_);
|
||||
StorageInMemoryMetadata(const ColumnsDescription & columns_, const IndicesDescription & secondary_indices_, const ConstraintsDescription & constraints_);
|
||||
|
||||
StorageInMemoryMetadata & operator=(const StorageInMemoryMetadata & other);
|
||||
};
|
||||
|
@ -501,7 +501,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
}
|
||||
|
||||
if (metadata_diff.skip_indices_changed)
|
||||
metadata.indices = IndicesDescription::parse(metadata_diff.new_skip_indices, new_columns, global_context);
|
||||
metadata.secondary_indices = IndicesDescription::parse(metadata_diff.new_skip_indices, new_columns, global_context);
|
||||
|
||||
if (metadata_diff.constraints_changed)
|
||||
metadata.constraints = ConstraintsDescription::parse(metadata_diff.new_constraints);
|
||||
@ -3319,8 +3319,8 @@ void StorageReplicatedMergeTree::alter(
|
||||
if (ast_to_str(future_metadata.ttl_for_table_ast) != ast_to_str(current_metadata.ttl_for_table_ast))
|
||||
future_metadata_in_zk.ttl_table = serializeAST(*future_metadata.ttl_for_table_ast);
|
||||
|
||||
String new_indices_str = future_metadata.indices.toString();
|
||||
if (new_indices_str != current_metadata.indices.toString())
|
||||
String new_indices_str = future_metadata.secondary_indices.toString();
|
||||
if (new_indices_str != current_metadata.secondary_indices.toString())
|
||||
future_metadata_in_zk.skip_indices = new_indices_str;
|
||||
|
||||
String new_constraints_str = future_metadata.constraints.toString();
|
||||
|
Loading…
Reference in New Issue
Block a user