Merge pull request #4286 from nikvas0/nikvas0/index_fix

Data Skipping Indices fix
This commit is contained in:
alexey-milovidov 2019-02-10 00:53:04 +03:00 committed by GitHub
commit 9650c4a0da
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
36 changed files with 346 additions and 327 deletions

View File

@ -460,14 +460,21 @@ ColumnsDescription InterpreterCreateQuery::setColumns(
ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const
{
ColumnsDescription res;
IndicesDescription indices;
if (create.columns_list && create.columns_list->columns)
if (create.columns_list)
{
res = getColumnsDescription(*create.columns_list->columns, context);
if (create.columns_list->columns)
res = getColumnsDescription(*create.columns_list->columns, context);
if (create.columns_list->indices)
for (const auto & index : create.columns_list->indices->children)
indices.indices.push_back(
std::dynamic_pointer_cast<ASTIndexDeclaration>(index->clone()));
}
else if (!create.as_table.empty())
{
res = as_storage->getColumns();
indices = as_storage->getIndicesDescription();
}
else if (create.select)
{
@ -479,6 +486,8 @@ ColumnsDescription InterpreterCreateQuery::setColumns(
/// Even if query has list of columns, canonicalize it (unfold Nested columns).
ASTPtr new_columns = formatColumns(res);
ASTPtr new_indices = formatIndices(indices);
if (!create.columns_list)
{
auto new_columns_list = std::make_shared<ASTColumns>();
@ -490,6 +499,11 @@ ColumnsDescription InterpreterCreateQuery::setColumns(
else
create.columns_list->set(create.columns_list->columns, new_columns);
if (new_indices && create.columns_list->indices)
create.columns_list->replace(create.columns_list->indices, new_indices);
else if (new_indices)
create.columns_list->set(create.columns_list->indices, new_indices);
/// Check for duplicates
std::set<String> all_columns;
auto check_column_already_exists = [&all_columns](const NameAndTypePair & column_name_and_type)

View File

@ -3,6 +3,7 @@
#include <Core/Field.h>
#include <Core/Types.h>
#include <Common/FieldVisitors.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/IAST.h>
@ -21,7 +22,7 @@ public:
String name;
IAST * expr;
ASTFunction * type;
Field granularity;
UInt64 granularity;
/** Get the text that identifies this element. */
String getID(char) const override { return "Index"; }
@ -52,7 +53,7 @@ public:
s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : "");
type->formatImpl(s, state, frame);
s.ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : "");
s.ostr << applyVisitor(FieldVisitorToString(), granularity);
s.ostr << toString(granularity);
}
};

View File

@ -127,7 +127,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
auto index = std::make_shared<ASTIndexDeclaration>();
index->name = typeid_cast<const ASTIdentifier &>(*name).name;
index->granularity = typeid_cast<const ASTLiteral &>(*granularity).value;
index->granularity = typeid_cast<const ASTLiteral &>(*granularity).value.get<UInt64>();
index->set(index->expr, expr);
index->set(index->type, type);
node = index;

View File

@ -6,11 +6,11 @@
namespace DB
{
using IndicesAsts = std::vector<std::shared_ptr<ASTIndexDeclaration>>;
using IndicesASTs = std::vector<std::shared_ptr<ASTIndexDeclaration>>;
struct IndicesDescription
{
IndicesAsts indices;
IndicesASTs indices;
IndicesDescription() = default;

View File

@ -1149,7 +1149,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands)
}
void MergeTreeData::createConvertExpression(const DataPartPtr & part, const NamesAndTypesList & old_columns, const NamesAndTypesList & new_columns,
const IndicesAsts & old_indices, const IndicesAsts & new_indices, ExpressionActionsPtr & out_expression,
const IndicesASTs & old_indices, const IndicesASTs & new_indices, ExpressionActionsPtr & out_expression,
NameToNameMap & out_rename_map, bool & out_force_update_metadata) const
{
out_expression = nullptr;
@ -1309,7 +1309,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name
MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
const DataPartPtr & part,
const NamesAndTypesList & new_columns,
const IndicesAsts & new_indices,
const IndicesASTs & new_indices,
bool skip_sanity_checks)
{
ExpressionActionsPtr expression;

View File

@ -490,7 +490,7 @@ public:
AlterDataPartTransactionPtr alterDataPart(
const DataPartPtr & part,
const NamesAndTypesList & new_columns,
const IndicesAsts & new_indices,
const IndicesASTs & new_indices,
bool skip_sanity_checks);
/// Freezes all parts.
@ -746,7 +746,7 @@ private:
/// Files to be deleted are mapped to an empty string in out_rename_map.
/// If part == nullptr, just checks that all type conversions are possible.
void createConvertExpression(const DataPartPtr & part, const NamesAndTypesList & old_columns, const NamesAndTypesList & new_columns,
const IndicesAsts & old_indices, const IndicesAsts & new_indices,
const IndicesASTs & old_indices, const IndicesASTs & new_indices,
ExpressionActionsPtr & out_expression, NameToNameMap & out_rename_map, bool & out_force_update_metadata) const;
/// Calculates column sizes in compressed form for the current state of data_parts. Call with data_parts mutex locked.

View File

@ -520,6 +520,14 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
RangesInDataParts parts_with_ranges;
std::vector<std::pair<MergeTreeIndexPtr, IndexConditionPtr>> useful_indices;
for (const auto & index : data.skip_indices)
{
auto condition = index->createIndexCondition(query_info, context);
if (!condition->alwaysUnknownOrTrue())
useful_indices.emplace_back(index, condition);
}
/// Let's find what range to read from each part.
size_t sum_marks = 0;
size_t sum_ranges = 0;
@ -532,16 +540,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
else
ranges.ranges = MarkRanges{MarkRange{0, part->marks_count}};
/// It can be done in multiple threads (one thread for each part).
/// Maybe it should be moved to BlockInputStream, but it can cause some problems.
for (const auto & index : data.skip_indices)
{
auto condition = index->createIndexCondition(query_info, context);
if (!condition->alwaysUnknownOrTrue())
{
ranges.ranges = filterMarksUsingIndex(index, condition, part, ranges.ranges, settings);
}
}
for (const auto & index_and_condition : useful_indices)
ranges.ranges = filterMarksUsingIndex(
index_and_condition.first, index_and_condition.second, part, ranges.ranges, settings);
if (!ranges.ranges.empty())
{

View File

@ -26,7 +26,7 @@ void MergeTreeIndexFactory::registerIndex(const std::string &name, Creator creat
ErrorCodes::LOGICAL_ERROR);
}
std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(
std::unique_ptr<IMergeTreeIndex> MergeTreeIndexFactory::get(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context) const
@ -54,4 +54,21 @@ std::unique_ptr<MergeTreeIndex> MergeTreeIndexFactory::get(
return it->second(columns, node, context);
}
std::unique_ptr<IMergeTreeIndex> minmaxIndexCreator(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context);
std::unique_ptr<IMergeTreeIndex> setIndexCreator(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context);
MergeTreeIndexFactory::MergeTreeIndexFactory()
{
registerIndex("minmax", minmaxIndexCreator);
registerIndex("set", setIndexCreator);
}
}

View File

@ -18,48 +18,51 @@ namespace DB
{
class MergeTreeData;
class MergeTreeIndex;
class IMergeTreeIndex;
using MergeTreeIndexPtr = std::shared_ptr<const MergeTreeIndex>;
using MutableMergeTreeIndexPtr = std::shared_ptr<MergeTreeIndex>;
using MergeTreeIndexPtr = std::shared_ptr<const IMergeTreeIndex>;
using MutableMergeTreeIndexPtr = std::shared_ptr<IMergeTreeIndex>;
struct MergeTreeIndexGranule
/// Stores some info about a single block of data.
struct IMergeTreeIndexGranule
{
virtual ~MergeTreeIndexGranule() = default;
virtual ~IMergeTreeIndexGranule() = default;
virtual void serializeBinary(WriteBuffer & ostr) const = 0;
virtual void deserializeBinary(ReadBuffer & istr) = 0;
virtual String toString() const = 0;
virtual bool empty() const = 0;
/// Updates the stored info using rows of the specified block.
/// Reads no more than `limit` rows.
/// After finishing updating `pos` will store the position of the first row which was not read.
virtual void update(const Block & block, size_t * pos, size_t limit) = 0;
};
using MergeTreeIndexGranulePtr = std::shared_ptr<MergeTreeIndexGranule>;
using MergeTreeIndexGranulePtr = std::shared_ptr<IMergeTreeIndexGranule>;
using MergeTreeIndexGranules = std::vector<MergeTreeIndexGranulePtr>;
/// Condition on the index.
class IndexCondition
class IIndexCondition
{
public:
virtual ~IndexCondition() = default;
virtual ~IIndexCondition() = default;
/// Checks if this index is useful for query.
virtual bool alwaysUnknownOrTrue() const = 0;
virtual bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr granule) const = 0;
};
using IndexConditionPtr = std::shared_ptr<IndexCondition>;
using IndexConditionPtr = std::shared_ptr<IIndexCondition>;
/// Structure for storing basic index info like columns, expression, arguments, ...
class MergeTreeIndex
class IMergeTreeIndex
{
public:
MergeTreeIndex(
IMergeTreeIndex(
String name,
ExpressionActionsPtr expr,
const Names & columns,
@ -73,7 +76,7 @@ public:
, header(header)
, granularity(granularity) {}
virtual ~MergeTreeIndex() = default;
virtual ~IMergeTreeIndex() = default;
/// gets filename without extension
String getFileName() const { return INDEX_FILE_PREFIX + name; }
@ -91,7 +94,6 @@ public:
size_t granularity;
};
using MergeTreeIndices = std::vector<MutableMergeTreeIndexPtr>;
@ -101,12 +103,12 @@ class MergeTreeIndexFactory : public ext::singleton<MergeTreeIndexFactory>
public:
using Creator = std::function<
std::unique_ptr<MergeTreeIndex>(
std::unique_ptr<IMergeTreeIndex>(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context)>;
std::unique_ptr<MergeTreeIndex> get(
std::unique_ptr<IMergeTreeIndex> get(
const NamesAndTypesList & columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context) const;
@ -116,7 +118,7 @@ public:
const auto & getAllIndexes() const { return indexes; }
protected:
MergeTreeIndexFactory() = default;
MergeTreeIndexFactory();
private:
using Indexes = std::unordered_map<std::string, Creator>;

View File

@ -17,7 +17,7 @@ namespace ErrorCodes
MergeTreeMinMaxGranule::MergeTreeMinMaxGranule(const MergeTreeMinMaxIndex & index)
: MergeTreeIndexGranule(), index(index), parallelogram()
: IMergeTreeIndexGranule(), index(index), parallelogram()
{
}
@ -52,22 +52,13 @@ void MergeTreeMinMaxGranule::deserializeBinary(ReadBuffer & istr)
}
}
String MergeTreeMinMaxGranule::toString() const
{
String res = "";
for (size_t i = 0; i < parallelogram.size(); ++i)
{
res += "["
+ applyVisitor(FieldVisitorToString(), parallelogram[i].left) + ", "
+ applyVisitor(FieldVisitorToString(), parallelogram[i].right) + "]";
}
return res;
}
void MergeTreeMinMaxGranule::update(const Block & block, size_t * pos, size_t limit)
{
if (*pos >= block.rows())
throw Exception(
"The provided position is not less than the number of block rows. Position: "
+ toString(*pos) + ", Block rows: " + toString(block.rows()) + ".", ErrorCodes::LOGICAL_ERROR);
size_t rows_read = std::min(limit, block.rows() - *pos);
for (size_t i = 0; i < index.columns.size(); ++i)
@ -96,7 +87,7 @@ MinMaxCondition::MinMaxCondition(
const SelectQueryInfo &query,
const Context &context,
const MergeTreeMinMaxIndex &index)
: IndexCondition(), index(index), condition(query, context, index.columns, index.expr) {}
: IIndexCondition(), index(index), condition(query, context, index.columns, index.expr) {}
bool MinMaxCondition::alwaysUnknownOrTrue() const
{
@ -109,7 +100,7 @@ bool MinMaxCondition::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) c
= std::dynamic_pointer_cast<MergeTreeMinMaxGranule>(idx_granule);
if (!granule)
throw Exception(
"Minmax index condition got wrong granule", ErrorCodes::LOGICAL_ERROR);
"Minmax index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR);
return condition.mayBeTrueInParallelogram(granule->parallelogram, index.data_types);
}
@ -123,11 +114,11 @@ MergeTreeIndexGranulePtr MergeTreeMinMaxIndex::createIndexGranule() const
IndexConditionPtr MergeTreeMinMaxIndex::createIndexCondition(
const SelectQueryInfo & query, const Context & context) const
{
return std::make_shared<MinMaxCondition>(query, context, *this);
return std::make_shared<MinMaxCondition>(query, context, *this);
};
std::unique_ptr<MergeTreeIndex> MergeTreeMinMaxIndexCreator(
std::unique_ptr<IMergeTreeIndex> minmaxIndexCreator(
const NamesAndTypesList & new_columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context)
@ -158,7 +149,7 @@ std::unique_ptr<MergeTreeIndex> MergeTreeMinMaxIndexCreator(
}
return std::make_unique<MergeTreeMinMaxIndex>(
node->name, std::move(minmax_expr), columns, data_types, sample, node->granularity.get<size_t>());
node->name, std::move(minmax_expr), columns, data_types, sample, node->granularity);
}
}

View File

@ -13,16 +13,14 @@ namespace DB
class MergeTreeMinMaxIndex;
struct MergeTreeMinMaxGranule : public MergeTreeIndexGranule
struct MergeTreeMinMaxGranule : public IMergeTreeIndexGranule
{
explicit MergeTreeMinMaxGranule(const MergeTreeMinMaxIndex & index);
void serializeBinary(WriteBuffer & ostr) const override;
void deserializeBinary(ReadBuffer & istr) override;
String toString() const override;
bool empty() const override { return parallelogram.empty(); }
void update(const Block & block, size_t * pos, size_t limit) override;
~MergeTreeMinMaxGranule() override = default;
@ -32,7 +30,7 @@ struct MergeTreeMinMaxGranule : public MergeTreeIndexGranule
};
class MinMaxCondition : public IndexCondition
class MinMaxCondition : public IIndexCondition
{
public:
MinMaxCondition(
@ -51,7 +49,7 @@ private:
};
class MergeTreeMinMaxIndex : public MergeTreeIndex
class MergeTreeMinMaxIndex : public IMergeTreeIndex
{
public:
MergeTreeMinMaxIndex(
@ -61,7 +59,7 @@ public:
const DataTypes & data_types_,
const Block & header_,
size_t granularity_)
: MergeTreeIndex(name_, expr_, columns_, data_types_, header_, granularity_) {}
: IMergeTreeIndex(name_, expr_, columns_, data_types_, header_, granularity_) {}
~MergeTreeMinMaxIndex() override = default;
@ -72,7 +70,4 @@ public:
};
std::unique_ptr<MergeTreeIndex> MergeTreeMinMaxIndexCreator(
const NamesAndTypesList & columns, std::shared_ptr<ASTIndexDeclaration> node, const Context & context);
}

View File

@ -9,28 +9,29 @@
namespace DB
{
/// Class for reading a single column (or index).
class MergeTreeReaderStream
{
public:
MergeTreeReaderStream(
const String &path_prefix_, const String &extension_, size_t marks_count_,
const MarkRanges &all_mark_ranges,
MarkCache *mark_cache, bool save_marks_in_cache,
UncompressedCache *uncompressed_cache,
const String & path_prefix_, const String & extension_, size_t marks_count_,
const MarkRanges & all_mark_ranges,
MarkCache * mark_cache, bool save_marks_in_cache,
UncompressedCache * uncompressed_cache,
size_t aio_threshold, size_t max_read_buffer_size,
const ReadBufferFromFileBase::ProfileCallback &profile_callback, clockid_t clock_type);
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type);
void seekToMark(size_t index);
void seekToStart();
ReadBuffer *data_buffer;
ReadBuffer * data_buffer;
private:
MergeTreeReaderStream() = default;
/// NOTE: lazily loads marks from the marks cache.
const MarkInCompressedFile &getMark(size_t index);
const MarkInCompressedFile & getMark(size_t index);
void loadMarks();
@ -39,7 +40,7 @@ private:
size_t marks_count;
MarkCache *mark_cache;
MarkCache * mark_cache;
bool save_marks_in_cache;
MarkCache::MappedPtr marks;

View File

@ -1,4 +1,4 @@
#include <Storages/MergeTree/MergeTreeUniqueIndex.h>
#include <Storages/MergeTree/MergeTreeSetSkippingIndex.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
@ -8,8 +8,6 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Poco/Logger.h>
namespace DB
{
@ -19,13 +17,17 @@ namespace ErrorCodes
extern const int INCORRECT_QUERY;
}
MergeTreeUniqueGranule::MergeTreeUniqueGranule(const MergeTreeUniqueIndex & index)
: MergeTreeIndexGranule(), index(index), set(new Set(SizeLimits{}, true))
/// 0b11 -- can be true and false at the same time
const Field UNKNOWN_FIELD(3);
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(
@ -49,7 +51,7 @@ void MergeTreeUniqueGranule::serializeBinary(WriteBuffer & ostr) const
}
}
void MergeTreeUniqueGranule::deserializeBinary(ReadBuffer & istr)
void MergeTreeSetIndexGranule::deserializeBinary(ReadBuffer & istr)
{
if (!set->empty())
{
@ -76,31 +78,13 @@ void MergeTreeUniqueGranule::deserializeBinary(ReadBuffer & istr)
set->insertFromBlock(block);
}
String MergeTreeUniqueGranule::toString() const
void MergeTreeSetIndexGranule::update(const Block & new_block, size_t * pos, size_t limit)
{
String res = "";
if (*pos >= new_block.rows())
throw Exception(
"The provided position is not less than the number of block rows. Position: "
+ toString(*pos) + ", Block rows: " + toString(new_block.rows()) + ".", ErrorCodes::LOGICAL_ERROR);
const auto & columns = set->getSetElements();
for (size_t i = 0; i < index.columns.size(); ++i)
{
const auto & column = columns[i];
res += " [";
for (size_t j = 0; j < column->size(); ++j)
{
if (j != 0)
res += ", ";
Field field;
column->get(j, field);
res += applyVisitor(FieldVisitorToString(), field);
}
res += "]\n";
}
return res;
}
void MergeTreeUniqueGranule::update(const Block & new_block, size_t * pos, size_t limit)
{
size_t rows_read = std::min(limit, new_block.rows() - *pos);
if (index.max_rows && size() > index.max_rows)
@ -126,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;
@ -134,11 +118,11 @@ Block MergeTreeUniqueGranule::getElementsBlock() const
}
UniqueCondition::UniqueCondition(
SetIndexCondition::SetIndexCondition(
const SelectQueryInfo & query,
const Context & context,
const MergeTreeUniqueIndex &index)
: IndexCondition(), index(index)
const MergeTreeSetSkippingIndex &index)
: IIndexCondition(), index(index)
{
for (size_t i = 0, size = index.columns.size(); i < size; ++i)
{
@ -151,30 +135,23 @@ UniqueCondition::UniqueCondition(
/// Replace logical functions with bit functions.
/// Working with UInt8: last bit = can be true, previous = can be false.
ASTPtr new_expression;
if (select.where_expression && select.prewhere_expression)
new_expression = makeASTFunction(
expression_ast = makeASTFunction(
"and",
select.where_expression->clone(),
select.prewhere_expression->clone());
else if (select.where_expression)
new_expression = select.where_expression->clone();
expression_ast = select.where_expression->clone();
else if (select.prewhere_expression)
new_expression = select.prewhere_expression->clone();
expression_ast = select.prewhere_expression->clone();
else
/// 0b11 -- can be true and false at the same time
new_expression = std::make_shared<ASTLiteral>(Field(3));
expression_ast = std::make_shared<ASTLiteral>(UNKNOWN_FIELD);
useless = checkASTAlwaysUnknownOrTrue(new_expression);
useless = checkASTUseless(expression_ast);
/// Do not proceed if index is useless for this query.
if (useless)
return;
expression_ast = makeASTFunction(
"bitAnd",
new_expression,
std::make_shared<ASTLiteral>(Field(1)));
traverseAST(expression_ast);
auto syntax_analyzer_result = SyntaxAnalyzer(context, {}).analyze(
@ -182,17 +159,17 @@ 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 wrong granule", ErrorCodes::LOGICAL_ERROR);
"Unique index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR);
if (useless)
return true;
@ -203,17 +180,16 @@ bool UniqueCondition::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) c
Block result = granule->getElementsBlock();
actions->execute(result);
const auto & column = result.getByName(expression_ast->getColumnName()).column;
for (size_t i = 0; i < column->size(); ++i)
if (column->getBool(i))
if (column->getInt(i) & 1)
return true;
return false;
}
void UniqueCondition::traverseAST(ASTPtr & node) const
void SetIndexCondition::traverseAST(ASTPtr & node) const
{
if (operatorFromAST(node))
{
@ -226,10 +202,10 @@ void UniqueCondition::traverseAST(ASTPtr & node) const
}
if (!atomFromAST(node))
node = std::make_shared<ASTLiteral>(Field(3)); /// can_be_true=1 can_be_false=1
node = std::make_shared<ASTLiteral>(UNKNOWN_FIELD);
}
bool UniqueCondition::atomFromAST(ASTPtr & node) const
bool SetIndexCondition::atomFromAST(ASTPtr & node) const
{
/// Function, literal or column
@ -260,14 +236,14 @@ 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);
if (!func)
return false;
const ASTs & args = typeid_cast<const ASTExpressionList &>(*func->arguments).children;
ASTs & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
if (func->name == "not")
{
@ -277,16 +253,50 @@ bool UniqueCondition::operatorFromAST(ASTPtr & node) const
func->name = "__bitSwapLastTwo";
}
else if (func->name == "and" || func->name == "indexHint")
func->name = "bitAnd";
{
auto last_arg = args.back();
args.pop_back();
ASTPtr new_func;
if (args.size() > 1)
new_func = makeASTFunction(
"bitAnd",
node,
last_arg);
else
new_func = makeASTFunction(
"bitAnd",
args.back(),
last_arg);
node = new_func;
}
else if (func->name == "or")
func->name = "bitOr";
{
auto last_arg = args.back();
args.pop_back();
ASTPtr new_func;
if (args.size() > 1)
new_func = makeASTFunction(
"bitOr",
node,
last_arg);
else
new_func = makeASTFunction(
"bitOr",
args.back(),
last_arg);
node = new_func;
}
else
return false;
return true;
}
bool checkAtomName(const String & name)
static bool checkAtomName(const String & name)
{
static std::set<String> atoms = {
"notEquals",
@ -302,7 +312,7 @@ bool checkAtomName(const String & name)
return atoms.find(name) != atoms.end();
}
bool UniqueCondition::checkASTAlwaysUnknownOrTrue(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()))
{
@ -312,16 +322,16 @@ bool UniqueCondition::checkASTAlwaysUnknownOrTrue(const ASTPtr & node, bool atom
const ASTs & args = typeid_cast<const ASTExpressionList &>(*func->arguments).children;
if (func->name == "and" || func->name == "indexHint")
return checkASTAlwaysUnknownOrTrue(args[0], atomic) && checkASTAlwaysUnknownOrTrue(args[1], atomic);
return checkASTUseless(args[0], atomic) && checkASTUseless(args[1], atomic);
else if (func->name == "or")
return checkASTAlwaysUnknownOrTrue(args[0], atomic) || checkASTAlwaysUnknownOrTrue(args[1], atomic);
return checkASTUseless(args[0], atomic) || checkASTUseless(args[1], atomic);
else if (func->name == "not")
return checkASTAlwaysUnknownOrTrue(args[0], atomic);
return checkASTUseless(args[0], atomic);
else if (!atomic && checkAtomName(func->name))
return checkASTAlwaysUnknownOrTrue(node, true);
return checkASTUseless(node, true);
else
return std::any_of(args.begin(), args.end(),
[this, &atomic](const auto & arg) { return checkASTAlwaysUnknownOrTrue(arg, atomic); });
[this, &atomic](const auto & arg) { return checkASTUseless(arg, atomic); });
}
else if (const auto * literal = typeid_cast<const ASTLiteral *>(node.get()))
return !atomic && literal->value.get<bool>();
@ -332,19 +342,19 @@ bool UniqueCondition::checkASTAlwaysUnknownOrTrue(const ASTPtr & node, bool atom
}
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<MergeTreeIndex> MergeTreeUniqueIndexCreator(
std::unique_ptr<IMergeTreeIndex> setIndexCreator(
const NamesAndTypesList & new_columns,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context)
@ -386,8 +396,8 @@ std::unique_ptr<MergeTreeIndex> MergeTreeUniqueIndexCreator(
header.insert(ColumnWithTypeAndName(column.type->createColumn(), column.type, column.name));
}
return std::make_unique<MergeTreeUniqueIndex>(
node->name, std::move(unique_expr), columns, data_types, header, node->granularity.get<size_t>(), max_rows);
return std::make_unique<MergeTreeSetSkippingIndex>(
node->name, std::move(unique_expr), columns, data_types, header, node->granularity, max_rows);
}
}

View File

@ -12,50 +12,49 @@
namespace DB
{
class MergeTreeUniqueIndex;
class MergeTreeSetSkippingIndex;
struct MergeTreeUniqueGranule : public MergeTreeIndexGranule
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;
String toString() const override;
size_t size() const { return set->getTotalRowCount(); }
bool empty() const override { return !size(); }
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 IndexCondition
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;
bool operatorFromAST(ASTPtr & node) const;
bool checkASTAlwaysUnknownOrTrue(const ASTPtr & node, bool atomic = false) const;
bool checkASTUseless(const ASTPtr &node, bool atomic = false) const;
const MergeTreeUniqueIndex & index;
const MergeTreeSetSkippingIndex & index;
bool useless;
std::set<String> key_columns;
@ -64,10 +63,10 @@ private:
};
class MergeTreeUniqueIndex : public MergeTreeIndex
class MergeTreeSetSkippingIndex : public IMergeTreeIndex
{
public:
MergeTreeUniqueIndex(
MergeTreeSetSkippingIndex(
String name_,
ExpressionActionsPtr expr_,
const Names & columns_,
@ -75,9 +74,9 @@ public:
const Block & header_,
size_t granularity_,
size_t max_rows_)
: MergeTreeIndex(std::move(name_), std::move(expr_), columns_, data_types_, header_, granularity_), max_rows(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;
@ -87,7 +86,4 @@ public:
size_t max_rows = 0;
};
std::unique_ptr<MergeTreeIndex> MergeTreeUniqueIndexCreator(
const NamesAndTypesList & columns, std::shared_ptr<ASTIndexDeclaration> node, const Context & context);
}

View File

@ -257,7 +257,7 @@ MergeTreeData::DataPart::Checksums checkDataPart(
{
stream.assertMark();
}
catch (Exception &e)
catch (Exception & e)
{
e.addMessage("Cannot read mark " + toString(mark_num)
+ " in file " + stream.mrk_file_path
@ -268,7 +268,7 @@ MergeTreeData::DataPart::Checksums checkDataPart(
{
index->createIndexGranule()->deserializeBinary(stream.uncompressed_hashing_buf);
}
catch (Exception &e)
catch (Exception & e)
{
e.addMessage("Cannot read granule " + toString(mark_num)
+ " in file " + stream.bin_file_path

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>
@ -579,7 +579,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (args.query.columns_list && args.query.columns_list->indices)
for (const auto & index : args.query.columns_list->indices->children)
indices_description.indices.push_back(
std::dynamic_pointer_cast<ASTIndexDeclaration>(index->ptr()));
std::dynamic_pointer_cast<ASTIndexDeclaration>(index->clone()));
storage_settings.loadFromQuery(*args.storage_def);
}
@ -624,14 +624,6 @@ static StoragePtr create(const StorageFactory::Arguments & args)
}
static void registerMergeTreeSkipIndices()
{
auto & factory = MergeTreeIndexFactory::instance();
factory.registerIndex("minmax", MergeTreeMinMaxIndexCreator);
factory.registerIndex("unique", MergeTreeUniqueIndexCreator);
}
void registerStorageMergeTree(StorageFactory & factory)
{
factory.registerStorage("MergeTree", create);
@ -649,8 +641,6 @@ void registerStorageMergeTree(StorageFactory & factory)
factory.registerStorage("ReplicatedSummingMergeTree", create);
factory.registerStorage("ReplicatedGraphiteMergeTree", create);
factory.registerStorage("ReplicatedVersionedCollapsingMergeTree", create);
registerMergeTreeSkipIndices();
}
}

View File

@ -113,7 +113,6 @@ namespace ErrorCodes
extern const int KEEPER_EXCEPTION;
extern const int ALL_REPLICAS_LOST;
extern const int REPLICA_STATUS_CHANGED;
extern const int INCORRECT_QUERY;
}
namespace ActionLocks

View File

@ -1,8 +0,0 @@
0 5 4.7 6.50 cba b 2014-01-04
0 5 4.7 6.50 cba b 2014-03-11
2 5 4.7 6.50 cba b 2014-06-11
2 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.50 cba b 2014-01-04
0 5 4.7 6.50 cba b 2014-03-11
2 5 4.7 6.50 cba b 2014-06-11
2 5 4.7 6.50 cba b 2015-01-01

View File

@ -1,41 +0,0 @@
DROP TABLE IF EXISTS test.minmax_idx;
CREATE TABLE test.minmax_idx
(
u64 UInt64,
i32 Int32,
f64 Float64,
d Decimal(10, 2),
s String,
e Enum8('a' = 1, 'b' = 2, 'c' = 3),
dt Date,
INDEX idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 4,
INDEX idx_all2 (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 2,
INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
) ENGINE = MergeTree()
ORDER BY u64
SETTINGS index_granularity = 2;
/* many small inserts => table will make merges */
INSERT INTO test.minmax_idx VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2014-01-01');
INSERT INTO test.minmax_idx VALUES (0, 5, 4.7, 6.5, 'cba', 'b', '2014-01-04');
INSERT INTO test.minmax_idx VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2017-01-01');
INSERT INTO test.minmax_idx VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2016-01-01');
INSERT INTO test.minmax_idx VALUES (2, 5, 4.7, 6.5, 'cba', 'b', '2015-01-01');
INSERT INTO test.minmax_idx VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2014-11-11');
INSERT INTO test.minmax_idx VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2014-02-11');
INSERT INTO test.minmax_idx VALUES (0, 5, 4.7, 6.5, 'cba', 'b', '2014-03-11');
INSERT INTO test.minmax_idx VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2014-04-11');
INSERT INTO test.minmax_idx VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2014-05-11');
INSERT INTO test.minmax_idx VALUES (2, 5, 4.7, 6.5, 'cba', 'b', '2014-06-11');
INSERT INTO test.minmax_idx VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2014-07-11');
/* simple select */
SELECT * FROM test.minmax_idx WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt;
/* select with hole made by primary key */
SELECT * FROM test.minmax_idx WHERE u64 != 1 AND e = 'b' ORDER BY dt;
DROP TABLE test.minmax_idx;

View File

@ -1,16 +0,0 @@
0 5 4.7 6.50 cba b 2014-01-04
0 5 4.7 6.50 cba b 2014-03-11
2 5 4.7 6.50 cba b 2014-06-11
2 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.50 cba b 2014-01-04
0 5 4.7 6.50 cba b 2014-03-11
2 5 4.7 6.50 cba b 2014-06-11
2 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.50 cba b 2014-01-04
0 5 4.7 6.50 cba b 2014-03-11
2 5 4.7 6.50 cba b 2014-06-11
2 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.50 cba b 2014-01-04
0 5 4.7 6.50 cba b 2014-03-11
2 5 4.7 6.50 cba b 2014-06-11
2 5 4.7 6.50 cba b 2015-01-01

View File

@ -1,8 +0,0 @@
0 5 4.7 6.50 cba b 2014-01-04
0 5 4.7 6.50 cba b 2014-03-11
2 5 4.7 6.50 cba b 2014-06-11
2 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.50 cba b 2014-01-04
0 5 4.7 6.50 cba b 2014-03-11
2 5 4.7 6.50 cba b 2014-06-11
2 5 4.7 6.50 cba b 2015-01-01

View File

@ -1,41 +0,0 @@
DROP TABLE IF EXISTS test.unique_idx;
CREATE TABLE test.unique_idx
(
u64 UInt64,
i32 Int32,
f64 Float64,
d Decimal(10, 2),
s String,
e Enum8('a' = 1, 'b' = 2, 'c' = 3),
dt Date,
INDEX idx_all (i32, i32 + f64, d, s, e, dt) TYPE unique GRANULARITY 4,
INDEX idx_all2 (i32, i32 + f64, d, s, e, dt) TYPE unique GRANULARITY 2,
INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE unique GRANULARITY 3
) ENGINE = MergeTree()
ORDER BY u64
SETTINGS index_granularity = 2;
/* many small inserts => table will make merges */
INSERT INTO test.unique_idx VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2014-01-01');
INSERT INTO test.unique_idx VALUES (0, 5, 4.7, 6.5, 'cba', 'b', '2014-01-04');
INSERT INTO test.unique_idx VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2017-01-01');
INSERT INTO test.unique_idx VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2016-01-01');
INSERT INTO test.unique_idx VALUES (2, 5, 4.7, 6.5, 'cba', 'b', '2015-01-01');
INSERT INTO test.unique_idx VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2014-11-11');
INSERT INTO test.unique_idx VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2014-02-11');
INSERT INTO test.unique_idx VALUES (0, 5, 4.7, 6.5, 'cba', 'b', '2014-03-11');
INSERT INTO test.unique_idx VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2014-04-11');
INSERT INTO test.unique_idx VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2014-05-11');
INSERT INTO test.unique_idx VALUES (2, 5, 4.7, 6.5, 'cba', 'b', '2014-06-11');
INSERT INTO test.unique_idx VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2014-07-11');
/* simple select */
SELECT * FROM test.unique_idx WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt;
/* select with hole made by primary key */
SELECT * FROM test.unique_idx WHERE u64 != 1 AND e = 'b' ORDER BY dt;
DROP TABLE test.unique_idx;

View File

@ -0,0 +1,6 @@
0 5 4.7 6.50 cba b 2014-01-04
1 5 4.7 6.50 cba b 2014-03-11
11 5 4.7 6.50 cba b 2014-06-11
12 5 4.7 6.50 cba b 2015-01-01
"rows_read": 4,
"rows_read": 2,

View File

@ -0,0 +1,47 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.minmax_idx
(
u64 UInt64,
i32 Int32,
f64 Float64,
d Decimal(10, 2),
s String,
e Enum8('a' = 1, 'b' = 2, 'c' = 3),
dt Date,
INDEX idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 1,
INDEX idx_all2 (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 2,
INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
) ENGINE = MergeTree()
ORDER BY u64
SETTINGS index_granularity = 2"
$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES
(0, 5, 4.7, 6.5, 'cba', 'b', '2014-01-04'),
(1, 5, 4.7, 6.5, 'cba', 'b', '2014-03-11'),
(2, 2, 4.5, 2.5, 'abc', 'a', '2014-01-01'),
(3, 5, 6.9, 1.57, 'bac', 'c', '2017-01-01'),
(4, 2, 4.5, 2.5, 'abc', 'a', '2016-01-01'),
(5, 5, 6.9, 1.57, 'bac', 'c', '2014-11-11'),
(6, 2, 4.5, 2.5, 'abc', 'a', '2014-02-11'),
(7, 5, 6.9, 1.57, 'bac', 'c', '2014-04-11'),
(8, 2, 4.5, 2.5, 'abc', 'a', '2014-05-11'),
(9, 5, 6.9, 1.57, 'bac', 'c', '2014-07-11'),
(11, 5, 4.7, 6.5, 'cba', 'b', '2014-06-11'),
(12, 5, 4.7, 6.5, 'cba', 'b', '2015-01-01')"
# simple select
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.minmax_idx WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt"
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.minmax_idx WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt FORMAT JSON" | grep "rows_read"
# select with hole made by primary key
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt"
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx"

View File

@ -0,0 +1,8 @@
0 5 4.7 6.50 cba b 2014-01-04
1 5 4.7 6.50 cba b 2014-03-11
12 5 4.7 6.50 cba b 2014-06-11
13 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.50 cba b 2014-01-04
1 5 4.7 6.50 cba b 2014-03-11
12 5 4.7 6.50 cba b 2014-06-11
13 5 4.7 6.50 cba b 2015-01-01

View File

@ -11,7 +11,7 @@ CREATE TABLE test.minmax_idx1
e Enum8('a' = 1, 'b' = 2, 'c' = 3),
dt Date,
INDEX
idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 2,
idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 1,
INDEX
idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/minmax', 'r1')
@ -28,7 +28,7 @@ CREATE TABLE test.minmax_idx2
e Enum8('a' = 1, 'b' = 2, 'c' = 3),
dt Date,
INDEX
idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 2,
idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 1,
INDEX
idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/minmax', 'r2')
@ -37,33 +37,36 @@ SETTINGS index_granularity = 2;
/* many small inserts => table will make merges */
INSERT INTO test.minmax_idx1 VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2014-01-01');
INSERT INTO test.minmax_idx1 VALUES (2, 2, 4.5, 2.5, 'abc', 'a', '2014-01-01');
INSERT INTO test.minmax_idx1 VALUES (0, 5, 4.7, 6.5, 'cba', 'b', '2014-01-04');
INSERT INTO test.minmax_idx2 VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2017-01-01');
INSERT INTO test.minmax_idx2 VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2016-01-01');
INSERT INTO test.minmax_idx2 VALUES (2, 5, 4.7, 6.5, 'cba', 'b', '2015-01-01');
INSERT INTO test.minmax_idx1 VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2014-11-11');
INSERT INTO test.minmax_idx2 VALUES (3, 5, 6.9, 1.57, 'bac', 'c', '2017-01-01');
INSERT INTO test.minmax_idx2 VALUES (4, 2, 4.5, 2.5, 'abc', 'a', '2016-01-01');
INSERT INTO test.minmax_idx2 VALUES (13, 5, 4.7, 6.5, 'cba', 'b', '2015-01-01');
INSERT INTO test.minmax_idx1 VALUES (5, 5, 6.9, 1.57, 'bac', 'c', '2014-11-11');
SYSTEM SYNC REPLICA test.minmax_idx1;
SYSTEM SYNC REPLICA test.minmax_idx2;
INSERT INTO test.minmax_idx1 VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2014-02-11');
INSERT INTO test.minmax_idx1 VALUES (0, 5, 4.7, 6.5, 'cba', 'b', '2014-03-11');
INSERT INTO test.minmax_idx1 VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2014-04-11');
INSERT INTO test.minmax_idx1 VALUES (1, 2, 4.5, 2.5, 'abc', 'a', '2014-05-11');
INSERT INTO test.minmax_idx2 VALUES (2, 5, 4.7, 6.5, 'cba', 'b', '2014-06-11');
INSERT INTO test.minmax_idx2 VALUES (1, 5, 6.9, 1.57, 'bac', 'c', '2014-07-11');
INSERT INTO test.minmax_idx1 VALUES (6, 2, 4.5, 2.5, 'abc', 'a', '2014-02-11');
INSERT INTO test.minmax_idx1 VALUES (1, 5, 4.7, 6.5, 'cba', 'b', '2014-03-11');
INSERT INTO test.minmax_idx1 VALUES (7, 5, 6.9, 1.57, 'bac', 'c', '2014-04-11');
INSERT INTO test.minmax_idx1 VALUES (8, 2, 4.5, 2.5, 'abc', 'a', '2014-05-11');
INSERT INTO test.minmax_idx2 VALUES (12, 5, 4.7, 6.5, 'cba', 'b', '2014-06-11');
INSERT INTO test.minmax_idx2 VALUES (9, 5, 6.9, 1.57, 'bac', 'c', '2014-07-11');
SYSTEM SYNC REPLICA test.minmax_idx1;
SYSTEM SYNC REPLICA test.minmax_idx2;
OPTIMIZE TABLE test.minmax_idx1;
OPTIMIZE TABLE test.minmax_idx2;
/* simple select */
SELECT * FROM test.minmax_idx1 WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt;
SELECT * FROM test.minmax_idx2 WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt;
/* select with hole made by primary key */
SELECT * FROM test.minmax_idx1 WHERE u64 != 1 AND e = 'b' ORDER BY dt;
SELECT * FROM test.minmax_idx2 WHERE u64 != 1 AND e = 'b' ORDER BY dt;
SELECT * FROM test.minmax_idx1 WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt;
SELECT * FROM test.minmax_idx2 WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt;
DROP TABLE test.minmax_idx1;
DROP TABLE test.minmax_idx2;

View File

@ -0,0 +1,6 @@
0 5 4.7 6.50 cba b 2014-01-04
1 5 4.7 6.50 cba b 2014-03-11
12 5 4.7 6.50 cba b 2014-06-11
13 5 4.7 6.50 cba b 2015-01-01
"rows_read": 4,
"rows_read": 2,

View File

@ -0,0 +1,46 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.set_idx;"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.set_idx
(
u64 UInt64,
i32 Int32,
f64 Float64,
d Decimal(10, 2),
s String,
e Enum8('a' = 1, 'b' = 2, 'c' = 3),
dt Date,
INDEX idx_all (i32, i32 + f64, d, s, e, dt) TYPE set GRANULARITY 1,
INDEX idx_all2 (i32, i32 + f64, d, s, e, dt) TYPE set GRANULARITY 2,
INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE set GRANULARITY 3
) ENGINE = MergeTree()
ORDER BY u64
SETTINGS index_granularity = 2;"
$CLICKHOUSE_CLIENT --query="INSERT INTO test.set_idx VALUES
(0, 5, 4.7, 6.5, 'cba', 'b', '2014-01-04'),
(1, 5, 4.7, 6.5, 'cba', 'b', '2014-03-11'),
(2, 2, 4.5, 2.5, 'abc', 'a', '2014-01-01'),
(3, 5, 6.9, 1.57, 'bac', 'c', '2017-01-01'),
(4, 2, 4.5, 2.5, 'abc', 'a', '2016-01-01'),
(5, 5, 6.9, 1.57, 'bac', 'c', '2014-11-11'),
(6, 2, 4.5, 2.5, 'abc', 'a', '2014-02-11'),
(7, 5, 6.9, 1.57, 'bac', 'c', '2014-04-11'),
(8, 2, 4.5, 2.5, 'abc', 'a', '2014-05-11'),
(9, 5, 6.9, 1.57, 'bac', 'c', '2014-07-11'),
(12, 5, 4.7, 6.5, 'cba', 'b', '2014-06-11'),
(13, 5, 4.7, 6.5, 'cba', 'b', '2015-01-01')"
# simple select
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.set_idx WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt"
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.set_idx WHERE i32 = 5 AND i32 + f64 < 12 AND 3 < d AND d < 7 AND (s = 'bac' OR s = 'cba') ORDER BY dt FORMAT JSON" | grep "rows_read"
# select with hole made by primary key
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.set_idx WHERE (u64 < 2 OR u64 > 10) AND s != 'cba' ORDER BY dt"
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.set_idx WHERE (u64 < 2 OR u64 > 10) AND s != 'cba' ORDER BY dt FORMAT JSON" | grep "rows_read"
$CLICKHOUSE_CLIENT --query="DROP TABLE test.set_idx;"

View File

@ -250,7 +250,7 @@ CREATE TABLE table_name
s String,
...
INDEX a (u64 * i32, s) TYPE minmax GRANULARITY 3,
INDEX b (u64 * length(s)) TYPE unique GRANULARITY 4
INDEX b (u64 * length(s)) TYPE set GRANULARITY 4
) ENGINE = MergeTree()
...
```
@ -266,14 +266,14 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
* `minmax`
Stores extremes of the specified expression (if the expression is `tuple`, then it stores extremes for each element of `tuple`), uses stored info for skipping blocks of the data like the primary key.
* `unique(max_rows)`
* `set(max_rows)`
Stores unique values of the specified expression (no more than `max_rows` rows), use them to check if the `WHERE` expression is not satisfiable on a block of the data.
If `max_rows=0`, then there are no limits for storing values. `unique` without parameters is equal to `unique(0)`.
If `max_rows=0`, then there are no limits for storing values. `set` without parameters is equal to `set(0)`.
```sql
INDEX sample_index (u64 * length(s)) TYPE minmax GRANULARITY 4
INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE unique GRANULARITY 4
INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE unique(100) GRANULARITY 4
INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set GRANULARITY 4
INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARITY 4
```

View File

@ -96,9 +96,9 @@ It only works for tables in the [`*MergeTree`](../operations/table_engines/merge
[replicated](../operations/table_engines/replication.md) tables). The following operations
are available:
* `ALTER ADD INDEX name expression TYPE type GRANULARITY value AFTER name [AFTER name2]` - Adds index description to tables metadata.
* `ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value AFTER name [AFTER name2]` - Adds index description to tables metadata.
* `ALTER DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk.
* `ALTER TABLE [db].name DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk.
These commands are lightweight in a sense that they only change metadata or remove files.
Also, they are replicated (syncing indices metadata through ZooKeeper).

View File

@ -241,7 +241,7 @@ CREATE TABLE table_name
s String,
...
INDEX a (u64 * i32, s) TYPE minmax GRANULARITY 3,
INDEX b (u64 * length(s), i32) TYPE unique GRANULARITY 4
INDEX b (u64 * length(s), i32) TYPE set GRANULARITY 4
) ENGINE = MergeTree()
...
```
@ -257,7 +257,7 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
* `minmax`
Хранит минимум и максимум выражения (если выражение - `tuple`, то для каждого элемента `tuple`), используя их для пропуска блоков аналогично первичному ключу.
* `unique(max_rows)`
* `set(max_rows)`
Хранит уникальные значения выражения на блоке в количестве не более `max_rows`, используя их для пропуска блоков, оценивая выполнимость `WHERE` выражения на хранимых данных.
Если `max_rows=0`, то хранит значения выражения без ограничений. Если параметров не передано, то полагается `max_rows=0`.
@ -265,8 +265,8 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
Примеры
```sql
INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE minmax GRANULARITY 4
INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE unique GRANULARITY 4
INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE unique(100) GRANULARITY 4
INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set GRANULARITY 4
INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARITY 4
```

View File

@ -83,12 +83,12 @@ MODIFY ORDER BY new_expression
Добавить или удалить индекс можно с помощью операций
```
ALTER ADD INDEX name expression TYPE type GRANULARITY value [AFTER name]
ALTER DROP INDEX name
ALTER TABLE [db].name ADD INDEX name expression TYPE type GRANULARITY value [AFTER name]
ALTER TABLE [db].name DROP INDEX name
```
Поддерживается только таблицами семейства `*MergeTree`.
Команда `ALTER ADD INDEX` добавляет описание индексов в метаданные, а `ALTER DROP INDEX` удаляет индекс из метаданных и стирает файлы индекса с диска, поэтому они легковесные и работают мгновенно.
Команда `ADD INDEX` добавляет описание индексов в метаданные, а `DROP INDEX` удаляет индекс из метаданных и стирает файлы индекса с диска, поэтому они легковесные и работают мгновенно.
Если индекс появился в метаданных, то он начнет считаться в последующих слияниях и записях в таблицу, а не сразу после выполнения операции `ALTER`.