2012-12-21 19:48:47 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2019-05-23 14:04:33 +00:00
|
|
|
#include <DataTypes/DataTypesDecimal.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataTypes/DataTypeArray.h>
|
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <DataTypes/DataTypeDate.h>
|
|
|
|
#include <DataTypes/DataTypeDateTime.h>
|
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
2017-11-14 00:08:54 +00:00
|
|
|
#include <DataTypes/DataTypeUUID.h>
|
2021-01-02 22:08:54 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2021-01-08 21:02:47 +00:00
|
|
|
#include <DataTypes/DataTypeLowCardinality.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2019-08-21 02:28:04 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <Columns/ColumnConst.h>
|
|
|
|
#include <Columns/ColumnArray.h>
|
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
#include <Columns/ColumnTuple.h>
|
|
|
|
|
2020-01-26 09:49:53 +00:00
|
|
|
#include <Access/AccessFlags.h>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2019-09-26 10:41:33 +00:00
|
|
|
#include <Interpreters/ExternalDictionariesLoader.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
2019-12-09 13:12:54 +00:00
|
|
|
#include <Functions/IFunctionImpl.h>
|
2017-07-21 06:35:58 +00:00
|
|
|
#include <Functions/FunctionHelpers.h>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Dictionaries/FlatDictionary.h>
|
|
|
|
#include <Dictionaries/HashedDictionary.h>
|
|
|
|
#include <Dictionaries/CacheDictionary.h>
|
2020-07-15 11:16:00 +00:00
|
|
|
#if defined(OS_LINUX) || defined(__FreeBSD__)
|
2020-01-05 13:59:49 +00:00
|
|
|
#include <Dictionaries/SSDCacheDictionary.h>
|
2020-04-30 20:50:31 +00:00
|
|
|
#include <Dictionaries/SSDComplexKeyCacheDictionary.h>
|
2020-05-24 08:56:34 +00:00
|
|
|
#endif
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Dictionaries/ComplexKeyHashedDictionary.h>
|
|
|
|
#include <Dictionaries/ComplexKeyCacheDictionary.h>
|
2020-05-09 23:53:01 +00:00
|
|
|
#include <Dictionaries/ComplexKeyDirectDictionary.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Dictionaries/RangeHashedDictionary.h>
|
2020-11-21 18:38:10 +00:00
|
|
|
#include <Dictionaries/IPAddressDictionary.h>
|
2020-02-18 13:30:09 +00:00
|
|
|
#include <Dictionaries/PolygonDictionaryImplementations.h>
|
2020-05-01 23:35:54 +00:00
|
|
|
#include <Dictionaries/DirectDictionary.h>
|
2012-12-21 19:48:47 +00:00
|
|
|
|
2017-06-06 17:18:32 +00:00
|
|
|
#include <ext/range.h>
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2018-09-13 13:33:44 +00:00
|
|
|
#include <type_traits>
|
2012-12-21 19:48:47 +00:00
|
|
|
|
|
|
|
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;
|
2017-04-01 07:20:54 +00:00
|
|
|
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;
|
2019-03-14 23:10:51 +00:00
|
|
|
extern const int ILLEGAL_COLUMN;
|
|
|
|
extern const int BAD_ARGUMENTS;
|
2021-01-21 14:42:50 +00:00
|
|
|
extern const int TYPE_MISMATCH;
|
|
|
|
extern const int NOT_IMPLEMENTED;
|
2016-01-12 02:21:15 +00:00
|
|
|
}
|
|
|
|
|
2020-05-14 14:03:06 +00:00
|
|
|
|
2019-09-26 10:41:33 +00:00
|
|
|
/** Functions that use plug-ins (external) dictionaries_loader.
|
2015-02-10 14:50:43 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* Get the value of the attribute of the specified type.
|
2017-04-01 07:20:54 +00:00
|
|
|
* 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.
|
2015-02-10 14:50:43 +00:00
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* Get an array of identifiers, consisting of the source and parents chain.
|
2015-02-10 14:50:43 +00:00
|
|
|
* dictGetHierarchy(dictionary, id).
|
|
|
|
*
|
2017-05-13 22:19:04 +00:00
|
|
|
* Is the first identifier the child of the second.
|
2015-02-10 14:50:43 +00:00
|
|
|
* dictIsIn(dictionary, child_id, parent_id).
|
2012-12-21 19:48:47 +00:00
|
|
|
*/
|
|
|
|
|
|
|
|
|
2020-05-14 14:03:06 +00:00
|
|
|
class FunctionDictHelper
|
|
|
|
{
|
|
|
|
public:
|
2020-10-18 14:49:19 +00:00
|
|
|
explicit FunctionDictHelper(const Context & context_) : context(context_), external_loader(context.getExternalDictionariesLoader()) {}
|
2020-05-14 14:03:06 +00:00
|
|
|
|
|
|
|
std::shared_ptr<const IDictionaryBase> getDictionary(const String & dictionary_name)
|
|
|
|
{
|
2020-07-15 19:25:31 +00:00
|
|
|
String resolved_name = DatabaseCatalog::instance().resolveDictionaryName(dictionary_name);
|
2020-10-16 21:14:49 +00:00
|
|
|
auto dict = external_loader.getDictionary(resolved_name);
|
|
|
|
if (!access_checked)
|
|
|
|
{
|
|
|
|
context.checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getDictionaryID().getTableName());
|
|
|
|
access_checked = true;
|
|
|
|
}
|
2020-05-14 14:03:06 +00:00
|
|
|
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());
|
2020-05-14 14:03:06 +00:00
|
|
|
return getDictionary(dict_name_col->getValue<String>());
|
|
|
|
}
|
|
|
|
|
2020-10-14 14:04:50 +00:00
|
|
|
bool isDictGetFunctionInjective(const Block & sample_columns)
|
2020-05-14 14:03:06 +00:00
|
|
|
{
|
2020-07-07 21:26:09 +00:00
|
|
|
/// Assume non-injective by default
|
2020-10-14 14:04:50 +00:00
|
|
|
if (!sample_columns)
|
2020-07-07 21:26:09 +00:00
|
|
|
return false;
|
|
|
|
|
2020-12-20 20:11:28 +00:00
|
|
|
if (sample_columns.columns() < 3)
|
|
|
|
throw Exception{"Wrong arguments count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
2020-05-14 14:03:06 +00:00
|
|
|
|
2020-10-18 14:49:19 +00:00
|
|
|
const auto * dict_name_col = checkAndGetColumnConst<ColumnString>(sample_columns.getByPosition(0).column.get());
|
2020-05-14 14:03:06 +00:00
|
|
|
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());
|
2020-05-14 14:03:06 +00:00
|
|
|
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>());
|
|
|
|
}
|
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
DictionaryAttribute getDictionaryAttribute(std::shared_ptr<const IDictionaryBase> dictionary, const String& attribute_name) const
|
|
|
|
{
|
|
|
|
const DictionaryStructure & structure = dictionary->getStructure();
|
|
|
|
|
|
|
|
auto find_iter = std::find_if(structure.attributes.begin(), structure.attributes.end(), [&](const auto &attribute)
|
|
|
|
{
|
|
|
|
return attribute.name == attribute_name;
|
|
|
|
});
|
|
|
|
|
|
|
|
if (find_iter == structure.attributes.end())
|
|
|
|
throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
|
|
|
|
|
|
|
|
return *find_iter;
|
|
|
|
}
|
2020-05-14 14:03:06 +00:00
|
|
|
private:
|
|
|
|
const Context & context;
|
|
|
|
const ExternalDictionariesLoader & external_loader;
|
2020-10-16 21:14:49 +00:00
|
|
|
/// 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;
|
2020-05-14 14:03:06 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2015-11-19 13:15:02 +00:00
|
|
|
class FunctionDictHas final : public IFunction
|
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
static constexpr auto name = "dictHas";
|
2015-11-19 13:15:02 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static FunctionPtr create(const Context & context)
|
|
|
|
{
|
2020-05-14 14:03:06 +00:00
|
|
|
return std::make_shared<FunctionDictHas>(context);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-11-19 13:15:02 +00:00
|
|
|
|
2020-10-18 14:49:19 +00:00
|
|
|
explicit FunctionDictHas(const Context & context_) : helper(context_) {}
|
2015-11-19 13:15:02 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getName() const override { return name; }
|
2015-11-19 13:15:02 +00:00
|
|
|
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t getNumberOfArguments() const override { return 2; }
|
|
|
|
|
2018-03-16 01:22:02 +00:00
|
|
|
bool useDefaultImplementationForConstants() const final { return true; }
|
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
|
|
|
{
|
2018-09-07 14:37:26 +00:00
|
|
|
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};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-07 14:37:26 +00:00
|
|
|
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};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
return std::make_shared<DataTypeUInt8>();
|
|
|
|
}
|
|
|
|
|
2018-08-08 11:26:18 +00:00
|
|
|
bool isDeterministic() const override { return false; }
|
2018-02-21 17:05:21 +00:00
|
|
|
|
2020-11-17 13:24:45 +00:00
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-10-14 14:04:50 +00:00
|
|
|
/** Do not require existence of the dictionary if the function is called for empty columns.
|
2018-03-16 03:13:13 +00:00
|
|
|
* 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();
|
2018-03-16 03:13:13 +00:00
|
|
|
|
2020-12-19 13:24:51 +00:00
|
|
|
auto dictionary = helper.getDictionary(arguments[0]);
|
2021-01-23 13:18:24 +00:00
|
|
|
auto dictionary_key_type = dictionary->getKeyType();
|
2019-09-11 11:14:55 +00:00
|
|
|
|
2020-12-20 20:11:28 +00:00
|
|
|
const ColumnWithTypeAndName & key_column_with_type = arguments[1];
|
|
|
|
const auto key_column = key_column_with_type.column;
|
2021-01-08 21:02:47 +00:00
|
|
|
const auto key_column_type = WhichDataType(key_column_with_type.type);
|
2020-10-18 14:49:19 +00:00
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
if (dictionary_key_type == DictionaryKeyType::simple)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-01-08 21:02:47 +00:00
|
|
|
if (!key_column_type.isUInt64())
|
|
|
|
throw Exception{"Second argument of function " + getName() + " must be " + dictionary->getStructure().getKeyDescription(),
|
|
|
|
ErrorCodes::TYPE_MISMATCH};
|
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
return dictionary->hasKeys({key_column}, {std::make_shared<DataTypeUInt64>()});
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2021-01-23 13:18:24 +00:00
|
|
|
else if (dictionary_key_type == DictionaryKeyType::complex)
|
2017-12-13 20:40:03 +00:00
|
|
|
{
|
2021-01-08 21:02:47 +00:00
|
|
|
if (!key_column_type.isTuple())
|
|
|
|
throw Exception{"Second argument of function " + getName() + " must be " + dictionary->getStructure().getKeyDescription(),
|
|
|
|
ErrorCodes::TYPE_MISMATCH};
|
|
|
|
|
2020-12-20 20:11:28 +00:00
|
|
|
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
|
2021-01-08 21:02:47 +00:00
|
|
|
ColumnPtr key_column_full = key_column->convertToFullColumnIfConst();
|
2020-12-20 20:11:28 +00:00
|
|
|
|
|
|
|
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();
|
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
return dictionary->hasKeys(key_columns, key_types);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
2021-01-08 21:02:47 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Has not supported for range dictionary", dictionary->getDictionaryID().getNameForLogs());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-05-14 14:03:06 +00:00
|
|
|
mutable FunctionDictHelper helper;
|
2015-11-19 13:15:02 +00:00
|
|
|
};
|
|
|
|
|
2020-12-18 21:43:08 +00:00
|
|
|
enum class DictionaryGetFunctionType
|
|
|
|
{
|
2021-01-02 22:08:54 +00:00
|
|
|
get,
|
2021-01-08 21:02:47 +00:00
|
|
|
getOrDefault
|
2019-05-23 14:04:33 +00:00
|
|
|
};
|
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
/// This variant of function derives the result type automatically.
|
|
|
|
template <DictionaryGetFunctionType dictionary_get_function_type>
|
|
|
|
class FunctionDictGetNoType final : public IFunction
|
2015-01-22 14:32:38 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-01-08 21:02:47 +00:00
|
|
|
static constexpr auto name = dictionary_get_function_type == DictionaryGetFunctionType::get ? "dictGet" : "dictGetOrDefault";
|
2015-01-22 14:32:38 +00:00
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
static FunctionPtr create(const Context & context)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-01-08 21:02:47 +00:00
|
|
|
return std::make_shared<FunctionDictGetNoType>(context);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-01-22 14:32:38 +00:00
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
explicit FunctionDictGetNoType(const Context & context_) : helper(context_) {}
|
2015-01-22 14:32:38 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getName() const override { return name; }
|
2015-01-22 14:32:38 +00:00
|
|
|
|
2020-12-20 20:11:28 +00:00
|
|
|
bool isVariadic() const override { return true; }
|
2021-01-08 21:02:47 +00:00
|
|
|
size_t getNumberOfArguments() const override { return 0; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-03-16 03:03:17 +00:00
|
|
|
bool useDefaultImplementationForConstants() const final { return true; }
|
2021-01-08 21:02:47 +00:00
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
|
2018-03-16 03:03:17 +00:00
|
|
|
|
2020-12-18 21:43:08 +00:00
|
|
|
bool isDeterministic() const override { return false; }
|
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
bool isInjective(const ColumnsWithTypeAndName & sample_columns) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2021-01-08 21:02:47 +00:00
|
|
|
return helper.isDictGetFunctionInjective(sample_columns);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-12-20 20:11:28 +00:00
|
|
|
if (arguments.size() < 3)
|
|
|
|
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
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};
|
2020-12-20 20:11:28 +00:00
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
if (const auto * name_col = checkAndGetColumnConst<ColumnString>(arguments[1].column.get()))
|
|
|
|
attribute_name = name_col->getValue<String>();
|
|
|
|
else
|
|
|
|
throw Exception{"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName()
|
|
|
|
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
2020-12-20 20:11:28 +00:00
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
dictionary = helper.getDictionary(dictionary_name);
|
2020-12-18 21:43:08 +00:00
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
return helper.getDictionaryAttribute(dictionary, attribute_name).type;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
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();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-12-20 20:11:28 +00:00
|
|
|
/// TODO: Use accurateCast if argument is integer
|
|
|
|
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(...).",
|
2020-12-20 20:11:28 +00:00
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
2019-05-23 14:04:33 +00:00
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
auto dictionary_key_type = dictionary->getKeyType();
|
2020-12-20 20:11:28 +00:00
|
|
|
|
|
|
|
size_t current_arguments_index = 3;
|
|
|
|
|
|
|
|
ColumnPtr range_col = nullptr;
|
|
|
|
DataTypePtr range_col_type = nullptr;
|
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
if (dictionary_key_type == DictionaryKeyType::range)
|
2020-12-20 20:11:28 +00:00
|
|
|
{
|
|
|
|
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.",
|
2020-12-20 20:11:28 +00:00
|
|
|
ErrorCodes::ILLEGAL_COLUMN};
|
|
|
|
|
|
|
|
++current_arguments_index;
|
|
|
|
}
|
|
|
|
|
2020-12-19 14:27:39 +00:00
|
|
|
ColumnPtr default_col = nullptr;
|
|
|
|
|
2021-01-02 22:08:54 +00:00
|
|
|
if (dictionary_get_function_type == DictionaryGetFunctionType::getOrDefault)
|
2020-12-19 14:27:39 +00:00
|
|
|
{
|
2020-12-20 20:11:28 +00:00
|
|
|
if (current_arguments_index >= arguments.size())
|
2021-01-08 21:02:47 +00:00
|
|
|
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
2020-12-20 20:11:28 +00:00
|
|
|
|
|
|
|
default_col = arguments[current_arguments_index].column;
|
2020-12-19 14:27:39 +00:00
|
|
|
}
|
2020-12-18 21:43:08 +00:00
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
ColumnPtr result;
|
2020-12-20 20:11:28 +00:00
|
|
|
|
|
|
|
const ColumnWithTypeAndName & key_col_with_type = arguments[2];
|
|
|
|
const auto key_column = key_col_with_type.column;
|
|
|
|
|
2021-01-23 13:18:24 +00:00
|
|
|
if (dictionary_key_type == DictionaryKeyType::simple)
|
2020-12-18 21:43:08 +00:00
|
|
|
{
|
2021-01-08 21:02:47 +00:00
|
|
|
result = dictionary->getColumn(attribute_name, result_type, {key_column}, {std::make_shared<DataTypeUInt64>()}, default_col);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2021-01-23 13:18:24 +00:00
|
|
|
else if (dictionary_key_type == DictionaryKeyType::complex)
|
2017-12-13 20:40:03 +00:00
|
|
|
{
|
2020-12-19 14:27:39 +00:00
|
|
|
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
|
|
|
|
ColumnPtr key_column_full = key_col_with_type.column->convertToFullColumnIfConst();
|
|
|
|
|
|
|
|
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();
|
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
result = dictionary->getColumn(attribute_name, result_type, key_columns, key_types, default_col);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2021-01-23 13:18:24 +00:00
|
|
|
else if (dictionary_key_type == DictionaryKeyType::range)
|
2020-12-18 21:43:08 +00:00
|
|
|
{
|
2021-01-08 21:02:47 +00:00
|
|
|
result = dictionary->getColumn(
|
|
|
|
attribute_name, result_type, {key_column, range_col}, {std::make_shared<DataTypeUInt64>(), range_col_type}, default_col);
|
2020-12-18 21:43:08 +00:00
|
|
|
}
|
2020-12-20 20:11:28 +00:00
|
|
|
else
|
|
|
|
throw Exception{"Unknown dictionary identifier type", ErrorCodes::BAD_ARGUMENTS};
|
2020-12-19 13:24:51 +00:00
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
return result;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2021-01-08 21:02:47 +00:00
|
|
|
private:
|
2020-05-14 14:03:06 +00:00
|
|
|
mutable FunctionDictHelper helper;
|
2021-01-08 21:02:47 +00:00
|
|
|
/// Initialized in getReturnTypeImpl
|
|
|
|
mutable std::shared_ptr<const IDictionaryBase> dictionary;
|
|
|
|
mutable String attribute_name;
|
|
|
|
};
|
|
|
|
|
|
|
|
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
|
|
|
|
2021-01-08 21:02:47 +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;
|
2015-01-22 14:32:38 +00:00
|
|
|
};
|
|
|
|
|
2020-12-18 21:43:08 +00:00
|
|
|
template<typename DataType, typename Name>
|
2021-01-02 22:08:54 +00:00
|
|
|
using FunctionDictGet = FunctionDictGetImpl<DataType, Name, DictionaryGetFunctionType::get>;
|
2020-12-18 21:43:08 +00:00
|
|
|
|
2017-11-22 21:09:23 +00:00
|
|
|
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"; };
|
2019-05-23 14:04:33 +00:00
|
|
|
struct NameDictGetDecimal32 { static constexpr auto name = "dictGetDecimal32"; };
|
|
|
|
struct NameDictGetDecimal64 { static constexpr auto name = "dictGetDecimal64"; };
|
|
|
|
struct NameDictGetDecimal128 { static constexpr auto name = "dictGetDecimal128"; };
|
2020-12-18 21:43:08 +00:00
|
|
|
struct NameDictGetString { static constexpr auto name = "dictGetString"; };
|
2017-11-22 21:09:23 +00:00
|
|
|
|
|
|
|
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>;
|
2019-05-23 14:04:33 +00:00
|
|
|
using FunctionDictGetDecimal32 = FunctionDictGet<DataTypeDecimal<Decimal32>, NameDictGetDecimal32>;
|
|
|
|
using FunctionDictGetDecimal64 = FunctionDictGet<DataTypeDecimal<Decimal64>, NameDictGetDecimal64>;
|
|
|
|
using FunctionDictGetDecimal128 = FunctionDictGet<DataTypeDecimal<Decimal128>, NameDictGetDecimal128>;
|
2020-12-18 21:43:08 +00:00
|
|
|
using FunctionDictGetString = FunctionDictGet<DataTypeString, NameDictGetString>;
|
2017-11-22 21:09:23 +00:00
|
|
|
|
2020-12-18 21:43:08 +00:00
|
|
|
template<typename DataType, typename Name>
|
2021-01-02 22:08:54 +00:00
|
|
|
using FunctionDictGetOrDefault = FunctionDictGetImpl<DataType, Name, DictionaryGetFunctionType::getOrDefault>;
|
2015-11-10 09:22:25 +00:00
|
|
|
|
2017-11-22 21:09:23 +00:00
|
|
|
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"; };
|
2019-05-23 14:04:33 +00:00
|
|
|
struct NameDictGetDecimal32OrDefault { static constexpr auto name = "dictGetDecimal32OrDefault"; };
|
|
|
|
struct NameDictGetDecimal64OrDefault { static constexpr auto name = "dictGetDecimal64OrDefault"; };
|
|
|
|
struct NameDictGetDecimal128OrDefault { static constexpr auto name = "dictGetDecimal128OrDefault"; };
|
2020-12-18 21:43:08 +00:00
|
|
|
struct NameDictGetStringOrDefault { static constexpr auto name = "dictGetStringOrDefault"; };
|
2017-11-22 21:09:23 +00:00
|
|
|
|
2017-12-23 23:16:40 +00:00
|
|
|
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>;
|
2019-05-23 14:04:33 +00:00
|
|
|
using FunctionDictGetDecimal32OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal32>, NameDictGetDecimal32OrDefault>;
|
|
|
|
using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal64>, NameDictGetDecimal64OrDefault>;
|
|
|
|
using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrDefault>;
|
2020-12-18 21:43:08 +00:00
|
|
|
using FunctionDictGetStringOrDefault = FunctionDictGetOrDefault<DataTypeString, NameDictGetStringOrDefault>;
|
2015-11-10 09:22:25 +00:00
|
|
|
|
2017-03-25 23:42:04 +00:00
|
|
|
/// Functions to work with hierarchies.
|
|
|
|
|
2015-01-28 13:20:20 +00:00
|
|
|
class FunctionDictGetHierarchy final : public IFunction
|
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
static constexpr auto name = "dictGetHierarchy";
|
2015-01-28 13:20:20 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static FunctionPtr create(const Context & context)
|
|
|
|
{
|
2020-05-14 14:03:06 +00:00
|
|
|
return std::make_shared<FunctionDictGetHierarchy>(context);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-01-28 13:20:20 +00:00
|
|
|
|
2020-10-18 14:49:19 +00:00
|
|
|
explicit FunctionDictGetHierarchy(const Context & context_) : helper(context_) {}
|
2015-01-28 13:20:20 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getName() const override { return name; }
|
2015-01-28 13:20:20 +00:00
|
|
|
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
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; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-03-16 03:03:17 +00:00
|
|
|
bool useDefaultImplementationForConstants() const final { return true; }
|
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
|
|
|
{
|
2018-09-07 14:37:26 +00:00
|
|
|
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};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-07 14:37:26 +00:00
|
|
|
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};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
|
|
|
}
|
|
|
|
|
2018-08-08 11:26:18 +00:00
|
|
|
bool isDeterministic() const override { return false; }
|
2018-02-21 17:05:21 +00:00
|
|
|
|
2020-11-17 13:24:45 +00:00
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
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();
|
2018-03-16 03:13:13 +00:00
|
|
|
|
2020-10-18 14:49:19 +00:00
|
|
|
auto dict = helper.getDictionary(arguments[0]);
|
|
|
|
ColumnPtr res;
|
2019-09-11 11:14:55 +00:00
|
|
|
|
2020-10-20 13:11:57 +00:00
|
|
|
if (!((res = executeDispatch<FlatDictionary>(arguments, result_type, dict))
|
2020-10-18 14:49:19 +00:00
|
|
|
|| (res = executeDispatch<DirectDictionary>(arguments, result_type, dict))
|
|
|
|
|| (res = executeDispatch<HashedDictionary>(arguments, result_type, dict))
|
|
|
|
|| (res = executeDispatch<CacheDictionary>(arguments, result_type, dict))))
|
2020-05-14 14:03:06 +00:00
|
|
|
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
2020-10-18 14:49:19 +00:00
|
|
|
|
|
|
|
return res;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename DictionaryType>
|
2020-11-17 13:24:45 +00:00
|
|
|
ColumnPtr executeDispatch(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const std::shared_ptr<const IDictionaryBase> & dict_ptr) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-10-18 14:49:19 +00:00
|
|
|
const auto * dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!dict)
|
2020-10-18 14:49:19 +00:00
|
|
|
return nullptr;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (!dict->hasHierarchy())
|
2018-03-16 03:03:17 +00:00
|
|
|
throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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
|
2017-07-21 06:35:58 +00:00
|
|
|
size_t total_count = 0;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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;
|
|
|
|
|
2017-08-04 16:59:50 +00:00
|
|
|
|
|
|
|
auto & hierarchy = hierarchies[i];
|
|
|
|
|
2017-08-10 03:23:02 +00:00
|
|
|
/// Checking for loop
|
2017-08-04 16:59:50 +00:00
|
|
|
if (std::find(std::begin(hierarchy), std::end(hierarchy), id) != std::end(hierarchy))
|
|
|
|
continue;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
all_zeroes = false;
|
|
|
|
/// place id at it's corresponding place
|
2017-08-04 16:59:50 +00:00
|
|
|
hierarchy.push_back(id);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
++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))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
const auto & in = id_col->getData();
|
2017-12-16 05:34:07 +00:00
|
|
|
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));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-10-18 14:49:19 +00:00
|
|
|
else if (const auto * id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-01-04 12:10:00 +00:00
|
|
|
const PaddedPODArray<UInt64> in(1, id_col_const->getValue<UInt64>());
|
2017-12-16 05:34:07 +00:00
|
|
|
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>());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
2018-03-16 03:03:17 +00:00
|
|
|
throw Exception{"Second argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-05-14 14:03:06 +00:00
|
|
|
mutable FunctionDictHelper helper;
|
2015-01-28 13:20:20 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
class FunctionDictIsIn final : public IFunction
|
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
static constexpr auto name = "dictIsIn";
|
2015-01-28 13:20:20 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
static FunctionPtr create(const Context & context)
|
|
|
|
{
|
2020-05-14 14:03:06 +00:00
|
|
|
return std::make_shared<FunctionDictIsIn>(context);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-01-28 13:20:20 +00:00
|
|
|
|
2020-10-18 14:49:19 +00:00
|
|
|
explicit FunctionDictIsIn(const Context & context_)
|
2020-05-14 14:03:06 +00:00
|
|
|
: helper(context_) {}
|
2015-01-28 13:20:20 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getName() const override { return name; }
|
2015-01-28 13:20:20 +00:00
|
|
|
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
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}; }
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
|
|
|
{
|
2018-09-07 14:37:26 +00:00
|
|
|
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};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-07 14:37:26 +00:00
|
|
|
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};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-07 14:37:26 +00:00
|
|
|
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};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
return std::make_shared<DataTypeUInt8>();
|
|
|
|
}
|
|
|
|
|
2018-08-08 11:26:18 +00:00
|
|
|
bool isDeterministic() const override { return false; }
|
2018-02-21 17:05:21 +00:00
|
|
|
|
2020-11-17 13:24:45 +00:00
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
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();
|
2018-03-16 03:13:13 +00:00
|
|
|
|
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))
|
2020-10-18 14:49:19 +00:00
|
|
|
|| (res = executeDispatch<DirectDictionary>(arguments, dict))
|
|
|
|
|| (res = executeDispatch<HashedDictionary>(arguments, dict))
|
|
|
|
|| (res = executeDispatch<CacheDictionary>(arguments, dict))))
|
2020-05-14 14:03:06 +00:00
|
|
|
throw Exception{"Unsupported dictionary type " + dict->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
|
2020-10-18 14:49:19 +00:00
|
|
|
|
|
|
|
return res;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename DictionaryType>
|
2020-11-17 13:24:45 +00:00
|
|
|
ColumnPtr executeDispatch(const ColumnsWithTypeAndName & arguments, const std::shared_ptr<const IDictionaryBase> & dict_ptr) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-10-18 14:49:19 +00:00
|
|
|
const auto * dict = typeid_cast<const DictionaryType *>(dict_ptr.get());
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!dict)
|
2020-10-18 14:49:19 +00:00
|
|
|
return nullptr;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (!dict->hasHierarchy())
|
2018-03-16 03:03:17 +00:00
|
|
|
throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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);
|
2017-04-01 07:20:54 +00:00
|
|
|
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};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename DictionaryType>
|
2020-10-18 14:49:19 +00:00
|
|
|
ColumnPtr execute(const DictionaryType * dict,
|
2020-10-14 13:09:11 +00:00
|
|
|
const ColumnUInt64 * child_id_col, const IColumn * ancestor_id_col_untyped) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-10-18 14:49:19 +00:00
|
|
|
if (const auto * ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-16 05:21:04 +00:00
|
|
|
auto out = ColumnUInt8::create();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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);
|
|
|
|
|
2020-05-14 14:03:06 +00:00
|
|
|
dict->isInVectorVector(child_ids, ancestor_ids, data);
|
2020-10-18 14:49:19 +00:00
|
|
|
return out;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-10-18 14:49:19 +00:00
|
|
|
else if (const auto * ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-16 05:21:04 +00:00
|
|
|
auto out = ColumnUInt8::create();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
const auto & child_ids = child_id_col->getData();
|
2019-01-04 12:10:00 +00:00
|
|
|
const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>();
|
2017-04-01 07:20:54 +00:00
|
|
|
auto & data = out->getData();
|
|
|
|
const auto size = child_id_col->size();
|
|
|
|
data.resize(size);
|
|
|
|
|
2020-05-14 14:03:06 +00:00
|
|
|
dict->isInVectorConstant(child_ids, ancestor_id, data);
|
2020-10-18 14:49:19 +00:00
|
|
|
return out;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
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};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
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
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-10-18 14:49:19 +00:00
|
|
|
if (const auto * ancestor_id_col = checkAndGetColumn<ColumnUInt64>(ancestor_id_col_untyped))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-16 05:21:04 +00:00
|
|
|
auto out = ColumnUInt8::create();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
const auto child_id = child_id_col->getValue<UInt64>();
|
2017-04-01 07:20:54 +00:00
|
|
|
const auto & ancestor_ids = ancestor_id_col->getData();
|
|
|
|
auto & data = out->getData();
|
|
|
|
const auto size = child_id_col->size();
|
|
|
|
data.resize(size);
|
|
|
|
|
2020-05-14 14:03:06 +00:00
|
|
|
dict->isInConstantVector(child_id, ancestor_ids, data);
|
2020-10-18 14:49:19 +00:00
|
|
|
return out;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-10-18 14:49:19 +00:00
|
|
|
else if (const auto * ancestor_id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(ancestor_id_col_untyped))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
const auto child_id = child_id_col->getValue<UInt64>();
|
2019-01-04 12:10:00 +00:00
|
|
|
const auto ancestor_id = ancestor_id_col_const->getValue<UInt64>();
|
2017-04-01 07:20:54 +00:00
|
|
|
UInt8 res = 0;
|
|
|
|
|
2020-05-14 14:03:06 +00:00
|
|
|
dict->isInConstantConstant(child_id, ancestor_id, res);
|
2020-10-18 14:49:19 +00:00
|
|
|
return DataTypeUInt8().createColumnConst(child_id_col->size(), res);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
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};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-05-14 14:03:06 +00:00
|
|
|
mutable FunctionDictHelper helper;
|
2015-01-28 13:20:20 +00:00
|
|
|
};
|
|
|
|
|
2018-08-26 00:52:58 +00:00
|
|
|
}
|