Merge pull request #12 from nikvas0/nikvas0/unique_index

Nikvas0/unique index
This commit is contained in:
Nikita Vasilev 2019-01-29 21:41:17 +03:00 committed by GitHub
commit f6a6a4467e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 610 additions and 16 deletions

View File

@ -157,7 +157,7 @@ protected:
using QueueWithCollation = std::priority_queue<SortCursorWithCollation>; using QueueWithCollation = std::priority_queue<SortCursorWithCollation>;
QueueWithCollation queue_with_collation; QueueWithCollation queue_with_collation;
/// Used in Vertical merge algorithm to gather non-PK columns (on next step) /// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step)
/// If it is not nullptr then it should be populated during execution /// If it is not nullptr then it should be populated during execution
WriteBuffer * out_row_sources_buf; WriteBuffer * out_row_sources_buf;

View File

@ -64,11 +64,13 @@ public:
ExpressionActionsPtr expr, ExpressionActionsPtr expr,
const Names & columns, const Names & columns,
const DataTypes & data_types, const DataTypes & data_types,
const Block & header,
size_t granularity) size_t granularity)
: name(name) : name(name)
, expr(expr) , expr(expr)
, columns(columns) , columns(columns)
, data_types(data_types) , data_types(data_types)
, header(header)
, granularity(granularity) {} , granularity(granularity) {}
virtual ~MergeTreeIndex() = default; virtual ~MergeTreeIndex() = default;
@ -85,6 +87,7 @@ public:
ExpressionActionsPtr expr; ExpressionActionsPtr expr;
Names columns; Names columns;
DataTypes data_types; DataTypes data_types;
Block header;
size_t granularity; size_t granularity;
}; };

View File

@ -116,14 +116,14 @@ void MergeTreeMinMaxGranule::update(const Block & block, size_t * pos, size_t li
LOG_DEBUG(log, "updated rows_read: " << rows_read); LOG_DEBUG(log, "updated rows_read: " << rows_read);
*pos += rows_read; *pos += rows_read;
}; }
MinMaxCondition::MinMaxCondition( MinMaxCondition::MinMaxCondition(
const SelectQueryInfo &query, const SelectQueryInfo &query,
const Context &context, const Context &context,
const MergeTreeMinMaxIndex &index) const MergeTreeMinMaxIndex &index)
: IndexCondition(), index(index), condition(query, context, index.columns, index.expr) {}; : IndexCondition(), index(index), condition(query, context, index.columns, index.expr) {}
bool MinMaxCondition::alwaysUnknownOrTrue() const bool MinMaxCondition::alwaysUnknownOrTrue() const
{ {
@ -188,7 +188,7 @@ std::unique_ptr<MergeTreeIndex> MergeTreeMinMaxIndexCreator(
} }
return std::make_unique<MergeTreeMinMaxIndex>( return std::make_unique<MergeTreeMinMaxIndex>(
node->name, std::move(minmax_expr), columns, data_types, node->granularity.get<size_t>());; node->name, std::move(minmax_expr), columns, data_types, sample, node->granularity.get<size_t>());;
} }
} }

View File

@ -12,6 +12,7 @@ namespace DB
class MergeTreeMinMaxIndex; class MergeTreeMinMaxIndex;
struct MergeTreeMinMaxGranule : public MergeTreeIndexGranule struct MergeTreeMinMaxGranule : public MergeTreeIndexGranule
{ {
explicit MergeTreeMinMaxGranule(const MergeTreeMinMaxIndex & index); explicit MergeTreeMinMaxGranule(const MergeTreeMinMaxIndex & index);
@ -30,6 +31,7 @@ struct MergeTreeMinMaxGranule : public MergeTreeIndexGranule
std::vector<Range> parallelogram; std::vector<Range> parallelogram;
}; };
class MinMaxCondition : public IndexCondition class MinMaxCondition : public IndexCondition
{ {
public: public:
@ -57,8 +59,9 @@ public:
ExpressionActionsPtr expr, ExpressionActionsPtr expr,
const Names & columns, const Names & columns,
const DataTypes & data_types, const DataTypes & data_types,
const Block & header,
size_t granularity) size_t granularity)
: MergeTreeIndex(name, expr, columns, data_types, granularity) {} : MergeTreeIndex(name, expr, columns, data_types, header, granularity) {}
~MergeTreeMinMaxIndex() override = default; ~MergeTreeMinMaxIndex() override = default;

