I'm able to build it

This commit is contained in:
alesapin 2020-05-28 15:37:05 +03:00
parent 380c60a33f
commit 52ca6b2051
37 changed files with 539 additions and 438 deletions

View File

@ -18,6 +18,11 @@ namespace ErrorCodes
static constexpr UInt64 SEED_GEN_A = 845897321;
static constexpr UInt64 SEED_GEN_B = 217728422;
BloomFilter::BloomFilter(const BloomFilterParameters & params)
: BloomFilter(params.filter_size, params.filter_hashes, params.seed)
{
}
BloomFilter::BloomFilter(size_t size_, size_t hashes_, size_t seed_)
: size(size_), hashes(hashes_), seed(seed_), words((size + sizeof(UnderType) - 1) / sizeof(UnderType)), filter(words, 0) {}

View File

@ -11,6 +11,15 @@
namespace DB
{
struct BloomFilterParameters
{
/// size of filter in bytes.
size_t filter_size;
/// number of used hash functions.
size_t filter_hashes;
/// random seed for hash functions generation.
size_t seed;
};
class BloomFilter
{
@ -19,6 +28,7 @@ public:
using UnderType = UInt64;
using Container = std::vector<UnderType>;
BloomFilter(const BloomFilterParameters & params);
/// size -- size of filter in bytes.
/// hashes -- number of used hash functions.
/// seed -- random seed for hash functions generation.

View File

@ -24,6 +24,7 @@
#include <Parsers/parseQuery.h>
#include <Storages/StorageFactory.h>
#include <Storages/StorageInMemoryMetadata.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
@ -252,7 +253,7 @@ ASTPtr InterpreterCreateQuery::formatIndices(const IndicesDescription & indices)
auto res = std::make_shared<ASTExpressionList>();
for (const auto & index : indices.indices)
res->children.push_back(index->clone());
res->children.push_back(index.definition_ast->clone());
return res;
}
@ -399,7 +400,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS
if (create.columns_list->indices)
for (const auto & index : create.columns_list->indices->children)
properties.indices.indices.push_back(
std::dynamic_pointer_cast<ASTIndexDeclaration>(index->clone()));
StorageMetadataSkipIndexField::getSkipIndexFromAST(index->clone(), properties.columns, context));
properties.constraints = getConstraintsDescription(create.columns_list->constraints);
}

View File

@ -3,7 +3,7 @@
#include <Interpreters/IInterpreter.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/IStorage_fwd.h>
#include <Storages/IndicesDescription.h>
#include <Storages/StorageInMemoryMetadata.h>
#include <Storages/ConstraintsDescription.h>
#include <Common/ThreadPool.h>
#include <Access/AccessRightsElement.h>

View File

@ -392,14 +392,14 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
{
auto it = std::find_if(
std::cbegin(indices_desc.indices), std::end(indices_desc.indices),
[&](const std::shared_ptr<ASTIndexDeclaration> & index)
[&](const StorageMetadataSkipIndexField & index)
{
return index->name == command.index_name;
return index.name == command.index_name;
});
if (it == std::cend(indices_desc.indices))
throw Exception("Unknown index: " + command.index_name, ErrorCodes::BAD_ARGUMENTS);
auto query = (*it)->expr->clone();
auto query = (*it).expression_list_ast->clone();
auto syntax_result = SyntaxAnalyzer(context).analyze(query, all_columns);
const auto required_columns = syntax_result->requiredSourceColumns();
for (const auto & column : required_columns)

View File

@ -253,7 +253,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
}
void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & context) const
{
if (type == ADD_COLUMN)
{
@ -334,9 +334,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
if (std::any_of(
metadata.indices.indices.cbegin(),
metadata.indices.indices.cend(),
[this](const ASTPtr & index_ast)
[this](const auto & index)
{
return index_ast->as<ASTIndexDeclaration &>().name == index_name;
return index.name == index_name;
}))
{
if (if_not_exists)
@ -353,9 +353,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
insert_it = std::find_if(
metadata.indices.indices.begin(),
metadata.indices.indices.end(),
[this](const ASTPtr & index_ast)
[this](const auto & index)
{
return index_ast->as<ASTIndexDeclaration &>().name == after_index_name;
return index.name == after_index_name;
});
if (insert_it == metadata.indices.indices.end())
@ -365,7 +365,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
++insert_it;
}
metadata.indices.indices.emplace(insert_it, std::dynamic_pointer_cast<ASTIndexDeclaration>(index_decl));
metadata.indices.indices.emplace(insert_it, StorageMetadataSkipIndexField::getSkipIndexFromAST(index_decl, metadata.columns, context));
}
else if (type == DROP_INDEX)
{
@ -374,9 +374,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
auto erase_it = std::find_if(
metadata.indices.indices.begin(),
metadata.indices.indices.end(),
[this](const ASTPtr & index_ast)
[this](const auto & index)
{
return index_ast->as<ASTIndexDeclaration &>().name == index_name;
return index.name == index_name;
});
if (erase_it == metadata.indices.indices.end())
@ -615,7 +615,7 @@ bool AlterCommand::isTTLAlter(const StorageInMemoryMetadata & metadata) const
return ttl_changed;
}
std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(StorageInMemoryMetadata & metadata) const
std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, const Context & context) const
{
if (!isRequireMutationStage(metadata))
return {};
@ -658,7 +658,7 @@ std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(Storage
}
result.ast = ast->clone();
apply(metadata);
apply(metadata, context);
return result;
}
@ -697,7 +697,7 @@ String alterTypeToString(const AlterCommand::Type type)
__builtin_unreachable();
}
void AlterCommands::apply(StorageInMemoryMetadata & metadata) const
void AlterCommands::apply(StorageInMemoryMetadata & metadata, const Context & context) const
{
if (!prepared)
throw DB::Exception("Alter commands is not prepared. Cannot apply. It's a bug", ErrorCodes::LOGICAL_ERROR);
@ -705,7 +705,7 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata) const
auto metadata_copy = metadata;
for (const AlterCommand & command : *this)
if (!command.ignore)
command.apply(metadata_copy);
command.apply(metadata_copy, context);
metadata = std::move(metadata_copy);
}
@ -975,11 +975,11 @@ static MutationCommand createMaterializeTTLCommand()
return command;
}
MutationCommands AlterCommands::getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl) const
MutationCommands AlterCommands::getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, const Context & context) const
{
MutationCommands result;
for (const auto & alter_cmd : *this)
if (auto mutation_cmd = alter_cmd.tryConvertToMutationCommand(metadata); mutation_cmd)
if (auto mutation_cmd = alter_cmd.tryConvertToMutationCommand(metadata, context); mutation_cmd)
result.push_back(*mutation_cmd);
if (materialize_ttl)

View File

@ -102,7 +102,7 @@ struct AlterCommand
static std::optional<AlterCommand> parse(const ASTAlterCommand * command, bool sanity_check_compression_codecs);
void apply(StorageInMemoryMetadata & metadata) const;
void apply(StorageInMemoryMetadata & metadata, const Context & context) const;
/// Checks that alter query changes data. For MergeTree:
/// * column files (data and marks)
@ -124,7 +124,7 @@ struct AlterCommand
/// If possible, convert alter command to mutation command. In other case
/// return empty optional. Some storages may execute mutations after
/// metadata changes.
std::optional<MutationCommand> tryConvertToMutationCommand(StorageInMemoryMetadata & metadata) const;
std::optional<MutationCommand> tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, const Context & context) const;
};
/// Return string representation of AlterCommand::Type
@ -151,7 +151,7 @@ public:
/// Apply all alter command in sequential order to storage metadata.
/// Commands have to be prepared before apply.
void apply(StorageInMemoryMetadata & metadata) const;
void apply(StorageInMemoryMetadata & metadata, const Context & context) const;
/// At least one command modify data on disk.
bool isModifyingData(const StorageInMemoryMetadata & metadata) const;
@ -166,7 +166,7 @@ public:
/// alter. If alter can be performed as pure metadata update, than result is
/// empty. If some TTL changes happened than, depending on materialize_ttl
/// additional mutation command (MATERIALIZE_TTL) will be returned.
MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl) const;
MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, const Context & context) const;
};
}

View File

