Merge pull request #29254 from kitaisreal/map-bloom-filter-index-updated

Updated bloom filter index for map type to work with mapKeys, mapValues functions
This commit is contained in:
Maksim Kita 2021-09-23 00:31:14 +03:00 committed by GitHub
commit 1bf375e2b7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 296 additions and 75 deletions

View File

@ -1,11 +1,9 @@
#include <Interpreters/BloomFilter.h>
#include <city.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnLowCardinality.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
@ -126,14 +124,6 @@ DataTypePtr BloomFilter::getPrimitiveType(const DataTypePtr & data_type)
throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS);
}
if (const auto * map_type = typeid_cast<const DataTypeMap *>(data_type.get()))
{
if (!typeid_cast<const DataTypeMap *>(map_type->getKeyType().get()))
return getPrimitiveType(map_type->getKeyType());
else
throw Exception("Unexpected key type " + data_type->getName() + " of bloom filter index for map.", ErrorCodes::BAD_ARGUMENTS);
}
if (const auto * nullable_type = typeid_cast<const DataTypeNullable *>(data_type.get()))
return getPrimitiveType(nullable_type->getNestedType());
@ -148,9 +138,6 @@ ColumnPtr BloomFilter::getPrimitiveColumn(const ColumnPtr & column)
if (const auto * array_col = typeid_cast<const ColumnArray *>(column.get()))
return getPrimitiveColumn(array_col->getDataPtr());
if (const auto * map_col = typeid_cast<const ColumnMap *>(column.get()))
return getPrimitiveColumn(map_col->getNestedData().getColumnPtr(0));
if (const auto * nullable_col = typeid_cast<const ColumnNullable *>(column.get()))
return getPrimitiveColumn(nullable_col->getNestedColumnPtr());

View File

