dbms: dictGet* functions for complex_key dictionary. No dictGet*OrDefault variants yet [#METR-17328]

This commit is contained in:
Andrey Mironov 2015-11-13 04:44:41 +03:00
parent 701bed3fae
commit 5c641ffc78
6 changed files with 279 additions and 66 deletions

View File

@ -134,10 +134,9 @@ private:
for (const auto & key : *dict_struct.key)
{
if (!first)
{
writeString(", ", out);
first = false;
}
first = false;
if (!key.expression.empty())
{

View File

@ -22,7 +22,7 @@ public:
const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime, bool require_nonempty)
: name{name}, dict_struct(dict_struct), source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
require_nonempty(require_nonempty)
require_nonempty(require_nonempty), key_description{createKeyDescription(dict_struct)}
{
createAttributes();
@ -43,6 +43,8 @@ public:
: ComplexKeyDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty}
{}
std::string getKeyDescription() const { return key_description; };
std::exception_ptr getCreationException() const override { return creation_exception; }
std::string getName() const override { return name; }
@ -79,9 +81,13 @@ public:
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
}
/*#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
PODArray<TYPE> & out) const\
{\
validateKeyColumns(key_types);\
\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
@ -89,7 +95,7 @@ public:
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, ids, out);\
getItems<TYPE>(attribute, key_columns, out);\
}
DECLARE_MULTIPLE_GETTER(UInt8)
DECLARE_MULTIPLE_GETTER(UInt16)
@ -102,8 +108,12 @@ public:
DECLARE_MULTIPLE_GETTER(Float32)
DECLARE_MULTIPLE_GETTER(Float64)
#undef DECLARE_MULTIPLE_GETTER
void getString(const std::string & attribute_name, const PODArray<id_t> & ids, ColumnString * out) const override
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
ColumnString * out) const
{
validateKeyColumns(key_types);
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
@ -114,21 +124,32 @@ public:
const auto & attr = *std::get<MapPointerType<StringRef>>(attribute.maps);
const auto & null_value = StringRef{std::get<String>(attribute.null_values)};
for (const auto i : ext::range(0, ids.size()))
const auto keys_size = key_columns.size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
const auto rows = key_columns.front()->size();
for (const auto i : ext::range(0, rows))
{
const auto it = attr.find(ids[i]);
const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool);
const auto it = attr.find(key);
const auto string_ref = it != attr.end() ? it->second : null_value;
out->insertData(string_ref.data, string_ref.size);
temporary_keys_pool.rollback(key.size);
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
query_count.fetch_add(rows, std::memory_order_relaxed);
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const PODArray<TYPE> & def,\
PODArray<TYPE> & out) const override\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const PODArray<TYPE> & def, PODArray<TYPE> & out) const\
{\
validateKeyColumns(key_types);\
\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
@ -136,7 +157,7 @@ public:
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, ids, def, out);\
getItems<TYPE>(attribute, key_columns, def, out);\
}
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt16)
@ -150,9 +171,11 @@ public:
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float64)
#undef DECLARE_MULTIPLE_GETTER_WITH_DEFAULT
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const ColumnString * const def,
ColumnString * const out) const override
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const ColumnString * const def, ColumnString * const out) const
{
validateKeyColumns(key_types);
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
@ -160,17 +183,26 @@ public:
ErrorCodes::TYPE_MISMATCH
};
const auto & attr = *std::get<std::unique_ptr<HashMapWithSavedHash<StringRef, StringRef>>>(attribute.maps);
const auto & attr = *std::get<MapPointerType<StringRef>>(attribute.maps);
for (const auto i : ext::range(0, ids.size()))
const auto keys_size = key_columns.size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
const auto rows = key_columns.front()->size();
for (const auto i : ext::range(0, rows))
{
const auto it = attr.find(ids[i]);
const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool);
const auto it = attr.find(key);
const auto string_ref = it != attr.end() ? it->second : def->getDataAt(i);
out->insertData(string_ref.data, string_ref.size);
temporary_keys_pool.rollback(key.size);
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
}*/
query_count.fetch_add(rows, std::memory_order_relaxed);
}
private:
template <typename Value> using MapType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
@ -239,7 +271,7 @@ private:
for (const auto row_idx : ext::range(0, rows))
{
/// calculate key once per row
const auto key = placeKeysInPool(row_idx, key_column_ptrs, keys);
const auto key = placeKeysInPool(row_idx, key_column_ptrs, keys, keys_pool);
auto should_rollback = false;
@ -254,7 +286,7 @@ private:
/// @note on multiple equal keys the mapped value for the first one is stored
if (should_rollback)
keys_pool.rollback(key.size + keys_size * sizeof(StringRef));
keys_pool.rollback(key.size);
}
}
@ -339,35 +371,103 @@ private:
return attr;
}
/*
static std::string createKeyDescription(const DictionaryStructure & dict_struct)
{
std::ostringstream out;
out << '(';
auto first = true;
for (const auto & key : *dict_struct.key)
{
if (!first)
out << ", ";
first = false;
out << key.type->getName();
}
out << ')';
return out.str();
}
void validateKeyColumns(const DataTypes & key_types) const
{
if (key_types.size() != dict_struct.key->size())
throw Exception{
"Key structure does not match, expected " + key_description,
ErrorCodes::TYPE_MISMATCH
};
for (const auto i : ext::range(0, key_types.size()))
{
const auto & expected_type = (*dict_struct.key)[i].type->getName();
const auto & actual_type = key_types[i]->getName();
if (expected_type != actual_type)
throw Exception{
"Key type at position " + std::to_string(i) + " does not match, expected " + expected_type +
", found " + actual_type,
ErrorCodes::TYPE_MISMATCH
};
}
}
template <typename T>
void getItems(const attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out) const
void getItems(const attribute_t & attribute, const ConstColumnPlainPtrs & key_columns, PODArray<T> & out) const
{
const auto & attr = *std::get<MapPointerType<T>>(attribute.maps);
const auto null_value = std::get<T>(attribute.null_values);
for (const auto i : ext::range(0, ids.size()))
const auto keys_size = key_columns.size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
const auto rows = key_columns.front()->size();
for (const auto i : ext::range(0, rows))
{
const auto it = attr.find(ids[i]);
/// copy key data to arena so it is contiguous and return StringRef to it
const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool);
const auto it = attr.find(key);
out[i] = it != attr.end() ? it->second : null_value;
/// free memory allocated for key
temporary_keys_pool.rollback(key.size);
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
query_count.fetch_add(rows, std::memory_order_relaxed);
}
template <typename T>
void getItems(const attribute_t & attribute, const PODArray<id_t> & ids, const PODArray<T> & def, PODArray<T> & out) const
void getItems(
const attribute_t & attribute, const ConstColumnPlainPtrs & key_columns, const PODArray<T> & def,
PODArray<T> & out) const
{
const auto & attr = *std::get<MapPointerType<T>>(attribute.maps);
for (const auto i : ext::range(0, ids.size()))
const auto keys_size = key_columns.size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
const auto rows = key_columns.front()->size();
for (const auto i : ext::range(0, rows))
{
const auto it = attr.find(ids[i]);
/// copy key data to arena so it is contiguous and return StringRef to it
const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool);
const auto it = attr.find(key);
out[i] = it != attr.end() ? it->second : def[i];
/// free memory allocated for key
temporary_keys_pool.rollback(key.size);
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
}*/
query_count.fetch_add(rows, std::memory_order_relaxed);
}
template <typename T>
bool setAttributeValueImpl(attribute_t & attribute, const StringRef key, const T value)
@ -416,7 +516,8 @@ private:
return attributes[it->second];
}
StringRef placeKeysInPool(const std::size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys)
static StringRef placeKeysInPool(
const std::size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
{
const auto keys_size = key_columns.size();
size_t sum_keys_size{};
@ -426,19 +527,15 @@ private:
sum_keys_size += keys[i].size;
}
const auto res = keys_pool.alloc(sum_keys_size + keys_size * sizeof(StringRef));
const auto res = pool.alloc(sum_keys_size);
auto place = res;
for (size_t j = 0; j < keys_size; ++j)
{
memcpy(place, keys[j].data, keys[j].size);
keys[j].data = place;
place += keys[j].size;
}
/// Размещаем в пуле StringRef-ы на только что скопированные ключи.
memcpy(place, &keys[0], keys_size * sizeof(StringRef));
return { res, sum_keys_size };
}
@ -447,6 +544,7 @@ private:
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const bool require_nonempty;
const std::string key_description;
std::map<std::string, std::size_t> attribute_index_by_name;
std::vector<attribute_t> attributes;
@ -462,4 +560,5 @@ private:
std::exception_ptr creation_exception;
};
}