@ -37,13 +37,13 @@ const ColumnsDescription & IStorage::getColumns() const
return columns;
}
const IndicesDescription & IStorage::getSkipIndices() const
const IndicesDescription & IStorage::getIndices() const
{
return indices;
}
bool IStorage::hasSkipIndices() const
bool IStorage::hasIndices() const
{
return !indices.empty();
}
@ -295,7 +295,7 @@ void IStorage::setColumns(ColumnsDescription columns_)
columns = std::move(columns_);
}
void IStorage::setSkipIndices(IndicesDescription indices_)
void IStorage::setIndices(IndicesDescription indices_)
{
indices = std::move(indices_);
}
@ -375,7 +375,7 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id,
StorageInMemoryMetadata IStorage::getInMemoryMetadata() const
{
return StorageInMemoryMetadata(getColumns(), getSkipIndices(), getConstraints());
return StorageInMemoryMetadata(getColumns(), getIndices(), getConstraints());
}
void IStorage::alter(
@ -386,7 +386,7 @@ void IStorage::alter(
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
auto table_id = getStorageID();
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
params.apply(metadata, context);
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
setColumns(std::move(metadata.columns));
}

View File

@ -143,9 +143,9 @@ public: /// thread-unsafe part. lockStructure must be acquired
virtual const ColumnsDescription & getColumns() const; /// returns combined set of columns
virtual void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones.
void setSkipIndices(StorageMetadataSkipIndices indices_);
const StorageMetadataSkipIndices & getSkipIndices() const;
bool hasSkipIndices() const;
void setIndices(IndicesDescription indices_);
const IndicesDescription & getIndices() const;
bool hasIndices() const;
const ConstraintsDescription & getConstraints() const;
void setConstraints(ConstraintsDescription constraints_);
@ -198,7 +198,7 @@ private:
mutable std::mutex id_mutex;
ColumnsDescription columns;
StorageMetadataSkipIndices indices;
IndicesDescription indices;
ConstraintsDescription constraints;
StorageMetadataKeyField partition_key;

View File

@ -11,46 +11,46 @@
namespace DB
{
bool IndicesDescription::empty() const
{
return indices.empty();
}
bool IndicesDescription::has(const String & name) const
{
return std::cend(indices) != std::find_if(
std::cbegin(indices), std::cend(indices),
[&name](const auto & index)
{
return index->name == name;
});
}
String IndicesDescription::toString() const
{
if (indices.empty())
return {};
ASTExpressionList list;
for (const auto & index : indices)
list.children.push_back(index);
return serializeAST(list, true);
}
IndicesDescription IndicesDescription::parse(const String & str)
{
if (str.empty())
return {};
IndicesDescription res;
ParserIndexDeclarationList parser;
ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
for (const auto & index : list->children)
res.indices.push_back(std::dynamic_pointer_cast<ASTIndexDeclaration>(index));
return res;
}
//bool IndicesDescription::empty() const
//{
// return indices.empty();
//}
//
//bool IndicesDescription::has(const String & name) const
//{
// return std::cend(indices) != std::find_if(
// std::cbegin(indices), std::cend(indices),
// [&name](const auto & index)
// {
// return index->name == name;
// });
//}
//
//String IndicesDescription::toString() const
//{
// if (indices.empty())
// return {};
//
// ASTExpressionList list;
// for (const auto & index : indices)
// list.children.push_back(index);
//
// return serializeAST(list, true);
//}
//
//IndicesDescription IndicesDescription::parse(const String & str)
//{
// if (str.empty())
// return {};
//
// IndicesDescription res;
// ParserIndexDeclarationList parser;
// ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
//
// for (const auto & index : list->children)
// res.indices.push_back(std::dynamic_pointer_cast<ASTIndexDeclaration>(index));
//
// return res;
//}
}

View File

@ -8,20 +8,20 @@
namespace DB
{
class ASTIndexDeclaration;
using IndicesASTs = std::vector<std::shared_ptr<ASTIndexDeclaration>>;
struct IndicesDescription
{
IndicesASTs indices;
IndicesDescription() = default;
bool empty() const;
bool has(const String & name) const;
String toString() const;
static IndicesDescription parse(const String & str);
};
//class ASTIndexDeclaration;
//using IndicesASTs = std::vector<std::shared_ptr<ASTIndexDeclaration>>;
//
//struct IndicesDescription
//{
// IndicesASTs indices;
//
// IndicesDescription() = default;
//
// bool empty() const;
// bool has(const String & name) const;
//
// String toString() const;
// static IndicesDescription parse(const String & str);
//};
}

View File

@ -294,7 +294,7 @@ void IMergeTreeDataPartWriter::calculateAndSerializeSkipIndices(
++skip_index_filling[i];
/// write index if it is filled
if (skip_index_filling[i] == index->granularity)
if (skip_index_filling[i] == index->index.granularity)
{
skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed);
skip_index_filling[i] = 0;

View File

@ -425,36 +425,27 @@ 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();
MergeTreeIndices new_indices;
if (!metadata.indices.indices.empty())
if (!metadata.indices.empty())
{
std::set<String> indices_names;
for (const auto & index_ast : metadata.indices.indices)
for (const auto & index : metadata.indices.indices)
{
const auto & index_decl = std::dynamic_pointer_cast<ASTIndexDeclaration>(index_ast);
new_indices.push_back(
MergeTreeIndexFactory::instance().get(
all_columns,
std::dynamic_pointer_cast<ASTIndexDeclaration>(index_decl),
global_context,
attach));
MergeTreeIndexFactory::instance().validate(index, attach);
if (indices_names.find(new_indices.back()->name) != indices_names.end())
if (indices_names.find(index.name) != indices_names.end())
throw Exception(
"Index with name " + backQuote(new_indices.back()->name) + " already exsists",
"Index with name " + backQuote(index.name) + " already exsists",
ErrorCodes::LOGICAL_ERROR);
ASTPtr expr_list = MergeTreeData::extractKeyExpressionList(index_decl->expr->clone());
for (const auto & expr : expr_list->children)
for (const auto & expr : index.expression_list_ast->children)
{
skip_indices_with_primary_key_expr_list->children.push_back(expr->clone());
skip_indices_with_sorting_key_expr_list->children.push_back(expr->clone());
}
indices_names.insert(new_indices.back()->name);
indices_names.insert(index.name);
}
}
auto syntax_primary = SyntaxAnalyzer(global_context).analyze(
@ -489,8 +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);
setSkipIndices(metadata.indices);
skip_indices = std::move(new_indices);
setIndices(metadata.indices);
setConstraints(metadata.constraints);
@ -1367,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);
commands.apply(metadata, global_context);
if (getIndices().empty() && !metadata.indices.empty() &&
!settings.allow_experimental_data_skipping_indices)
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
@ -1389,9 +1379,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
columns_alter_type_forbidden.insert(col);
}
for (const auto & index : skip_indices)
for (const auto & index : getIndices().indices)
{
for (const String & col : index->expr->getRequiredColumns())
for (const String & col : index.expression->getRequiredColumns())
columns_alter_type_forbidden.insert(col);
}
@ -3067,14 +3057,15 @@ bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con
/// If there is a tuple on the left side of the IN operator, at least one item of the tuple
/// must be part of the key (probably wrapped by a chain of some acceptable functions).
const auto * left_in_operand_tuple = left_in_operand->as<ASTFunction>();
const auto & index_wrapper_factory = MergeTreeIndexFactory::instance();
if (left_in_operand_tuple && left_in_operand_tuple->name == "tuple")
{
for (const auto & item : left_in_operand_tuple->arguments->children)
{
if (isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(item))
return true;
for (const auto & index : skip_indices)
if (index->mayBenefitFromIndexForIn(item))
for (const auto & index : getIndices().indices)
if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(item))
return true;
}
/// The tuple itself may be part of the primary key, so check that as a last resort.
@ -3082,8 +3073,8 @@ bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con
}
else
{
for (const auto & index : skip_indices)
if (index->mayBenefitFromIndexForIn(left_in_operand))
for (const auto & index : getIndices().indices)
if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(left_in_operand))
return true;
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand);
@ -3496,8 +3487,8 @@ ColumnDependencies MergeTreeData::getColumnDependencies(const NameSet & updated_
return false;
};
for (const auto & index : skip_indices)
add_dependent_columns(index->expr, indices_columns);
for (const auto & index : getIndices().indices)
add_dependent_columns(index.expression, indices_columns);
if (hasRowsTTL())
{

View File

@ -607,8 +607,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
NamesAndTypesList gathering_columns;
NamesAndTypesList merging_columns;
Names gathering_column_names, merging_column_names;
const auto & index_factory = MergeTreeIndexFactory::instance();
extractMergingAndGatheringColumns(
storage_columns, data.getSortingKey().expression, data.skip_indices,
storage_columns, data.getSortingKey().expression, index_factory.getMany(data.getIndices().indices),
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);
@ -715,7 +716,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
Pipe pipe(std::move(input));
if (data.hasPrimaryKey() || data.hasSkipIndices())
if (data.hasPrimaryKey() || data.hasIndices())
{
auto expr = std::make_shared<ExpressionTransform>(pipe.getHeader(), data.sorting_key_and_skip_indices_expr);
pipe.addSimpleTransform(std::move(expr));
@ -799,7 +800,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
MergedBlockOutputStream to{
new_data_part,
merging_columns,
data.skip_indices,
index_factory.getMany(data.getIndices().indices),
compression_codec,
merged_column_to_size,
data_settings->min_merge_bytes_to_use_direct_io,
@ -1063,7 +1064,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.skip_indices, for_file_renames);
auto part_indices = getIndicesForNewDataPart(data.getIndices(), for_file_renames);
mutateAllPartColumns(
new_data_part,
part_indices,
@ -1079,7 +1080,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
else /// TODO: check that we modify only non-key columns in this case.
{
/// We will modify only some of the columns. Other columns and key values can be copied as-is.
auto indices_to_recalc = getIndicesToRecalculate(in, storage_from_source_part, updated_header.getNamesAndTypesList(), context);
auto indices_to_recalc = getIndicesToRecalculate(in, updated_header.getNamesAndTypesList(), context);
NameSet files_to_skip = collectFilesToSkip(updated_header, indices_to_recalc, mrk_extension);
NameToNameVector files_to_rename = collectFilesForRenames(source_part, for_file_renames, mrk_extension);
@ -1482,7 +1483,7 @@ NamesAndTypesList MergeTreeDataMergerMutator::getColumnsForNewDataPart(
}
MergeTreeIndices MergeTreeDataMergerMutator::getIndicesForNewDataPart(
const MergeTreeIndices & all_indices,
const IndicesDescription & all_indices,
const MutationCommands & commands_for_removes)
{
NameSet removed_indices;
@ -1491,33 +1492,35 @@ MergeTreeIndices MergeTreeDataMergerMutator::getIndicesForNewDataPart(
removed_indices.insert(command.column_name);
MergeTreeIndices new_indices;
for (const auto & index : all_indices)
if (!removed_indices.count(index->name))
new_indices.push_back(index);
for (const auto & index : all_indices.indices)
if (!removed_indices.count(index.name))
new_indices.push_back(MergeTreeIndexFactory::instance().get(index));
return new_indices;
}
std::set<MergeTreeIndexPtr> MergeTreeDataMergerMutator::getIndicesToRecalculate(
BlockInputStreamPtr & input_stream,
StoragePtr storage_from_source_part,
const NamesAndTypesList & updated_columns,
const Context & context) const
{
/// Checks if columns used in skipping indexes modified.
const auto & index_factory = MergeTreeIndexFactory::instance();
std::set<MergeTreeIndexPtr> indices_to_recalc;
ASTPtr indices_recalc_expr_list = std::make_shared<ASTExpressionList>();
for (const auto & col : updated_columns.getNames())
{
for (size_t i = 0; i < data.skip_indices.size(); ++i)
const auto & indices = data.getIndices().indices;
for (size_t i = 0; i < indices.size(); ++i)
{
const auto & index = data.skip_indices[i];
const auto & index_cols = index->getColumnsRequiredForIndexCalc();
const auto & index = indices[i];
const auto & index_cols = index.expression->getRequiredColumns();
auto it = std::find(std::cbegin(index_cols), std::cend(index_cols), col);
if (it != std::cend(index_cols) && indices_to_recalc.insert(index).second)
if (it != std::cend(index_cols)
&& indices_to_recalc.insert(index_factory.get(index)).second)
{
ASTPtr expr_list = MergeTreeData::extractKeyExpressionList(
storage_from_source_part->getIndices().indices[i]->expr->clone());
ASTPtr expr_list = index.expression_list_ast->clone();
for (const auto & expr : expr_list->children)
indices_recalc_expr_list->children.push_back(expr->clone());
}
@ -1571,7 +1574,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.hasSkipIndices())
if (data.hasPrimaryKey() || data.hasIndices())
mutating_stream = std::make_shared<MaterializingBlockInputStream>(
std::make_shared<ExpressionBlockInputStream>(mutating_stream, data.primary_key_and_skip_indices_expr));

View File

@ -162,7 +162,7 @@ private:
/// Get skip indices, that should exists in the resulting data part.
static MergeTreeIndices getIndicesForNewDataPart(
const MergeTreeIndices & all_indices,
const IndicesDescription & all_indices,
const MutationCommands & commands_for_removes);
bool shouldExecuteTTL(const Names & columns, const MutationCommands & commands) const;
@ -171,7 +171,6 @@ private:
/// wraps input stream into additional expression stream
std::set<MergeTreeIndexPtr> getIndicesToRecalculate(
BlockInputStreamPtr & input_stream,
StoragePtr storage_from_source_part,
const NamesAndTypesList & updated_columns,
const Context & context) const;

View File

@ -548,11 +548,13 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
RangesInDataParts parts_with_ranges;
std::vector<std::pair<MergeTreeIndexPtr, MergeTreeIndexConditionPtr>> useful_indices;
for (const auto & index : data.skip_indices)
for (const auto & index : data.getIndices().indices)
{
auto condition = index->createIndexCondition(query_info, context);
auto index_helper = MergeTreeIndexFactory::instance().get(index);
auto condition = index_helper->createIndexCondition(query_info, context);
if (!condition->alwaysUnknownOrTrue())
useful_indices.emplace_back(index, condition);
useful_indices.emplace_back(index_helper, condition);
}
/// Let's find what range to read from each part.
@ -1385,10 +1387,12 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
{
if (!part->volume->getDisk()->exists(part->getFullRelativePath() + index->getFileName() + ".idx"))
{
LOG_DEBUG(log, "File for index {} does not exist. Skipping it.", backQuote(index->name));
LOG_DEBUG(log, "File for index {} does not exist. Skipping it.", backQuote(index->index.name));
return ranges;
}
auto index_granularity = index->index.granularity;
const size_t min_marks_for_seek = roundRowsOrBytesToMarks(
settings.merge_tree_min_rows_for_seek,
settings.merge_tree_min_bytes_for_seek,
@ -1399,7 +1403,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
size_t marks_count = part->getMarksCount();
size_t final_mark = part->index_granularity.hasFinalMark();
size_t index_marks_count = (marks_count - final_mark + index->granularity - 1) / index->granularity;
size_t index_marks_count = (marks_count - final_mark + index_granularity - 1) / index_granularity;
MergeTreeIndexReader reader(
index, part,
@ -1415,8 +1419,8 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
for (const auto & range : ranges)
{
MarkRange index_range(
range.begin / index->granularity,
(range.end + index->granularity - 1) / index->granularity);
range.begin / index_granularity,
(range.end + index_granularity - 1) / index_granularity);
if (last_index_mark != index_range.begin || !granule)
reader.seek(index_range.begin);
@ -1427,8 +1431,8 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
granule = reader.read();
MarkRange data_range(
std::max(range.begin, index_mark * index->granularity),
std::min(range.end, (index_mark + 1) * index->granularity));
std::max(range.begin, index_mark * index_granularity),
std::min(range.end, (index_mark + 1) * index_granularity));
if (!condition->mayBeTrueOnGranule(granule))
{
@ -1445,7 +1449,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex(
last_index_mark = index_range.end - 1;
}
LOG_DEBUG(log, "Index {} has dropped {} granules.", backQuote(index->name), granules_dropped);
LOG_DEBUG(log, "Index {} has dropped {} granules.", backQuote(index->index.name), granules_dropped);
return res;
}

View File

@ -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.hasSkipIndices())
if (data.hasSortingKey() || data.hasIndices())
data.sorting_key_and_skip_indices_expr->execute(block);
Names sort_columns = data.getSortingKeyColumns();
@ -302,7 +302,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
/// either default lz4 or compression method with zero thresholds on absolute and relative part size.
auto compression_codec = data.global_context.chooseCompressionCodec(0, 0);
MergedBlockOutputStream out(new_data_part, columns, data.skip_indices, compression_codec);
const auto & index_factory = MergeTreeIndexFactory::instance();
MergedBlockOutputStream out(new_data_part, columns, index_factory.getMany(data.getIndices().indices), compression_codec);
out.writePrefix();
out.writeWithPermutation(block, perm_ptr);

View File

@ -31,7 +31,7 @@ MergeTreeIndexBloomFilter::MergeTreeIndexBloomFilter(
const StorageMetadataSkipIndexField & index_,
size_t bits_per_row_,
size_t hash_functions_)
: IMergeTreeIndex(index)
: IMergeTreeIndex(index_)
, bits_per_row(bits_per_row_)
, hash_functions(hash_functions_)
{
@ -89,40 +89,41 @@ static void assertIndexColumnsType(const Block & header)
}
}
std::unique_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(
const StorageMetadataSkipIndexField & index, bool attach)
std::shared_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(
const StorageMetadataSkipIndexField & index)
{
assertIndexColumnsType(index.sample_block);
double max_conflict_probability = 0.025;
if (index.arguments.size() > 1)
{
if (!attach) /// This is for backward compatibility.
throw Exception("BloomFilter index cannot have more than one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
if (!index.arguments.empty())
{
auto * argument = arguments->children[0]->as<ASTLiteral>();
if (!argument || (argument->value.safeGet<Float64>() < 0 || argument->value.safeGet<Float64>() > 1))
{
if (!attach || !argument) /// This is for backward compatibility.
throw Exception("The BloomFilter false positive must be a double number between 0 and 1.", ErrorCodes::BAD_ARGUMENTS);
argument->value = Field(std::min(Float64(1), std::max(argument->value.safeGet<Float64>(), Float64(0))));
}
max_conflict_probability = argument->value.safeGet<Float64>();
const auto & argument = index.arguments[0];
max_conflict_probability = std::min(Float64(1), std::max(argument.safeGet<Float64>(), Float64(0)));
}
const auto & bits_per_row_and_size_of_hash_functions = BloomFilterHash::calculationBestPractices(max_conflict_probability);
return std::make_unique<MergeTreeIndexBloomFilter>(
node->name, std::move(index_expr), index_sample.getNames(), index_sample.getDataTypes(), index_sample, node->granularity,
bits_per_row_and_size_of_hash_functions.first, bits_per_row_and_size_of_hash_functions.second);
return std::make_shared<MergeTreeIndexBloomFilter>(
index, bits_per_row_and_size_of_hash_functions.first, bits_per_row_and_size_of_hash_functions.second);
}
void bloomFilterIndexValidatorNew(const StorageMetadataSkipIndexField & index, bool attach)
{
assertIndexColumnsType(index.sample_block);
if (index.arguments.size() > 1)
{
if (!attach) /// This is for backward compatibility.
throw Exception("BloomFilter index cannot have more than one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
if (!index.arguments.empty())
{
const auto & argument = index.arguments[0];
if (!attach && (argument.getType() != Field::Types::Float64 || argument.get<Float64>() < 0 || argument.get<Float64>() > 1))
throw Exception("The BloomFilter false positive must be a double number between 0 and 1.", ErrorCodes::BAD_ARGUMENTS);
}
}
}

View File

@ -42,7 +42,7 @@ namespace ErrorCodes
/// Adds all tokens from string to bloom filter.
static void stringToBloomFilter(
const char * data, size_t size, const std::unique_ptr<ITokenExtractor> & token_extractor, BloomFilter & bloom_filter)
const char * data, size_t size, TokenExtractorPtr token_extractor, BloomFilter & bloom_filter)
{
size_t cur = 0;
size_t token_start = 0;
@ -53,7 +53,7 @@ static void stringToBloomFilter(
/// Adds all tokens from like pattern string to bloom filter. (Because like pattern can contain `\%` and `\_`.)
static void likeStringToBloomFilter(
const String & data, const std::unique_ptr<ITokenExtractor> & token_extractor, BloomFilter & bloom_filter)
const String & data, TokenExtractorPtr token_extractor, BloomFilter & bloom_filter)
{
size_t cur = 0;
String token;
@ -61,48 +61,67 @@ static void likeStringToBloomFilter(
bloom_filter.add(token.c_str(), token.size());
}
/// Unified condition for equals, startsWith and endsWith
bool MergeTreeConditionFullText::createFunctionEqualsCondition(RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx)
bool MergeTreeConditionFullText::createFunctionEqualsCondition(RPNElement & out, const Field & value, const BloomFilterParameters & params, TokenExtractorPtr token_extractor)
{
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(
idx.bloom_filter_size, idx.bloom_filter_hashes, idx.seed);
out.bloom_filter = std::make_unique<BloomFilter>(params);
const auto & str = value.get<String>();
stringToBloomFilter(str.c_str(), str.size(), idx.token_extractor_func, *out.bloom_filter);
stringToBloomFilter(str.c_str(), str.size(), token_extractor, *out.bloom_filter);
return true;
}
MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText(const MergeTreeIndexFullText & index_)
: index(index_)
MergeTreeIndexGranuleFullText::MergeTreeIndexGranuleFullText(
const String & index_name_,
size_t columns_number,
const BloomFilterParameters & params_)
: index_name(index_name_)
, params(params_)
, bloom_filters(
index.columns.size(), BloomFilter(index.bloom_filter_size, index.bloom_filter_hashes, index.seed))
, has_elems(false) {}
columns_number, BloomFilter(params))
, has_elems(false)
{
}
void MergeTreeIndexGranuleFullText::serializeBinary(WriteBuffer & ostr) const
{
if (empty())
throw Exception("Attempt to write empty minmax index " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
throw Exception("Attempt to write empty fulltext index " + backQuote(index_name), ErrorCodes::LOGICAL_ERROR);
for (const auto & bloom_filter : bloom_filters)
ostr.write(reinterpret_cast<const char *>(bloom_filter.getFilter().data()), index.bloom_filter_size);
ostr.write(reinterpret_cast<const char *>(bloom_filter.getFilter().data()), params.filter_size);
}
void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr)
{
for (auto & bloom_filter : bloom_filters)
{
istr.read(reinterpret_cast<char *>(bloom_filter.getFilter().data()), index.bloom_filter_size);
istr.read(reinterpret_cast<char *>(
bloom_filter.getFilter().data()), params.filter_size);
}
has_elems = true;
}
MergeTreeIndexAggregatorFullText::MergeTreeIndexAggregatorFullText(const MergeTreeIndexFullText & index_)
: index(index_), granule(std::make_shared<MergeTreeIndexGranuleFullText>(index)) {}
MergeTreeIndexAggregatorFullText::MergeTreeIndexAggregatorFullText(
const Names & index_columns_,
const String & index_name_,
const BloomFilterParameters & params_,
TokenExtractorPtr token_extractor_)
: index_columns(index_columns_)
, index_name (index_name_)
, params(params_)
, token_extractor(token_extractor_)
, granule(
std::make_shared<MergeTreeIndexGranuleFullText>(
index_name, index_columns.size(), params))
{
}
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorFullText::getGranuleAndReset()
{
auto new_granule = std::make_shared<MergeTreeIndexGranuleFullText>(index);
auto new_granule = std::make_shared<MergeTreeIndexGranuleFullText>(
index_name, index_columns.size(), params);
new_granule.swap(granule);
return new_granule;
}
@ -116,13 +135,13 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos,
size_t rows_read = std::min(limit, block.rows() - *pos);
for (size_t col = 0; col < index.columns.size(); ++col)
for (size_t col = 0; col < index_columns.size(); ++col)
{
const auto & column = block.getByName(index.columns[col]).column;
const auto & column = block.getByName(index_columns[col]).column;
for (size_t i = 0; i < rows_read; ++i)
{
auto ref = column->getDataAt(*pos + i);
stringToBloomFilter(ref.data, ref.size, index.token_extractor_func, granule->bloom_filters[col]);
stringToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]);
}
}
granule->has_elems = true;
@ -133,7 +152,14 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos,
MergeTreeConditionFullText::MergeTreeConditionFullText(
const SelectQueryInfo & query_info,
const Context & context,
const MergeTreeIndexFullText & index_) : index(index_), prepared_sets(query_info.sets)
const Block & index_sample_block,
const BloomFilterParameters & params_,
TokenExtractorPtr token_extactor_)
: index_columns(index_sample_block.getNames())
, index_data_types(index_sample_block.getNamesAndTypesList().getTypes())
, params(params_)
, token_extractor(token_extactor_)
, prepared_sets(query_info.sets)
{
rpn = std::move(
RPNBuilder<RPNElement>(
@ -283,11 +309,11 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx
bool MergeTreeConditionFullText::getKey(const ASTPtr & node, size_t & key_column_num)
{
auto it = std::find(index.columns.begin(), index.columns.end(), node->getColumnName());
if (it == index.columns.end())
auto it = std::find(index_columns.begin(), index_columns.end(), node->getColumnName());
if (it == index_columns.end())
return false;
key_column_num = static_cast<size_t>(it - index.columns.begin());
key_column_num = static_cast<size_t>(it - index_columns.begin());
return true;
}
@ -331,67 +357,63 @@ bool MergeTreeConditionFullText::atomFromAST(
if (key_arg_pos == 1 && (func_name != "equals" || func_name != "notEquals"))
return false;
else if (!index.token_extractor_func->supportLike() && (func_name == "like" || func_name == "notLike"))
else if (!token_extractor->supportLike() && (func_name == "like" || func_name == "notLike"))
return false;
if (func_name == "notEquals")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_NOT_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(
index.bloom_filter_size, index.bloom_filter_hashes, index.seed);
out.bloom_filter = std::make_unique<BloomFilter>(params);
const auto & str = const_value.get<String>();
stringToBloomFilter(str.c_str(), str.size(), index.token_extractor_func, *out.bloom_filter);
stringToBloomFilter(str.c_str(), str.size(), token_extractor, *out.bloom_filter);
return true;
}
else if (func_name == "equals")
{
out.key_column = key_column_num;
return createFunctionEqualsCondition(out, const_value, index);
return createFunctionEqualsCondition(out, const_value, params, token_extractor);
}
else if (func_name == "like")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(
index.bloom_filter_size, index.bloom_filter_hashes, index.seed);
out.bloom_filter = std::make_unique<BloomFilter>(params);
const auto & str = const_value.get<String>();
likeStringToBloomFilter(str, index.token_extractor_func, *out.bloom_filter);
likeStringToBloomFilter(str, token_extractor, *out.bloom_filter);
return true;
}
else if (func_name == "notLike")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_NOT_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(
index.bloom_filter_size, index.bloom_filter_hashes, index.seed);
out.bloom_filter = std::make_unique<BloomFilter>(params);
const auto & str = const_value.get<String>();
likeStringToBloomFilter(str, index.token_extractor_func, *out.bloom_filter);
likeStringToBloomFilter(str, token_extractor, *out.bloom_filter);
return true;
}
else if (func_name == "hasToken")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(
index.bloom_filter_size, index.bloom_filter_hashes, index.seed);
out.bloom_filter = std::make_unique<BloomFilter>(params);
const auto & str = const_value.get<String>();
stringToBloomFilter(str.c_str(), str.size(), index.token_extractor_func, *out.bloom_filter);
stringToBloomFilter(str.c_str(), str.size(), token_extractor, *out.bloom_filter);
return true;
}
else if (func_name == "startsWith")
{
out.key_column = key_column_num;
return createFunctionEqualsCondition(out, const_value, index);
return createFunctionEqualsCondition(out, const_value, params, token_extractor);
}
else if (func_name == "endsWith")
{
out.key_column = key_column_num;
return createFunctionEqualsCondition(out, const_value, index);
return createFunctionEqualsCondition(out, const_value, params, token_extractor);
}
else if (func_name == "multiSearchAny")
{
@ -406,9 +428,9 @@ bool MergeTreeConditionFullText::atomFromAST(
if (element.getType() != Field::Types::String)
return false;
bloom_filters.back().emplace_back(index.bloom_filter_size, index.bloom_filter_hashes, index.seed);
bloom_filters.back().emplace_back(params);
const auto & str = element.get<String>();
stringToBloomFilter(str.c_str(), str.size(), index.token_extractor_func, bloom_filters.back().back());
stringToBloomFilter(str.c_str(), str.size(), token_extractor, bloom_filters.back().back());
}
out.set_bloom_filters = std::move(bloom_filters);
return true;
@ -467,7 +489,7 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
if (getKey(tuple_elements[i], key))
{
key_tuple_mapping.emplace_back(i, key);
data_types.push_back(index.data_types[key]);
data_types.push_back(index_data_types[key]);
}
}
}
@ -477,7 +499,7 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
if (getKey(left_arg, key))
{
key_tuple_mapping.emplace_back(0, key);
data_types.push_back(index.data_types[key]);
data_types.push_back(index_data_types[key]);
}
}
@ -515,9 +537,9 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
const auto & column = columns[tuple_idx];
for (size_t row = 0; row < prepared_set->getTotalRowCount(); ++row)
{
bloom_filters.back().emplace_back(index.bloom_filter_size, index.bloom_filter_hashes, index.seed);
bloom_filters.back().emplace_back(params);
auto ref = column->getDataAt(row);
stringToBloomFilter(ref.data, ref.size, index.token_extractor_func, bloom_filters.back().back());
stringToBloomFilter(ref.data, ref.size, token_extractor, bloom_filters.back().back());
}
}
@ -529,23 +551,23 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
MergeTreeIndexGranulePtr MergeTreeIndexFullText::createIndexGranule() const
{
return std::make_shared<MergeTreeIndexGranuleFullText>(*this);
return std::make_shared<MergeTreeIndexGranuleFullText>(index.name, index.column_names.size(), params);
}
MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator() const
{
return std::make_shared<MergeTreeIndexAggregatorFullText>(*this);
return std::make_shared<MergeTreeIndexAggregatorFullText>(index.column_names, index.name, params, token_extractor.get());
}
MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition(
const SelectQueryInfo & query, const Context & context) const
{
return std::make_shared<MergeTreeConditionFullText>(query, context, *this);
return std::make_shared<MergeTreeConditionFullText>(query, context, index.sample_block, params, token_extractor.get());
};
bool MergeTreeIndexFullText::mayBenefitFromIndexForIn(const ASTPtr & node) const
{
return std::find(std::cbegin(columns), std::cend(columns), node->getColumnName()) != std::cend(columns);
return std::find(std::cbegin(index.column_names), std::cend(index.column_names), node->getColumnName()) != std::cend(index.column_names);
}
@ -746,80 +768,63 @@ bool SplitTokenExtractor::nextLike(const String & str, size_t * pos, String & to
}
std::unique_ptr<IMergeTreeIndex> bloomFilterIndexCreator(
const NamesAndTypesList & new_columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context,
bool /*attach*/)
std::shared_ptr<IMergeTreeIndex> bloomFilterIndexCreator(
const StorageMetadataSkipIndexField & index)
{
if (node->name.empty())
throw Exception("Index must have unique name", ErrorCodes::INCORRECT_QUERY);
ASTPtr expr_list = MergeTreeData::extractKeyExpressionList(node->expr->clone());
auto syntax = SyntaxAnalyzer(context).analyze(expr_list, new_columns);
auto index_expr = ExpressionAnalyzer(expr_list, syntax, context).getActions(false);
auto sample = ExpressionAnalyzer(expr_list, syntax, context)
.getActions(true)->getSampleBlock();
Names columns;
DataTypes data_types;
for (size_t i = 0; i < expr_list->children.size(); ++i)
if (index.type == NgramTokenExtractor::getName())
{
const auto & column = sample.getByPosition(i);
columns.emplace_back(column.name);
data_types.emplace_back(column.type);
if (data_types.back()->getTypeId() != TypeIndex::String
&& data_types.back()->getTypeId() != TypeIndex::FixedString)
throw Exception("Bloom filter index can be used only with `String` or `FixedString` column.", ErrorCodes::INCORRECT_QUERY);
}
boost::algorithm::to_lower(node->type->name);
if (node->type->name == NgramTokenExtractor::getName())
{
if (!node->type->arguments || node->type->arguments->children.size() != 4)
throw Exception("`ngrambf` index must have exactly 4 arguments.", ErrorCodes::INCORRECT_QUERY);
size_t n = typeid_cast<const ASTLiteral &>(
*node->type->arguments->children[0]).value.get<size_t>();
size_t bloom_filter_size = typeid_cast<const ASTLiteral &>(
*node->type->arguments->children[1]).value.get<size_t>();
size_t bloom_filter_hashes = typeid_cast<const ASTLiteral &>(
*node->type->arguments->children[2]).value.get<size_t>();
size_t seed = typeid_cast<const ASTLiteral &>(
*node->type->arguments->children[3]).value.get<size_t>();
size_t n = index.arguments[0].get<size_t>();
BloomFilterParameters params
{
.filter_size = index.arguments[1].get<size_t>(),
.filter_hashes = index.arguments[2].get<size_t>(),
.seed = index.arguments[3].get<size_t>(),
};
auto tokenizer = std::make_unique<NgramTokenExtractor>(n);
return std::make_unique<MergeTreeIndexFullText>(
node->name, std::move(index_expr), columns, data_types, sample, node->granularity,
bloom_filter_size, bloom_filter_hashes, seed, std::move(tokenizer));
return std::make_shared<MergeTreeIndexFullText>(index, params, std::move(tokenizer));
}
else if (node->type->name == SplitTokenExtractor::getName())
else if (index.type == SplitTokenExtractor::getName())
{
if (!node->type->arguments || node->type->arguments->children.size() != 3)
throw Exception("`tokenbf` index must have exactly 3 arguments.", ErrorCodes::INCORRECT_QUERY);
size_t bloom_filter_size = typeid_cast<const ASTLiteral &>(
*node->type->arguments->children[0]).value.get<size_t>();
size_t bloom_filter_hashes = typeid_cast<const ASTLiteral &>(
*node->type->arguments->children[1]).value.get<size_t>();
size_t seed = typeid_cast<const ASTLiteral &>(
*node->type->arguments->children[2]).value.get<size_t>();
BloomFilterParameters params
{
.filter_size = index.arguments[0].get<size_t>(),
.filter_hashes = index.arguments[1].get<size_t>(),
.seed = index.arguments[2].get<size_t>(),
};
auto tokenizer = std::make_unique<SplitTokenExtractor>();
return std::make_unique<MergeTreeIndexFullText>(
node->name, std::move(index_expr), columns, data_types, sample, node->granularity,
bloom_filter_size, bloom_filter_hashes, seed, std::move(tokenizer));
return std::make_shared<MergeTreeIndexFullText>(index, params, std::move(tokenizer));
}
else
{
throw Exception("Unknown index type: " + backQuote(node->name), ErrorCodes::LOGICAL_ERROR);
throw Exception("Unknown index type: " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
}
}
void bloomFilterIndexValidator(const StorageMetadataSkipIndexField & index, bool /*attach*/)
{
for (const auto & data_type : index.data_types)
{
if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString)
throw Exception("Bloom filter index can be used only with `String` or `FixedString` column.", ErrorCodes::INCORRECT_QUERY);
}
if (index.type == NgramTokenExtractor::getName())
{
if (index.arguments.size() != 4)
throw Exception("`ngrambf` index must have exactly 4 arguments.", ErrorCodes::INCORRECT_QUERY);
}
else if (index.type == SplitTokenExtractor::getName())
{
if (index.arguments.size() != 3)
throw Exception("`tokenbf` index must have exactly 3 arguments.", ErrorCodes::INCORRECT_QUERY);
}
else
{
throw Exception("Unknown index type: " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -10,13 +10,33 @@
namespace DB
{
/// Interface for string parsers.
struct ITokenExtractor
{
virtual ~ITokenExtractor() = default;
/// Fast inplace implementation for regular use.
/// Gets string (data ptr and len) and start position for extracting next token (state of extractor).
/// Returns false if parsing is finished, otherwise returns true.
virtual bool next(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const = 0;
/// Special implementation for creating bloom filter for LIKE function.
/// It skips unescaped `%` and `_` and supports escaping symbols, but it is less lightweight.
virtual bool nextLike(const String & str, size_t * pos, String & out) const = 0;
virtual bool supportLike() const = 0;
};
using TokenExtractorPtr = const ITokenExtractor *;
class MergeTreeIndexFullText;
struct MergeTreeIndexGranuleFullText : public IMergeTreeIndexGranule
{
explicit MergeTreeIndexGranuleFullText(
const MergeTreeIndexFullText & index_);
const String & index_name_,
size_t columns_number,
const BloomFilterParameters & params_);
~MergeTreeIndexGranuleFullText() override = default;
@ -25,7 +45,9 @@ struct MergeTreeIndexGranuleFullText : public IMergeTreeIndexGranule
bool empty() const override { return !has_elems; }
const MergeTreeIndexFullText & index;
String index_name;
BloomFilterParameters params;
std::vector<BloomFilter> bloom_filters;
bool has_elems;
};
@ -35,7 +57,11 @@ using MergeTreeIndexGranuleFullTextPtr = std::shared_ptr<MergeTreeIndexGranuleFu
struct MergeTreeIndexAggregatorFullText : IMergeTreeIndexAggregator
{
explicit MergeTreeIndexAggregatorFullText(const MergeTreeIndexFullText & index);
explicit MergeTreeIndexAggregatorFullText(
const Names & index_columns_,
const String & index_name_,
const BloomFilterParameters & params_,
TokenExtractorPtr token_extractor_);
~MergeTreeIndexAggregatorFullText() override = default;
@ -44,7 +70,11 @@ struct MergeTreeIndexAggregatorFullText : IMergeTreeIndexAggregator
void update(const Block & block, size_t * pos, size_t limit) override;
const MergeTreeIndexFullText & index;
Names index_columns;
String index_name;
BloomFilterParameters params;
TokenExtractorPtr token_extractor;
MergeTreeIndexGranuleFullTextPtr granule;
};
@ -55,7 +85,9 @@ public:
MergeTreeConditionFullText(
const SelectQueryInfo & query_info,
const Context & context,
const MergeTreeIndexFullText & index_);
const Block & index_sample_block,
const BloomFilterParameters & params_,
TokenExtractorPtr token_extactor_);
~MergeTreeConditionFullText() override = default;
@ -115,29 +147,19 @@ private:
bool getKey(const ASTPtr & node, size_t & key_column_num);
bool tryPrepareSetBloomFilter(const ASTs & args, RPNElement & out);
static bool createFunctionEqualsCondition(RPNElement & out, const Field & value, const MergeTreeIndexFullText & idx);
static bool createFunctionEqualsCondition(
RPNElement & out, const Field & value, const BloomFilterParameters & params, TokenExtractorPtr token_extractor);
const MergeTreeIndexFullText & index;
Names index_columns;
DataTypes index_data_types;
BloomFilterParameters params;
TokenExtractorPtr token_extractor;
RPN rpn;
/// Sets from syntax analyzer.
PreparedSets prepared_sets;
};
/// Interface for string parsers.
struct ITokenExtractor
{
virtual ~ITokenExtractor() = default;
/// Fast inplace implementation for regular use.
/// Gets string (data ptr and len) and start position for extracting next token (state of extractor).
/// Returns false if parsing is finished, otherwise returns true.
virtual bool next(const char * data, size_t len, size_t * pos, size_t * token_start, size_t * token_len) const = 0;
/// Special implementation for creating bloom filter for LIKE function.
/// It skips unescaped `%` and `_` and supports escaping symbols, but it is less lightweight.
virtual bool nextLike(const String & str, size_t * pos, String & out) const = 0;
virtual bool supportLike() const = 0;
};
/// Parser extracting all ngrams from string.
struct NgramTokenExtractor : public ITokenExtractor
@ -170,21 +192,12 @@ class MergeTreeIndexFullText : public IMergeTreeIndex
{
public:
MergeTreeIndexFullText(
String name_,
ExpressionActionsPtr expr_,
const Names & columns_,
const DataTypes & data_types_,
const Block & header_,
size_t granularity_,
size_t bloom_filter_size_,
size_t bloom_filter_hashes_,
size_t seed_,
std::unique_ptr<ITokenExtractor> && token_extractor_func_)
: IMergeTreeIndex(name_, expr_, columns_, data_types_, header_, granularity_)
, bloom_filter_size(bloom_filter_size_)
, bloom_filter_hashes(bloom_filter_hashes_)
, seed(seed_)
, token_extractor_func(std::move(token_extractor_func_)) {}
const StorageMetadataSkipIndexField & index_,
const BloomFilterParameters & params_,
std::unique_ptr<ITokenExtractor> && token_extractor_)
: IMergeTreeIndex(index_)
, params(params_)
, token_extractor(std::move(token_extractor_)) {}
~MergeTreeIndexFullText() override = default;
@ -196,14 +209,9 @@ public:
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;
/// Bloom filter size in bytes.
size_t bloom_filter_size;
/// Number of bloom filter hash functions.
size_t bloom_filter_hashes;
/// Bloom filter seed.
size_t seed;
BloomFilterParameters params;
/// Function for selecting next token.
std::unique_ptr<ITokenExtractor> token_extractor_func;
std::unique_ptr<ITokenExtractor> token_extractor;
};
}

View File

@ -16,20 +16,24 @@ namespace ErrorCodes
}
MergeTreeIndexGranuleMinMax::MergeTreeIndexGranuleMinMax(const Block & index_sample_block_)
: index_sample_block(index_sample_block_)
MergeTreeIndexGranuleMinMax::MergeTreeIndexGranuleMinMax(const String & index_name_, const Block & index_sample_block_)
: index_name(index_name_)
, index_sample_block(index_sample_block_)
{}
MergeTreeIndexGranuleMinMax::MergeTreeIndexGranuleMinMax(
const Block & index_sample_block_, std::vector<Range> && hyperrectangle_)
: index_sample_block(index_sample_block_),
hyperrectangle(std::move(hyperrectangle_)) {}
const String & index_name_,
const Block & index_sample_block_,
std::vector<Range> && hyperrectangle_)
: index_name(index_name_)
, index_sample_block(index_sample_block_)
, hyperrectangle(std::move(hyperrectangle_)) {}
void MergeTreeIndexGranuleMinMax::serializeBinary(WriteBuffer & ostr) const
{
if (empty())
throw Exception(
"Attempt to write empty minmax index " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
"Attempt to write empty minmax index " + backQuote(index_name), ErrorCodes::LOGICAL_ERROR);
for (size_t i = 0; i < index_sample_block.columns(); ++i)
{
@ -84,13 +88,14 @@ void MergeTreeIndexGranuleMinMax::deserializeBinary(ReadBuffer & istr)
}
}
MergeTreeIndexAggregatorMinMax::MergeTreeIndexAggregatorMinMax(const Block & index_sample_block_)
: index_sample_block(index_sample_block_)
MergeTreeIndexAggregatorMinMax::MergeTreeIndexAggregatorMinMax(const String & index_name_, const Block & index_sample_block_)
: index_name(index_name_)
, index_sample_block(index_sample_block_)
{}
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorMinMax::getGranuleAndReset()
{
return std::make_shared<MergeTreeIndexGranuleMinMax>(index_sample_block, std::move(hyperrectangle));
return std::make_shared<MergeTreeIndexGranuleMinMax>(index_name, index_sample_block, std::move(hyperrectangle));
}
void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, size_t limit)
@ -149,19 +154,19 @@ bool MergeTreeIndexConditionMinMax::mayBeTrueOnGranule(MergeTreeIndexGranulePtr
for (const auto & range : granule->hyperrectangle)
if (range.left.isNull() || range.right.isNull())
return true;
return condition.checkInHyperrectangle(granule->hyperrectangle, index.data_types).can_be_true;
return condition.checkInHyperrectangle(granule->hyperrectangle, index_data_types).can_be_true;
}
MergeTreeIndexGranulePtr MergeTreeIndexMinMax::createIndexGranule() const
{
return std::make_shared<MergeTreeIndexGranuleMinMax>(index.sample_block);
return std::make_shared<MergeTreeIndexGranuleMinMax>(index.name, index.sample_block);
}
MergeTreeIndexAggregatorPtr MergeTreeIndexMinMax::createIndexAggregator() const
{
return std::make_shared<MergeTreeIndexAggregatorMinMax>(index.sample_block);
return std::make_shared<MergeTreeIndexAggregatorMinMax>(index.name, index.sample_block);
}
MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition(
@ -185,11 +190,13 @@ bool MergeTreeIndexMinMax::mayBenefitFromIndexForIn(const ASTPtr & node) const
return false;
}
std::unique_ptr<IMergeTreeIndex> minmaxIndexCreator(
const StorageMetadataSkipIndexField & index,
bool /*attach*/)
std::shared_ptr<IMergeTreeIndex> minmaxIndexCreator(
const StorageMetadataSkipIndexField & index)
{
return std::make_unique<MergeTreeIndexMinMax>(index);
return std::make_shared<MergeTreeIndexMinMax>(index);
}
void minmaxIndexValidator(const StorageMetadataSkipIndexField & /* index */, bool /* attach */)
{
}
}

