mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Better naming
This commit is contained in:
parent
5b5f749592
commit
f5e5290c0a
@ -400,7 +400,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS
|
||||
if (create.columns_list->indices)
|
||||
for (const auto & index : create.columns_list->indices->children)
|
||||
properties.indices.push_back(
|
||||
StorageMetadataSkipIndexField::getSkipIndexFromAST(index->clone(), properties.columns, context));
|
||||
IndexDescription::getSkipIndexFromAST(index->clone(), properties.columns, context));
|
||||
|
||||
properties.constraints = getConstraintsDescription(create.columns_list->constraints);
|
||||
}
|
||||
|
@ -392,7 +392,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
{
|
||||
auto it = std::find_if(
|
||||
std::cbegin(indices_desc), std::end(indices_desc),
|
||||
[&](const StorageMetadataSkipIndexField & index)
|
||||
[&](const IndexDescription & index)
|
||||
{
|
||||
return index.name == command.index_name;
|
||||
});
|
||||
|
@ -365,7 +365,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con
|
||||
++insert_it;
|
||||
}
|
||||
|
||||
metadata.indices.emplace(insert_it, StorageMetadataSkipIndexField::getSkipIndexFromAST(index_decl, metadata.columns, context));
|
||||
metadata.indices.emplace(insert_it, IndexDescription::getSkipIndexFromAST(index_decl, metadata.columns, context));
|
||||
}
|
||||
else if (type == DROP_INDEX)
|
||||
{
|
||||
|
@ -19,7 +19,7 @@ namespace ErrorCodes
|
||||
};
|
||||
|
||||
|
||||
StorageMetadataSkipIndexField StorageMetadataSkipIndexField::getSkipIndexFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context)
|
||||
IndexDescription IndexDescription::getSkipIndexFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context)
|
||||
{
|
||||
const auto * index_definition = definition_ast->as<ASTIndexDeclaration>();
|
||||
if (!index_definition)
|
||||
@ -34,7 +34,7 @@ StorageMetadataSkipIndexField StorageMetadataSkipIndexField::getSkipIndexFromAST
|
||||
if (index_definition->type->parameters && !index_definition->type->parameters->children.empty())
|
||||
throw Exception("Index type cannot have parameters", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
StorageMetadataSkipIndexField result;
|
||||
IndexDescription result;
|
||||
result.definition_ast = index_definition->clone();
|
||||
result.name = index_definition->name;
|
||||
result.type = Poco::toLower(index_definition->type->name);
|
||||
@ -101,7 +101,7 @@ IndicesDescription IndicesDescription::parse(const String & str, const ColumnsDe
|
||||
ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||
|
||||
for (const auto & index : list->children)
|
||||
result.emplace_back(StorageMetadataSkipIndexField::getSkipIndexFromAST(index, columns, context));
|
||||
result.emplace_back(IndexDescription::getSkipIndexFromAST(index, columns, context));
|
||||
|
||||
return result;
|
||||
}
|
||||
|
@ -12,7 +12,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct StorageMetadataSkipIndexField
|
||||
struct IndexDescription
|
||||
{
|
||||
ASTPtr definition_ast;
|
||||
|
||||
@ -34,11 +34,10 @@ struct StorageMetadataSkipIndexField
|
||||
|
||||
size_t granularity;
|
||||
|
||||
static StorageMetadataSkipIndexField
|
||||
getSkipIndexFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context);
|
||||
static IndexDescription getSkipIndexFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context);
|
||||
};
|
||||
|
||||
struct IndicesDescription : public std::vector<StorageMetadataSkipIndexField>
|
||||
struct IndicesDescription : public std::vector<IndexDescription>
|
||||
{
|
||||
bool has(const String & name) const;
|
||||
String toString() const;
|
||||
|
@ -28,7 +28,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
MergeTreeIndexBloomFilter::MergeTreeIndexBloomFilter(
|
||||
const StorageMetadataSkipIndexField & index_,
|
||||
const IndexDescription & index_,
|
||||
size_t bits_per_row_,
|
||||
size_t hash_functions_)
|
||||
: IMergeTreeIndex(index_)
|
||||
@ -90,7 +90,7 @@ static void assertIndexColumnsType(const Block & header)
|
||||
}
|
||||
|
||||
std::shared_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(
|
||||
const StorageMetadataSkipIndexField & index)
|
||||
const IndexDescription & index)
|
||||
{
|
||||
|
||||
double max_conflict_probability = 0.025;
|
||||
@ -107,7 +107,7 @@ std::shared_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(
|
||||
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)
|
||||
void bloomFilterIndexValidatorNew(const IndexDescription & index, bool attach)
|
||||
{
|
||||
assertIndexColumnsType(index.sample_block);
|
||||
|
||||
|
@ -12,7 +12,7 @@ class MergeTreeIndexBloomFilter : public IMergeTreeIndex
|
||||
{
|
||||
public:
|
||||
MergeTreeIndexBloomFilter(
|
||||
const StorageMetadataSkipIndexField & index_,
|
||||
const IndexDescription & index_,
|
||||
size_t bits_per_row_,
|
||||
size_t hash_functions_);
|
||||
|
||||
|
@ -769,7 +769,7 @@ bool SplitTokenExtractor::nextLike(const String & str, size_t * pos, String & to
|
||||
|
||||
|
||||
std::shared_ptr<IMergeTreeIndex> bloomFilterIndexCreator(
|
||||
const StorageMetadataSkipIndexField & index)
|
||||
const IndexDescription & index)
|
||||
{
|
||||
if (index.type == NgramTokenExtractor::getName())
|
||||
{
|
||||
@ -804,7 +804,7 @@ std::shared_ptr<IMergeTreeIndex> bloomFilterIndexCreator(
|
||||
}
|
||||
}
|
||||
|
||||
void bloomFilterIndexValidator(const StorageMetadataSkipIndexField & index, bool /*attach*/)
|
||||
void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/)
|
||||
{
|
||||
for (const auto & data_type : index.data_types)
|
||||
{
|
||||
|
@ -192,7 +192,7 @@ class MergeTreeIndexFullText : public IMergeTreeIndex
|
||||
{
|
||||
public:
|
||||
MergeTreeIndexFullText(
|
||||
const StorageMetadataSkipIndexField & index_,
|
||||
const IndexDescription & index_,
|
||||
const BloomFilterParameters & params_,
|
||||
std::unique_ptr<ITokenExtractor> && token_extractor_)
|
||||
: IMergeTreeIndex(index_)
|
||||
|
@ -131,7 +131,7 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s
|
||||
|
||||
|
||||
MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax(
|
||||
const StorageMetadataSkipIndexField & index,
|
||||
const IndexDescription & index,
|
||||
const SelectQueryInfo & query,
|
||||
const Context & context)
|
||||
: index_data_types(index.data_types)
|
||||
@ -191,12 +191,12 @@ bool MergeTreeIndexMinMax::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
||||
}
|
||||
|
||||
std::shared_ptr<IMergeTreeIndex> minmaxIndexCreator(
|
||||
const StorageMetadataSkipIndexField & index)
|
||||
const IndexDescription & index)
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexMinMax>(index);
|
||||
}
|
||||
|
||||
void minmaxIndexValidator(const StorageMetadataSkipIndexField & /* index */, bool /* attach */)
|
||||
void minmaxIndexValidator(const IndexDescription & /* index */, bool /* attach */)
|
||||
{
|
||||
}
|
||||
}
|
||||
|
@ -50,7 +50,7 @@ class MergeTreeIndexConditionMinMax : public IMergeTreeIndexCondition
|
||||
{
|
||||
public:
|
||||
MergeTreeIndexConditionMinMax(
|
||||
const StorageMetadataSkipIndexField & index,
|
||||
const IndexDescription & index,
|
||||
const SelectQueryInfo & query,
|
||||
const Context & context);
|
||||
|
||||
@ -68,7 +68,7 @@ private:
|
||||
class MergeTreeIndexMinMax : public IMergeTreeIndex
|
||||
{
|
||||
public:
|
||||
MergeTreeIndexMinMax(const StorageMetadataSkipIndexField & index_)
|
||||
MergeTreeIndexMinMax(const IndexDescription & index_)
|
||||
: IMergeTreeIndex(index_)
|
||||
{}
|
||||
|
||||
|
@ -477,13 +477,13 @@ bool MergeTreeIndexSet::mayBenefitFromIndexForIn(const ASTPtr &) const
|
||||
return false;
|
||||
}
|
||||
|
||||
std::shared_ptr<IMergeTreeIndex> setIndexCreator(const StorageMetadataSkipIndexField & index)
|
||||
std::shared_ptr<IMergeTreeIndex> setIndexCreator(const IndexDescription & 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*/)
|
||||
void setIndexValidator(const IndexDescription & index, bool /*attach*/)
|
||||
{
|
||||
if (index.arguments.size() != 1)
|
||||
throw Exception("Set index must have exactly one argument.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
@ -117,7 +117,7 @@ class MergeTreeIndexSet : public IMergeTreeIndex
|
||||
{
|
||||
public:
|
||||
MergeTreeIndexSet(
|
||||
const StorageMetadataSkipIndexField & index_,
|
||||
const IndexDescription & index_,
|
||||
size_t max_rows_)
|
||||
: IMergeTreeIndex(index_)
|
||||
, max_rows(max_rows_)
|
||||
|
@ -32,7 +32,7 @@ void MergeTreeIndexFactory::registerValidator(const std::string & index_type, Va
|
||||
|
||||
|
||||
std::shared_ptr<IMergeTreeIndex> MergeTreeIndexFactory::get(
|
||||
const StorageMetadataSkipIndexField & index) const
|
||||
const IndexDescription & index) const
|
||||
{
|
||||
auto it = creators.find(index.type);
|
||||
if (it == creators.end())
|
||||
@ -52,7 +52,7 @@ std::shared_ptr<IMergeTreeIndex> MergeTreeIndexFactory::get(
|
||||
}
|
||||
|
||||
|
||||
MergeTreeIndices MergeTreeIndexFactory::getMany(const std::vector<StorageMetadataSkipIndexField> & indices) const
|
||||
MergeTreeIndices MergeTreeIndexFactory::getMany(const std::vector<IndexDescription> & indices) const
|
||||
{
|
||||
MergeTreeIndices result;
|
||||
for (const auto & index : indices)
|
||||
@ -60,7 +60,7 @@ MergeTreeIndices MergeTreeIndexFactory::getMany(const std::vector<StorageMetadat
|
||||
return result;
|
||||
}
|
||||
|
||||
void MergeTreeIndexFactory::validate(const StorageMetadataSkipIndexField & index, bool attach) const
|
||||
void MergeTreeIndexFactory::validate(const IndexDescription & index, bool attach) const
|
||||
{
|
||||
auto it = validators.find(index.type);
|
||||
if (it == validators.end())
|
||||
|
@ -76,7 +76,7 @@ using MergeTreeIndexConditionPtr = std::shared_ptr<IMergeTreeIndexCondition>;
|
||||
class IMergeTreeIndex
|
||||
{
|
||||
public:
|
||||
IMergeTreeIndex(const StorageMetadataSkipIndexField & index_)
|
||||
IMergeTreeIndex(const IndexDescription & index_)
|
||||
: index(index_)
|
||||
{
|
||||
}
|
||||
@ -98,7 +98,7 @@ public:
|
||||
|
||||
Names getColumnsRequiredForIndexCalc() const { return index.expression->getRequiredColumns(); }
|
||||
|
||||
const StorageMetadataSkipIndexField & index;
|
||||
const IndexDescription & index;
|
||||
};
|
||||
|
||||
using MergeTreeIndices = std::vector<MergeTreeIndexPtr>;
|
||||
@ -111,15 +111,15 @@ public:
|
||||
|
||||
using Creator = std::function<
|
||||
std::shared_ptr<IMergeTreeIndex>(
|
||||
const StorageMetadataSkipIndexField & index)>;
|
||||
const IndexDescription & index)>;
|
||||
|
||||
using Validator = std::function<void(const StorageMetadataSkipIndexField & index, bool attach)>;
|
||||
using Validator = std::function<void(const IndexDescription & index, bool attach)>;
|
||||
|
||||
void validate(const StorageMetadataSkipIndexField & index, bool attach) const;
|
||||
void validate(const IndexDescription & index, bool attach) const;
|
||||
|
||||
std::shared_ptr<IMergeTreeIndex> get(const StorageMetadataSkipIndexField & index) const;
|
||||
std::shared_ptr<IMergeTreeIndex> get(const IndexDescription & index) const;
|
||||
|
||||
MergeTreeIndices getMany(const std::vector<StorageMetadataSkipIndexField> & indices) const;
|
||||
MergeTreeIndices getMany(const std::vector<IndexDescription> & indices) const;
|
||||
|
||||
void registerCreator(const std::string & index_type, Creator creator);
|
||||
void registerValidator(const std::string & index_type, Validator creator);
|
||||
@ -135,21 +135,21 @@ private:
|
||||
};
|
||||
|
||||
std::shared_ptr<IMergeTreeIndex> minmaxIndexCreator(
|
||||
const StorageMetadataSkipIndexField & index);
|
||||
void minmaxIndexValidator(const StorageMetadataSkipIndexField & index, bool attach);
|
||||
const IndexDescription & index);
|
||||
void minmaxIndexValidator(const IndexDescription & index, bool attach);
|
||||
|
||||
|
||||
std::shared_ptr<IMergeTreeIndex> setIndexCreator(
|
||||
const StorageMetadataSkipIndexField & index);
|
||||
void setIndexValidator(const StorageMetadataSkipIndexField & index, bool attach);
|
||||
const IndexDescription & index);
|
||||
void setIndexValidator(const IndexDescription & index, bool attach);
|
||||
|
||||
std::shared_ptr<IMergeTreeIndex> bloomFilterIndexCreator(
|
||||
const StorageMetadataSkipIndexField & index);
|
||||
const IndexDescription & index);
|
||||
|
||||
void bloomFilterIndexValidator(const StorageMetadataSkipIndexField & index, bool attach);
|
||||
void bloomFilterIndexValidator(const IndexDescription & index, bool attach);
|
||||
|
||||
|
||||
std::shared_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(
|
||||
const StorageMetadataSkipIndexField & index);
|
||||
void bloomFilterIndexValidatorNew(const StorageMetadataSkipIndexField & index, bool attach);
|
||||
const IndexDescription & index);
|
||||
void bloomFilterIndexValidatorNew(const IndexDescription & index, bool attach);
|
||||
}
|
||||
|
@ -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.push_back(StorageMetadataSkipIndexField::getSkipIndexFromAST(index, args.columns, args.context));
|
||||
indices_description.push_back(IndexDescription::getSkipIndexFromAST(index, args.columns, args.context));
|
||||
|
||||
storage_settings->loadFromQuery(*args.storage_def);
|
||||
|
||||
|
27
src/Storages/extractKeyExpressionList.cpp
Normal file
27
src/Storages/extractKeyExpressionList.cpp
Normal file
@ -0,0 +1,27 @@
|
||||
#include <Storages/extractKeyExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
ASTPtr extractKeyExpressionList(const ASTPtr & node)
|
||||
{
|
||||
if (!node)
|
||||
return std::make_shared<ASTExpressionList>();
|
||||
|
||||
const auto * expr_func = node->as<ASTFunction>();
|
||||
|
||||
if (expr_func && expr_func->name == "tuple")
|
||||
{
|
||||
/// Primary key is specified in tuple, extract its arguments.
|
||||
return expr_func->arguments->clone();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Primary key consists of one column.
|
||||
auto res = std::make_shared<ASTExpressionList>();
|
||||
res->children.push_back(node);
|
||||
return res;
|
||||
}
|
||||
}
|
||||
}
|
8
src/Storages/extractKeyExpressionList.h
Normal file
8
src/Storages/extractKeyExpressionList.h
Normal file
@ -0,0 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
ASTPtr extractKeyExpressionList(const ASTPtr & node);
|
||||
}
|
Loading…
Reference in New Issue
Block a user