View File

@ -32,7 +32,7 @@ Block createSampleBlock(const DictionaryStructure & dict_struct)
for (const auto & attribute : *dict_struct.key)
{
auto column = attribute.type->createColumn();
column->insert(attribute.null_value);
column->insertDefault();
block.insert(ColumnWithTypeAndName{column, attribute.type, attribute.name});
}

View File

@ -155,7 +155,7 @@ struct DictionaryStructure final
id.emplace(config, config_prefix + ".id");
else if (has_key)
{
key.emplace(getAttributes(config, config_prefix + ".key", false));
key.emplace(getAttributes(config, config_prefix + ".key", false, false));
if (key->empty())
throw Exception{"Empty 'key' supplied", ErrorCodes::BAD_ARGUMENTS};
}
@ -187,7 +187,7 @@ struct DictionaryStructure final
private:
std::vector<DictionaryAttribute> getAttributes(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
const bool hierarchy_allowed = true)
const bool hierarchy_allowed = true, const bool allow_null_values = true)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
@ -211,19 +211,22 @@ private:
if (!expression.empty())
has_expressions = true;
const auto null_value_string = config.getString(prefix + "null_value");
Field null_value;
try
if (allow_null_values)
{
ReadBufferFromString null_value_buffer{null_value_string};
type->deserializeText(null_value, null_value_buffer);
}
catch (const std::exception & e)
{
throw Exception{
std::string{"Error parsing null_value: "} + e.what(),
ErrorCodes::BAD_ARGUMENTS
};
const auto null_value_string = config.getString(prefix + "null_value");
try
{
ReadBufferFromString null_value_buffer{null_value_string};
type->deserializeText(null_value, null_value_buffer);
}
catch (const std::exception & e)
{
throw Exception{
std::string{"Error parsing null_value: "} + e.what(),
ErrorCodes::BAD_ARGUMENTS
};
}
}
const auto hierarchical = config.getBool(prefix + "hierarchical", false);

