mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
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:
commit
1bf375e2b7
@ -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());
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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
|
73
tests/queries/0_stateless/2021_map_bloom_filter_index.sql
Normal file
73
tests/queries/0_stateless/2021_map_bloom_filter_index.sql
Normal 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;
|
Loading…
Reference in New Issue
Block a user