fix float in bloom filter

This commit is contained in:
zhang2014 2019-06-19 23:09:07 +08:00
parent d145295163
commit a50aea09f1
7 changed files with 119 additions and 100 deletions

View File

@ -1,6 +1,5 @@
#include <Interpreters/BloomFilter.h>
#include <city.h>
#include "BloomFilter.h"
namespace DB
@ -72,72 +71,16 @@ bool operator== (const BloomFilter & a, const BloomFilter & b)
return true;
}
void BloomFilter::addHashWithSeed(const UInt64 & hash, const UInt64 & seed)
void BloomFilter::addHashWithSeed(const UInt64 & hash, const UInt64 & hash_seed)
{
size_t pos = CityHash_v1_0_2::Hash128to64(CityHash_v1_0_2::uint128(hash, seed)) % (8 * size);
size_t pos = CityHash_v1_0_2::Hash128to64(CityHash_v1_0_2::uint128(hash, hash_seed)) % (8 * size);
filter[pos / (8 * sizeof(UnderType))] |= (1ULL << (pos % (8 * sizeof(UnderType))));
}
bool BloomFilter::containsWithSeed(const UInt64 & hash, const UInt64 & seed)
bool BloomFilter::findHashWithSeed(const UInt64 & hash, const UInt64 & hash_seed)
{
size_t pos = CityHash_v1_0_2::Hash128to64(CityHash_v1_0_2::uint128(hash, seed)) % (8 * size);
size_t pos = CityHash_v1_0_2::Hash128to64(CityHash_v1_0_2::uint128(hash, hash_seed)) % (8 * size);
return bool(filter[pos / (8 * sizeof(UnderType))] & (1ULL << (pos % (8 * sizeof(UnderType)))));
}
static std::pair<sMergeTreeIndexFullText.cppize_t, size_t> calculationBestPracticesImpl(double max_conflict_probability)
{
static const size_t MAX_BITS_PER_ROW = 20;
static const size_t MAX_HASH_FUNCTION_COUNT = 15;
/// For the smallest index per level in probability_lookup_table
static const size_t min_probability_index_each_bits[] = {0, 0, 1, 2, 3, 3, 4, 5, 6, 6, 7, 8, 8, 9, 10, 10, 11, 12, 12, 13, 14};
static const long double probability_lookup_table[MAX_BITS_PER_ROW + 1][MAX_HASH_FUNCTION_COUNT] =
{
{1.0}, /// dummy, 0 bits per row
{1.0, 1.0},
{1.0, 0.393, 0.400},
{1.0, 0.283, 0.237, 0.253},
{1.0, 0.221, 0.155, 0.147, 0.160},
{1.0, 0.181, 0.109, 0.092, 0.092, 0.101}, // 5
{1.0, 0.154, 0.0804, 0.0609, 0.0561, 0.0578, 0.0638},
{1.0, 0.133, 0.0618, 0.0423, 0.0359, 0.0347, 0.0364},
{1.0, 0.118, 0.0489, 0.0306, 0.024, 0.0217, 0.0216, 0.0229},
{1.0, 0.105, 0.0397, 0.0228, 0.0166, 0.0141, 0.0133, 0.0135, 0.0145},
{1.0, 0.0952, 0.0329, 0.0174, 0.0118, 0.00943, 0.00844, 0.00819, 0.00846}, // 10
{1.0, 0.0869, 0.0276, 0.0136, 0.00864, 0.0065, 0.00552, 0.00513, 0.00509},
{1.0, 0.08, 0.0236, 0.0108, 0.00646, 0.00459, 0.00371, 0.00329, 0.00314},
{1.0, 0.074, 0.0203, 0.00875, 0.00492, 0.00332, 0.00255, 0.00217, 0.00199, 0.00194},
{1.0, 0.0689, 0.0177, 0.00718, 0.00381, 0.00244, 0.00179, 0.00146, 0.00129, 0.00121, 0.0012},
{1.0, 0.0645, 0.0156, 0.00596, 0.003, 0.00183, 0.00128, 0.001, 0.000852, 0.000775, 0.000744}, // 15
{1.0, 0.0606, 0.0138, 0.005, 0.00239, 0.00139, 0.000935, 0.000702, 0.000574, 0.000505, 0.00047, 0.000459},
{1.0, 0.0571, 0.0123, 0.00423, 0.00193, 0.00107, 0.000692, 0.000499, 0.000394, 0.000335, 0.000302, 0.000287, 0.000284},
{1.0, 0.054, 0.0111, 0.00362, 0.00158, 0.000839, 0.000519, 0.00036, 0.000275, 0.000226, 0.000198, 0.000183, 0.000176},
{1.0, 0.0513, 0.00998, 0.00312, 0.0013, 0.000663, 0.000394, 0.000264, 0.000194, 0.000155, 0.000132, 0.000118, 0.000111, 0.000109},
{1.0, 0.0488, 0.00906, 0.0027, 0.00108, 0.00053, 0.000303, 0.000196, 0.00014, 0.000108, 8.89e-05, 7.77e-05, 7.12e-05, 6.79e-05, 6.71e-05} // 20
};
for (size_t bits_per_row = 1; bits_per_row < MAX_BITS_PER_ROW; ++bits_per_row)
{
if (probability_lookup_table[bits_per_row][min_probability_index_each_bits[bits_per_row]] <= max_conflict_probability)
{
size_t max_size_of_hash_functions = min_probability_index_each_bits[bits_per_row];
for (size_t size_of_hash_functions = max_size_of_hash_functions; size_of_hash_functions > 0; --size_of_hash_functions)
if (probability_lookup_table[bits_per_row][size_of_hash_functions] > max_conflict_probability)
{
std::cout << "Best bf:" << bits_per_row << ", " << (size_of_hash_functions + 1) << "\n";
return std::pair<size_t, size_t>(bits_per_row, size_of_hash_functions + 1);
}
}
}
return std::pair<size_t, size_t>(MAX_BITS_PER_ROW - 1, min_probability_index_each_bits[MAX_BITS_PER_ROW - 1]);
}
std::pair<size_t, size_t> calculationBestPractices(double max_conflict_probability)
{
return calculationBestPracticesImpl(max_conflict_probability);
}
}