View File

@ -0,0 +1,431 @@
#include <Storages/MergeTree/MergeTreeUniqueIndex.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Poco/Logger.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_QUERY;
}
MergeTreeUniqueGranule::MergeTreeUniqueGranule(const MergeTreeUniqueIndex & index)
: MergeTreeIndexGranule(), index(index), set(new Set(SizeLimits{}, true))
{
set->setHeader(index.header);
}
void MergeTreeUniqueGranule::serializeBinary(WriteBuffer & ostr) const
{
if (empty())
throw Exception(
"Attempt to write empty unique index `" + index.name + "`", ErrorCodes::LOGICAL_ERROR);
Poco::Logger * log = &Poco::Logger::get("unique_idx");
LOG_DEBUG(log, "serializeBinary Granule");
const auto & columns = set->getSetElements();
const auto & size_type = DataTypePtr(std::make_shared<DataTypeUInt64>());
if (index.max_rows && size() > index.max_rows)
{
size_type->serializeBinary(0, ostr);
return;
}
size_type->serializeBinary(size(), ostr);
for (size_t i = 0; i < index.columns.size(); ++i)
{
const auto & type = index.data_types[i];
type->serializeBinaryBulk(*columns[i], ostr, 0, size());
}
}
void MergeTreeUniqueGranule::deserializeBinary(ReadBuffer & istr)
{
Poco::Logger * log = &Poco::Logger::get("unique_idx");
LOG_DEBUG(log, "deserializeBinary Granule");
if (!set->empty())
{
auto new_set = std::make_unique<Set>(SizeLimits{}, true);
new_set->setHeader(index.header);
set.swap(new_set);
}
Block block;
Field field_rows;
const auto & size_type = DataTypePtr(std::make_shared<DataTypeUInt64>());
size_type->deserializeBinary(field_rows, istr);
size_t rows_to_read = field_rows.get<size_t>();
for (size_t i = 0; i < index.columns.size(); ++i)
{
const auto & type = index.data_types[i];
auto new_column = type->createColumn();
type->deserializeBinaryBulk(*new_column, istr, rows_to_read, 0);
block.insert(ColumnWithTypeAndName(new_column->getPtr(), type, index.columns[i]));
}
set->insertFromBlock(block);
}
String MergeTreeUniqueGranule::toString() const
{
String res = "unique granule:\n";
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)
{
Poco::Logger * log = &Poco::Logger::get("unique_idx");
LOG_DEBUG(log, "update Granule " << new_block.columns()
<< " pos: "<< *pos << " limit: " << limit << " rows: " << new_block.rows());
size_t rows_read = std::min(limit, new_block.rows() - *pos);
if (index.max_rows && size() > index.max_rows)
{
*pos += rows_read;
return;
}
Block key_block;
for (size_t i = 0; i < index.columns.size(); ++i)
{
const auto & name = index.columns[i];
const auto & type = index.data_types[i];
key_block.insert(
ColumnWithTypeAndName(
new_block.getByName(name).column->cut(*pos, rows_read),
type,
name));
}
set->insertFromBlock(key_block);
LOG_DEBUG(log, "unique rows: " << set->getTotalRowCount());
*pos += rows_read;
}
Block MergeTreeUniqueGranule::getElementsBlock() const
{
if (index.max_rows && size() > index.max_rows)
return index.header;
return index.header.cloneWithColumns(set->getSetElements());
}
UniqueCondition::UniqueCondition(
const SelectQueryInfo & query,
const Context & context,
const MergeTreeUniqueIndex &index)
: IndexCondition(), index(index)
{
for (size_t i = 0, size = index.columns.size(); i < size; ++i)
{
std::string name = index.columns[i];
if (!key_columns.count(name))
key_columns.insert(name);
}
const ASTSelectQuery & select = typeid_cast<const ASTSelectQuery &>(*query.query);
/// 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(
"and",
select.where_expression->clone(),
select.prewhere_expression->clone());
else if (select.where_expression)
new_expression = select.where_expression->clone();
else if (select.prewhere_expression)
new_expression = select.prewhere_expression->clone();
else
/// 11_2 -- can be true and false at the same time
new_expression = std::make_shared<ASTLiteral>(Field(3));
useless = checkASTAlwaysUnknownOrTrue(new_expression);
/// 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(
expression_ast, index.header.getNamesAndTypesList());
actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true);
}
bool UniqueCondition::alwaysUnknownOrTrue() const
{
return useless;
}
bool UniqueCondition::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
{
auto granule = std::dynamic_pointer_cast<MergeTreeUniqueGranule>(idx_granule);
if (!granule)
throw Exception(
"Unique index condition got wrong granule", ErrorCodes::LOGICAL_ERROR);
if (useless)
return true;
if (index.max_rows && granule->size() > index.max_rows)
return true;
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))
return true;
return false;
}
void UniqueCondition::traverseAST(ASTPtr & node) const
{
if (operatorFromAST(node))
{
auto * func = typeid_cast<ASTFunction *>(&*node);
auto & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
for (auto & arg : args)
traverseAST(arg);
return;
}
if (!atomFromAST(node))
node = std::make_shared<ASTLiteral>(Field(3)); /// can_be_true=1 can_be_false=1
}
bool UniqueCondition::atomFromAST(ASTPtr & node) const
{
/// Function, literal or column
if (typeid_cast<const ASTLiteral *>(node.get()))
return true;
if (const auto * identifier = typeid_cast<const ASTIdentifier *>(node.get()))
return key_columns.count(identifier->getColumnName()) != 0;
if (auto * func = typeid_cast<ASTFunction *>(node.get()))
{
if (key_columns.count(func->getColumnName()))
{
/// Function is already calculated.
node = std::make_shared<ASTIdentifier>(func->getColumnName());
return true;
}
ASTs & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
for (auto & arg : args)
if (!atomFromAST(arg))
return false;
return true;
}
return false;
}
bool UniqueCondition::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;
if (func->name == "not")
{
if (args.size() != 1)
return false;
const auto one = std::make_shared<ASTLiteral>(Field(1));
const auto two = std::make_shared<ASTLiteral>(Field(2));
node = makeASTFunction(
"bitOr",
makeASTFunction(
"bitShiftLeft",
makeASTFunction(
"bitAnd",
node->clone(),
one->clone()),
one->clone()),
makeASTFunction(
"bitShiftRight",
makeASTFunction(
"bitAnd",
node->clone(),
two->clone()),
one->clone()));
}
else if (func->name == "and" || func->name == "indexHint")
func->name = "bitAnd";
else if (func->name == "or")
func->name = "bitOr";
else
return false;
return true;
}
bool checkAtomName(const String & name)
{
static std::set<String> atoms = {
"notEquals",
"equals",
"less",
"greater",
"lessOrEquals",
"greaterOrEquals",
"in",
"notIn",
"like"
};
return atoms.find(name) != atoms.end();
}
bool UniqueCondition::checkASTAlwaysUnknownOrTrue(const ASTPtr & node, bool atomic) const
{
if (const auto * func = typeid_cast<const ASTFunction *>(node.get()))
{
if (key_columns.count(func->getColumnName()))
return false;
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);
else if (func->name == "or")
return checkASTAlwaysUnknownOrTrue(args[0], atomic) || checkASTAlwaysUnknownOrTrue(args[1], atomic);
else if (func->name == "not")
return checkASTAlwaysUnknownOrTrue(args[0], atomic);
else if (!atomic && checkAtomName(func->name))
return checkASTAlwaysUnknownOrTrue(node, true);
else
return std::any_of(args.begin(), args.end(),
[this, &atomic](const auto & arg) { return checkASTAlwaysUnknownOrTrue(arg, atomic); });
}
else if (const auto * literal = typeid_cast<const ASTLiteral *>(node.get()))
return !atomic && literal->value.get<bool>();
else if (const auto * identifier = typeid_cast<const ASTIdentifier *>(node.get()))
return key_columns.find(identifier->getColumnName()) == key_columns.end();
else
return true;
}
MergeTreeIndexGranulePtr MergeTreeUniqueIndex::createIndexGranule() const
{
return std::make_shared<MergeTreeUniqueGranule>(*this);
}
IndexConditionPtr MergeTreeUniqueIndex::createIndexCondition(
const SelectQueryInfo & query, const Context & context) const
{
return std::make_shared<UniqueCondition>(query, context, *this);
};
std::unique_ptr<MergeTreeIndex> MergeTreeUniqueIndexCreator(
const MergeTreeData & data,
std::shared_ptr<ASTIndexDeclaration> node,
const Context & context)
{
if (node->name.empty())
throw Exception("Index must have unique name", ErrorCodes::INCORRECT_QUERY);
size_t max_rows = 0;
if (node->type->arguments)
{
if (node->type->arguments->children.size() > 1)
throw Exception("Unique index cannot have only 0 or 1 argument", ErrorCodes::INCORRECT_QUERY);
else if (node->type->arguments->children.size() == 1)
max_rows = typeid_cast<const ASTLiteral &>(
*node->type->arguments->children[0]).value.get<size_t>();
}
ASTPtr expr_list = MergeTreeData::extractKeyExpressionList(node->expr->clone());
auto syntax = SyntaxAnalyzer(context, {}).analyze(
expr_list, data.getColumns().getAllPhysical());
auto unique_expr = ExpressionAnalyzer(expr_list, syntax, context).getActions(false);
auto sample = ExpressionAnalyzer(expr_list, syntax, context)
.getActions(true)->getSampleBlock();
Block header;
Names columns;
DataTypes data_types;
Poco::Logger * log = &Poco::Logger::get("unique_idx");
LOG_DEBUG(log, "new unique index" << node->name);
for (size_t i = 0; i < expr_list->children.size(); ++i)
{
const auto & column = sample.getByPosition(i);
columns.emplace_back(column.name);
data_types.emplace_back(column.type);
header.insert(ColumnWithTypeAndName(column.type->createColumn(), column.type, column.name));
LOG_DEBUG(log, ">" << column.name << " " << column.type->getName());
}
return std::make_unique<MergeTreeUniqueIndex>(
node->name, std::move(unique_expr), columns, data_types, header, node->granularity.get<size_t>(), max_rows);;
}
}

