ClickHouse/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

248 lines
8.4 KiB
C++
Raw Normal View History

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/TreeRewriter.h>
2019-01-10 13:50:41 +00:00
#include <Parsers/ASTFunction.h>
2019-01-09 14:15:23 +00:00
#include <Poco/Logger.h>
#include <Common/FieldVisitorsAccurateComparison.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;
}
2020-05-28 12:37:05 +00:00
MergeTreeIndexGranuleMinMax::MergeTreeIndexGranuleMinMax(const String & index_name_, const Block & index_sample_block_)
: index_name(index_name_)
, index_sample_block(index_sample_block_)
2020-05-27 18:38:34 +00:00
{}
2019-03-08 19:52:21 +00:00
2019-06-19 15:30:48 +00:00
MergeTreeIndexGranuleMinMax::MergeTreeIndexGranuleMinMax(
2020-05-28 12:37:05 +00:00
const String & index_name_,
const Block & index_sample_block_,
std::vector<Range> && hyperrectangle_)
: index_name(index_name_)
, index_sample_block(index_sample_block_)
, 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(ErrorCodes::LOGICAL_ERROR, "Attempt to write empty minmax index {}", backQuote(index_name));
2019-01-08 17:27:44 +00:00
2020-05-27 18:38:34 +00:00
for (size_t i = 0; i < index_sample_block.columns(); ++i)
2019-01-08 17:27:44 +00:00
{
2020-05-27 18:38:34 +00:00
const DataTypePtr & type = index_sample_block.getByPosition(i).type;
2021-03-09 14:46:52 +00:00
auto serialization = type->getDefaultSerialization();
serialization->serializeBinary(hyperrectangle[i].left, ostr, {});
serialization->serializeBinary(hyperrectangle[i].right, ostr, {});
2019-01-08 17:27:44 +00:00
}
}
void MergeTreeIndexGranuleMinMax::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version)
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;
2021-03-09 14:46:52 +00:00
2020-05-27 18:38:34 +00:00
for (size_t i = 0; i < index_sample_block.columns(); ++i)
2019-01-08 17:27:44 +00:00
{
2020-05-27 18:38:34 +00:00
const DataTypePtr & type = index_sample_block.getByPosition(i).type;
2021-03-09 14:46:52 +00:00
auto serialization = type->getDefaultSerialization();
switch (version)
2019-05-12 17:01:36 +00:00
{
case 1:
if (!type->isNullable())
{
serialization->deserializeBinary(min_val, istr, {});
serialization->deserializeBinary(max_val, istr, {});
}
else
{
/// NOTE: that this serialization differs from
/// IMergeTreeDataPart::MinMaxIndex::load() to preserve
/// backward compatibility.
///
/// But this is deprecated format, so this is OK.
bool is_null;
readBinary(is_null, istr);
if (!is_null)
{
serialization->deserializeBinary(min_val, istr, {});
serialization->deserializeBinary(max_val, istr, {});
}
else
{
min_val = Null();
max_val = Null();
}
}
break;
/// New format with proper Nullable support for values that includes Null values
case 2:
serialization->deserializeBinary(min_val, istr, {});
serialization->deserializeBinary(max_val, istr, {});
// NULL_LAST
if (min_val.isNull())
min_val = POSITIVE_INFINITY;
if (max_val.isNull())
max_val = POSITIVE_INFINITY;
break;
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index version {}.", version);
2019-05-12 17:01:36 +00:00
}
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
}
2020-05-28 12:37:05 +00:00
MergeTreeIndexAggregatorMinMax::MergeTreeIndexAggregatorMinMax(const String & index_name_, const Block & index_sample_block_)
: index_name(index_name_)
, index_sample_block(index_sample_block_)
2020-05-27 18:38:34 +00:00
{}
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-05-28 12:37:05 +00:00
return std::make_shared<MergeTreeIndexGranuleMinMax>(index_name, index_sample_block, 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(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. "
"Position: {}, Block rows: {}.", toString(*pos), toString(block.rows()));
2019-02-06 07:49:18 +00:00
2019-01-26 19:31:12 +00:00
size_t rows_read = std::min(limit, block.rows() - *pos);
FieldRef field_min;
FieldRef field_max;
2020-05-27 18:38:34 +00:00
for (size_t i = 0; i < index_sample_block.columns(); ++i)
2019-01-08 17:27:44 +00:00
{
2020-05-27 18:38:34 +00:00
auto index_column_name = index_sample_block.getByPosition(i).name;
const auto & column = block.getByName(index_column_name).column->cut(*pos, rows_read);
if (const auto * column_nullable = typeid_cast<const ColumnNullable *>(column.get()))
column_nullable->getExtremesNullLast(field_min, field_max);
else
column->getExtremes(field_min, field_max);
2019-01-26 19:31:12 +00:00
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
{
hyperrectangle[i].left
= applyVisitor(FieldVisitorAccurateLess(), hyperrectangle[i].left, field_min) ? hyperrectangle[i].left : field_min;
hyperrectangle[i].right
= applyVisitor(FieldVisitorAccurateLess(), hyperrectangle[i].right, field_max) ? field_max : hyperrectangle[i].right;
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
2022-11-17 18:44:26 +00:00
namespace
{
KeyCondition buildCondition(const IndexDescription & index, const SelectQueryInfo & query_info, ContextPtr context)
{
if (context->getSettingsRef().allow_experimental_analyzer)
{
NameSet array_join_name_set;
if (query_info.syntax_analyzer_result)
array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet();
return KeyCondition{query_info.filter_actions_dag, context, index.column_names, index.expression, array_join_name_set};
}
return KeyCondition{query_info, context, index.column_names, index.expression};
}
}
2019-01-08 17:27:44 +00:00
2019-06-19 15:30:48 +00:00
MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax(
2022-11-17 18:44:26 +00:00
const IndexDescription & index, const SelectQueryInfo & query_info, ContextPtr context)
2020-05-27 18:38:34 +00:00
: index_data_types(index.data_types)
2022-11-17 18:44:26 +00:00
, condition(buildCondition(index, query_info, context))
2020-05-27 18:38:34 +00:00
{
}
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)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Minmax index condition got a granule with the wrong type.");
2020-05-28 12:37:05 +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
{
2020-05-28 12:37:05 +00:00
return std::make_shared<MergeTreeIndexGranuleMinMax>(index.name, index.sample_block);
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
{
2020-05-28 12:37:05 +00:00
return std::make_shared<MergeTreeIndexAggregatorMinMax>(index.name, index.sample_block);
2019-03-08 19:52:21 +00:00
}
2019-06-19 15:30:48 +00:00
MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const
2019-01-08 17:27:44 +00:00
{
2020-05-27 18:38:34 +00:00
return std::make_shared<MergeTreeIndexConditionMinMax>(index, query, context);
2022-05-16 18:59:27 +00:00
}
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
2020-05-27 18:38:34 +00:00
for (const auto & cname : index.column_names)
2019-08-03 11:02:40 +00:00
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
MergeTreeIndexFormat MergeTreeIndexMinMax::getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & relative_path_prefix) const
{
if (data_part_storage.exists(relative_path_prefix + ".idx2"))
return {2, ".idx2"};
else if (data_part_storage.exists(relative_path_prefix + ".idx"))
return {1, ".idx"};
return {0 /* unknown */, ""};
}
2020-05-28 13:45:08 +00:00
MergeTreeIndexPtr minmaxIndexCreator(
2020-05-28 13:09:03 +00:00
const IndexDescription & index)
2019-01-08 17:27:44 +00:00
{
2020-05-28 12:37:05 +00:00
return std::make_shared<MergeTreeIndexMinMax>(index);
2019-01-08 17:27:44 +00:00
}
2020-05-28 13:09:03 +00:00
void minmaxIndexValidator(const IndexDescription & /* index */, bool /* attach */)
2020-05-28 12:37:05 +00:00
{
}
}