mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
Added function dictGetOrNull
This commit is contained in:
parent
6acb9bc827
commit
f2e6a0a97d
@ -40,6 +40,7 @@ void registerFunctionsExternalDictionaries(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionDictGetStringOrDefault>();
|
||||
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::get>>();
|
||||
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::getOrDefault>>();
|
||||
factory.registerFunction<FunctionDictGetOrNull>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,6 +19,7 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
|
||||
#include <Access/AccessFlags.h>
|
||||
|
||||
@ -148,7 +149,6 @@ public:
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
private:
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool isVariadic() const override { return true; }
|
||||
|
||||
@ -238,6 +238,7 @@ private:
|
||||
return dictionary->hasKeys({key_column, range_col}, {std::make_shared<DataTypeUInt64>(), range_col_type});
|
||||
}
|
||||
|
||||
private:
|
||||
mutable FunctionDictHelper helper;
|
||||
};
|
||||
|
||||
@ -302,7 +303,7 @@ public:
|
||||
}
|
||||
|
||||
if (types.size() > 1)
|
||||
return std::make_shared<DataTypeTuple>(types);
|
||||
return std::make_shared<DataTypeTuple>(types, attribute_names);
|
||||
else
|
||||
return types.front();
|
||||
}
|
||||
@ -701,6 +702,147 @@ using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault<DataTypeDecim
|
||||
using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrDefault>;
|
||||
using FunctionDictGetStringOrDefault = FunctionDictGetOrDefault<DataTypeString, NameDictGetStringOrDefault>;
|
||||
|
||||
class FunctionDictGetOrNull final : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "dictGetOrNull";
|
||||
|
||||
static FunctionPtr create(const Context &context)
|
||||
{
|
||||
return std::make_shared<FunctionDictGetOrNull>(context);
|
||||
}
|
||||
|
||||
explicit FunctionDictGetOrNull(const Context & context_)
|
||||
: dictionary_get_func_impl(context_)
|
||||
, dictionary_has_func_impl(context_)
|
||||
{}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
private:
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; }
|
||||
|
||||
bool isInjective(const ColumnsWithTypeAndName & sample_columns) const override
|
||||
{
|
||||
return dictionary_get_func_impl.isInjective(sample_columns);
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
auto result_type = dictionary_get_func_impl.getReturnTypeImpl(arguments);
|
||||
|
||||
WhichDataType result_data_type(result_type);
|
||||
if (result_data_type.isTuple())
|
||||
{
|
||||
const auto & data_type_tuple = static_cast<const DataTypeTuple &>(*result_type);
|
||||
auto elements_types_copy = data_type_tuple.getElements();
|
||||
for (auto & element_type : elements_types_copy)
|
||||
element_type = makeNullable(element_type);
|
||||
|
||||
result_type = std::make_shared<DataTypeTuple>(elements_types_copy, data_type_tuple.getElementNames());
|
||||
}
|
||||
else
|
||||
result_type = makeNullable(result_type);
|
||||
|
||||
return result_type;
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
auto dictionary_name_argument_column = arguments[0];
|
||||
auto keys_argument_column = arguments[2];
|
||||
|
||||
ColumnsWithTypeAndName dict_has_arguments;
|
||||
dict_has_arguments.reserve(arguments.size() - 1);
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
if (i == 1)
|
||||
continue;
|
||||
|
||||
dict_has_arguments.emplace_back(arguments[i]);
|
||||
}
|
||||
|
||||
auto is_key_in_dictionary_column = dictionary_has_func_impl.executeImpl(dict_has_arguments, std::make_shared<DataTypeUInt8>(), input_rows_count);
|
||||
auto is_key_in_dictionary_column_mutable = is_key_in_dictionary_column->assumeMutable();
|
||||
ColumnVector<UInt8> & is_key_in_dictionary_column_typed = assert_cast<ColumnVector<UInt8> &>(*is_key_in_dictionary_column_mutable);
|
||||
auto & is_key_in_dictionary_data = is_key_in_dictionary_column_typed.getData();
|
||||
|
||||
auto result_type = dictionary_get_func_impl.getReturnTypeImpl(arguments);
|
||||
auto dictionary_get_result_column = dictionary_get_func_impl.executeImpl(arguments, result_type, input_rows_count);
|
||||
|
||||
for (auto & key : is_key_in_dictionary_data)
|
||||
key = !key;
|
||||
|
||||
ColumnPtr result;
|
||||
|
||||
WhichDataType result_data_type(result_type);
|
||||
auto dictionary_get_result_column_mutable = dictionary_get_result_column->assumeMutable();
|
||||
|
||||
if (result_data_type.isTuple())
|
||||
{
|
||||
ColumnTuple & column_tuple = assert_cast<ColumnTuple &>(*dictionary_get_result_column_mutable);
|
||||
|
||||
const auto & columns = column_tuple.getColumns();
|
||||
size_t tuple_size = columns.size();
|
||||
|
||||
MutableColumns new_columns(tuple_size);
|
||||
for (size_t tuple_column_index = 0; tuple_column_index < tuple_size; ++tuple_column_index)
|
||||
{
|
||||
auto nullable_column_map = ColumnVector<UInt8>::create();
|
||||
auto & nullable_column_map_data = nullable_column_map->getData();
|
||||
nullable_column_map_data.assign(is_key_in_dictionary_data);
|
||||
|
||||
auto mutable_column = columns[tuple_column_index]->assumeMutable();
|
||||
if (ColumnNullable * nullable_column = typeid_cast<ColumnNullable *>(mutable_column.get()))
|
||||
{
|
||||
auto & null_map_data = nullable_column->getNullMapData();
|
||||
addNullMap(null_map_data, is_key_in_dictionary_data);
|
||||
new_columns[tuple_column_index] = std::move(mutable_column);
|
||||
}
|
||||
else
|
||||
new_columns[tuple_column_index] = ColumnNullable::create(std::move(mutable_column), std::move(nullable_column_map));
|
||||
}
|
||||
|
||||
result = ColumnTuple::create(std::move(new_columns));
|
||||
}
|
||||
else
|
||||
{
|
||||
if (ColumnNullable * nullable_column = typeid_cast<ColumnNullable *>(dictionary_get_result_column_mutable.get()))
|
||||
{
|
||||
auto & null_map_data = nullable_column->getNullMapData();
|
||||
addNullMap(null_map_data, is_key_in_dictionary_data);
|
||||
result = std::move(dictionary_get_result_column);
|
||||
}
|
||||
else
|
||||
result = ColumnNullable::create(std::move(dictionary_get_result_column), std::move(is_key_in_dictionary_column_mutable));
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
static void addNullMap(PaddedPODArray<UInt8> & null_map, PaddedPODArray<UInt8> & null_map_to_add)
|
||||
{
|
||||
assert(null_map.size() == null_map_to_add.size());
|
||||
|
||||
for (size_t i = 0; i < null_map.size(); ++i)
|
||||
null_map[i] = null_map[i] || null_map_to_add[i];
|
||||
}
|
||||
|
||||
const FunctionDictGetNoType<DictionaryGetFunctionType::get> dictionary_get_func_impl;
|
||||
const FunctionDictHas dictionary_has_func_impl;
|
||||
};
|
||||
/// Functions to work with hierarchies.
|
||||
|
||||
class FunctionDictGetHierarchy final : public IFunction
|
||||
|
18
tests/queries/0_stateless/01780_dict_get_or_null.reference
Normal file
18
tests/queries/0_stateless/01780_dict_get_or_null.reference
Normal file
@ -0,0 +1,18 @@
|
||||
Simple key dictionary dictGetOrNull
|
||||
0 0 \N \N (NULL,NULL)
|
||||
1 1 First First ('First','First')
|
||||
2 1 Second \N ('Second',NULL)
|
||||
3 1 Third Third ('Third','Third')
|
||||
4 0 \N \N (NULL,NULL)
|
||||
Complex key dictionary dictGetOrNull
|
||||
(0,'key') 0 \N \N (NULL,NULL)
|
||||
(1,'key') 1 First First ('First','First')
|
||||
(2,'key') 1 Second \N ('Second',NULL)
|
||||
(3,'key') 1 Third Third ('Third','Third')
|
||||
(4,'key') 0 \N \N (NULL,NULL)
|
||||
Range key dictionary dictGetOrNull
|
||||
(0,'2019-05-20') 0 \N \N (NULL,NULL)
|
||||
(1,'2019-05-20') 1 First First ('First','First')
|
||||
(2,'2019-05-20') 1 Second \N ('Second',NULL)
|
||||
(3,'2019-05-20') 1 Third Third ('Third','Third')
|
||||
(4,'2019-05-20') 0 \N \N (NULL,NULL)
|
116
tests/queries/0_stateless/01780_dict_get_or_null.sql
Normal file
116
tests/queries/0_stateless/01780_dict_get_or_null.sql
Normal file
@ -0,0 +1,116 @@
|
||||
DROP TABLE IF EXISTS simple_key_dictionary_source_table;
|
||||
CREATE TABLE simple_key_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value String,
|
||||
value_nullable Nullable(String)
|
||||
) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO simple_key_dictionary_source_table VALUES (1, 'First', 'First');
|
||||
INSERT INTO simple_key_dictionary_source_table VALUES (2, 'Second', NULL);
|
||||
INSERT INTO simple_key_dictionary_source_table VALUES (3, 'Third', 'Third');
|
||||
|
||||
DROP DICTIONARY IF EXISTS simple_key_dictionary;
|
||||
CREATE DICTIONARY simple_key_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String,
|
||||
value_nullable Nullable(String)
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'simple_key_dictionary_source_table'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT 'Simple key dictionary dictGetOrNull';
|
||||
|
||||
SELECT
|
||||
number,
|
||||
dictHas('simple_key_dictionary', number),
|
||||
dictGetOrNull('simple_key_dictionary', 'value', number),
|
||||
dictGetOrNull('simple_key_dictionary', 'value_nullable', number),
|
||||
dictGetOrNull('simple_key_dictionary', ('value', 'value_nullable'), number)
|
||||
FROM system.numbers LIMIT 5;
|
||||
|
||||
DROP DICTIONARY simple_key_dictionary;
|
||||
DROP TABLE simple_key_dictionary_source_table;
|
||||
|
||||
DROP TABLE IF EXISTS complex_key_dictionary_source_table;
|
||||
CREATE TABLE complex_key_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value String,
|
||||
value_nullable Nullable(String)
|
||||
) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO complex_key_dictionary_source_table VALUES (1, 'key', 'First', 'First');
|
||||
INSERT INTO complex_key_dictionary_source_table VALUES (2, 'key', 'Second', NULL);
|
||||
INSERT INTO complex_key_dictionary_source_table VALUES (3, 'key', 'Third', 'Third');
|
||||
|
||||
DROP DICTIONARY IF EXISTS complex_key_dictionary;
|
||||
CREATE DICTIONARY complex_key_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value String,
|
||||
value_nullable Nullable(String)
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'complex_key_dictionary_source_table'))
|
||||
LAYOUT(COMPLEX_KEY_DIRECT());
|
||||
|
||||
SELECT 'Complex key dictionary dictGetOrNull';
|
||||
|
||||
SELECT
|
||||
(number, 'key'),
|
||||
dictHas('complex_key_dictionary', (number, 'key')),
|
||||
dictGetOrNull('complex_key_dictionary', 'value', (number, 'key')),
|
||||
dictGetOrNull('complex_key_dictionary', 'value_nullable', (number, 'key')),
|
||||
dictGetOrNull('complex_key_dictionary', ('value', 'value_nullable'), (number, 'key'))
|
||||
FROM system.numbers LIMIT 5;
|
||||
|
||||
DROP DICTIONARY complex_key_dictionary;
|
||||
DROP TABLE complex_key_dictionary_source_table;
|
||||
|
||||
DROP TABLE IF EXISTS range_key_dictionary_source_table;
|
||||
CREATE TABLE range_key_dictionary_source_table
|
||||
(
|
||||
key UInt64,
|
||||
start_date Date,
|
||||
end_date Date,
|
||||
value String,
|
||||
value_nullable Nullable(String)
|
||||
)
|
||||
ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO range_key_dictionary_source_table VALUES(1, toDate('2019-05-20'), toDate('2019-05-20'), 'First', 'First');
|
||||
INSERT INTO range_key_dictionary_source_table VALUES(2, toDate('2019-05-20'), toDate('2019-05-20'), 'Second', NULL);
|
||||
INSERT INTO range_key_dictionary_source_table VALUES(3, toDate('2019-05-20'), toDate('2019-05-20'), 'Third', 'Third');
|
||||
|
||||
DROP DICTIONARY IF EXISTS range_key_dictionary;
|
||||
CREATE DICTIONARY range_key_dictionary
|
||||
(
|
||||
key UInt64,
|
||||
start_date Date,
|
||||
end_date Date,
|
||||
value String,
|
||||
value_nullable Nullable(String)
|
||||
)
|
||||
PRIMARY KEY key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'range_key_dictionary_source_table'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(RANGE_HASHED())
|
||||
RANGE(MIN start_date MAX end_date);
|
||||
|
||||
SELECT 'Range key dictionary dictGetOrNull';
|
||||
|
||||
SELECT
|
||||
(number, toDate('2019-05-20')),
|
||||
dictHas('range_key_dictionary', number, toDate('2019-05-20')),
|
||||
dictGetOrNull('range_key_dictionary', 'value', number, toDate('2019-05-20')),
|
||||
dictGetOrNull('range_key_dictionary', 'value_nullable', number, toDate('2019-05-20')),
|
||||
dictGetOrNull('range_key_dictionary', ('value', 'value_nullable'), number, toDate('2019-05-20'))
|
||||
FROM system.numbers LIMIT 5;
|
||||
|
||||
DROP DICTIONARY range_key_dictionary;
|
||||
DROP TABLE range_key_dictionary_source_table;
|
Loading…
Reference in New Issue
Block a user