unique -> set

This commit is contained in:
Nikita Vasilev 2019-02-06 23:05:50 +03:00
parent 6ef6536460
commit a675ba2323
4 changed files with 37 additions and 37 deletions

View File

@ -60,7 +60,7 @@ std::unique_ptr<IMergeTreeIndex> minmaxIndexCreator(
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context);
std::unique_ptr<IMergeTreeIndex> uniqueIndexCreator(
std::unique_ptr<IMergeTreeIndex> setIndexCreator(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context);
@ -68,7 +68,7 @@ std::unique_ptr<IMergeTreeIndex> uniqueIndexCreator(
MergeTreeIndexFactory::MergeTreeIndexFactory()
{
registerIndex("minmax", minmaxIndexCreator);
registerIndex("unique", uniqueIndexCreator);
registerIndex("set", setIndexCreator);
}
}

View File

@ -1,4 +1,4 @@
#include <Storages/MergeTree/MergeTreeUniqueIndex.h>
#include <Storages/MergeTree/MergeTreeSetSkippingIndex.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
@ -21,13 +21,13 @@ namespace ErrorCodes
const Field UNKNOWN_FIELD(3);
MergeTreeUniqueGranule::MergeTreeUniqueGranule(const MergeTreeUniqueIndex & index)
MergeTreeSetIndexGranule::MergeTreeSetIndexGranule(const MergeTreeSetSkippingIndex & index)
: IMergeTreeIndexGranule(), index(index), set(new Set(SizeLimits{}, true))
{
set->setHeader(index.header);
}
void MergeTreeUniqueGranule::serializeBinary(WriteBuffer & ostr) const
void MergeTreeSetIndexGranule::serializeBinary(WriteBuffer & ostr) const
{
if (empty())
throw Exception(
@ -51,7 +51,7 @@ void MergeTreeUniqueGranule::serializeBinary(WriteBuffer & ostr) const
}
}
void MergeTreeUniqueGranule::deserializeBinary(ReadBuffer & istr)
void MergeTreeSetIndexGranule::deserializeBinary(ReadBuffer & istr)
{
if (!set->empty())
{
@ -78,7 +78,7 @@ void MergeTreeUniqueGranule::deserializeBinary(ReadBuffer & istr)
set->insertFromBlock(block);
}
void MergeTreeUniqueGranule::update(const Block & new_block, size_t * pos, size_t limit)
void MergeTreeSetIndexGranule::update(const Block & new_block, size_t * pos, size_t limit)
{
if (*pos >= new_block.rows())
throw Exception(
@ -110,7 +110,7 @@ void MergeTreeUniqueGranule::update(const Block & new_block, size_t * pos, size_
*pos += rows_read;
}
Block MergeTreeUniqueGranule::getElementsBlock() const
Block MergeTreeSetIndexGranule::getElementsBlock() const
{
if (index.max_rows && size() > index.max_rows)
return index.header;
@ -118,10 +118,10 @@ Block MergeTreeUniqueGranule::getElementsBlock() const
}
UniqueCondition::UniqueCondition(
SetIndexCondition::SetIndexCondition(
const SelectQueryInfo & query,
const Context & context,
const MergeTreeUniqueIndex &index)
const MergeTreeSetSkippingIndex &index)
: IIndexCondition(), index(index)
{
for (size_t i = 0, size = index.columns.size(); i < size; ++i)
@ -159,14 +159,14 @@ UniqueCondition::UniqueCondition(
actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true);
}
bool UniqueCondition::alwaysUnknownOrTrue() const
bool SetIndexCondition::alwaysUnknownOrTrue() const
{
return useless;
}
bool UniqueCondition::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
bool SetIndexCondition::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
{
auto granule = std::dynamic_pointer_cast<MergeTreeUniqueGranule>(idx_granule);
auto granule = std::dynamic_pointer_cast<MergeTreeSetIndexGranule>(idx_granule);
if (!granule)
throw Exception(
"Unique index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR);
@ -189,7 +189,7 @@ bool UniqueCondition::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) c
return false;
}
void UniqueCondition::traverseAST(ASTPtr & node) const
void SetIndexCondition::traverseAST(ASTPtr & node) const
{
if (operatorFromAST(node))
{
@ -205,7 +205,7 @@ void UniqueCondition::traverseAST(ASTPtr & node) const
node = std::make_shared<ASTLiteral>(UNKNOWN_FIELD);
}
bool UniqueCondition::atomFromAST(ASTPtr & node) const
bool SetIndexCondition::atomFromAST(ASTPtr & node) const
{
/// Function, literal or column
@ -236,7 +236,7 @@ bool UniqueCondition::atomFromAST(ASTPtr & node) const
return false;
}
bool UniqueCondition::operatorFromAST(ASTPtr & node) const
bool SetIndexCondition::operatorFromAST(ASTPtr & node) const
{
/// Functions AND, OR, NOT. Replace with bit*.
auto * func = typeid_cast<ASTFunction *>(&*node);
@ -312,7 +312,7 @@ static bool checkAtomName(const String & name)
return atoms.find(name) != atoms.end();
}
bool UniqueCondition::checkASTUseless(const ASTPtr &node, bool atomic) const
bool SetIndexCondition::checkASTUseless(const ASTPtr &node, bool atomic) const
{
if (const auto * func = typeid_cast<const ASTFunction *>(node.get()))
{
@ -342,19 +342,19 @@ bool UniqueCondition::checkASTUseless(const ASTPtr &node, bool atomic) const
}
MergeTreeIndexGranulePtr MergeTreeUniqueIndex::createIndexGranule() const
MergeTreeIndexGranulePtr MergeTreeSetSkippingIndex::createIndexGranule() const
{
return std::make_shared<MergeTreeUniqueGranule>(*this);
return std::make_shared<MergeTreeSetIndexGranule>(*this);
}
IndexConditionPtr MergeTreeUniqueIndex::createIndexCondition(
IndexConditionPtr MergeTreeSetSkippingIndex::createIndexCondition(
const SelectQueryInfo & query, const Context & context) const
{
return std::make_shared<UniqueCondition>(query, context, *this);
return std::make_shared<SetIndexCondition>(query, context, *this);
};
std::unique_ptr<IMergeTreeIndex> uniqueIndexCreator(
std::unique_ptr<IMergeTreeIndex> setIndexCreator(
const NamesAndTypesList & new_columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context)
@ -396,7 +396,7 @@ std::unique_ptr<IMergeTreeIndex> uniqueIndexCreator(
header.insert(ColumnWithTypeAndName(column.type->createColumn(), column.type, column.name));
}
return std::make_unique<MergeTreeUniqueIndex>(
return std::make_unique<MergeTreeSetSkippingIndex>(
node->name, std::move(unique_expr), columns, data_types, header, node->granularity, max_rows);
}

View File

@ -12,11 +12,11 @@
namespace DB
{
class MergeTreeUniqueIndex;
class MergeTreeSetSkippingIndex;
struct MergeTreeUniqueGranule : public IMergeTreeIndexGranule
struct MergeTreeSetIndexGranule : public IMergeTreeIndexGranule
{
explicit MergeTreeUniqueGranule(const MergeTreeUniqueIndex & index);
explicit MergeTreeSetIndexGranule(const MergeTreeSetSkippingIndex & index);
void serializeBinary(WriteBuffer & ostr) const override;
void deserializeBinary(ReadBuffer & istr) override;
@ -27,26 +27,26 @@ struct MergeTreeUniqueGranule : public IMergeTreeIndexGranule
void update(const Block & block, size_t * pos, size_t limit) override;
Block getElementsBlock() const;
~MergeTreeUniqueGranule() override = default;
~MergeTreeSetIndexGranule() override = default;
const MergeTreeUniqueIndex & index;
const MergeTreeSetSkippingIndex & index;
std::unique_ptr<Set> set;
};
class UniqueCondition : public IIndexCondition
class SetIndexCondition : public IIndexCondition
{
public:
UniqueCondition(
SetIndexCondition(
const SelectQueryInfo & query,
const Context & context,
const MergeTreeUniqueIndex & index);
const MergeTreeSetSkippingIndex & index);
bool alwaysUnknownOrTrue() const override;
bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override;
~UniqueCondition() override = default;
~SetIndexCondition() override = default;
private:
void traverseAST(ASTPtr & node) const;
bool atomFromAST(ASTPtr & node) const;
@ -54,7 +54,7 @@ private:
bool checkASTUseless(const ASTPtr &node, bool atomic = false) const;
const MergeTreeUniqueIndex & index;
const MergeTreeSetSkippingIndex & index;
bool useless;
std::set<String> key_columns;
@ -63,10 +63,10 @@ private:
};
class MergeTreeUniqueIndex : public IMergeTreeIndex
class MergeTreeSetSkippingIndex : public IMergeTreeIndex
{
public:
MergeTreeUniqueIndex(
MergeTreeSetSkippingIndex(
String name_,
ExpressionActionsPtr expr_,
const Names & columns_,
@ -76,7 +76,7 @@ public:
size_t max_rows_)
: IMergeTreeIndex(std::move(name_), std::move(expr_), columns_, data_types_, header_, granularity_), max_rows(max_rows_) {}
~MergeTreeUniqueIndex() override = default;
~MergeTreeSetSkippingIndex() override = default;
MergeTreeIndexGranulePtr createIndexGranule() const override;

View File

@ -3,7 +3,7 @@
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeIndices.h>
#include <Storages/MergeTree/MergeTreeMinMaxIndex.h>
#include <Storages/MergeTree/MergeTreeUniqueIndex.h>
#include <Storages/MergeTree/MergeTreeSetSkippingIndex.h>
#include <Common/typeid_cast.h>
#include <Common/OptimizedRegularExpression.h>