@ -5,7 +5,6 @@
#include <Columns/IColumn.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
@ -124,26 +123,6 @@ struct BloomFilterHash
}
}
if (which.isMap())
{
const auto * map_col = typeid_cast<const ColumnMap *>(column.get());
const auto & keys_data = map_col->getNestedData().getColumn(0);
if (checkAndGetColumn<ColumnNullable>(keys_data))
throw Exception("Unexpected key type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS);
const auto & offsets = map_col->getNestedColumn().getOffsets();
limit = offsets[pos + limit - 1] - offsets[pos - 1]; /// PaddedPODArray allows access on index -1.
pos = offsets[pos - 1];
if (limit == 0)
{
auto index_column = ColumnUInt64::create(1);
ColumnUInt64::Container & index_column_vec = index_column->getData();
index_column_vec[0] = 0;
return index_column;
}
}
const ColumnPtr actual_col = BloomFilter::getPrimitiveColumn(column);
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(data_type);

View File

@ -46,20 +46,29 @@ MergeTreeIndexGranulePtr MergeTreeIndexBloomFilter::createIndexGranule() const
bool MergeTreeIndexBloomFilter::mayBenefitFromIndexForIn(const ASTPtr & node) const
{
const String & column_name = node->getColumnName();
Names required_columns = index.expression->getRequiredColumns();
NameSet required_columns_set(required_columns.begin(), required_columns.end());
for (const auto & cname : index.column_names)
if (column_name == cname)
std::vector<ASTPtr> nodes_to_check;
nodes_to_check.emplace_back(node);
while (!nodes_to_check.empty())
{
auto node_to_check = nodes_to_check.back();
nodes_to_check.pop_back();
const auto & column_name = node_to_check->getColumnName();
if (required_columns_set.find(column_name) != required_columns_set.end())
return true;
if (const auto * func = typeid_cast<const ASTFunction *>(node.get()))
{
for (const auto & children : func->arguments->children)
if (mayBenefitFromIndexForIn(children))
return true;
if (const auto * function = typeid_cast<const ASTFunction *>(node_to_check.get()))
{
auto & function_arguments_children = function->arguments->children;
nodes_to_check.insert(nodes_to_check.end(), function_arguments_children.begin(), function_arguments_children.end());
}
}
return false;
return true;
}
MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator() const

View File

@ -276,7 +276,9 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B
if (functionIsInOrGlobalInOperator(function->name))
{
if (const auto & prepared_set = getPreparedSet(arguments[1]))
auto prepared_set = getPreparedSet(arguments[1]);
if (prepared_set)
{
if (traverseASTIn(function->name, arguments[0], prepared_set, out))
maybe_useful = true;
@ -285,6 +287,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B
else if (function->name == "equals" ||
function->name == "notEquals" ||
function->name == "has" ||
function->name == "mapContains" ||
function->name == "indexOf" ||
function->name == "hasAny" ||
function->name == "hasAll")
@ -308,14 +311,22 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B
}
bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
const String & function_name, const ASTPtr & key_ast, const SetPtr & prepared_set, RPNElement & out)
const String & function_name,
const ASTPtr & key_ast,
const SetPtr & prepared_set,
RPNElement & out)
{
const auto prepared_info = getPreparedSetInfo(prepared_set);
return traverseASTIn(function_name, key_ast, prepared_info.type, prepared_info.column, out);
return traverseASTIn(function_name, key_ast, prepared_set, prepared_info.type, prepared_info.column, out);
}
bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
const String & function_name, const ASTPtr & key_ast, const DataTypePtr & type, const ColumnPtr & column, RPNElement & out)
const String & function_name,
const ASTPtr & key_ast,
const SetPtr & prepared_set,
const DataTypePtr & type,
const ColumnPtr & column,
RPNElement & out)
{
if (header.has(key_ast->getColumnName()))
{
@ -352,10 +363,83 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
const auto & sub_data_types = tuple_data_type->getElements();
for (size_t index = 0; index < arguments.size(); ++index)
match_with_subtype |= traverseASTIn(function_name, arguments[index], sub_data_types[index], sub_columns[index], out);
match_with_subtype |= traverseASTIn(function_name, arguments[index], nullptr, sub_data_types[index], sub_columns[index], out);
return match_with_subtype;
}
if (function->name == "arrayElement")
{
/** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] IN ('') because if key does not exists in map
* we return default value for arrayElement.
*
* We cannot skip keys that does not exist in map if comparison is with default type value because
* that way we skip necessary granules where map key does not exists.
*/
if (!prepared_set)
return false;
auto default_column_to_check = type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst();
ColumnWithTypeAndName default_column_with_type_to_check { default_column_to_check, type, "" };
ColumnsWithTypeAndName default_columns_with_type_to_check = {default_column_with_type_to_check};
auto set_contains_default_value_predicate_column = prepared_set->execute(default_columns_with_type_to_check, false /*negative*/);
const auto & set_contains_default_value_predicate_column_typed = assert_cast<const ColumnUInt8 &>(*set_contains_default_value_predicate_column);
bool set_contain_default_value = set_contains_default_value_predicate_column_typed.getData()[0];
if (set_contain_default_value)
return false;
const auto & col_name = assert_cast<ASTIdentifier *>(function->arguments.get()->children[0].get())->name();
auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name);
auto map_values_index_column_name = fmt::format("mapValues({})", col_name);
if (header.has(map_keys_index_column_name))
{
/// For mapKeys we serialize key argument with bloom filter
auto & argument = function->arguments.get()->children[1];
if (const auto * literal = argument->as<ASTLiteral>())
{
size_t position = header.getPositionByName(map_keys_index_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type;
auto element_key = literal->value;
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key)));
return true;
}
else
{
return false;
}
}
else if (header.has(map_values_index_column_name))
{
/// For mapValues we serialize set with bloom filter
size_t row_size = column->size();
size_t position = header.getPositionByName(map_values_index_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type;
const auto & array_type = assert_cast<const DataTypeArray &>(*index_type);
const auto & array_nested_type = array_type.getNestedType();
const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, array_nested_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(array_nested_type, converted_column, 0, row_size)));
}
else
{
return false;
}
if (function_name == "in" || function_name == "globalIn")
out.function = RPNElement::FUNCTION_IN;
if (function_name == "notIn" || function_name == "globalNotIn")
out.function = RPNElement::FUNCTION_NOT_IN;
return true;
}
}
return false;
@ -420,7 +504,12 @@ static bool indexOfCanUseBloomFilter(const ASTPtr & parent)
bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
const String & function_name, const ASTPtr & key_ast, const DataTypePtr & value_type, const Field & value_field, RPNElement & out, const ASTPtr & parent)
const String & function_name,
const ASTPtr & key_ast,
const DataTypePtr & value_type,
const Field & value_field,
RPNElement & out,
const ASTPtr & parent)
{
if (header.has(key_ast->getColumnName()))
{
@ -488,6 +577,29 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
return true;
}
if (function_name == "mapContains")
{
const auto & col_name = assert_cast<ASTIdentifier *>(key_ast.get())->name();
auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name);
if (!header.has(map_keys_index_column_name))
return false;
size_t position = header.getPositionByName(map_keys_index_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type;
const auto * array_type = typeid_cast<const DataTypeArray *>(index_type.get());
if (!array_type)
return false;
out.function = RPNElement::FUNCTION_HAS;
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType());
Field converted_field = convertFieldToType(value_field, *actual_type, value_type.get());
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field)));
return true;
}
if (const auto * function = key_ast->as<ASTFunction>())
{
WhichDataType which(value_type);
@ -512,33 +624,38 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
if (function->name == "arrayElement")
{
/** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map
* we return default value for arrayElement.
*
* We cannot skip keys that does not exist in map if comparison is with default type value because
* that way we skip necessary granules where map key does not exists.
*/
if (value_field == value_type->getDefault())
return false;
const auto & col_name = assert_cast<ASTIdentifier *>(function->arguments.get()->children[0].get())->name();
if (header.has(col_name))
auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name);
if (!header.has(map_keys_index_column_name))
return false;
size_t position = header.getPositionByName(map_keys_index_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type;
out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS;
auto & argument = function->arguments.get()->children[1];
if (const auto * literal = argument->as<ASTLiteral>())
{
size_t position = header.getPositionByName(col_name);
const DataTypePtr & index_type = header.getByPosition(position).type;
const auto * map_type = typeid_cast<const DataTypeMap *>(index_type.get());
if (map_type)
{
out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS;
auto & argument = function->arguments.get()->children[1];
if (const auto * literal = argument->as<ASTLiteral>())
{
auto element_key = literal->value;
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key)));
}
else
{
return false;
}
return true;
}
auto element_key = literal->value;
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key)));
return true;
}
return false;
}
}