View File

@ -12,8 +12,12 @@ namespace DB
struct MergeTreeIndexGranuleMinMax : public IMergeTreeIndexGranule
{
explicit MergeTreeIndexGranuleMinMax(const Block & index_sample_block_);
MergeTreeIndexGranuleMinMax(const Block & index_sample_block_, std::vector<Range> && hyperrectangle_);
MergeTreeIndexGranuleMinMax(const String & index_name_, const Block & index_sample_block_);
MergeTreeIndexGranuleMinMax(
const String & index_name_,
const Block & index_sample_block_,
std::vector<Range> && hyperrectangle_);
~MergeTreeIndexGranuleMinMax() override = default;
void serializeBinary(WriteBuffer & ostr) const override;
@ -21,6 +25,7 @@ struct MergeTreeIndexGranuleMinMax : public IMergeTreeIndexGranule
bool empty() const override { return hyperrectangle.empty(); }
String index_name;
Block index_sample_block;
std::vector<Range> hyperrectangle;
};
@ -28,13 +33,14 @@ struct MergeTreeIndexGranuleMinMax : public IMergeTreeIndexGranule
struct MergeTreeIndexAggregatorMinMax : IMergeTreeIndexAggregator
{
explicit MergeTreeIndexAggregatorMinMax(const Block & index_sample_block);
MergeTreeIndexAggregatorMinMax(const String & index_name_, const Block & index_sample_block);
~MergeTreeIndexAggregatorMinMax() override = default;
bool empty() const override { return hyperrectangle.empty(); }
MergeTreeIndexGranulePtr getGranuleAndReset() override;
void update(const Block & block, size_t * pos, size_t limit) override;
String index_name;
Block index_sample_block;
std::vector<Range> hyperrectangle;
};

