Added function dictGetOrNull

This commit is contained in:
Maksim Kita 2021-03-31 17:46:13 +03:00
parent 6acb9bc827
commit f2e6a0a97d
4 changed files with 279 additions and 2 deletions

View File

@ -40,6 +40,7 @@ void registerFunctionsExternalDictionaries(FunctionFactory & factory)
factory.registerFunction<FunctionDictGetStringOrDefault>();
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::get>>();
factory.registerFunction<FunctionDictGetNoType<DictionaryGetFunctionType::getOrDefault>>();
factory.registerFunction<FunctionDictGetOrNull>();
}
}

View File

@ -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

View 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)

View 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;