mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
convert type with condition
This commit is contained in:
parent
6c8ff6dc31
commit
d145295163
@ -30,10 +30,12 @@ struct BloomFilterHash
|
||||
{
|
||||
WhichDataType which(data_type);
|
||||
|
||||
if (which.isUInt())
|
||||
if (which.isUInt() || which.isDateOrDateTime())
|
||||
return ColumnConst::create(ColumnUInt64::create(1, intHash64(field.safeGet<UInt64>())), 1);
|
||||
else if (which.isInt())
|
||||
else if (which.isInt() || which.isEnum())
|
||||
return ColumnConst::create(ColumnUInt64::create(1, intHash64(ext::bit_cast<UInt64>(field.safeGet<Int64>()))), 1);
|
||||
else if (which.isFloat())
|
||||
return ColumnConst::create(ColumnUInt64::create(1, intHash64(ext::bit_cast<UInt64>(field.safeGet<Float64>()))), 1);
|
||||
else if (which.isString() || which.isFixedString())
|
||||
{
|
||||
const auto & value = field.safeGet<String>();
|
||||
|
@ -36,15 +36,18 @@ MergeTreeIndexGranulePtr MergeTreeIndexBloomFilter::createIndexGranule() const
|
||||
|
||||
bool MergeTreeIndexBloomFilter::mayBenefitFromIndexForIn(const ASTPtr & node) const
|
||||
{
|
||||
const String column_name = node->getColumnName();
|
||||
const String & column_name = node->getColumnName();
|
||||
|
||||
for (const auto & name : columns)
|
||||
if (column_name == name)
|
||||
return true;
|
||||
|
||||
if (const auto * func = typeid_cast<const ASTFunction *>(node.get()))
|
||||
if (func->arguments->children.size() == 1)
|
||||
return mayBenefitFromIndexForIn(func->arguments->children.front());
|
||||
{
|
||||
for (const auto & children : func->arguments->children)
|
||||
if (mayBenefitFromIndexForIn(children))
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
@ -59,6 +62,24 @@ IndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const SelectQu
|
||||
return std::make_shared<MergeTreeIndexConditionBloomFilter>(query_info, context, header, hash_functions);
|
||||
}
|
||||
|
||||
static void assertIndexColumnsType(const Block &header)
|
||||
{
|
||||
if (!header || !header.columns())
|
||||
throw Exception("Index must have columns.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
const DataTypes & columns_data_types = header.getDataTypes();
|
||||
|
||||
for (size_t index = 0; index < columns_data_types.size(); ++index)
|
||||
{
|
||||
WhichDataType which(columns_data_types[index]);
|
||||
|
||||
if (!which.isUInt() && !which.isInt() && !which.isString() && !which.isFixedString() && !which.isFloat() &&
|
||||
!which.isDateOrDateTime() && !which.isEnum())
|
||||
throw Exception("Unexpected type " + columns_data_types[index]->getName() + " of bloom filter index.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
std::unique_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(const NamesAndTypesList & columns, std::shared_ptr<ASTIndexDeclaration> node, const Context & context)
|
||||
{
|
||||
if (node->name.empty())
|
||||
@ -70,8 +91,7 @@ std::unique_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(const NamesAndTypesL
|
||||
auto index_expr = ExpressionAnalyzer(expr_list, syntax, context).getActions(false);
|
||||
auto index_sample = ExpressionAnalyzer(expr_list, syntax, context).getActions(true)->getSampleBlock();
|
||||
|
||||
if (!index_sample || !index_sample.columns())
|
||||
throw Exception("Index must have columns.", ErrorCodes::INCORRECT_QUERY);
|
||||
assertIndexColumnsType(index_sample);
|
||||
|
||||
double max_conflict_probability = 0.025;
|
||||
if (node->type->arguments && !node->type->arguments->children.empty())
|
||||
|
@ -10,7 +10,8 @@
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include "MergeTreeIndexConditionBloomFilter.h"
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -76,7 +77,7 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr &
|
||||
|
||||
MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter(
|
||||
const SelectQueryInfo & info, const Context & context, const Block & header, size_t hash_functions)
|
||||
: header(header), query_info(info), hash_functions(hash_functions)
|
||||
: header(header), context(context), query_info(info), hash_functions(hash_functions)
|
||||
{
|
||||
auto atomFromAST = [this](auto & node, auto &, auto & constants, auto & out) { return traverseAtomAST(node, constants, out); };
|
||||
rpn = std::move(RPNBuilder<RPNElement>(info, context, atomFromAST).extractRPN());
|
||||
@ -249,7 +250,9 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
|
||||
{
|
||||
size_t row_size = column->size();
|
||||
size_t position = header.getPositionByName(key_ast->getColumnName());
|
||||
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(type, column, 0, row_size)));
|
||||
const DataTypePtr & index_type = header.getByPosition(position).type;
|
||||
const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type, context);
|
||||
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size)));
|
||||
|
||||
if (function_name == "in" || function_name == "globalIn")
|
||||
out.function = RPNElement::FUNCTION_IN;
|
||||
@ -293,7 +296,9 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
|
||||
if (header.has(key_ast->getColumnName()))
|
||||
{
|
||||
size_t position = header.getPositionByName(key_ast->getColumnName());
|
||||
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(&*value_type, value_field)));
|
||||
const DataTypePtr & index_type = header.getByPosition(position).type;
|
||||
Field converted_field = convertFieldToType(value_field, *index_type, &*value_type);
|
||||
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(&*index_type, converted_field)));
|
||||
out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS;
|
||||
return true;
|
||||
}
|
||||
|
@ -53,6 +53,7 @@ public:
|
||||
|
||||
private:
|
||||
const Block & header;
|
||||
const Context & context;
|
||||
const SelectQueryInfo & query_info;
|
||||
const size_t hash_functions;
|
||||
std::vector<RPNElement> rpn;
|
||||
@ -65,11 +66,9 @@ private:
|
||||
|
||||
bool traverseASTIn(const String & function_name, const ASTPtr & key_ast, const SetPtr & prepared_set, RPNElement & out);
|
||||
|
||||
bool traverseASTIn(const String &function_name, const ASTPtr &key_ast, const DataTypePtr &type, const ColumnPtr &column,
|
||||
RPNElement &out);
|
||||
bool traverseASTIn(const String & function_name, const ASTPtr & key_ast, const DataTypePtr & type, const ColumnPtr & column, RPNElement & out);
|
||||
|
||||
bool traverseASTEquals(const String &function_name, const ASTPtr &key_ast, const DataTypePtr &value_type, const Field &value_field,
|
||||
RPNElement &out);
|
||||
bool traverseASTEquals(const String & function_name, const ASTPtr & key_ast, const DataTypePtr & value_type, const Field & value_field, RPNElement & out);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,7 +1,7 @@
|
||||
DROP TABLE IF EXISTS test.single_column_bloom_filter;
|
||||
|
||||
SET allow_experimental_data_skipping_indices = 1;
|
||||
|
||||
DROP TABLE IF EXISTS test.single_column_bloom_filter;
|
||||
|
||||
CREATE TABLE test.single_column_bloom_filter (u64 UInt64, i32 Int32, i64 UInt64, INDEX idx (i32) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 6;
|
||||
|
||||
INSERT INTO test.single_column_bloom_filter SELECT number AS u64, number AS i32, number AS i64 FROM system.numbers LIMIT 100;
|
||||
@ -19,13 +19,49 @@ SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) = (1
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN (1, 2) SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) IN ((1, 2), (2, 3)) SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) IN ((1, 1), (2, 2)) SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) = (1, (1, 1)) SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN (SELECT arrayJoin([1, 2])) SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) IN (SELECT arrayJoin([(1, 1), (2, 2)])) SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) IN ((1, (1, 1)), (2, (2, 2))) SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN (SELECT arrayJoin([toInt32(1), toInt32(2)])) SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) IN (SELECT arrayJoin([(toInt32(1), toInt32(2)), (toInt32(2), toInt32(3))])) SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) IN (SELECT arrayJoin([(toInt32(1), toUInt64(1)), (toInt32(2), toUInt64(2))])) SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) IN (SELECT arrayJoin([(toUInt64(1), (toUInt64(1), toInt32(1))), (toUInt64(2), (toUInt64(2), toInt32(2)))])) SETTINGS max_rows_to_read = 6;
|
||||
WITH (1, 2) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN liter_prepared_set SETTINGS max_rows_to_read = 6;
|
||||
WITH ((1, 2), (2, 3)) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) IN liter_prepared_set SETTINGS max_rows_to_read = 6;
|
||||
WITH ((1, 1), (2, 2)) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) IN liter_prepared_set SETTINGS max_rows_to_read = 6;
|
||||
WITH ((1, (1, 1)), (2, (2, 2))) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) IN liter_prepared_set SETTINGS max_rows_to_read = 6;
|
||||
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN (1, 2) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN (SELECT arrayJoin([1, 2])) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) IN ((1, 2), (2, 3)) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) IN ((1, 1), (2, 2)) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) IN ((1, (1, 1)), (2, (2, 2))) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN (SELECT arrayJoin([toInt32(1), toInt32(2)])) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) IN (SELECT arrayJoin([(toInt32(1), toInt32(2)), (toInt32(2), toInt32(3))])) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) IN (SELECT arrayJoin([(toInt32(1), toUInt64(1)), (toInt32(2), toUInt64(2))])) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) IN (SELECT arrayJoin([(toUInt64(1), (toUInt64(1), toInt32(1))), (toUInt64(2), (toUInt64(2), toInt32(2)))])) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
WITH (1, 2) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN liter_prepared_set SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
WITH ((1, 2), (2, 3)) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) IN liter_prepared_set SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
WITH ((1, 1), (2, 2)) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) IN liter_prepared_set SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
WITH ((1, (1, 1)), (2, (2, 2))) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) IN liter_prepared_set SETTINGS max_rows_to_read = 5; -- { serverError 158 }
|
||||
|
||||
DROP TABLE IF EXISTS test.single_column_bloom_filter;
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS test.bloom_filter_types_test;
|
||||
|
||||
CREATE TABLE test.bloom_filter_types_test (order_key UInt64, i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, f32 Float32, f64 Float64, date Date, date_time DateTime, str String, fixed_string FixedString(3), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6;
|
||||
INSERT INTO test.bloom_filter_types_test SELECT number AS order_key, toInt8(number) AS i8, toInt16(number) AS i16, toInt32(number) AS i32, toInt64(number) AS i64, toUInt8(number) AS u8, toUInt16(number) AS u16, toUInt32(number) AS u32, toUInt64(number) AS u64, toFloat32(number) AS f32, toFloat64(number) AS f64, toDate(number) AS date, toDateTime(number) AS date_time, toString(number) AS str, toFixedString(toString(number), 3) AS fixed_string FROM system.numbers LIMIT 100;
|
||||
|
||||
SELECT COUNT() FROM test.bloom_filter_types_test WHERE i8 = 1 SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.bloom_filter_types_test WHERE i16 = 1 SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.bloom_filter_types_test WHERE i32 = 1 SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.bloom_filter_types_test WHERE i64 = 1 SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.bloom_filter_types_test WHERE u8 = 1 SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.bloom_filter_types_test WHERE u16 = 1 SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.bloom_filter_types_test WHERE u32 = 1 SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.bloom_filter_types_test WHERE u64 = 1 SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.bloom_filter_types_test WHERE f32 = 1.0 SETTINGS max_rows_to_read = 6;
|
||||
SELECT COUNT() FROM test.bloom_filter_types_test WHERE f64 = 1.0 SETTINGS max_rows_to_read = 6;
|
||||
|
||||
SELECT * FROM test.bloom_filter_types_test WHERE f32 = 1 SETTINGS max_rows_to_read = 6;
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS test.bloom_filter_types_test;
|
||||
|
Loading…
Reference in New Issue
Block a user