View File

@ -477,19 +477,18 @@ bool MergeTreeIndexSet::mayBenefitFromIndexForIn(const ASTPtr &) const
return false;
}
std::unique_ptr<IMergeTreeIndex> setIndexCreator(
const StorageMetadataSkipIndexField & index,
bool /*attach*/)
std::shared_ptr<IMergeTreeIndex> setIndexCreator(const StorageMetadataSkipIndexField & index)
{
size_t max_rows = index.arguments[0].get<size_t>();
return std::make_shared<MergeTreeIndexSet>(index, max_rows);
}
void setIndexValidator(const StorageMetadataSkipIndexField & index, bool /*attach*/)
{
size_t max_rows = 0;
if (index.arguments.size() != 1)
throw Exception("Set index must have exactly one argument.", ErrorCodes::INCORRECT_QUERY);
else if (index.arguments[0].getType() != Field::Types::UInt64)
throw Exception("Set index argument must be positive integer.", ErrorCodes::INCORRECT_QUERY);
else ()
max_rows = index.arguments[0].get<size_t>();
return std::make_unique<MergeTreeIndexSet>(index, max_rows);
}
}

View File

@ -18,21 +18,27 @@ namespace ErrorCodes
extern const int INCORRECT_QUERY;
}
void MergeTreeIndexFactory::registerIndex(const std::string & index_type, Creator creator)
void MergeTreeIndexFactory::registerCreator(const std::string & index_type, Creator creator)
{
if (!indexes.emplace(index_type, std::move(creator)).second)
if (!creators.emplace(index_type, std::move(creator)).second)
throw Exception("MergeTreeIndexFactory: the Index creator name '" + index_type + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
}
std::unique_ptr<IMergeTreeIndex> MergeTreeIndexFactory::get(
const StorageMetadataSkipIndexField & index, bool attach) const
void MergeTreeIndexFactory::registerValidator(const std::string & index_type, Validator validator)
{
auto it = indexes.find(index.type);
if (it == indexes.end())
if (!validators.emplace(index_type, std::move(validator)).second)
throw Exception("MergeTreeIndexFactory: the Index validator name '" + index_type + "' is not unique", ErrorCodes::LOGICAL_ERROR);
}
std::shared_ptr<IMergeTreeIndex> MergeTreeIndexFactory::get(
const StorageMetadataSkipIndexField & index) const
{
auto it = creators.find(index.type);
if (it == creators.end())
throw Exception(
"Unknown Index type '" + index.type + "'. Available index types: " +
std::accumulate(indexes.cbegin(), indexes.cend(), std::string{},
std::accumulate(creators.cbegin(), creators.cend(), std::string{},
[] (auto && left, const auto & right) -> std::string
{
if (left.empty())
@ -42,16 +48,56 @@ std::unique_ptr<IMergeTreeIndex> MergeTreeIndexFactory::get(
}),
ErrorCodes::INCORRECT_QUERY);
return it->second(index, attach);
return it->second(index);
}
MergeTreeIndices MergeTreeIndexFactory::getMany(const std::vector<StorageMetadataSkipIndexField> & indices) const
{
MergeTreeIndices result;
for (const auto & index : indices)
result.emplace_back(get(index));
return result;
}
void MergeTreeIndexFactory::validate(const StorageMetadataSkipIndexField & index, bool attach) const
{
auto it = validators.find(index.type);
if (it == validators.end())
throw Exception(
"Unknown Index type '" + index.type + "'. Available index types: "
+ std::accumulate(
validators.cbegin(),
validators.cend(),
std::string{},
[](auto && left, const auto & right) -> std::string
{
if (left.empty())
return right.first;
else
return left + ", " + right.first;
}),
ErrorCodes::INCORRECT_QUERY);
it->second(index, attach);
}
MergeTreeIndexFactory::MergeTreeIndexFactory()
{
registerIndex("minmax", minmaxIndexCreator);
registerIndex("set", setIndexCreator);
registerIndex("ngrambf_v1", bloomFilterIndexCreator);
registerIndex("tokenbf_v1", bloomFilterIndexCreator);
registerIndex("bloom_filter", bloomFilterIndexCreatorNew);
registerCreator("minmax", minmaxIndexCreator);
registerValidator("minmax", minmaxIndexValidator);
registerCreator("set", setIndexCreator);
registerValidator("set", setIndexValidator);
registerCreator("ngrambf_v1", bloomFilterIndexCreator);
registerValidator("ngrambf_v1", bloomFilterIndexValidator);
registerCreator("tokenbf_v1", bloomFilterIndexCreator);
registerValidator("tokenbf_v1", bloomFilterIndexValidator);
registerCreator("bloom_filter", bloomFilterIndexCreatorNew);
registerValidator("bloom_filter", bloomFilterIndexValidatorNew);
}
MergeTreeIndexFactory & MergeTreeIndexFactory::instance()

View File

@ -110,35 +110,46 @@ public:
static MergeTreeIndexFactory & instance();
using Creator = std::function<
std::unique_ptr<IMergeTreeIndex>(
const StorageMetadataSkipIndexField & index,
bool attach)>;
std::shared_ptr<IMergeTreeIndex>(
const StorageMetadataSkipIndexField & index)>;
std::unique_ptr<IMergeTreeIndex> get(
const StorageMetadataSkipIndexField & index,
bool attach) const;
using Validator = std::function<void(const StorageMetadataSkipIndexField & index, bool attach)>;
void registerIndex(const std::string & index_type, Creator creator);
void validate(const StorageMetadataSkipIndexField & index, bool attach) const;
const auto & getAllIndexes() const { return indexes; }
std::shared_ptr<IMergeTreeIndex> get(const StorageMetadataSkipIndexField & index) const;
MergeTreeIndices getMany(const std::vector<StorageMetadataSkipIndexField> & indices) const;
void registerCreator(const std::string & index_type, Creator creator);
void registerValidator(const std::string & index_type, Validator creator);
protected:
MergeTreeIndexFactory();
private:
using Indexes = std::unordered_map<std::string, Creator>;
Indexes indexes;
using Creators = std::unordered_map<std::string, Creator>;
using Validators = std::unordered_map<std::string, Validator>;
Creators creators;
Validators validators;
};
std::unique_ptr<IMergeTreeIndex> minmaxIndexCreator(
const StorageMetadataSkipIndexField & index, bool attach);
std::shared_ptr<IMergeTreeIndex> minmaxIndexCreator(
const StorageMetadataSkipIndexField & index);
void minmaxIndexValidator(const StorageMetadataSkipIndexField & index, bool attach);
std::unique_ptr<IMergeTreeIndex> setIndexCreator(
const StorageMetadataSkipIndexField & index, bool attach);
std::unique_ptr<IMergeTreeIndex> bloomFilterIndexCreator(
const StorageMetadataSkipIndexField & index, bool attach);
std::shared_ptr<IMergeTreeIndex> setIndexCreator(
const StorageMetadataSkipIndexField & index);
void setIndexValidator(const StorageMetadataSkipIndexField & index, bool attach);
std::unique_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(
const StorageMetadataSkipIndexField & index, bool attach);
std::shared_ptr<IMergeTreeIndex> bloomFilterIndexCreator(
const StorageMetadataSkipIndexField & index);
void bloomFilterIndexValidator(const StorageMetadataSkipIndexField & index, bool attach);
std::shared_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(
const StorageMetadataSkipIndexField & index);
void bloomFilterIndexValidatorNew(const StorageMetadataSkipIndexField & index, bool attach);
}

View File

@ -157,8 +157,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
return;
std::unordered_set<String> skip_indexes_column_names_set;
for (const auto & index : storage.skip_indices)
std::copy(index->columns.cbegin(), index->columns.cend(),
for (const auto & index : storage.getIndices().indices)
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());

View File

@ -39,7 +39,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block)
{
std::unordered_set<String> skip_indexes_column_names_set;
for (const auto & index : writer->getSkipIndices())
std::copy(index->columns.cbegin(), index->columns.cend(),
std::copy(index->index.column_names.cbegin(), index->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());

View File

@ -607,7 +607,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (args.query.columns_list && args.query.columns_list->indices)
for (auto & index : args.query.columns_list->indices->children)
indices_description.indices.push_back(std::dynamic_pointer_cast<ASTIndexDeclaration>(index));
indices_description.indices.push_back(StorageMetadataSkipIndexField::getSkipIndexFromAST(index, args.columns, args.context));
storage_settings->loadFromQuery(*args.storage_def);

View File

@ -777,7 +777,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context,
optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context);
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
params.apply(metadata, context);
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
setColumns(std::move(metadata.columns));
}