View File

@ -195,10 +195,9 @@ private:
for (const auto & key : *dict_struct.key)
{
if (!first)
{
writeString(", ", out);
first = false;
}
first = false;
if (!key.expression.empty())
{

View File

@ -5,6 +5,7 @@
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeTuple.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnArray.h>
@ -19,6 +20,7 @@
#include <DB/Dictionaries/HashedDictionary.h>
#include <DB/Dictionaries/CacheDictionary.h>
#include <DB/Dictionaries/RangeHashedDictionary.h>
#include <DB/Dictionaries/ComplexKeyDictionary.h>
#include <ext/range.hpp>
@ -781,11 +783,12 @@ private:
};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()))
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
!typeid_cast<const DataTypeTuple *>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64.",
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
@ -794,7 +797,7 @@ private:
{
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be Date.",
+ ", must be Date.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
@ -817,6 +820,7 @@ private:
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
@ -876,6 +880,52 @@ private:
return true;
}
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const ComplexKeyDictionary *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 3)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & attr_name = attr_name_col->getData();
const auto key_col_with_type = block.getByPosition(arguments[2]);
if (const auto key_col = typeid_cast<const ColumnTuple *>(key_col_with_type.column.get()))
{
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col->getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = new ColumnString;
block.getByPosition(result).column = out;
dict->getString(attr_name, key_columns, key_types, out);
}
else
throw Exception{
"Third argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH
};
return true;
}
template <typename DictionaryType>
bool executeDispatchRange(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
@ -1186,6 +1236,13 @@ template <> struct DictGetTraits<DATA_TYPE>\
dict->get##TYPE(name, ids, out);\
}\
template <typename DictionaryType>\
static void get(\
const DictionaryType * const dict, const std::string & name, const ConstColumnPlainPtrs & key_columns,\
const DataTypes & key_types, PODArray<TYPE> & out)\
{\
dict->get##TYPE(name, key_columns, key_types, out);\
}\
template <typename DictionaryType>\
static void get(\
const DictionaryType * const dict, const std::string & name, const PODArray<UInt64> & ids,\
const PODArray<UInt16> & dates, PODArray<TYPE> & out)\
@ -1199,6 +1256,13 @@ template <> struct DictGetTraits<DATA_TYPE>\
{\
dict->get##TYPE(name, ids, def, out);\
}\
template <typename DictionaryType>\
static void getOrDefault(\
const DictionaryType * const dict, const std::string & name, const ConstColumnPlainPtrs & key_columns,\
const DataTypes & key_types, const PODArray<TYPE> & def, PODArray<TYPE> & out)\
{\
dict->get##TYPE(name, key_columns, key_types, def, out);\
}\
};
DECLARE_DICT_GET_TRAITS(UInt8, DataTypeUInt8)
DECLARE_DICT_GET_TRAITS(UInt16, DataTypeUInt16)
@ -1259,11 +1323,12 @@ private:
};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()))
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
!typeid_cast<const DataTypeTuple *>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64.",
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
@ -1295,6 +1360,7 @@ private:
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
@ -1329,13 +1395,11 @@ private:
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped))
{
const auto out = new ColumnVector<Type>;
const auto out = new ColumnVector<Type>(id_col->size());
block.getByPosition(result).column = out;
const auto & ids = id_col->getData();
auto & data = out->getData();
const auto size = ids.size();
data.resize(size);
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
}
@ -1358,6 +1422,54 @@ private:
return true;
}
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const ComplexKeyDictionary *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 3)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & attr_name = attr_name_col->getData();
const auto key_col_with_type = block.getByPosition(arguments[2]);
if (const auto key_col = typeid_cast<const ColumnTuple *>(key_col_with_type.column.get()))
{
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col->getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = new ColumnVector<Type>(key_columns.front()->size());
block.getByPosition(result).column = out;
auto & data = out->getData();
DictGetTraits<DataType>::get(dict, attr_name, key_columns, key_types, data);
}
else
throw Exception{
"Third argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH
};
return true;
}
template <typename DictionaryType>
bool executeDispatchRange(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
@ -1538,11 +1650,12 @@ private:
};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()))
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
!typeid_cast<const DataTypeTuple *>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64.",
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}