ClickHouse/dbms/src/Storages/MergeTree/MergeTreeSetSkippingIndex.cpp

399 lines
12 KiB
C++
Raw Normal View History

2019-02-06 20:05:50 +00:00
#include <Storages/MergeTree/MergeTreeSetSkippingIndex.h>
2019-01-26 13:12:13 +00:00
2019-01-26 15:45:38 +00:00
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/SyntaxAnalyzer.h>
2019-01-27 21:08:14 +00:00
#include <Parsers/ASTIdentifier.h>
2019-01-27 20:24:33 +00:00
#include <Parsers/ASTFunction.h>
2019-01-27 21:08:14 +00:00
#include <Parsers/ASTLiteral.h>
2019-01-26 15:45:38 +00:00
2019-02-06 11:08:04 +00:00
2019-01-26 13:12:13 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_QUERY;
}
2019-02-05 13:52:45 +00:00
/// 0b11 -- can be true and false at the same time
const Field UNKNOWN_FIELD(3);
2019-02-06 20:05:50 +00:00
MergeTreeSetIndexGranule::MergeTreeSetIndexGranule(const MergeTreeSetSkippingIndex & index)
2019-02-06 07:49:18 +00:00
: IMergeTreeIndexGranule(), index(index), set(new Set(SizeLimits{}, true))
2019-01-26 13:12:13 +00:00
{
2019-01-27 18:02:15 +00:00
set->setHeader(index.header);
2019-01-26 13:12:13 +00:00
}
2019-02-06 20:05:50 +00:00
void MergeTreeSetIndexGranule::serializeBinary(WriteBuffer & ostr) const
2019-01-26 13:12:13 +00:00
{
if (empty())
throw Exception(
2019-02-13 19:26:24 +00:00
"Attempt to write empty set index `" + index.name + "`", ErrorCodes::LOGICAL_ERROR);
2019-01-26 13:12:13 +00:00
2019-01-27 18:02:15 +00:00
const auto & columns = set->getSetElements();
const auto & size_type = DataTypePtr(std::make_shared<DataTypeUInt64>());
2019-01-29 17:26:45 +00:00
2019-02-13 19:26:24 +00:00
if (size() > index.max_rows)
2019-01-29 17:26:45 +00:00
{
size_type->serializeBinary(0, ostr);
return;
}
2019-01-27 18:02:15 +00:00
size_type->serializeBinary(size(), ostr);
2019-01-26 13:12:13 +00:00
for (size_t i = 0; i < index.columns.size(); ++i)
{
2019-01-27 18:02:15 +00:00
const auto & type = index.data_types[i];
type->serializeBinaryBulk(*columns[i], ostr, 0, size());
2019-01-26 13:12:13 +00:00
}
}
2019-02-06 20:05:50 +00:00
void MergeTreeSetIndexGranule::deserializeBinary(ReadBuffer & istr)
2019-01-26 13:12:13 +00:00
{
2019-01-27 18:02:15 +00:00
if (!set->empty())
{
auto new_set = std::make_unique<Set>(SizeLimits{}, true);
2019-01-28 10:35:19 +00:00
new_set->setHeader(index.header);
2019-01-27 18:02:15 +00:00
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>();
2019-01-26 13:12:13 +00:00
for (size_t i = 0; i < index.columns.size(); ++i)
{
2019-01-27 18:02:15 +00:00
const auto & type = index.data_types[i];
2019-01-26 13:12:13 +00:00
auto new_column = type->createColumn();
2019-01-27 18:02:15 +00:00
type->deserializeBinaryBulk(*new_column, istr, rows_to_read, 0);
2019-01-26 13:12:13 +00:00
block.insert(ColumnWithTypeAndName(new_column->getPtr(), type, index.columns[i]));
}
2019-01-27 18:02:15 +00:00
set->insertFromBlock(block);
2019-01-26 13:12:13 +00:00
}
void MergeTreeSetIndexGranule::update(const Block & new_block, size_t * pos, size_t limit)
2019-01-26 13:12:13 +00:00
{
2019-02-06 07:49:18 +00:00
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);
2019-01-27 18:02:15 +00:00
size_t rows_read = std::min(limit, new_block.rows() - *pos);
2019-01-26 13:12:13 +00:00
2019-02-13 19:26:24 +00:00
if (size() > index.max_rows)
2019-01-26 13:12:13 +00:00
{
2019-01-27 18:02:15 +00:00
*pos += rows_read;
return;
2019-01-26 13:12:13 +00:00
}
2019-01-27 18:02:15 +00:00
Block key_block;
for (size_t i = 0; i < index.columns.size(); ++i)
2019-01-26 13:12:13 +00:00
{
2019-01-27 18:02:15 +00:00
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));
2019-01-26 13:12:13 +00:00
}
2019-01-27 18:02:15 +00:00
set->insertFromBlock(key_block);
*pos += rows_read;
2019-01-27 18:23:08 +00:00
}
2019-02-06 20:05:50 +00:00
Block MergeTreeSetIndexGranule::getElementsBlock() const
2019-01-27 18:23:08 +00:00
{
2019-02-13 19:26:24 +00:00
if (size() > index.max_rows)
2019-01-27 18:23:08 +00:00
return index.header;
return index.header.cloneWithColumns(set->getSetElements());
}
2019-01-26 15:45:38 +00:00
2019-02-06 20:05:50 +00:00
SetIndexCondition::SetIndexCondition(
2019-01-27 20:24:33 +00:00
const SelectQueryInfo & query,
2019-01-28 08:15:13 +00:00
const Context & context,
2019-02-06 20:05:50 +00:00
const MergeTreeSetSkippingIndex &index)
2019-02-06 07:49:18 +00:00
: IIndexCondition(), index(index)
2019-01-27 20:24:33 +00:00
{
for (size_t i = 0, size = index.columns.size(); i < size; ++i)
{
std::string name = index.columns[i];
if (!key_columns.count(name))
2019-01-29 17:26:45 +00:00
key_columns.insert(name);
2019-01-27 20:24:33 +00:00
}
const ASTSelectQuery & select = typeid_cast<const ASTSelectQuery &>(*query.query);
/// Replace logical functions with bit functions.
2019-01-29 16:53:44 +00:00
/// Working with UInt8: last bit = can be true, previous = can be false.
2019-01-27 20:24:33 +00:00
if (select.where_expression && select.prewhere_expression)
2019-02-05 18:34:10 +00:00
expression_ast = makeASTFunction(
2019-01-27 20:24:33 +00:00
"and",
select.where_expression->clone(),
select.prewhere_expression->clone());
else if (select.where_expression)
2019-02-05 18:34:10 +00:00
expression_ast = select.where_expression->clone();
2019-01-27 20:24:33 +00:00
else if (select.prewhere_expression)
2019-02-05 18:34:10 +00:00
expression_ast = select.prewhere_expression->clone();
2019-01-27 20:24:33 +00:00
else
2019-02-05 18:34:10 +00:00
expression_ast = std::make_shared<ASTLiteral>(UNKNOWN_FIELD);
2019-01-27 20:24:33 +00:00
2019-02-05 18:34:10 +00:00
useless = checkASTUseless(expression_ast);
2019-01-29 16:53:44 +00:00
/// Do not proceed if index is useless for this query.
2019-02-05 13:52:45 +00:00
if (useless)
2019-01-29 16:53:44 +00:00
return;
2019-01-28 08:15:13 +00:00
traverseAST(expression_ast);
2019-01-29 13:13:18 +00:00
auto syntax_analyzer_result = SyntaxAnalyzer(context, {}).analyze(
expression_ast, index.header.getNamesAndTypesList());
actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true);
2019-01-27 20:24:33 +00:00
}
2019-01-26 15:45:38 +00:00
2019-02-06 20:05:50 +00:00
bool SetIndexCondition::alwaysUnknownOrTrue() const
2019-01-26 15:45:38 +00:00
{
2019-01-29 16:53:44 +00:00
return useless;
2019-01-26 15:45:38 +00:00
}
2019-01-26 13:12:13 +00:00
2019-02-06 20:05:50 +00:00
bool SetIndexCondition::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
2019-01-26 15:45:38 +00:00
{
2019-02-06 20:05:50 +00:00
auto granule = std::dynamic_pointer_cast<MergeTreeSetIndexGranule>(idx_granule);
2019-01-26 15:45:38 +00:00
if (!granule)
throw Exception(
2019-02-13 19:26:24 +00:00
"Set index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR);
2019-01-26 15:45:38 +00:00
2019-02-14 09:06:32 +00:00
if (useless || !granule->size() || granule->size() > index.max_rows)
2019-01-28 08:15:13 +00:00
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)
2019-02-05 18:34:10 +00:00
if (column->getInt(i) & 1)
2019-01-28 08:15:13 +00:00
return true;
return false;
2019-01-26 15:45:38 +00:00
}
2019-02-06 20:05:50 +00:00
void SetIndexCondition::traverseAST(ASTPtr & node) const
2019-01-27 20:24:33 +00:00
{
2019-01-29 18:40:10 +00:00
if (operatorFromAST(node))
{
2019-01-29 17:09:17 +00:00
auto * func = typeid_cast<ASTFunction *>(&*node);
2019-01-28 07:24:32 +00:00
auto & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
2019-01-27 20:24:33 +00:00
2019-01-29 17:09:17 +00:00
for (auto & arg : args)
traverseAST(arg);
2019-01-28 07:24:32 +00:00
return;
2019-01-27 20:24:33 +00:00
}
2019-01-29 13:13:18 +00:00
if (!atomFromAST(node))
2019-02-05 13:52:45 +00:00
node = std::make_shared<ASTLiteral>(UNKNOWN_FIELD);
2019-01-27 20:24:33 +00:00
}
2019-02-06 20:05:50 +00:00
bool SetIndexCondition::atomFromAST(ASTPtr & node) const
2019-01-27 20:24:33 +00:00
{
2019-01-27 21:08:14 +00:00
/// Function, literal or column
2019-01-28 07:05:24 +00:00
if (typeid_cast<const ASTLiteral *>(node.get()))
2019-01-27 21:08:14 +00:00
return true;
2019-01-29 17:09:17 +00:00
if (const auto * identifier = typeid_cast<const ASTIdentifier *>(node.get()))
2019-01-28 07:05:24 +00:00
return key_columns.count(identifier->getColumnName()) != 0;
2019-01-27 21:08:14 +00:00
2019-01-29 17:09:17 +00:00
if (auto * func = typeid_cast<ASTFunction *>(node.get()))
2019-01-29 13:13:18 +00:00
{
2019-01-28 07:05:24 +00:00
if (key_columns.count(func->getColumnName()))
2019-01-29 13:13:18 +00:00
{
/// Function is already calculated.
node = std::make_shared<ASTIdentifier>(func->getColumnName());
2019-01-27 21:08:14 +00:00
return true;
2019-01-29 13:13:18 +00:00
}
2019-01-27 21:08:14 +00:00
2019-01-29 13:13:18 +00:00
ASTs & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
2019-01-27 21:08:14 +00:00
2019-01-29 17:09:17 +00:00
for (auto & arg : args)
if (!atomFromAST(arg))
2019-01-27 20:24:33 +00:00
return false;
return true;
}
return false;
}
2019-02-06 20:05:50 +00:00
bool SetIndexCondition::operatorFromAST(ASTPtr & node) const
2019-01-27 20:24:33 +00:00
{
/// Functions AND, OR, NOT. Replace with bit*.
2019-01-29 17:09:17 +00:00
auto * func = typeid_cast<ASTFunction *>(&*node);
2019-01-28 07:24:32 +00:00
if (!func)
return false;
2019-02-05 18:34:10 +00:00
ASTs & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
2019-01-27 20:24:33 +00:00
if (func->name == "not")
{
if (args.size() != 1)
return false;
2019-01-28 07:24:32 +00:00
2019-01-30 19:40:01 +00:00
func->name = "__bitSwapLastTwo";
2019-01-27 20:24:33 +00:00
}
else if (func->name == "and" || func->name == "indexHint")
2019-02-05 18:34:10 +00:00
{
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;
}
2019-01-27 20:24:33 +00:00
else if (func->name == "or")
2019-02-05 18:34:10 +00:00
{
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;
}
2019-01-27 20:24:33 +00:00
else
return false;
return true;
}
2019-02-06 09:05:05 +00:00
static bool checkAtomName(const String & name)
2019-01-29 16:53:44 +00:00
{
static std::set<String> atoms = {
"notEquals",
"equals",
"less",
"greater",
"lessOrEquals",
"greaterOrEquals",
"in",
"notIn",
"like"
};
return atoms.find(name) != atoms.end();
}
2019-02-06 20:05:50 +00:00
bool SetIndexCondition::checkASTUseless(const ASTPtr &node, bool atomic) const
2019-01-29 16:53:44 +00:00
{
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")
2019-02-05 13:52:45 +00:00
return checkASTUseless(args[0], atomic) && checkASTUseless(args[1], atomic);
2019-01-29 16:53:44 +00:00
else if (func->name == "or")
2019-02-05 13:52:45 +00:00
return checkASTUseless(args[0], atomic) || checkASTUseless(args[1], atomic);
2019-01-29 16:53:44 +00:00
else if (func->name == "not")
2019-02-05 13:52:45 +00:00
return checkASTUseless(args[0], atomic);
2019-01-29 16:53:44 +00:00
else if (!atomic && checkAtomName(func->name))
2019-02-05 13:52:45 +00:00
return checkASTUseless(node, true);
2019-01-29 16:53:44 +00:00
else
return std::any_of(args.begin(), args.end(),
2019-02-05 13:52:45 +00:00
[this, &atomic](const auto & arg) { return checkASTUseless(arg, atomic); });
2019-01-29 16:53:44 +00:00
}
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;
}
2019-01-26 15:45:38 +00:00
2019-02-06 20:05:50 +00:00
MergeTreeIndexGranulePtr MergeTreeSetSkippingIndex::createIndexGranule() const
2019-01-26 15:45:38 +00:00
{
2019-02-06 20:05:50 +00:00
return std::make_shared<MergeTreeSetIndexGranule>(*this);
2019-01-26 15:45:38 +00:00
}
2019-01-26 13:12:13 +00:00
2019-02-06 20:05:50 +00:00
IndexConditionPtr MergeTreeSetSkippingIndex::createIndexCondition(
2019-01-26 15:45:38 +00:00
const SelectQueryInfo & query, const Context & context) const
{
2019-02-06 20:05:50 +00:00
return std::make_shared<SetIndexCondition>(query, context, *this);
2019-01-26 13:12:13 +00:00
};
2019-01-26 15:45:38 +00:00
2019-02-06 20:05:50 +00:00
std::unique_ptr<IMergeTreeIndex> setIndexCreator(
2019-01-30 10:18:59 +00:00
const NamesAndTypesList & new_columns,
2019-01-26 15:45:38 +00:00
std::shared_ptr<ASTIndexDeclaration> node,
2019-02-20 12:48:50 +00:00
const MergeTreeData &,
2019-01-26 15:45:38 +00:00
const Context & context)
{
if (node->name.empty())
throw Exception("Index must have unique name", ErrorCodes::INCORRECT_QUERY);
size_t max_rows = 0;
2019-02-13 19:26:24 +00:00
if (!node->type->arguments || node->type->arguments->children.size() != 1)
throw Exception("Set index must have exactly one 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>();
2019-01-26 15:45:38 +00:00
ASTPtr expr_list = MergeTreeData::extractKeyExpressionList(node->expr->clone());
auto syntax = SyntaxAnalyzer(context, {}).analyze(
2019-01-30 10:18:59 +00:00
expr_list, new_columns);
2019-01-26 15:45:38 +00:00
auto unique_expr = ExpressionAnalyzer(expr_list, syntax, context).getActions(false);
auto sample = ExpressionAnalyzer(expr_list, syntax, context)
.getActions(true)->getSampleBlock();
2019-01-27 18:02:15 +00:00
Block header;
2019-01-26 15:45:38 +00:00
Names columns;
DataTypes data_types;
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);
2019-01-27 18:02:15 +00:00
header.insert(ColumnWithTypeAndName(column.type->createColumn(), column.type, column.name));
2019-01-26 15:45:38 +00:00
}
2019-02-06 20:05:50 +00:00
return std::make_unique<MergeTreeSetSkippingIndex>(
2019-02-06 08:43:54 +00:00
node->name, std::move(unique_expr), columns, data_types, header, node->granularity, max_rows);
2019-01-26 15:45:38 +00:00
}
2019-01-29 17:28:13 +00:00
}