View File

@ -561,7 +561,7 @@ void StorageDistributed::alter(const AlterCommands & params, const Context & con
checkAlterIsPossible(params, context.getSettingsRef());
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
params.apply(metadata, context);
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
setColumns(std::move(metadata.columns));
}

View File

@ -242,6 +242,7 @@ StorageMetadataSkipIndexField StorageMetadataSkipIndexField::getSkipIndexFromAST
result.granularity = index_definition->granularity;
ASTPtr expr_list = extractKeyExpressionList(index_definition->expr->clone());
result.expression_list_ast = expr_list->clone();
auto syntax = SyntaxAnalyzer(context).analyze(expr_list, columns.getAllPhysical());
result.expression = ExpressionAnalyzer(expr_list, syntax, context).getActions(true);
@ -269,12 +270,12 @@ StorageMetadataSkipIndexField StorageMetadataSkipIndexField::getSkipIndexFromAST
return result;
}
bool StorageMetadataSkipIndices::empty() const
bool IndicesDescription::empty() const
{
return indices.empty();
}
bool StorageMetadataSkipIndices::has(const String & name) const
bool IndicesDescription::has(const String & name) const
{
for (const auto & index : indices)
if (index.name == name)
@ -282,7 +283,7 @@ bool StorageMetadataSkipIndices::has(const String & name) const
return false;
}
String StorageMetadataSkipIndices::toString() const
String IndicesDescription::toString() const
{
if (indices.empty())
return {};
@ -295,9 +296,9 @@ String StorageMetadataSkipIndices::toString() const
}
StorageMetadataSkipIndices StorageMetadataSkipIndices::parse(const String & str, const ColumnsDescription & columns, const Context & context)
IndicesDescription IndicesDescription::parse(const String & str, const ColumnsDescription & columns, const Context & context)
{
StorageMetadataSkipIndices result;
IndicesDescription result;
if (str.empty())
return result;

View File

@ -9,6 +9,42 @@
namespace DB
{
struct StorageMetadataSkipIndexField
{
ASTPtr definition_ast;
ASTPtr expression_list_ast;
String name;
String type;
ExpressionActionsPtr expression;
FieldVector arguments;
Names column_names;
DataTypes data_types;
Block sample_block;
size_t granularity;
static StorageMetadataSkipIndexField
getSkipIndexFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context);
};
struct IndicesDescription
{
std::vector<StorageMetadataSkipIndexField> indices;
bool empty() const;
bool has(const String & name) const;
String toString() const;
static IndicesDescription parse(const String & str, const ColumnsDescription & columns, const Context & context);
};
/// Structure represent table metadata stored in memory.
/// Only one storage engine support all fields -- MergeTree.
@ -119,38 +155,5 @@ struct StorageMetadataTableTTL
StorageMetadataTTLFields move_ttl;
};
struct StorageMetadataSkipIndexField
{
ASTPtr definition_ast;
String name;
String type;
ExpressionActionsPtr expression;
FieldVector arguments;
Names column_names;
DataTypes data_types;
Block sample_block;
size_t granularity;
static StorageMetadataSkipIndexField getSkipIndexFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context);
};
struct StorageMetadataSkipIndices
{
std::vector<StorageMetadataSkipIndexField> indices;
bool empty() const;
bool has(const String & name) const;
String toString() const;
static StorageMetadataSkipIndices parse(const String & str, const ColumnsDescription & columns, const Context & context);
};
}