View File

@ -70,13 +70,27 @@ private:
bool traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out, const ASTPtr & parent);
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 SetPtr & prepared_set,
RPNElement & out);
bool traverseASTIn(
const String & function_name, const ASTPtr & key_ast, const DataTypePtr & type, const ColumnPtr & column, RPNElement & out);
const String & function_name,
const ASTPtr & key_ast,
const SetPtr & prepared_set,
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, const ASTPtr & parent);
const String & function_name,
const ASTPtr & key_ast,
const DataTypePtr & value_type,
const Field & value_field,
RPNElement & out,
const ASTPtr & parent);
};
}

View File

@ -0,0 +1,42 @@
Map bloom filter mapKeys
Equals with existing key
0 {'K0':'V0'}
Equals with non existing key
Equals with non existing key and default value
0 {'K0':'V0'}
1 {'K1':'V1'}
Not equals with existing key
1 {'K1':'V1'}
Not equals with non existing key
0 {'K0':'V0'}
1 {'K1':'V1'}
Not equals with non existing key and default value
IN with existing key
0 {'K0':'V0'}
IN with non existing key
IN with non existing key and default value
0 {'K0':'V0'}
1 {'K1':'V1'}
NOT IN with existing key
1 {'K1':'V1'}
NOT IN with non existing key
0 {'K0':'V0'}
1 {'K1':'V1'}
NOT IN with non existing key and default value
Map bloom filter mapValues
IN with existing key
0 {'K0':'V0'}
IN with non existing key
IN with non existing key and default value
0 {'K0':'V0'}
1 {'K1':'V1'}
NOT IN with existing key
1 {'K1':'V1'}
NOT IN with non existing key
0 {'K0':'V0'}
1 {'K1':'V1'}
NOT IN with non existing key and default value
MapContains with existing key
0 {'K0':'V0'}
MapContains with non existing key
MapContains with non existing key and default value