View File

@ -0,0 +1,93 @@
#pragma once
#include <Storages/MergeTree/MergeTreeIndices.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Interpreters/Set.h>
#include <memory>
#include <set>
namespace DB
{
class MergeTreeUniqueIndex;
struct MergeTreeUniqueGranule : public MergeTreeIndexGranule
{
explicit MergeTreeUniqueGranule(const MergeTreeUniqueIndex & 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;
const MergeTreeUniqueIndex & index;
std::unique_ptr<Set> set;
};
class UniqueCondition : public IndexCondition
{
public:
UniqueCondition(
const SelectQueryInfo & query,
const Context & context,
const MergeTreeUniqueIndex & index);
bool alwaysUnknownOrTrue() const override;
bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override;
~UniqueCondition() 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;
const MergeTreeUniqueIndex & index;
bool useless;
std::set<String> key_columns;
ASTPtr expression_ast;
ExpressionActionsPtr actions;
};
class MergeTreeUniqueIndex : public MergeTreeIndex
{
public:
MergeTreeUniqueIndex(
String name,
ExpressionActionsPtr expr,
const Names & columns,
const DataTypes & data_types,
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) {}
~MergeTreeUniqueIndex() override = default;
MergeTreeIndexGranulePtr createIndexGranule() const override;
IndexConditionPtr createIndexCondition(
const SelectQueryInfo & query, const Context & context) const override;
size_t max_rows = 0;
};
std::unique_ptr<MergeTreeIndex> MergeTreeUniqueIndexCreator(
const MergeTreeData & data, std::shared_ptr<ASTIndexDeclaration> node, const Context & context);
}

View File

@ -3,6 +3,7 @@
#include <Storages/StorageReplicatedMergeTree.h> #include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeIndices.h> #include <Storages/MergeTree/MergeTreeIndices.h>
#include <Storages/MergeTree/MergeTreeMinMaxIndex.h> #include <Storages/MergeTree/MergeTreeMinMaxIndex.h>
#include <Storages/MergeTree/MergeTreeUniqueIndex.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Common/OptimizedRegularExpression.h> #include <Common/OptimizedRegularExpression.h>
@ -627,6 +628,7 @@ static void registerMergeTreeSkipIndices()
{ {
auto & factory = MergeTreeIndexFactory::instance(); auto & factory = MergeTreeIndexFactory::instance();
factory.registerIndex("minmax", MergeTreeMinMaxIndexCreator); factory.registerIndex("minmax", MergeTreeMinMaxIndexCreator);
factory.registerIndex("unique", MergeTreeUniqueIndexCreator);
} }

View File

@ -9,12 +9,9 @@ CREATE TABLE test.minmax_idx
s String, s String,
e Enum8('a' = 1, 'b' = 2, 'c' = 3), e Enum8('a' = 1, 'b' = 2, 'c' = 3),
dt Date, dt Date,
INDEX INDEX idx_all (i32, i32 + f64, d, s, e, dt) TYPE minmax GRANULARITY 4,
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 INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3
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() ) ENGINE = MergeTree()
ORDER BY u64 ORDER BY u64
SETTINGS index_granularity = 2; SETTINGS index_granularity = 2;