View File

@ -257,7 +257,7 @@ void StorageMaterializedView::alter(
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
auto table_id = getStorageID();
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
params.apply(metadata, context);
/// start modify query
if (context.getSettingsRef().allow_experimental_alter_materialized_view_structure)

View File

@ -412,7 +412,7 @@ void StorageMerge::alter(
auto table_id = getStorageID();
StorageInMemoryMetadata storage_metadata = getInMemoryMetadata();
params.apply(storage_metadata);
params.apply(storage_metadata, context);
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, storage_metadata);
setColumns(storage_metadata.columns);
}

View File

@ -230,8 +230,8 @@ void StorageMergeTree::alter(
auto table_id = getStorageID();
StorageInMemoryMetadata metadata = getInMemoryMetadata();
auto maybe_mutation_commands = commands.getMutationCommands(metadata, context.getSettingsRef().materialize_ttl_after_modify);
commands.apply(metadata);
auto maybe_mutation_commands = commands.getMutationCommands(metadata, context.getSettingsRef().materialize_ttl_after_modify, context);
commands.apply(metadata, context);
/// This alter can be performed at metadata level only
if (commands.isSettingsAlter())

View File

@ -52,7 +52,7 @@ void StorageNull::alter(
auto table_id = getStorageID();
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
params.apply(metadata, context);
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
setColumns(std::move(metadata.columns));
}

View File

@ -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);
metadata.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);
@ -3276,7 +3276,7 @@ void StorageReplicatedMergeTree::alter(
/// We don't replicate storage_settings_ptr ALTER. It's local operation.
/// Also we don't upgrade alter lock to table structure lock.
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
params.apply(metadata, query_context);
changeSettings(metadata.settings_ast, table_lock_holder);
@ -3310,7 +3310,7 @@ void StorageReplicatedMergeTree::alter(
StorageInMemoryMetadata current_metadata = getInMemoryMetadata();
StorageInMemoryMetadata future_metadata = current_metadata;
params.apply(future_metadata);
params.apply(future_metadata, query_context);
ReplicatedMergeTreeTableMetadata future_metadata_in_zk(*this);
if (ast_to_str(future_metadata.order_by_ast) != ast_to_str(current_metadata.order_by_ast))
@ -3357,7 +3357,7 @@ void StorageReplicatedMergeTree::alter(
alter_entry->alter_version = new_metadata_version;
alter_entry->create_time = time(nullptr);
auto maybe_mutation_commands = params.getMutationCommands(current_metadata, query_context.getSettingsRef().materialize_ttl_after_modify);
auto maybe_mutation_commands = params.getMutationCommands(current_metadata, query_context.getSettingsRef().materialize_ttl_after_modify, query_context);
alter_entry->have_mutation = !maybe_mutation_commands.empty();
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", alter_entry->toString(), zkutil::CreateMode::PersistentSequential));