2019-06-19 15:30:48 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeIndexMinMax.h>
|
2019-01-10 13:50:41 +00:00
|
|
|
|
|
|
|
#include <Interpreters/ExpressionActions.h>
|
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
|
|
|
#include <Interpreters/SyntaxAnalyzer.h>
|
|
|
|
|
2019-01-09 14:15:23 +00:00
|
|
|
#include <Poco/Logger.h>
|
2019-01-08 17:27:44 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-01-22 19:43:52 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int INCORRECT_QUERY;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-08-03 11:02:40 +00:00
|
|
|
MergeTreeIndexGranuleMinMax::MergeTreeIndexGranuleMinMax(const MergeTreeIndexMinMax & index_)
|
2020-03-09 02:26:50 +00:00
|
|
|
: index(index_) {}
|
2019-03-08 19:52:21 +00:00
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
MergeTreeIndexGranuleMinMax::MergeTreeIndexGranuleMinMax(
|
2020-03-10 14:56:55 +00:00
|
|
|
const MergeTreeIndexMinMax & index_, std::vector<Range> && hyperrectangle_)
|
2020-03-11 08:55:37 +00:00
|
|
|
: index(index_), hyperrectangle(std::move(hyperrectangle_)) {}
|
2019-01-08 17:27:44 +00:00
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
void MergeTreeIndexGranuleMinMax::serializeBinary(WriteBuffer & ostr) const
|
2019-01-08 17:27:44 +00:00
|
|
|
{
|
|
|
|
if (empty())
|
|
|
|
throw Exception(
|
2019-06-15 12:06:22 +00:00
|
|
|
"Attempt to write empty minmax index " + backQuote(index.name), ErrorCodes::LOGICAL_ERROR);
|
2019-01-08 17:27:44 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < index.columns.size(); ++i)
|
|
|
|
{
|
|
|
|
const DataTypePtr & type = index.data_types[i];
|
2019-05-12 17:01:36 +00:00
|
|
|
if (!type->isNullable())
|
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
type->serializeBinary(hyperrectangle[i].left, ostr);
|
|
|
|
type->serializeBinary(hyperrectangle[i].right, ostr);
|
2019-05-12 17:01:36 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
bool is_null = hyperrectangle[i].left.isNull() || hyperrectangle[i].right.isNull(); // one is enough
|
2019-05-12 17:01:36 +00:00
|
|
|
writeBinary(is_null, ostr);
|
|
|
|
if (!is_null)
|
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
type->serializeBinary(hyperrectangle[i].left, ostr);
|
|
|
|
type->serializeBinary(hyperrectangle[i].right, ostr);
|
2019-05-12 17:01:36 +00:00
|
|
|
}
|
|
|
|
}
|
2019-01-08 17:27:44 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
void MergeTreeIndexGranuleMinMax::deserializeBinary(ReadBuffer & istr)
|
2019-01-08 17:27:44 +00:00
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle.clear();
|
2019-02-14 17:35:50 +00:00
|
|
|
Field min_val;
|
|
|
|
Field max_val;
|
2019-01-08 17:27:44 +00:00
|
|
|
for (size_t i = 0; i < index.columns.size(); ++i)
|
|
|
|
{
|
|
|
|
const DataTypePtr & type = index.data_types[i];
|
2019-05-12 17:01:36 +00:00
|
|
|
if (!type->isNullable())
|
|
|
|
{
|
|
|
|
type->deserializeBinary(min_val, istr);
|
|
|
|
type->deserializeBinary(max_val, istr);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
bool is_null;
|
|
|
|
readBinary(is_null, istr);
|
|
|
|
if (!is_null)
|
|
|
|
{
|
|
|
|
type->deserializeBinary(min_val, istr);
|
|
|
|
type->deserializeBinary(max_val, istr);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
min_val = Null();
|
|
|
|
max_val = Null();
|
|
|
|
}
|
|
|
|
}
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle.emplace_back(min_val, true, max_val, true);
|
2019-01-08 17:27:44 +00:00
|
|
|
}
|
2019-01-09 14:15:23 +00:00
|
|
|
}
|
|
|
|
|
2019-03-08 19:52:21 +00:00
|
|
|
|
2019-08-03 11:02:40 +00:00
|
|
|
MergeTreeIndexAggregatorMinMax::MergeTreeIndexAggregatorMinMax(const MergeTreeIndexMinMax & index_)
|
|
|
|
: index(index_) {}
|
2019-03-08 19:52:21 +00:00
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorMinMax::getGranuleAndReset()
|
2019-03-08 19:52:21 +00:00
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
return std::make_shared<MergeTreeIndexGranuleMinMax>(index, std::move(hyperrectangle));
|
2019-03-08 19:52:21 +00:00
|
|
|
}
|
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, size_t limit)
|
2019-01-08 17:27:44 +00:00
|
|
|
{
|
2019-02-06 07:49:18 +00:00
|
|
|
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);
|
|
|
|
|
2019-01-26 19:31:12 +00:00
|
|
|
size_t rows_read = std::min(limit, block.rows() - *pos);
|
|
|
|
|
2019-02-14 17:35:50 +00:00
|
|
|
Field field_min;
|
|
|
|
Field field_max;
|
2019-01-08 17:27:44 +00:00
|
|
|
for (size_t i = 0; i < index.columns.size(); ++i)
|
|
|
|
{
|
2019-01-26 19:31:12 +00:00
|
|
|
const auto & column = block.getByName(index.columns[i]).column;
|
|
|
|
column->cut(*pos, rows_read)->getExtremes(field_min, field_max);
|
|
|
|
|
2020-03-10 14:56:55 +00:00
|
|
|
if (hyperrectangle.size() <= i)
|
2019-01-08 17:27:44 +00:00
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle.emplace_back(field_min, true, field_max, true);
|
2019-01-08 17:27:44 +00:00
|
|
|
}
|
2019-01-26 19:31:12 +00:00
|
|
|
else
|
|
|
|
{
|
2020-03-10 14:56:55 +00:00
|
|
|
hyperrectangle[i].left = std::min(hyperrectangle[i].left, field_min);
|
|
|
|
hyperrectangle[i].right = std::max(hyperrectangle[i].right, field_max);
|
2019-01-26 19:31:12 +00:00
|
|
|
}
|
2019-01-08 17:27:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
*pos += rows_read;
|
2019-01-27 18:23:08 +00:00
|
|
|
}
|
2019-01-08 17:27:44 +00:00
|
|
|
|
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax(
|
2019-01-08 17:27:44 +00:00
|
|
|
const SelectQueryInfo &query,
|
|
|
|
const Context &context,
|
2019-08-03 11:02:40 +00:00
|
|
|
const MergeTreeIndexMinMax &index_)
|
2020-03-09 02:26:50 +00:00
|
|
|
: index(index_), condition(query, context, index.columns, index.expr) {}
|
2019-01-08 17:27:44 +00:00
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
bool MergeTreeIndexConditionMinMax::alwaysUnknownOrTrue() const
|
2019-01-08 17:27:44 +00:00
|
|
|
{
|
|
|
|
return condition.alwaysUnknownOrTrue();
|
|
|
|
}
|
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
bool MergeTreeIndexConditionMinMax::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const
|
2019-01-08 17:27:44 +00:00
|
|
|
{
|
2019-06-19 15:30:48 +00:00
|
|
|
std::shared_ptr<MergeTreeIndexGranuleMinMax> granule
|
|
|
|
= std::dynamic_pointer_cast<MergeTreeIndexGranuleMinMax>(idx_granule);
|
2019-01-26 06:26:49 +00:00
|
|
|
if (!granule)
|
2019-01-08 17:27:44 +00:00
|
|
|
throw Exception(
|
2019-02-06 07:49:18 +00:00
|
|
|
"Minmax index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR);
|
2020-03-10 14:56:55 +00:00
|
|
|
for (const auto & range : granule->hyperrectangle)
|
2019-05-12 17:08:28 +00:00
|
|
|
if (range.left.isNull() || range.right.isNull())
|
|
|
|
return true;
|
2020-03-10 14:56:55 +00:00
|
|
|
return condition.checkInHyperrectangle(granule->hyperrectangle, index.data_types).can_be_true;
|
2019-01-08 17:27:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
MergeTreeIndexGranulePtr MergeTreeIndexMinMax::createIndexGranule() const
|
2019-01-08 17:27:44 +00:00
|
|
|
{
|
2019-06-19 15:30:48 +00:00
|
|
|
return std::make_shared<MergeTreeIndexGranuleMinMax>(*this);
|
2019-01-08 17:27:44 +00:00
|
|
|
}
|
|
|
|
|
2019-03-08 19:52:21 +00:00
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
MergeTreeIndexAggregatorPtr MergeTreeIndexMinMax::createIndexAggregator() const
|
2019-03-08 19:52:21 +00:00
|
|
|
{
|
2019-06-19 15:30:48 +00:00
|
|
|
return std::make_shared<MergeTreeIndexAggregatorMinMax>(*this);
|
2019-03-08 19:52:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition(
|
2019-01-09 17:05:52 +00:00
|
|
|
const SelectQueryInfo & query, const Context & context) const
|
2019-01-08 17:27:44 +00:00
|
|
|
{
|
2019-06-19 15:30:48 +00:00
|
|
|
return std::make_shared<MergeTreeIndexConditionMinMax>(query, context, *this);
|
2019-01-08 17:27:44 +00:00
|
|
|
};
|
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
bool MergeTreeIndexMinMax::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
2019-02-25 08:43:19 +00:00
|
|
|
{
|
|
|
|
const String column_name = node->getColumnName();
|
2019-02-25 10:12:05 +00:00
|
|
|
|
2019-08-03 11:02:40 +00:00
|
|
|
for (const auto & cname : columns)
|
|
|
|
if (column_name == cname)
|
2019-02-25 10:12:05 +00:00
|
|
|
return true;
|
|
|
|
|
|
|
|
if (const auto * func = typeid_cast<const ASTFunction *>(node.get()))
|
|
|
|
if (func->arguments->children.size() == 1)
|
|
|
|
return mayBenefitFromIndexForIn(func->arguments->children.front());
|
|
|
|
|
|
|
|
return false;
|
2019-02-25 08:43:19 +00:00
|
|
|
}
|
2019-01-08 17:27:44 +00:00
|
|
|
|
2019-02-06 09:05:05 +00:00
|
|
|
std::unique_ptr<IMergeTreeIndex> minmaxIndexCreator(
|
2019-01-30 10:18:59 +00:00
|
|
|
const NamesAndTypesList & new_columns,
|
2019-01-09 17:05:52 +00:00
|
|
|
std::shared_ptr<ASTIndexDeclaration> node,
|
|
|
|
const Context & context)
|
2019-01-08 17:27:44 +00:00
|
|
|
{
|
|
|
|
if (node->name.empty())
|
|
|
|
throw Exception("Index must have unique name", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
|
|
|
if (node->type->arguments)
|
|
|
|
throw Exception("Minmax index have not any arguments", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
|
|
|
ASTPtr expr_list = MergeTreeData::extractKeyExpressionList(node->expr->clone());
|
2020-02-26 19:33:09 +00:00
|
|
|
auto syntax = SyntaxAnalyzer(context).analyze(expr_list, new_columns);
|
2019-01-08 17:27:44 +00:00
|
|
|
auto minmax_expr = ExpressionAnalyzer(expr_list, syntax, context).getActions(false);
|
|
|
|
|
2019-01-09 14:15:23 +00:00
|
|
|
auto sample = ExpressionAnalyzer(expr_list, syntax, context)
|
2019-02-05 16:43:52 +00:00
|
|
|
.getActions(true)->getSampleBlock();
|
2019-01-08 17:27:44 +00:00
|
|
|
|
2019-01-10 12:57:12 +00:00
|
|
|
Names columns;
|
|
|
|
DataTypes data_types;
|
|
|
|
|
2019-01-09 14:15:23 +00:00
|
|
|
for (size_t i = 0; i < expr_list->children.size(); ++i)
|
2019-01-08 17:27:44 +00:00
|
|
|
{
|
2019-01-09 14:15:23 +00:00
|
|
|
const auto & column = sample.getByPosition(i);
|
|
|
|
|
2019-01-10 12:57:12 +00:00
|
|
|
columns.emplace_back(column.name);
|
|
|
|
data_types.emplace_back(column.type);
|
2019-01-08 17:27:44 +00:00
|
|
|
}
|
|
|
|
|
2019-06-19 15:30:48 +00:00
|
|
|
return std::make_unique<MergeTreeIndexMinMax>(
|
2019-02-06 08:43:54 +00:00
|
|
|
node->name, std::move(minmax_expr), columns, data_types, sample, node->granularity);
|
2019-01-08 17:27:44 +00:00
|
|
|
}
|
|
|
|
|
2019-01-29 20:00:12 +00:00
|
|
|
}
|