mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Rename to methods
This commit is contained in:
parent
d9a817f578
commit
663e92b1c5
@ -418,7 +418,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS
|
||||
/// Secondary indices make sense only for MergeTree family of storage engines.
|
||||
/// We should not copy them for other storages.
|
||||
if (create.storage && endsWith(create.storage->engine->name, "MergeTree"))
|
||||
properties.indices = as_storage->getIndices();
|
||||
properties.indices = as_storage->getSecondaryIndices();
|
||||
|
||||
properties.constraints = as_storage->getConstraints();
|
||||
}
|
||||
|
@ -294,7 +294,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
|
||||
|
||||
const ColumnsDescription & columns_desc = storage->getColumns();
|
||||
const IndicesDescription & indices_desc = storage->getIndices();
|
||||
const IndicesDescription & indices_desc = storage->getSecondaryIndices();
|
||||
NamesAndTypesList all_columns = columns_desc.getAllPhysical();
|
||||
|
||||
NameSet updated_columns;
|
||||
|
@ -37,13 +37,13 @@ const ColumnsDescription & IStorage::getColumns() const
|
||||
return columns;
|
||||
}
|
||||
|
||||
const IndicesDescription & IStorage::getIndices() const
|
||||
const IndicesDescription & IStorage::getSecondaryIndices() const
|
||||
{
|
||||
return indices;
|
||||
}
|
||||
|
||||
|
||||
bool IStorage::hasIndices() const
|
||||
bool IStorage::hasSecondaryIndices() const
|
||||
{
|
||||
return !indices.empty();
|
||||
}
|
||||
@ -295,7 +295,7 @@ void IStorage::setColumns(ColumnsDescription columns_)
|
||||
columns = std::move(columns_);
|
||||
}
|
||||
|
||||
void IStorage::setIndices(IndicesDescription indices_)
|
||||
void IStorage::setSecondaryIndices(IndicesDescription indices_)
|
||||
{
|
||||
indices = std::move(indices_);
|
||||
}
|
||||
@ -375,7 +375,7 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id,
|
||||
|
||||
StorageInMemoryMetadata IStorage::getInMemoryMetadata() const
|
||||
{
|
||||
return StorageInMemoryMetadata(getColumns(), getIndices(), getConstraints());
|
||||
return StorageInMemoryMetadata(getColumns(), getSecondaryIndices(), getConstraints());
|
||||
}
|
||||
|
||||
void IStorage::alter(
|
||||
@ -621,7 +621,7 @@ ColumnDependencies IStorage::getColumnDependencies(const NameSet & updated_colum
|
||||
return false;
|
||||
};
|
||||
|
||||
for (const auto & index : getIndices())
|
||||
for (const auto & index : getSecondaryIndices())
|
||||
add_dependent_columns(index.expression, indices_columns);
|
||||
|
||||
if (hasRowsTTL())
|
||||
|
@ -144,10 +144,10 @@ 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 setIndices(IndicesDescription indices_);
|
||||
const IndicesDescription & getIndices() const;
|
||||
void setSecondaryIndices(IndicesDescription indices_);
|
||||
const IndicesDescription & getSecondaryIndices() const;
|
||||
/// Has at least one non primary index
|
||||
bool hasIndices() const;
|
||||
bool hasSecondaryIndices() const;
|
||||
|
||||
const ConstraintsDescription & getConstraints() const;
|
||||
void setConstraints(ConstraintsDescription constraints_);
|
||||
|
@ -248,7 +248,7 @@ MergeTreeData::MergeTreeData(
|
||||
|
||||
StorageInMemoryMetadata MergeTreeData::getInMemoryMetadata() const
|
||||
{
|
||||
StorageInMemoryMetadata metadata(getColumns(), getIndices(), getConstraints());
|
||||
StorageInMemoryMetadata metadata(getColumns(), getSecondaryIndices(), getConstraints());
|
||||
|
||||
if (isPartitionKeyDefined())
|
||||
metadata.partition_by_ast = getPartitionKeyAST()->clone();
|
||||
@ -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);
|
||||
|
||||
setIndices(metadata.indices);
|
||||
setSecondaryIndices(metadata.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 (getIndices().empty() && !metadata.indices.empty() &&
|
||||
if (getSecondaryIndices().empty() && !metadata.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);
|
||||
@ -1378,7 +1378,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
columns_alter_type_forbidden.insert(col);
|
||||
}
|
||||
|
||||
for (const auto & index : getIndices())
|
||||
for (const auto & index : getSecondaryIndices())
|
||||
{
|
||||
for (const String & col : index.expression->getRequiredColumns())
|
||||
columns_alter_type_forbidden.insert(col);
|
||||
@ -3062,7 +3062,7 @@ bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con
|
||||
{
|
||||
if (isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(item))
|
||||
return true;
|
||||
for (const auto & index : getIndices())
|
||||
for (const auto & index : getSecondaryIndices())
|
||||
if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(item))
|
||||
return true;
|
||||
}
|
||||
@ -3071,7 +3071,7 @@ bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & index : getIndices())
|
||||
for (const auto & index : getSecondaryIndices())
|
||||
if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(left_in_operand))
|
||||
return true;
|
||||
|
||||
|
@ -607,7 +607,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
NamesAndTypesList merging_columns;
|
||||
Names gathering_column_names, merging_column_names;
|
||||
extractMergingAndGatheringColumns(
|
||||
storage_columns, data.getSortingKey().expression, data.getIndices(),
|
||||
storage_columns, data.getSortingKey().expression, data.getSecondaryIndices(),
|
||||
data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names);
|
||||
|
||||
auto single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + future_part.name, disk);
|
||||
@ -793,7 +793,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
merged_stream = std::make_shared<TTLBlockInputStream>(merged_stream, data, new_data_part, time_of_merge, force_ttl);
|
||||
|
||||
const auto & index_factory = MergeTreeIndexFactory::instance();
|
||||
if (data.hasIndices())
|
||||
if (data.hasSecondaryIndices())
|
||||
{
|
||||
merged_stream = std::make_shared<ExpressionBlockInputStream>(merged_stream, data.primary_key_and_skip_indices_expr);
|
||||
merged_stream = std::make_shared<MaterializingBlockInputStream>(merged_stream);
|
||||
@ -802,7 +802,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
MergedBlockOutputStream to{
|
||||
new_data_part,
|
||||
merging_columns,
|
||||
index_factory.getMany(data.getIndices()),
|
||||
index_factory.getMany(data.getSecondaryIndices()),
|
||||
compression_codec,
|
||||
merged_column_to_size,
|
||||
data_settings->min_merge_bytes_to_use_direct_io,
|
||||
@ -1074,7 +1074,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
/// All columns from part are changed and may be some more that were missing before in part
|
||||
if (isCompactPart(source_part) || source_part->getColumns().isSubsetOf(updated_header.getNamesAndTypesList()))
|
||||
{
|
||||
auto part_indices = getIndicesForNewDataPart(data.getIndices(), for_file_renames);
|
||||
auto part_indices = getIndicesForNewDataPart(data.getSecondaryIndices(), for_file_renames);
|
||||
mutateAllPartColumns(
|
||||
new_data_part,
|
||||
part_indices,
|
||||
@ -1520,7 +1520,7 @@ std::set<MergeTreeIndexPtr> MergeTreeDataMergerMutator::getIndicesToRecalculate(
|
||||
ASTPtr indices_recalc_expr_list = std::make_shared<ASTExpressionList>();
|
||||
for (const auto & col : updated_columns.getNames())
|
||||
{
|
||||
const auto & indices = data.getIndices();
|
||||
const auto & indices = data.getSecondaryIndices();
|
||||
for (size_t i = 0; i < indices.size(); ++i)
|
||||
{
|
||||
const auto & index = indices[i];
|
||||
@ -1584,7 +1584,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns(
|
||||
if (mutating_stream == nullptr)
|
||||
throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (data.hasPrimaryKey() || data.hasIndices())
|
||||
if (data.hasPrimaryKey() || data.hasSecondaryIndices())
|
||||
mutating_stream = std::make_shared<MaterializingBlockInputStream>(
|
||||
std::make_shared<ExpressionBlockInputStream>(mutating_stream, data.primary_key_and_skip_indices_expr));
|
||||
|
||||
|
@ -549,7 +549,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
std::vector<std::pair<MergeTreeIndexPtr, MergeTreeIndexConditionPtr>> useful_indices;
|
||||
|
||||
for (const auto & index : data.getIndices())
|
||||
for (const auto & index : data.getSecondaryIndices())
|
||||
{
|
||||
auto index_helper = MergeTreeIndexFactory::instance().get(index);
|
||||
auto condition = index_helper->createIndexCondition(query_info, context);
|
||||
|
@ -263,7 +263,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
new_data_part->volume->getDisk()->createDirectories(full_path);
|
||||
|
||||
/// If we need to calculate some columns to sort.
|
||||
if (data.hasSortingKey() || data.hasIndices())
|
||||
if (data.hasSortingKey() || data.hasSecondaryIndices())
|
||||
data.sorting_key_and_skip_indices_expr->execute(block);
|
||||
|
||||
Names sort_columns = data.getSortingKeyColumns();
|
||||
@ -303,7 +303,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
auto compression_codec = data.global_context.chooseCompressionCodec(0, 0);
|
||||
|
||||
const auto & index_factory = MergeTreeIndexFactory::instance();
|
||||
MergedBlockOutputStream out(new_data_part, columns, index_factory.getMany(data.getIndices()), compression_codec);
|
||||
MergedBlockOutputStream out(new_data_part, columns, index_factory.getMany(data.getSecondaryIndices()), compression_codec);
|
||||
|
||||
out.writePrefix();
|
||||
out.writeWithPermutation(block, perm_ptr);
|
||||
|
@ -157,7 +157,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
||||
return;
|
||||
|
||||
std::unordered_set<String> skip_indexes_column_names_set;
|
||||
for (const auto & index : storage.getIndices())
|
||||
for (const auto & index : storage.getSecondaryIndices())
|
||||
std::copy(index.column_names.cbegin(), index.column_names.cend(),
|
||||
std::inserter(skip_indexes_column_names_set, skip_indexes_column_names_set.end()));
|
||||
Names skip_indexes_column_names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end());
|
||||
|
@ -55,7 +55,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
|
||||
|
||||
ttl_table = formattedAST(data.getTableTTLs().definition_ast);
|
||||
|
||||
skip_indices = data.getIndices().toString();
|
||||
skip_indices = data.getSecondaryIndices().toString();
|
||||
if (data.canUseAdaptiveGranularity())
|
||||
index_granularity_bytes = data_settings->index_granularity_bytes;
|
||||
else
|
||||
|
@ -55,7 +55,7 @@ protected:
|
||||
, part(part_)
|
||||
{
|
||||
setColumns(part_->storage.getColumns());
|
||||
setIndices(part_->storage.getIndices());
|
||||
setSecondaryIndices(part_->storage.getSecondaryIndices());
|
||||
setPrimaryKey(part_->storage.getPrimaryKey());
|
||||
setSortingKey(part_->storage.getSortingKey());
|
||||
setColumnTTLs(part->storage.getColumnTTLs());
|
||||
|
@ -158,7 +158,7 @@ StorageMaterializedView::StorageMaterializedView(
|
||||
|
||||
StorageInMemoryMetadata StorageMaterializedView::getInMemoryMetadata() const
|
||||
{
|
||||
StorageInMemoryMetadata result(getColumns(), getIndices(), getConstraints());
|
||||
StorageInMemoryMetadata result(getColumns(), getSecondaryIndices(), getConstraints());
|
||||
result.select = getSelectQuery();
|
||||
return result;
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user