View File

@ -26,8 +26,8 @@ public:
void add(const char * data, size_t len);
void clear();
void addHashWithSeed(const UInt64 & hash, const UInt64 & seed);
bool containsWithSeed(const UInt64 & hash, const UInt64 & seed);
void addHashWithSeed(const UInt64 & hash, const UInt64 & hash_seed);
bool findHashWithSeed(const UInt64 & hash, const UInt64 & hash_seed);
/// Checks if this contains everything from another bloom filter.
/// Bloom filters must have equal size and seed.
@ -53,6 +53,4 @@ using BloomFilterPtr = std::shared_ptr<BloomFilter>;
bool operator== (const BloomFilter & a, const BloomFilter & b);
std::pair<size_t, size_t> calculationBestPractices(double max_conflict_probability);
}

View File

@ -34,7 +34,7 @@ struct BloomFilterHash
return ColumnConst::create(ColumnUInt64::create(1, intHash64(field.safeGet<UInt64>())), 1);
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())
else if (which.isFloat32() || which.isFloat64())
return ColumnConst::create(ColumnUInt64::create(1, intHash64(ext::bit_cast<UInt64>(field.safeGet<Float64>()))), 1);
else if (which.isString() || which.isFixedString())
{
@ -87,14 +87,31 @@ struct BloomFilterHash
const typename ColumnVector<Type>::Container & vec_from = index_column->getData();
for (size_t index = 0, size = vec.size(); index < size; ++index)
/// Because we're missing the precision of float in the Field.h
/// to be consistent, we need to convert Float32 to Float64 processing, also see: BloomFilterHash::hashWithField
if constexpr (std::is_same_v<ColumnVector<Type>, ColumnFloat32>)
{
UInt64 hash = intHash64(ext::bit_cast<UInt64>(vec_from[index + pos]));
for (size_t index = 0, size = vec.size(); index < size; ++index)
{
UInt64 hash = intHash64(ext::bit_cast<UInt64>(Float64(vec_from[index + pos])));
if constexpr (is_first)
vec[index] = hash;
else
vec[index] = CityHash_v1_0_2::Hash128to64(CityHash_v1_0_2::uint128(vec[index], hash));
if constexpr (is_first)
vec[index] = hash;
else
vec[index] = CityHash_v1_0_2::Hash128to64(CityHash_v1_0_2::uint128(vec[index], hash));
}
}
else
{
for (size_t index = 0, size = vec.size(); index < size; ++index)
{
UInt64 hash = intHash64(ext::bit_cast<UInt64>(vec_from[index + pos]));
if constexpr (is_first)
vec[index] = hash;
else
vec[index] = CityHash_v1_0_2::Hash128to64(CityHash_v1_0_2::uint128(vec[index], hash));
}
}
}
@ -138,6 +155,53 @@ struct BloomFilterHash
else
throw Exception("Illegal column type was passed to the bloom filter index.", ErrorCodes::ILLEGAL_COLUMN);
}
static std::pair<size_t, size_t> calculationBestPractices(double max_conflict_probability)
{
static const size_t MAX_BITS_PER_ROW = 20;
static const size_t MAX_HASH_FUNCTION_COUNT = 15;
/// For the smallest index per level in probability_lookup_table
static const size_t min_probability_index_each_bits[] = {0, 0, 1, 2, 3, 3, 4, 5, 6, 6, 7, 8, 8, 9, 10, 10, 11, 12, 12, 13, 14};
static const long double probability_lookup_table[MAX_BITS_PER_ROW + 1][MAX_HASH_FUNCTION_COUNT] =
{
{1.0}, /// dummy, 0 bits per row
{1.0, 1.0},
{1.0, 0.393, 0.400},
{1.0, 0.283, 0.237, 0.253},
{1.0, 0.221, 0.155, 0.147, 0.160},
{1.0, 0.181, 0.109, 0.092, 0.092, 0.101}, // 5
{1.0, 0.154, 0.0804, 0.0609, 0.0561, 0.0578, 0.0638},
{1.0, 0.133, 0.0618, 0.0423, 0.0359, 0.0347, 0.0364},
{1.0, 0.118, 0.0489, 0.0306, 0.024, 0.0217, 0.0216, 0.0229},
{1.0, 0.105, 0.0397, 0.0228, 0.0166, 0.0141, 0.0133, 0.0135, 0.0145},
{1.0, 0.0952, 0.0329, 0.0174, 0.0118, 0.00943, 0.00844, 0.00819, 0.00846}, // 10
{1.0, 0.0869, 0.0276, 0.0136, 0.00864, 0.0065, 0.00552, 0.00513, 0.00509},
{1.0, 0.08, 0.0236, 0.0108, 0.00646, 0.00459, 0.00371, 0.00329, 0.00314},
{1.0, 0.074, 0.0203, 0.00875, 0.00492, 0.00332, 0.00255, 0.00217, 0.00199, 0.00194},
{1.0, 0.0689, 0.0177, 0.00718, 0.00381, 0.00244, 0.00179, 0.00146, 0.00129, 0.00121, 0.0012},
{1.0, 0.0645, 0.0156, 0.00596, 0.003, 0.00183, 0.00128, 0.001, 0.000852, 0.000775, 0.000744}, // 15
{1.0, 0.0606, 0.0138, 0.005, 0.00239, 0.00139, 0.000935, 0.000702, 0.000574, 0.000505, 0.00047, 0.000459},
{1.0, 0.0571, 0.0123, 0.00423, 0.00193, 0.00107, 0.000692, 0.000499, 0.000394, 0.000335, 0.000302, 0.000287, 0.000284},
{1.0, 0.054, 0.0111, 0.00362, 0.00158, 0.000839, 0.000519, 0.00036, 0.000275, 0.000226, 0.000198, 0.000183, 0.000176},
{1.0, 0.0513, 0.00998, 0.00312, 0.0013, 0.000663, 0.000394, 0.000264, 0.000194, 0.000155, 0.000132, 0.000118, 0.000111, 0.000109},
{1.0, 0.0488, 0.00906, 0.0027, 0.00108, 0.00053, 0.000303, 0.000196, 0.00014, 0.000108, 8.89e-05, 7.77e-05, 7.12e-05, 6.79e-05, 6.71e-05} // 20
};
for (size_t bits_per_row = 1; bits_per_row < MAX_BITS_PER_ROW; ++bits_per_row)
{
if (probability_lookup_table[bits_per_row][min_probability_index_each_bits[bits_per_row]] <= max_conflict_probability)
{
size_t max_size_of_hash_functions = min_probability_index_each_bits[bits_per_row];
for (size_t size_of_hash_functions = max_size_of_hash_functions; size_of_hash_functions > 0; --size_of_hash_functions)
if (probability_lookup_table[bits_per_row][size_of_hash_functions] > max_conflict_probability)
return std::pair<size_t, size_t>(bits_per_row, size_of_hash_functions + 1);
}
}
return std::pair<size_t, size_t>(MAX_BITS_PER_ROW - 1, min_probability_index_each_bits[MAX_BITS_PER_ROW - 1]);
}
};
}

