ClickHouse/src/Functions/FunctionsExternalDictionaries.h

1006 lines
44 KiB
C++
Raw Normal View History

#pragma once
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
2017-07-13 20:58:19 +00:00
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnTuple.h>
#include <Access/AccessFlags.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
2021-02-27 20:39:34 +00:00
#include <Interpreters/castColumn.h>
#include <Functions/IFunctionImpl.h>
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
#include <Functions/FunctionHelpers.h>
#include <Dictionaries/FlatDictionary.h>
#include <Dictionaries/HashedDictionary.h>
#include <Dictionaries/CacheDictionary.h>
#include <Dictionaries/ComplexKeyHashedDictionary.h>
#include <Dictionaries/RangeHashedDictionary.h>
#include <Dictionaries/IPAddressDictionary.h>
2020-02-18 13:30:09 +00:00
#include <Dictionaries/PolygonDictionaryImplementations.h>
#include <Dictionaries/DirectDictionary.h>
2017-06-06 17:18:32 +00:00
#include <ext/range.h>
2015-04-16 06:12:35 +00:00
#include <type_traits>
namespace DB
{
2016-01-12 02:21:15 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int UNSUPPORTED_METHOD;
extern const int UNKNOWN_TYPE;
2017-06-13 02:06:53 +00:00
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
2021-01-21 14:42:50 +00:00
extern const int TYPE_MISMATCH;
2016-01-12 02:21:15 +00:00
}
/** Functions that use plug-ins (external) dictionaries_loader.
*
2017-05-13 22:19:04 +00:00
* Get the value of the attribute of the specified type.
* dictGetType(dictionary, attribute, id),
2017-05-13 22:19:04 +00:00
* Type - placeholder for the type name, any numeric and string types are currently supported.
* The type must match the actual attribute type with which it was declared in the dictionary structure.
*
2017-05-13 22:19:04 +00:00
* Get an array of identifiers, consisting of the source and parents chain.
* dictGetHierarchy(dictionary, id).
*
2017-05-13 22:19:04 +00:00
* Is the first identifier the child of the second.
* dictIsIn(dictionary, child_id, parent_id).
*/
class FunctionDictHelper
{
public:
2021-03-19 12:47:27 +00:00
explicit FunctionDictHelper(const Context & context_) : context(context_) {}
std::shared_ptr<const IDictionaryBase> getDictionary(const String & dictionary_name)
{
2021-03-19 12:47:27 +00:00
auto dict = context.getExternalDictionariesLoader().getDictionary(dictionary_name);
if (!access_checked)
{
context.checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getDictionaryID().getTableName());
access_checked = true;
}
return dict;
}
std::shared_ptr<const IDictionaryBase> getDictionary(const ColumnWithTypeAndName & column)
{
2020-10-18 14:49:19 +00:00
const auto * dict_name_col = checkAndGetColumnConst<ColumnString>(column.column.get());
return getDictionary(dict_name_col->getValue<String>());
}
2020-10-14 14:04:50 +00:00
bool isDictGetFunctionInjective(const Block & sample_columns)
{
/// Assume non-injective by default
2020-10-14 14:04:50 +00:00
if (!sample_columns)
return false;
if (sample_columns.columns() < 3)
throw Exception{"Wrong arguments count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
2020-10-18 14:49:19 +00:00
const auto * dict_name_col = checkAndGetColumnConst<ColumnString>(sample_columns.getByPosition(0).column.get());
if (!dict_name_col)
throw Exception{"First argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
2020-10-18 14:49:19 +00:00
const auto * attr_name_col = checkAndGetColumnConst<ColumnString>(sample_columns.getByPosition(1).column.get());
if (!attr_name_col)
throw Exception{"Second argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
return getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>());
}
DictionaryStructure getDictionaryStructure(const String & dictionary_name) const
{
2021-03-19 12:47:27 +00:00
return context.getExternalDictionariesLoader().getDictionaryStructure(dictionary_name);
}
private:
2021-03-19 12:47:27 +00:00
const Context & context;
/// Access cannot be not granted, since in this case checkAccess() will throw and access_checked will not be updated.
std::atomic<bool> access_checked = false;
/// We must not cache dictionary or dictionary's structure here, because there are places
/// where ExpressionActionsPtr is cached (StorageDistributed caching it for sharding_key_expr and
/// optimize_skip_unused_shards), and if the dictionary will be cached within "query" then
/// cached ExpressionActionsPtr will always have first version of the query and the dictionary
/// will not be updated after reload (see https://github.com/ClickHouse/ClickHouse/pull/16205)
};
class FunctionDictHas final : public IFunction
{
public:
static constexpr auto name = "dictHas";
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictHas>(context);
}
2020-10-18 14:49:19 +00:00
explicit FunctionDictHas(const Context & context_) : helper(context_) {}
String getName() const override { return name; }
private:
size_t getNumberOfArguments() const override { return 0; }
bool isVariadic() const override { return true; }
bool isDeterministic() const override { return false; }
2018-03-16 01:22:02 +00:00
bool useDefaultImplementationForConstants() const final { return true; }
2018-03-16 01:22:02 +00:00
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() < 2)
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (!isString(arguments[0]))
2018-03-16 03:03:17 +00:00
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!WhichDataType(arguments[1]).isUInt64() &&
!isTuple(arguments[1]))
2018-03-16 03:03:17 +00:00
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeUInt8>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
2020-10-14 14:04:50 +00:00
/** Do not require existence of the dictionary if the function is called for empty columns.
* This is needed to allow successful query analysis on a server,
* that is the initiator of a distributed query,
* in the case when the function will be invoked for real data only at the remote servers.
* This feature is controversial and implemented specially
* for backward compatibility with the case in Yandex Banner System.
*/
2018-09-14 18:55:12 +00:00
if (input_rows_count == 0)
2020-10-18 14:49:19 +00:00
return result_type->createColumn();
2020-12-19 13:24:51 +00:00
auto dictionary = helper.getDictionary(arguments[0]);
auto dictionary_key_type = dictionary->getKeyType();
2019-09-11 11:14:55 +00:00
const ColumnWithTypeAndName & key_column_with_type = arguments[1];
const auto key_column = key_column_with_type.column;
const auto key_column_type = WhichDataType(key_column_with_type.type);
2020-10-18 14:49:19 +00:00
ColumnPtr range_col = nullptr;
DataTypePtr range_col_type = nullptr;
if (dictionary_key_type == DictionaryKeyType::range)
{
if (arguments.size() != 3)
throw Exception{"Wrong argument count for function " + getName()
+ " when dictionary has key type range", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
range_col = arguments[2].column;
range_col_type = arguments[2].type;
if (!(range_col_type->isValueRepresentedByInteger() && range_col_type->getSizeOfValueInMemory() <= sizeof(Int64)))
throw Exception{"Illegal type " + range_col_type->getName() + " of fourth argument of function "
+ getName() + " must be convertible to Int64.",
ErrorCodes::ILLEGAL_COLUMN};
}
if (dictionary_key_type == DictionaryKeyType::simple)
{
if (!key_column_type.isUInt64())
throw Exception{"Second argument of function " + getName() + " must be " + dictionary->getStructure().getKeyDescription(),
ErrorCodes::TYPE_MISMATCH};
return dictionary->hasKeys({key_column}, {std::make_shared<DataTypeUInt64>()});
}
else if (dictionary_key_type == DictionaryKeyType::complex)
{
if (!key_column_type.isTuple())
throw Exception{"Second argument of function " + getName() + " must be " + dictionary->getStructure().getKeyDescription(),
ErrorCodes::TYPE_MISMATCH};
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column_full = key_column->convertToFullColumnIfConst();
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_column_full).getColumnsCopy();
const auto & key_types = static_cast<const DataTypeTuple &>(*key_column_with_type.type).getElements();
return dictionary->hasKeys(key_columns, key_types);
}
else
return dictionary->hasKeys({key_column, range_col}, {std::make_shared<DataTypeUInt64>(), range_col_type});
}
mutable FunctionDictHelper helper;
};
enum class DictionaryGetFunctionType
{
get,
getOrDefault
};
/// This variant of function derives the result type automatically.
template <DictionaryGetFunctionType dictionary_get_function_type>
class FunctionDictGetNoType final : public IFunction
{
public:
static constexpr auto name = dictionary_get_function_type == DictionaryGetFunctionType::get ? "dictGet" : "dictGetOrDefault";
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictGetNoType>(context);
}
explicit FunctionDictGetNoType(const Context & context_) : helper(context_) {}
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
2018-03-16 03:03:17 +00:00
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
2018-03-16 03:03:17 +00:00
bool isDeterministic() const override { return false; }
bool isInjective(const ColumnsWithTypeAndName & sample_columns) const override
{
return helper.isDictGetFunctionInjective(sample_columns);
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() < 3)
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
String dictionary_name;
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()))
dictionary_name = name_col->getValue<String>();
else
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName()
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
Strings attribute_names = getAttributeNamesFromColumn(arguments[1].column, arguments[1].type);
DataTypes types;
auto dictionary_structure = helper.getDictionaryStructure(dictionary_name);
for (auto & attribute_name : attribute_names)
{
/// We're extracting the return type from the dictionary's config, without loading the dictionary.
auto attribute = dictionary_structure.getAttribute(attribute_name);
types.emplace_back(attribute.type);
}
if (types.size() > 1)
return std::make_shared<DataTypeTuple>(types);
else
return types.front();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
if (input_rows_count == 0)
return result_type->createColumn();
String dictionary_name;
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()))
dictionary_name = name_col->getValue<String>();
else
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName()
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
Strings attribute_names = getAttributeNamesFromColumn(arguments[1].column, arguments[1].type);
auto dictionary = helper.getDictionary(dictionary_name);
if (!WhichDataType(arguments[2].type).isUInt64() && !isTuple(arguments[2].type))
2020-12-29 15:21:49 +00:00
throw Exception{"Illegal type " + arguments[2].type->getName() + " of third argument of function "
+ getName() + ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto dictionary_key_type = dictionary->getKeyType();
size_t current_arguments_index = 3;
ColumnPtr range_col = nullptr;
DataTypePtr range_col_type = nullptr;
if (dictionary_key_type == DictionaryKeyType::range)
{
if (current_arguments_index >= arguments.size())
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
range_col = arguments[current_arguments_index].column;
range_col_type = arguments[current_arguments_index].type;
if (!(range_col_type->isValueRepresentedByInteger() && range_col_type->getSizeOfValueInMemory() <= sizeof(Int64)))
2020-12-29 15:21:49 +00:00
throw Exception{"Illegal type " + range_col_type->getName() + " of fourth argument of function "
+ getName() + " must be convertible to Int64.",
ErrorCodes::ILLEGAL_COLUMN};
++current_arguments_index;
}
Columns default_cols;
if (dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault)
{
if (current_arguments_index >= arguments.size())
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
2021-02-27 20:39:34 +00:00
const auto & column_before_cast = arguments[current_arguments_index];
2021-02-27 20:39:34 +00:00
if (const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(column_before_cast.type.get()))
{
const DataTypes & nested_types = type_tuple->getElements();
for (const auto & nested_type : nested_types)
if (nested_type->isNullable())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong argument for function ({}) default values column nullable is not supported", getName());
}
else if (column_before_cast.type->isNullable())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong argument for function ({}) default values column nullable is not supported", getName());
auto result_type_no_nullable = removeNullable(result_type);
ColumnWithTypeAndName column_to_cast = {column_before_cast.column->convertToFullColumnIfConst(), column_before_cast.type, column_before_cast.name};
auto result = castColumnAccurate(column_to_cast, result_type_no_nullable);
if (attribute_names.size() > 1)
{
2021-02-27 20:39:34 +00:00
const auto * tuple_column = checkAndGetColumn<ColumnTuple>(result.get());
if (!tuple_column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function ({}) default values column must be tuple", getName());
if (tuple_column->tupleSize() != attribute_names.size())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function ({}) default values tuple column must contain same column size as requested attributes",
getName());
default_cols = tuple_column->getColumnsCopy();
}
else
2021-02-27 20:39:34 +00:00
default_cols.emplace_back(result);
}
else
{
for (size_t i = 0; i < attribute_names.size(); ++i)
default_cols.emplace_back(nullptr);
}
ColumnPtr result;
const ColumnWithTypeAndName & key_col_with_type = arguments[2];
const auto key_column = key_col_with_type.column;
if (dictionary_key_type == DictionaryKeyType::simple)
{
2021-03-05 14:12:50 +00:00
if (!WhichDataType(key_col_with_type.type).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function ({}) must be uint64 when dictionary is simple. Actual type ({}).",
getName(),
key_col_with_type.type->getName());
if (attribute_names.size() > 1)
{
const auto & result_tuple_type = assert_cast<const DataTypeTuple &>(*result_type);
Columns result_columns = dictionary->getColumns(
attribute_names,
result_tuple_type.getElements(),
{key_column},
{std::make_shared<DataTypeUInt64>()},
default_cols);
result = ColumnTuple::create(std::move(result_columns));
}
else
result = dictionary->getColumn(
attribute_names[0],
result_type,
{key_column},
{std::make_shared<DataTypeUInt64>()},
default_cols.front());
}
else if (dictionary_key_type == DictionaryKeyType::complex)
{
if (!isTuple(key_col_with_type.type))
2021-03-05 14:12:50 +00:00
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function ({}) must be tuple when dictionary is complex. Actual type ({}).",
getName(),
key_col_with_type.type->getName());
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column_full = key_col_with_type.column->convertToFullColumnIfConst();
2021-03-01 11:34:34 +00:00
if (!isTuple(key_col_with_type.type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function ({}) must be tuple when dictionary is complex. Actual type ({}).",
getName(),
key_col_with_type.type->getName());
const auto & key_columns = typeid_cast<const ColumnTuple &>(*key_column_full).getColumnsCopy();
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
if (attribute_names.size() > 1)
{
const auto & result_tuple_type = assert_cast<const DataTypeTuple &>(*result_type);
Columns result_columns = dictionary->getColumns(
attribute_names,
result_tuple_type.getElements(),
key_columns,
key_types,
default_cols);
result = ColumnTuple::create(std::move(result_columns));
}
else
result = dictionary->getColumn(
attribute_names[0],
result_type,
key_columns,
key_types,
default_cols.front());
}
else if (dictionary_key_type == DictionaryKeyType::range)
{
2021-03-05 14:12:50 +00:00
if (!WhichDataType(key_col_with_type.type).isUInt64())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function ({}) must be uint64 when dictionary is range. Actual type ({}).",
getName(),
key_col_with_type.type->getName());
if (attribute_names.size() > 1)
{
const auto & result_tuple_type = assert_cast<const DataTypeTuple &>(*result_type);
Columns result_columns = dictionary->getColumns(
attribute_names,
result_tuple_type.getElements(),
{key_column, range_col},
{std::make_shared<DataTypeUInt64>(), range_col_type},
default_cols);
result = ColumnTuple::create(std::move(result_columns));
}
else
result = dictionary->getColumn(
attribute_names[0],
result_type,
{key_column, range_col},
{std::make_shared<DataTypeUInt64>(), range_col_type},
default_cols.front());
}
else
throw Exception{"Unknown dictionary identifier type", ErrorCodes::BAD_ARGUMENTS};
2020-12-19 13:24:51 +00:00
return result;
}
private:
Strings getAttributeNamesFromColumn(const ColumnPtr & column, const DataTypePtr & type) const
{
Strings attribute_names;
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(column.get()))
attribute_names.emplace_back(name_col->getValue<String>());
else if (const auto * tuple_col_const = checkAndGetColumnConst<ColumnTuple>(column.get()))
{
const ColumnTuple & tuple_col = assert_cast<const ColumnTuple &>(tuple_col_const->getDataColumn());
size_t tuple_size = tuple_col.tupleSize();
if (tuple_size < 1)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Tuple second argument of function ({}) must contain multiple constant string columns");
for (size_t i = 0; i < tuple_col.tupleSize(); ++i)
{
const auto * tuple_column = tuple_col.getColumnPtr(i).get();
const auto * attribute_name_column = checkAndGetColumn<ColumnString>(tuple_column);
if (!attribute_name_column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Tuple second argument of function ({}) must contain multiple constant string columns",
getName());
attribute_names.emplace_back(attribute_name_column->getDataAt(0));
}
}
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type ({}) of second argument of function ({}), expected a const string or const tuple of const strings.",
type->getName(),
getName());
return attribute_names;
}
mutable FunctionDictHelper helper;
};
template <typename DataType, typename Name, DictionaryGetFunctionType dictionary_get_function_type>
class FunctionDictGetImpl final : public IFunction
{
using Type = typename DataType::FieldType;
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &context)
{
return std::make_shared<FunctionDictGetImpl>(context);
}
explicit FunctionDictGetImpl(const Context & context_) : 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 final { return true; }
bool isDeterministic() const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
bool isInjective(const ColumnsWithTypeAndName & sample_columns) const override
{
return impl.isInjective(sample_columns);
}
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
{
DataTypePtr result;
2021-01-12 21:26:32 +00:00
if constexpr (IsDataTypeDecimal<DataType>)
result = std::make_shared<DataType>(DataType::maxPrecision(), 0);
else
result = std::make_shared<DataType>();
return result;
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
auto return_type = impl.getReturnTypeImpl(arguments);
if (!areTypesEqual(return_type, result_type))
throw Exception{"Dictionary attribute has different type " + return_type->getName() + " expected " + result_type->getName(),
ErrorCodes::TYPE_MISMATCH};
return impl.executeImpl(arguments, return_type, input_rows_count);
}
static bool areTypesEqual(const DataTypePtr & lhs, const DataTypePtr & rhs)
{
return removeNullable(recursiveRemoveLowCardinality(lhs))->equals(*removeNullable(recursiveRemoveLowCardinality(rhs)));
}
const FunctionDictGetNoType<dictionary_get_function_type> impl;
};
template<typename DataType, typename Name>
using FunctionDictGet = FunctionDictGetImpl<DataType, Name, DictionaryGetFunctionType::get>;
struct NameDictGetUInt8 { static constexpr auto name = "dictGetUInt8"; };
struct NameDictGetUInt16 { static constexpr auto name = "dictGetUInt16"; };
struct NameDictGetUInt32 { static constexpr auto name = "dictGetUInt32"; };
struct NameDictGetUInt64 { static constexpr auto name = "dictGetUInt64"; };
struct NameDictGetInt8 { static constexpr auto name = "dictGetInt8"; };
struct NameDictGetInt16 { static constexpr auto name = "dictGetInt16"; };
struct NameDictGetInt32 { static constexpr auto name = "dictGetInt32"; };
struct NameDictGetInt64 { static constexpr auto name = "dictGetInt64"; };
struct NameDictGetFloat32 { static constexpr auto name = "dictGetFloat32"; };
struct NameDictGetFloat64 { static constexpr auto name = "dictGetFloat64"; };
struct NameDictGetDate { static constexpr auto name = "dictGetDate"; };
struct NameDictGetDateTime { static constexpr auto name = "dictGetDateTime"; };
struct NameDictGetUUID { static constexpr auto name = "dictGetUUID"; };
struct NameDictGetDecimal32 { static constexpr auto name = "dictGetDecimal32"; };
struct NameDictGetDecimal64 { static constexpr auto name = "dictGetDecimal64"; };
struct NameDictGetDecimal128 { static constexpr auto name = "dictGetDecimal128"; };
struct NameDictGetString { static constexpr auto name = "dictGetString"; };
using FunctionDictGetUInt8 = FunctionDictGet<DataTypeUInt8, NameDictGetUInt8>;
using FunctionDictGetUInt16 = FunctionDictGet<DataTypeUInt16, NameDictGetUInt16>;
using FunctionDictGetUInt32 = FunctionDictGet<DataTypeUInt32, NameDictGetUInt32>;
using FunctionDictGetUInt64 = FunctionDictGet<DataTypeUInt64, NameDictGetUInt64>;
using FunctionDictGetInt8 = FunctionDictGet<DataTypeInt8, NameDictGetInt8>;
using FunctionDictGetInt16 = FunctionDictGet<DataTypeInt16, NameDictGetInt16>;
using FunctionDictGetInt32 = FunctionDictGet<DataTypeInt32, NameDictGetInt32>;
using FunctionDictGetInt64 = FunctionDictGet<DataTypeInt64, NameDictGetInt64>;
using FunctionDictGetFloat32 = FunctionDictGet<DataTypeFloat32, NameDictGetFloat32>;
using FunctionDictGetFloat64 = FunctionDictGet<DataTypeFloat64, NameDictGetFloat64>;
using FunctionDictGetDate = FunctionDictGet<DataTypeDate, NameDictGetDate>;
using FunctionDictGetDateTime = FunctionDictGet<DataTypeDateTime, NameDictGetDateTime>;
using FunctionDictGetUUID = FunctionDictGet<DataTypeUUID, NameDictGetUUID>;
using FunctionDictGetDecimal32 = FunctionDictGet<DataTypeDecimal<Decimal32>, NameDictGetDecimal32>;
using FunctionDictGetDecimal64 = FunctionDictGet<DataTypeDecimal<Decimal64>, NameDictGetDecimal64>;
using FunctionDictGetDecimal128 = FunctionDictGet<DataTypeDecimal<Decimal128>, NameDictGetDecimal128>;
using FunctionDictGetString = FunctionDictGet<DataTypeString, NameDictGetString>;
template<typename DataType, typename Name>
using FunctionDictGetOrDefault = FunctionDictGetImpl<DataType, Name, DictionaryGetFunctionType::getOrDefault>;
struct NameDictGetUInt8OrDefault { static constexpr auto name = "dictGetUInt8OrDefault"; };
struct NameDictGetUInt16OrDefault { static constexpr auto name = "dictGetUInt16OrDefault"; };
struct NameDictGetUInt32OrDefault { static constexpr auto name = "dictGetUInt32OrDefault"; };
struct NameDictGetUInt64OrDefault { static constexpr auto name = "dictGetUInt64OrDefault"; };
struct NameDictGetInt8OrDefault { static constexpr auto name = "dictGetInt8OrDefault"; };
struct NameDictGetInt16OrDefault { static constexpr auto name = "dictGetInt16OrDefault"; };
struct NameDictGetInt32OrDefault { static constexpr auto name = "dictGetInt32OrDefault"; };
struct NameDictGetInt64OrDefault { static constexpr auto name = "dictGetInt64OrDefault"; };
struct NameDictGetFloat32OrDefault { static constexpr auto name = "dictGetFloat32OrDefault"; };
struct NameDictGetFloat64OrDefault { static constexpr auto name = "dictGetFloat64OrDefault"; };
struct NameDictGetDateOrDefault { static constexpr auto name = "dictGetDateOrDefault"; };
struct NameDictGetDateTimeOrDefault { static constexpr auto name = "dictGetDateTimeOrDefault"; };
struct NameDictGetUUIDOrDefault { static constexpr auto name = "dictGetUUIDOrDefault"; };
struct NameDictGetDecimal32OrDefault { static constexpr auto name = "dictGetDecimal32OrDefault"; };
struct NameDictGetDecimal64OrDefault { static constexpr auto name = "dictGetDecimal64OrDefault"; };
struct NameDictGetDecimal128OrDefault { static constexpr auto name = "dictGetDecimal128OrDefault"; };
struct NameDictGetStringOrDefault { static constexpr auto name = "dictGetStringOrDefault"; };
using FunctionDictGetUInt8OrDefault = FunctionDictGetOrDefault<DataTypeUInt8, NameDictGetUInt8OrDefault>;
using FunctionDictGetUInt16OrDefault = FunctionDictGetOrDefault<DataTypeUInt16, NameDictGetUInt16OrDefault>;
using FunctionDictGetUInt32OrDefault = FunctionDictGetOrDefault<DataTypeUInt32, NameDictGetUInt32OrDefault>;
using FunctionDictGetUInt64OrDefault = FunctionDictGetOrDefault<DataTypeUInt64, NameDictGetUInt64OrDefault>;
using FunctionDictGetInt8OrDefault = FunctionDictGetOrDefault<DataTypeInt8, NameDictGetInt8OrDefault>;
using FunctionDictGetInt16OrDefault = FunctionDictGetOrDefault<DataTypeInt16, NameDictGetInt16OrDefault>;
using FunctionDictGetInt32OrDefault = FunctionDictGetOrDefault<DataTypeInt32, NameDictGetInt32OrDefault>;
using FunctionDictGetInt64OrDefault = FunctionDictGetOrDefault<DataTypeInt64, NameDictGetInt64OrDefault>;
using FunctionDictGetFloat32OrDefault = FunctionDictGetOrDefault<DataTypeFloat32, NameDictGetFloat32OrDefault>;
using FunctionDictGetFloat64OrDefault = FunctionDictGetOrDefault<DataTypeFloat64, NameDictGetFloat64OrDefault>;
using FunctionDictGetDateOrDefault = FunctionDictGetOrDefault<DataTypeDate, NameDictGetDateOrDefault>;
using FunctionDictGetDateTimeOrDefault = FunctionDictGetOrDefault<DataTypeDateTime, NameDictGetDateTimeOrDefault>;
using FunctionDictGetUUIDOrDefault = FunctionDictGetOrDefault<DataTypeUUID, NameDictGetUUIDOrDefault>;
using FunctionDictGetDecimal32OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal32>, NameDictGetDecimal32OrDefault>;
using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal64>, NameDictGetDecimal64OrDefault>;
using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrDefault>;
using FunctionDictGetStringOrDefault = FunctionDictGetOrDefault<DataTypeString, NameDictGetStringOrDefault>;
/// Functions to work with hierarchies.
class FunctionDictGetHierarchy final : public IFunction
{
public:
static constexpr auto name = "dictGetHierarchy";
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictGetHierarchy>(context);
}
2020-10-18 14:49:19 +00:00
explicit FunctionDictGetHierarchy(const Context & context_) : helper(context_) {}
String getName() const override { return name; }
private:
size_t getNumberOfArguments() const override { return 2; }
2020-10-14 14:04:50 +00:00
bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const override { return true; }
2018-03-16 03:03:17 +00:00
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
2018-03-16 03:03:17 +00:00
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!WhichDataType(arguments[1]).isUInt64())
2018-03-16 03:03:17 +00:00
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
bool isDeterministic() const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
2018-04-24 07:16:39 +00:00
if (input_rows_count == 0)
2020-10-18 14:49:19 +00:00
return result_type->createColumn();
2020-10-18 14:49:19 +00:00
auto dict = helper.getDictionary(arguments[0]);
ColumnPtr res;
2019-09-11 11:14:55 +00:00
2021-03-07 22:32:13 +00:00
/// TODO: Rewrite this
2020-10-20 13:11:57 +00:00
if (!((res = executeDispatch<FlatDictionary>(arguments, result_type, dict))
2021-03-07 22:32:13 +00:00
|| (res = executeDispatch<DirectDictionary<DictionaryKeyType::simple>>(arguments, result_type, dict))
2020-10-18 14:49:19 +00:00
|| (res = executeDispatch<HashedDictionary>(arguments, result_type, dict))
2021-02-16 21:33:02 +00:00
|| (res = executeDispatch<CacheDictionary<DictionaryKeyType::simple>>(arguments, result_type, dict))))
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
2020-10-18 14:49:19 +00:00
return res;
}
template <typename DictionaryType>
ColumnPtr executeDispatch(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const std::shared_ptr<const IDictionaryBase> & dict_ptr) const
{
2020-10-18 14:49:19 +00:00
const auto * dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
2020-10-18 14:49:19 +00:00
return nullptr;
if (!dict->hasHierarchy())
2018-03-16 03:03:17 +00:00
throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD};
const auto get_hierarchies = [&] (const PaddedPODArray<UInt64> & in, PaddedPODArray<UInt64> & out, PaddedPODArray<UInt64> & offsets)
{
const auto size = in.size();
/// copy of `in` array
auto in_array = std::make_unique<PaddedPODArray<UInt64>>(std::begin(in), std::end(in));
/// used for storing and handling result of ::toParent call
auto out_array = std::make_unique<PaddedPODArray<UInt64>>(size);
/// resulting hierarchies
std::vector<std::vector<IDictionary::Key>> hierarchies(size); /// TODO Bad code, poor performance.
/// total number of non-zero elements, used for allocating all the required memory upfront
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
size_t total_count = 0;
while (true)
{
auto all_zeroes = true;
/// erase zeroed identifiers, store non-zeroed ones
for (const auto i : ext::range(0, size))
{
const auto id = (*in_array)[i];
if (0 == id)
continue;
auto & hierarchy = hierarchies[i];
2017-08-10 03:23:02 +00:00
/// Checking for loop
if (std::find(std::begin(hierarchy), std::end(hierarchy), id) != std::end(hierarchy))
continue;
all_zeroes = false;
/// place id at it's corresponding place
hierarchy.push_back(id);
++total_count;
}
if (all_zeroes)
break;
/// translate all non-zero identifiers at once
dict->toParent(*in_array, *out_array);
/// we're going to use the `in_array` from this iteration as `out_array` on the next one
std::swap(in_array, out_array);
}
out.reserve(total_count);
offsets.resize(size);
for (const auto i : ext::range(0, size))
{
const auto & ids = hierarchies[i];
out.insert_assume_reserved(std::begin(ids), std::end(ids));
offsets[i] = out.size();
}
};
2020-10-18 14:49:19 +00:00
const auto * id_col_untyped = arguments[1].column.get();
if (const auto * id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
{
const auto & in = id_col->getData();
auto backend = ColumnUInt64::create();
auto offsets = ColumnArray::ColumnOffsets::create();
get_hierarchies(in, backend->getData(), offsets->getData());
2020-10-18 14:49:19 +00:00
return ColumnArray::create(std::move(backend), std::move(offsets));
}
2020-10-18 14:49:19 +00:00
else if (const auto * id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
{
const PaddedPODArray<UInt64> in(1, id_col_const->getValue<UInt64>());
auto backend = ColumnUInt64::create();
auto offsets = ColumnArray::ColumnOffsets::create();
get_hierarchies(in, backend->getData(), offsets->getData());
auto array = ColumnArray::create(std::move(backend), std::move(offsets));
2020-10-18 14:49:19 +00:00
return result_type->createColumnConst(id_col_const->size(), (*array)[0].get<Array>());
}
else
2018-03-16 03:03:17 +00:00
throw Exception{"Second argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
}
mutable FunctionDictHelper helper;
};
class FunctionDictIsIn final : public IFunction
{
public:
static constexpr auto name = "dictIsIn";
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictIsIn>(context);
}
2020-10-18 14:49:19 +00:00
explicit FunctionDictIsIn(const Context & context_)
: helper(context_) {}
String getName() const override { return name; }
private:
size_t getNumberOfArguments() const override { return 3; }
2018-03-16 03:03:17 +00:00
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
2018-03-16 03:03:17 +00:00
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!WhichDataType(arguments[1]).isUInt64())
2018-03-16 03:03:17 +00:00
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!WhichDataType(arguments[2]).isUInt64())
2018-03-16 03:03:17 +00:00
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeUInt8>();
}
bool isDeterministic() const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
2018-04-24 07:16:39 +00:00
if (input_rows_count == 0)
2020-10-18 14:49:19 +00:00
return result_type->createColumn();
2020-10-18 14:49:19 +00:00
auto dict = helper.getDictionary(arguments[0]);
2019-09-11 11:14:55 +00:00
2020-10-18 14:49:19 +00:00
ColumnPtr res;
2020-10-20 13:11:57 +00:00
if (!((res = executeDispatch<FlatDictionary>(arguments, dict))
2021-03-07 22:32:13 +00:00
|| (res = executeDispatch<DirectDictionary<DictionaryKeyType::simple>>(arguments, dict))
2020-10-18 14:49:19 +00:00
|| (res = executeDispatch<HashedDictionary>(arguments, dict))
2021-02-16 21:33:02 +00:00
|| (res = executeDispatch<CacheDictionary<DictionaryKeyType::simple>>(arguments, dict))))
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
2020-10-18 14:49:19 +00:00
return res;
}
template <typename DictionaryType>
ColumnPtr executeDispatch(const ColumnsWithTypeAndName & arguments, const std::shared_ptr<const IDictionaryBase> & dict_ptr) const
{
2020-10-18 14:49:19 +00:00
const auto * dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
if (!dict)
2020-10-18 14:49:19 +00:00
return nullptr;
if (!dict->hasHierarchy())
2018-03-16 03:03:17 +00:00
throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD};
2020-10-18 14:49:19 +00:00
const auto * child_id_col_untyped = arguments[1].column.get();
const auto * ancestor_id_col_untyped = arguments[2].column.get();
2020-10-18 14:49:19 +00:00
if (const auto * child_id_col = checkAndGetColumn<ColumnUInt64>(child_id_col_untyped))
return execute(dict, child_id_col, ancestor_id_col_untyped);
else if (const auto * child_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(child_id_col_untyped))
return execute(dict, child_id_col_const, ancestor_id_col_untyped);
else
2018-03-16 03:03:17 +00:00
throw Exception{"Illegal column " + child_id_col_untyped->getName()
+ " of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
}
template <typename DictionaryType>
2020-10-18 14:49:19 +00:00
ColumnPtr execute(const DictionaryType * dict,
const ColumnUInt64 * child_id_col, const IColumn * ancestor_id_col_untyped) const
{
2020-10-18 14:49:19 +00:00
if (const auto * ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
{
auto out = ColumnUInt8::create();
const auto & child_ids = child_id_col->getData();
const auto & ancestor_ids = ancestor_id_col->getData();
auto & data = out->getData();
const auto size = child_id_col->size();
data.resize(size);
dict->isInVectorVector(child_ids, ancestor_ids, data);
2020-10-18 14:49:19 +00:00
return out;
}
2020-10-18 14:49:19 +00:00
else if (const auto * ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
{
auto out = ColumnUInt8::create();
const auto & child_ids = child_id_col->getData();
const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>();
auto & data = out->getData();
const auto size = child_id_col->size();
data.resize(size);
dict->isInVectorConstant(child_ids, ancestor_id, data);
2020-10-18 14:49:19 +00:00
return out;
}
else
{
2018-03-16 03:03:17 +00:00
throw Exception{"Illegal column " + ancestor_id_col_untyped->getName()
+ " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
}
}
template <typename DictionaryType>
2020-10-18 14:49:19 +00:00
ColumnPtr execute(const DictionaryType * dict, const ColumnConst * child_id_col, const IColumn * ancestor_id_col_untyped) const
{
2020-10-18 14:49:19 +00:00
if (const auto * ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
{
auto out = ColumnUInt8::create();
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
const auto child_id = child_id_col->getValue<UInt64>();
const auto & ancestor_ids = ancestor_id_col->getData();
auto & data = out->getData();
const auto size = child_id_col->size();
data.resize(size);
dict->isInConstantVector(child_id, ancestor_ids, data);
2020-10-18 14:49:19 +00:00
return out;
}
2020-10-18 14:49:19 +00:00
else if (const auto * ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
{
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
const auto child_id = child_id_col->getValue<UInt64>();
const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>();
UInt8 res = 0;
dict->isInConstantConstant(child_id, ancestor_id, res);
2020-10-18 14:49:19 +00:00
return DataTypeUInt8().createColumnConst(child_id_col->size(), res);
}
else
2018-03-16 03:03:17 +00:00
throw Exception{"Illegal column " + ancestor_id_col_untyped->getName()
+ " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
}
mutable FunctionDictHelper helper;
};
2018-08-26 00:52:58 +00:00
}