View File

@ -0,0 +1,73 @@
DROP TABLE IF EXISTS map_test_index_map_keys;
CREATE TABLE map_test_index_map_keys
(
row_id UInt32,
map Map(String, String),
INDEX map_bloom_filter_keys mapKeys(map) TYPE bloom_filter GRANULARITY 1
) Engine=MergeTree() ORDER BY row_id;
INSERT INTO map_test_index_map_keys VALUES (0, {'K0':'V0'}), (1, {'K1':'V1'});
SELECT 'Map bloom filter mapKeys';
SELECT 'Equals with existing key';
SELECT * FROM map_test_index_map_keys WHERE map['K0'] = 'V0';
SELECT 'Equals with non existing key';
SELECT * FROM map_test_index_map_keys WHERE map['K2'] = 'V2';
SELECT 'Equals with non existing key and default value';
SELECT * FROM map_test_index_map_keys WHERE map['K3'] = '';
SELECT 'Not equals with existing key';
SELECT * FROM map_test_index_map_keys WHERE map['K0'] != 'V0';
SELECT 'Not equals with non existing key';
SELECT * FROM map_test_index_map_keys WHERE map['K2'] != 'V2';
SELECT 'Not equals with non existing key and default value';
SELECT * FROM map_test_index_map_keys WHERE map['K3'] != '';
SELECT 'IN with existing key';
SELECT * FROM map_test_index_map_keys WHERE map['K0'] IN 'V0';
SELECT 'IN with non existing key';
SELECT * FROM map_test_index_map_keys WHERE map['K2'] IN 'V2';
SELECT 'IN with non existing key and default value';
SELECT * FROM map_test_index_map_keys WHERE map['K3'] IN '';
SELECT 'NOT IN with existing key';
SELECT * FROM map_test_index_map_keys WHERE map['K0'] NOT IN 'V0';
SELECT 'NOT IN with non existing key';
SELECT * FROM map_test_index_map_keys WHERE map['K2'] NOT IN 'V2';
SELECT 'NOT IN with non existing key and default value';
SELECT * FROM map_test_index_map_keys WHERE map['K3'] NOT IN '';
DROP TABLE map_test_index_map_keys;
DROP TABLE IF EXISTS map_test_index_map_values;
CREATE TABLE map_test_index_map_values
(
row_id UInt32,
map Map(String, String),
INDEX map_bloom_filter_values mapValues(map) TYPE bloom_filter GRANULARITY 1
) Engine=MergeTree() ORDER BY row_id;
INSERT INTO map_test_index_map_values VALUES (0, {'K0':'V0'}), (1, {'K1':'V1'});
SELECT 'Map bloom filter mapValues';
SELECT 'IN with existing key';
SELECT * FROM map_test_index_map_values WHERE map['K0'] IN 'V0';
SELECT 'IN with non existing key';
SELECT * FROM map_test_index_map_values WHERE map['K2'] IN 'V2';
SELECT 'IN with non existing key and default value';
SELECT * FROM map_test_index_map_values WHERE map['K3'] IN '';
SELECT 'NOT IN with existing key';
SELECT * FROM map_test_index_map_values WHERE map['K0'] NOT IN 'V0';
SELECT 'NOT IN with non existing key';
SELECT * FROM map_test_index_map_values WHERE map['K2'] NOT IN 'V2';
SELECT 'NOT IN with non existing key and default value';
SELECT * FROM map_test_index_map_values WHERE map['K3'] NOT IN '';
SELECT 'MapContains with existing key';
SELECT * FROM map_test_index_map_values WHERE mapContains(map, 'K0');
SELECT 'MapContains with non existing key';
SELECT * FROM map_test_index_map_values WHERE mapContains(map, 'K2');
SELECT 'MapContains with non existing key and default value';
SELECT * FROM map_test_index_map_values WHERE mapContains(map, 'K3');
DROP TABLE map_test_index_map_values;