View File

@ -0,0 +1,8 @@
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

@ -0,0 +1,41 @@
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

@ -250,7 +250,7 @@ CREATE TABLE table_name
s String, s String,
... ...
INDEX a (u64 * i32, s) TYPE minmax GRANULARITY 3, INDEX a (u64 * i32, s) TYPE minmax GRANULARITY 3,
INDEX b (u64 * length(s)) TYPE minmax GRANULARITY 4 INDEX b (u64 * length(s)) TYPE unique GRANULARITY 4
) ENGINE = MergeTree() ) ENGINE = MergeTree()
... ...
``` ```
@ -263,10 +263,17 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
#### Available Types of Indices #### Available Types of Indices
* `minmax` Stores extremes of specified expression (if the expression is `tuple`, then it stores extremes for each element of `tuple`), uses stored info for skipping blocks of data like primary key. * `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)`
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)`.
```sql ```sql
INDEX sample_index (u64 * length(s)) TYPE minmax GRANULARITY 4 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
``` ```

View File

@ -241,7 +241,7 @@ CREATE TABLE table_name
s String, s String,
... ...
INDEX a (u64 * i32, s) TYPE minmax GRANULARITY 3, INDEX a (u64 * i32, s) TYPE minmax GRANULARITY 3,
INDEX b (u64 * length(s)) TYPE minmax GRANULARITY 4 INDEX b (u64 * length(s), i32) TYPE unique GRANULARITY 4
) ENGINE = MergeTree() ) ENGINE = MergeTree()
... ...
``` ```
@ -254,13 +254,22 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
#### Доступные индексы #### Доступные индексы
* `minmax` Хранит минимум и максимум выражения (если выражение - `tuple`, то для каждого элемента `tuple`), используя их для пропуска кусков аналогично первичному ключу. * `minmax`
Хранит минимум и максимум выражения (если выражение - `tuple`, то для каждого элемента `tuple`), используя их для пропуска блоков аналогично первичному ключу.
Пример * `unique(max_rows)`
Хранит уникальные значения выражения на блоке в количестве не более `max_rows`, используя их для пропуска блоков, оценивая выполнимость `WHERE` выражения на хранимых данных.
Если `max_rows=0`, то хранит значения выражения без ограничений. Если параметров не передано, то полагается `max_rows=0`.
Примеры
```sql ```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 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
``` ```
## Конкурентный доступ к данным ## Конкурентный доступ к данным
Для конкурентного доступа к таблице используется мультиверсионность. То есть, при одновременном чтении и обновлении таблицы, данные будут читаться из набора кусочков, актуального на момент запроса. Длинных блокировок нет. Вставки никак не мешают чтениям. Для конкурентного доступа к таблице используется мультиверсионность. То есть, при одновременном чтении и обновлении таблицы, данные будут читаться из набора кусочков, актуального на момент запроса. Длинных блокировок нет. Вставки никак не мешают чтениям.