View File

@ -11,6 +11,7 @@
#include <Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h>
#include <Parsers/queryToString.h>
#include <Columns/ColumnConst.h>
#include <Interpreters/BloomFilterHash.h>
namespace DB
@ -97,7 +98,7 @@ std::unique_ptr<IMergeTreeIndex> bloomFilterIndexCreatorNew(const NamesAndTypesL
if (node->type->arguments && !node->type->arguments->children.empty())
max_conflict_probability = typeid_cast<const ASTLiteral &>(*node->type->arguments->children[0]).value.get<Float64>();
const auto & bits_per_row_and_size_of_hash_functions = calculationBestPractices(max_conflict_probability);
const auto & bits_per_row_and_size_of_hash_functions = BloomFilterHash::calculationBestPractices(max_conflict_probability);
return std::make_unique<MergeTreeIndexBloomFilter>(
node->name, std::move(index_expr), index_sample.getNames(), index_sample.getDataTypes(), index_sample, node->granularity,

View File

@ -51,7 +51,7 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr &
if (const_column)
{
for (size_t index = 0; index < hash_functions; ++index)
if (!bloom_filter->containsWithSeed(const_column->getValue<UInt64>(), BloomFilterHash::bf_hash_seed[index]))
if (!bloom_filter->findHashWithSeed(const_column->getValue<UInt64>(), BloomFilterHash::bf_hash_seed[index]))
return false;
return true;
}
@ -64,7 +64,7 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr &
{
bool match_row = true;
for (size_t hash_index = 0; match_row && hash_index < hash_functions; ++hash_index)
match_row = bloom_filter->containsWithSeed(data[index], BloomFilterHash::bf_hash_seed[hash_index]);
match_row = bloom_filter->findHashWithSeed(data[index], BloomFilterHash::bf_hash_seed[hash_index]);
missing_rows = !match_row;
}

View File

@ -0,0 +1,30 @@
1
0
1
1
2
0
2
2
2
0
2
2
2
0
2
2
1
1
1
1
1
1
1
1
1
1
1
1
1
1

View File

@ -11,11 +11,6 @@ SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) = (1, 2) SE
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) = (1, 1) 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 = 1 SETTINGS max_rows_to_read = 5; -- { serverError 158 }
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) = (1, 2) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) = (1, 1) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) = (1, (1, 1)) SETTINGS max_rows_to_read = 5; -- { serverError 158 }
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;
@ -29,26 +24,13 @@ WITH ((1, 2), (2, 3)) AS liter_prepared_set SELECT COUNT() FROM test.single_colu
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, 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;
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('Europe/Moscow'), str String, fixed_string FixedString(5), 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, 'Europe/Moscow') AS date, toDateTime(number, 'Europe/Moscow') AS date_time, toString(number) AS str, toFixedString(toString(number), 5) 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;
@ -58,10 +40,11 @@ SELECT COUNT() FROM test.bloom_filter_types_test WHERE u8 = 1 SETTINGS max_rows_
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;
SELECT COUNT() FROM test.bloom_filter_types_test WHERE f32 = 1 SETTINGS max_rows_to_read = 6;
SELECT COUNT() FROM test.bloom_filter_types_test WHERE f64 = 1 SETTINGS max_rows_to_read = 6;
SELECT COUNT() FROM test.bloom_filter_types_test WHERE date = '1970-01-02' SETTINGS max_rows_to_read = 6;
SELECT COUNT() FROM test.bloom_filter_types_test WHERE date_time = toDateTime('1970-01-01 03:00:01', 'Europe/Moscow') SETTINGS max_rows_to_read = 6;
SELECT COUNT() FROM test.bloom_filter_types_test WHERE str = '1' SETTINGS max_rows_to_read = 6;
SELECT COUNT() FROM test.bloom_filter_types_test WHERE fixed_string = toFixedString('1', 5) SETTINGS max_rows_to_read = 12;
DROP TABLE IF EXISTS test.bloom_filter_types_test;