2017-04-27 17:16:24 +00:00
|
|
|
#pragma once
|
|
|
|
#include <Columns/ColumnVector.h>
|
|
|
|
#include <Columns/ColumnString.h>
|
2017-04-28 18:33:31 +00:00
|
|
|
#include <Columns/IColumn.h>
|
2017-04-27 17:16:24 +00:00
|
|
|
#include <DataStreams/IProfilingBlockInputStream.h>
|
2017-04-28 18:33:31 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <Dictionaries/DictionaryBlockInputStreamBase.h>
|
2017-04-27 17:16:24 +00:00
|
|
|
#include <Dictionaries/DictionaryStructure.h>
|
2017-04-28 18:33:31 +00:00
|
|
|
#include <Dictionaries/IDictionary.h>
|
2017-06-08 13:35:35 +00:00
|
|
|
#include <ext/range.h>
|
2017-05-04 18:14:23 +00:00
|
|
|
#include <common/logger_useful.h>
|
|
|
|
#include <Core/Names.h>
|
2017-06-15 19:39:35 +00:00
|
|
|
#include <memory>
|
2017-04-27 17:16:24 +00:00
|
|
|
|
2017-05-26 16:08:56 +00:00
|
|
|
namespace DB
|
2017-04-28 18:33:31 +00:00
|
|
|
{
|
2017-04-27 17:16:24 +00:00
|
|
|
|
2018-06-05 19:46:49 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-05-26 16:08:56 +00:00
|
|
|
/*
|
|
|
|
* BlockInputStream implementation for external dictionaries
|
2017-04-27 17:16:24 +00:00
|
|
|
* read() returns single block consisting of the in-memory contents of the dictionaries
|
|
|
|
*/
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
2017-04-28 18:33:31 +00:00
|
|
|
class DictionaryBlockInputStream : public DictionaryBlockInputStreamBase
|
2017-04-27 17:16:24 +00:00
|
|
|
{
|
|
|
|
public:
|
2017-05-26 16:08:56 +00:00
|
|
|
using DictionatyPtr = std::shared_ptr<DictionaryType const>;
|
|
|
|
|
|
|
|
DictionaryBlockInputStream(std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
|
|
|
|
PaddedPODArray<Key> && ids, const Names & column_names);
|
|
|
|
DictionaryBlockInputStream(std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
|
2017-05-04 18:14:23 +00:00
|
|
|
const std::vector<StringRef> & keys, const Names & column_names);
|
2017-04-27 17:16:24 +00:00
|
|
|
|
2017-06-05 09:02:05 +00:00
|
|
|
using GetColumnsFunction =
|
2018-01-10 00:04:08 +00:00
|
|
|
std::function<ColumnsWithTypeAndName(const Columns &, const std::vector<DictionaryAttribute> & attributes)>;
|
2017-06-05 09:02:05 +00:00
|
|
|
// Used to separate key columns format for storage and view.
|
|
|
|
// Calls get_key_columns_function to get key column for dictionary get fuction call
|
|
|
|
// and get_view_columns_function to get key representation.
|
|
|
|
// Now used in trie dictionary, where columns are stored as ip and mask, and are showed as string
|
|
|
|
DictionaryBlockInputStream(std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
|
|
|
|
const Columns & data_columns, const Names & column_names,
|
|
|
|
GetColumnsFunction && get_key_columns_function,
|
|
|
|
GetColumnsFunction && get_view_columns_function);
|
|
|
|
|
2018-01-10 00:04:08 +00:00
|
|
|
String getName() const override
|
|
|
|
{
|
2018-02-21 20:23:27 +00:00
|
|
|
return "Dictionary";
|
2017-05-26 16:08:56 +00:00
|
|
|
}
|
2017-04-27 17:16:24 +00:00
|
|
|
|
2017-05-26 16:08:56 +00:00
|
|
|
protected:
|
|
|
|
Block getBlock(size_t start, size_t size) const override;
|
2017-05-04 18:14:23 +00:00
|
|
|
|
2017-05-26 16:08:56 +00:00
|
|
|
private:
|
2017-05-04 18:14:23 +00:00
|
|
|
// pointer types to getXXX functions
|
|
|
|
// for single key dictionaries
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename Type>
|
2017-05-04 18:14:23 +00:00
|
|
|
using DictionaryGetter = void (DictionaryType::*)(
|
2017-05-26 16:08:56 +00:00
|
|
|
const std::string &, const PaddedPODArray<Key> &, PaddedPODArray<Type> &) const;
|
2017-05-04 18:14:23 +00:00
|
|
|
using DictionaryStringGetter = void (DictionaryType::*)(
|
2017-05-26 16:08:56 +00:00
|
|
|
const std::string &, const PaddedPODArray<Key> &, ColumnString *) const;
|
2017-05-04 18:14:23 +00:00
|
|
|
// for complex complex key dictionaries
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename Type>
|
2017-05-04 18:14:23 +00:00
|
|
|
using GetterByKey = void (DictionaryType::*)(
|
2017-05-29 17:26:45 +00:00
|
|
|
const std::string &, const Columns &, const DataTypes &, PaddedPODArray<Type> & out) const;
|
2017-05-04 18:14:23 +00:00
|
|
|
using StringGetterByKey = void (DictionaryType::*)(
|
2017-05-29 17:26:45 +00:00
|
|
|
const std::string &, const Columns &, const DataTypes &, ColumnString * out) const;
|
2017-05-04 18:14:23 +00:00
|
|
|
|
|
|
|
// call getXXX
|
|
|
|
// for single key dictionaries
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename Type, typename Container>
|
2017-05-04 18:14:23 +00:00
|
|
|
void callGetter(DictionaryGetter<Type> getter, const PaddedPODArray<Key> & ids,
|
2017-05-29 17:26:45 +00:00
|
|
|
const Columns & keys, const DataTypes & data_types,
|
2017-05-26 16:08:56 +00:00
|
|
|
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename Container>
|
2017-05-04 18:14:23 +00:00
|
|
|
void callGetter(DictionaryStringGetter getter, const PaddedPODArray<Key> & ids,
|
2017-05-29 17:26:45 +00:00
|
|
|
const Columns & keys, const DataTypes & data_types,
|
2017-05-26 16:08:56 +00:00
|
|
|
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
|
2017-05-04 18:14:23 +00:00
|
|
|
// for complex complex key dictionaries
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename Type, typename Container>
|
2017-05-04 18:14:23 +00:00
|
|
|
void callGetter(GetterByKey<Type> getter, const PaddedPODArray<Key> & ids,
|
2017-05-29 17:26:45 +00:00
|
|
|
const Columns & keys, const DataTypes & data_types,
|
2017-05-26 16:08:56 +00:00
|
|
|
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename Container>
|
2017-05-04 18:14:23 +00:00
|
|
|
void callGetter(StringGetterByKey getter, const PaddedPODArray<Key> & ids,
|
2017-05-29 17:26:45 +00:00
|
|
|
const Columns & keys, const DataTypes & data_types,
|
2017-05-26 16:08:56 +00:00
|
|
|
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
|
2017-05-04 18:14:23 +00:00
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <template <typename> class Getter, typename StringGetter>
|
2017-06-05 09:02:05 +00:00
|
|
|
Block fillBlock(const PaddedPODArray<Key> & ids, const Columns & keys,
|
|
|
|
const DataTypes & types, ColumnsWithTypeAndName && view) const;
|
2017-05-04 18:14:23 +00:00
|
|
|
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename AttributeType, typename Getter>
|
2017-05-26 16:08:56 +00:00
|
|
|
ColumnPtr getColumnFromAttribute(Getter getter, const PaddedPODArray<Key> & ids,
|
2017-05-29 17:26:45 +00:00
|
|
|
const Columns & keys, const DataTypes & data_types,
|
2017-05-26 16:08:56 +00:00
|
|
|
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename Getter>
|
2017-05-04 18:14:23 +00:00
|
|
|
ColumnPtr getColumnFromStringAttribute(Getter getter, const PaddedPODArray<Key> & ids,
|
2017-05-29 17:26:45 +00:00
|
|
|
const Columns & keys, const DataTypes & data_types,
|
2017-05-26 16:08:56 +00:00
|
|
|
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const;
|
2018-01-10 00:04:08 +00:00
|
|
|
ColumnPtr getColumnFromIds(const PaddedPODArray<Key> & ids) const;
|
2017-05-04 18:14:23 +00:00
|
|
|
|
2017-05-26 16:08:56 +00:00
|
|
|
void fillKeyColumns(const std::vector<StringRef> & keys, size_t start, size_t size,
|
2017-12-15 02:52:38 +00:00
|
|
|
const DictionaryStructure & dictionary_structure, ColumnsWithTypeAndName & columns) const;
|
2017-05-04 18:14:23 +00:00
|
|
|
|
2017-05-26 16:08:56 +00:00
|
|
|
DictionatyPtr dictionary;
|
|
|
|
Names column_names;
|
|
|
|
PaddedPODArray<Key> ids;
|
|
|
|
ColumnsWithTypeAndName key_columns;
|
2017-05-29 17:26:45 +00:00
|
|
|
Poco::Logger * logger;
|
2017-05-26 16:08:56 +00:00
|
|
|
Block (DictionaryBlockInputStream<DictionaryType, Key>::*fillBlockFunction)(
|
2018-01-10 00:04:08 +00:00
|
|
|
const PaddedPODArray<Key> & ids, const Columns& keys,
|
2017-06-05 09:02:05 +00:00
|
|
|
const DataTypes & types, ColumnsWithTypeAndName && view) const;
|
|
|
|
|
|
|
|
Columns data_columns;
|
|
|
|
GetColumnsFunction get_key_columns_function;
|
|
|
|
GetColumnsFunction get_view_columns_function;
|
2018-03-07 14:29:00 +00:00
|
|
|
|
|
|
|
enum class DictionaryKeyType
|
|
|
|
{
|
|
|
|
Id,
|
|
|
|
ComplexKey,
|
|
|
|
Callback
|
|
|
|
};
|
|
|
|
|
|
|
|
DictionaryKeyType key_type;
|
2017-04-27 17:16:24 +00:00
|
|
|
};
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
2017-05-04 18:14:23 +00:00
|
|
|
DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
|
2017-05-26 16:08:56 +00:00
|
|
|
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
|
|
|
|
PaddedPODArray<Key> && ids, const Names& column_names)
|
|
|
|
: DictionaryBlockInputStreamBase(ids.size(), max_block_size),
|
2017-05-29 17:26:45 +00:00
|
|
|
dictionary(std::static_pointer_cast<const DictionaryType>(dictionary)),
|
|
|
|
column_names(column_names), ids(std::move(ids)),
|
|
|
|
logger(&Poco::Logger::get("DictionaryBlockInputStream")),
|
2018-03-07 14:29:00 +00:00
|
|
|
fillBlockFunction(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<DictionaryGetter, DictionaryStringGetter>),
|
|
|
|
key_type(DictionaryKeyType::Id)
|
2017-04-27 17:16:24 +00:00
|
|
|
{
|
2017-05-04 18:14:23 +00:00
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
2017-05-04 18:14:23 +00:00
|
|
|
DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
|
2017-05-26 16:08:56 +00:00
|
|
|
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
|
|
|
|
const std::vector<StringRef> & keys, const Names& column_names)
|
|
|
|
: DictionaryBlockInputStreamBase(keys.size(), max_block_size),
|
2017-05-29 17:26:45 +00:00
|
|
|
dictionary(std::static_pointer_cast<const DictionaryType>(dictionary)), column_names(column_names),
|
|
|
|
logger(&Poco::Logger::get("DictionaryBlockInputStream")),
|
2018-03-07 14:29:00 +00:00
|
|
|
fillBlockFunction(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<GetterByKey, StringGetterByKey>),
|
|
|
|
key_type(DictionaryKeyType::ComplexKey)
|
2017-05-04 18:14:23 +00:00
|
|
|
{
|
2017-12-15 02:52:38 +00:00
|
|
|
const DictionaryStructure & dictionaty_structure = dictionary->getStructure();
|
2017-05-26 16:08:56 +00:00
|
|
|
fillKeyColumns(keys, 0, keys.size(), dictionaty_structure, key_columns);
|
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
2017-06-05 09:02:05 +00:00
|
|
|
DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
|
|
|
|
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
|
|
|
|
const Columns & data_columns, const Names & column_names,
|
|
|
|
GetColumnsFunction && get_key_columns_function,
|
|
|
|
GetColumnsFunction && get_view_columns_function)
|
|
|
|
: DictionaryBlockInputStreamBase(data_columns.front()->size(), max_block_size),
|
|
|
|
dictionary(std::static_pointer_cast<const DictionaryType>(dictionary)), column_names(column_names),
|
|
|
|
logger(&Poco::Logger::get("DictionaryBlockInputStream")),
|
|
|
|
fillBlockFunction(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<GetterByKey, StringGetterByKey>),
|
|
|
|
data_columns(data_columns),
|
2018-03-07 14:29:00 +00:00
|
|
|
get_key_columns_function(get_key_columns_function), get_view_columns_function(get_view_columns_function),
|
|
|
|
key_type(DictionaryKeyType::Callback)
|
2017-06-05 09:02:05 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
2017-05-26 16:08:56 +00:00
|
|
|
Block DictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, size_t length) const
|
|
|
|
{
|
2018-03-07 14:29:00 +00:00
|
|
|
switch (key_type)
|
2017-05-26 16:08:56 +00:00
|
|
|
{
|
2018-03-07 14:29:00 +00:00
|
|
|
case DictionaryKeyType::ComplexKey:
|
2017-06-05 09:02:05 +00:00
|
|
|
{
|
2018-03-07 14:29:00 +00:00
|
|
|
Columns columns;
|
|
|
|
ColumnsWithTypeAndName view_columns;
|
|
|
|
columns.reserve(key_columns.size());
|
|
|
|
for (const auto & key_column : key_columns)
|
|
|
|
{
|
|
|
|
ColumnPtr column = key_column.column->cut(start, length);
|
|
|
|
columns.emplace_back(column);
|
|
|
|
view_columns.emplace_back(column, key_column.type, key_column.name);
|
|
|
|
}
|
|
|
|
return (this->*fillBlockFunction)({}, columns, {}, std::move(view_columns));
|
2017-06-05 09:02:05 +00:00
|
|
|
}
|
2018-03-07 14:29:00 +00:00
|
|
|
case DictionaryKeyType::Id:
|
|
|
|
{
|
|
|
|
PaddedPODArray<Key> block_ids(ids.begin() + start, ids.begin() + start + length);
|
|
|
|
return (this->*fillBlockFunction)(block_ids, {}, {}, {});
|
|
|
|
}
|
|
|
|
case DictionaryKeyType::Callback:
|
2017-06-05 09:02:05 +00:00
|
|
|
{
|
2018-03-07 14:29:00 +00:00
|
|
|
Columns columns;
|
|
|
|
columns.reserve(data_columns.size());
|
|
|
|
for (const auto & data_column : data_columns)
|
|
|
|
columns.push_back(data_column->cut(start, length));
|
|
|
|
const DictionaryStructure & dictionaty_structure = dictionary->getStructure();
|
|
|
|
const auto & attributes = *dictionaty_structure.key;
|
|
|
|
ColumnsWithTypeAndName keys_with_type_and_name = get_key_columns_function(columns, attributes);
|
|
|
|
ColumnsWithTypeAndName view_with_type_and_name = get_view_columns_function(columns, attributes);
|
|
|
|
DataTypes types;
|
|
|
|
columns.clear();
|
|
|
|
for (const auto & key_column : keys_with_type_and_name)
|
|
|
|
{
|
|
|
|
columns.push_back(key_column.column);
|
|
|
|
types.push_back(key_column.type);
|
|
|
|
}
|
|
|
|
return (this->*fillBlockFunction)({}, columns, types, std::move(view_with_type_and_name));
|
2017-06-05 09:02:05 +00:00
|
|
|
}
|
2017-05-26 16:08:56 +00:00
|
|
|
}
|
2018-03-07 14:29:00 +00:00
|
|
|
throw Exception("Unexpected DictionaryKeyType.", ErrorCodes::LOGICAL_ERROR);
|
2017-05-04 18:14:23 +00:00
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
|
|
|
template <typename Type, typename Container>
|
2017-05-04 18:14:23 +00:00
|
|
|
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
|
|
|
|
DictionaryGetter<Type> getter, const PaddedPODArray<Key> & ids,
|
2017-12-01 20:21:35 +00:00
|
|
|
const Columns & /*keys*/, const DataTypes & /*data_types*/,
|
2017-05-26 16:08:56 +00:00
|
|
|
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const
|
2017-05-04 18:14:23 +00:00
|
|
|
{
|
|
|
|
(dictionary.*getter)(attribute.name, ids, container);
|
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
|
|
|
template <typename Container>
|
2017-05-04 18:14:23 +00:00
|
|
|
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
|
|
|
|
DictionaryStringGetter getter, const PaddedPODArray<Key> & ids,
|
2017-12-01 20:21:35 +00:00
|
|
|
const Columns & /*keys*/, const DataTypes & /*data_types*/,
|
2017-05-26 16:08:56 +00:00
|
|
|
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const
|
2017-05-04 18:14:23 +00:00
|
|
|
{
|
|
|
|
(dictionary.*getter)(attribute.name, ids, container);
|
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
|
|
|
template <typename Type, typename Container>
|
2017-05-04 18:14:23 +00:00
|
|
|
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
|
2017-12-01 20:21:35 +00:00
|
|
|
GetterByKey<Type> getter, const PaddedPODArray<Key> & /*ids*/,
|
2017-05-29 17:26:45 +00:00
|
|
|
const Columns & keys, const DataTypes & data_types,
|
2017-05-26 16:08:56 +00:00
|
|
|
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const
|
2017-05-04 18:14:23 +00:00
|
|
|
{
|
|
|
|
(dictionary.*getter)(attribute.name, keys, data_types, container);
|
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
|
|
|
template <typename Container>
|
2017-05-04 18:14:23 +00:00
|
|
|
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
|
2017-12-01 20:21:35 +00:00
|
|
|
StringGetterByKey getter, const PaddedPODArray<Key> & /*ids*/,
|
2017-05-29 17:26:45 +00:00
|
|
|
const Columns & keys, const DataTypes & data_types,
|
2017-05-26 16:08:56 +00:00
|
|
|
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const
|
2017-05-04 18:14:23 +00:00
|
|
|
{
|
|
|
|
(dictionary.*getter)(attribute.name, keys, data_types, container);
|
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
|
|
|
template <template <typename> class Getter, typename StringGetter>
|
2017-05-26 16:08:56 +00:00
|
|
|
Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
|
2018-01-10 00:04:08 +00:00
|
|
|
const PaddedPODArray<Key> & ids, const Columns & keys, const DataTypes & types, ColumnsWithTypeAndName && view) const
|
2017-05-04 18:14:23 +00:00
|
|
|
{
|
|
|
|
std::unordered_set<std::string> names(column_names.begin(), column_names.end());
|
|
|
|
|
2017-06-05 09:02:05 +00:00
|
|
|
DataTypes data_types = types;
|
|
|
|
ColumnsWithTypeAndName block_columns;
|
|
|
|
|
|
|
|
data_types.reserve(keys.size());
|
2017-12-15 02:52:38 +00:00
|
|
|
const DictionaryStructure & dictionaty_structure = dictionary->getStructure();
|
2017-06-05 09:02:05 +00:00
|
|
|
if (data_types.empty() && dictionaty_structure.key)
|
2017-09-08 03:47:27 +00:00
|
|
|
for (const auto & key : *dictionaty_structure.key)
|
2017-06-05 09:02:05 +00:00
|
|
|
data_types.push_back(key.type);
|
|
|
|
|
|
|
|
for (const auto & column : view)
|
|
|
|
if (names.find(column.name) != names.end())
|
|
|
|
block_columns.push_back(column);
|
2017-05-04 18:14:23 +00:00
|
|
|
|
2017-12-15 02:52:38 +00:00
|
|
|
const DictionaryStructure & structure = dictionary->getStructure();
|
2017-04-27 17:16:24 +00:00
|
|
|
|
2017-05-04 18:14:23 +00:00
|
|
|
if (structure.id && names.find(structure.id->name) != names.end())
|
2017-06-05 09:02:05 +00:00
|
|
|
block_columns.emplace_back(getColumnFromIds(ids), std::make_shared<DataTypeUInt64>(), structure.id->name);
|
2017-04-27 17:16:24 +00:00
|
|
|
|
|
|
|
for (const auto idx : ext::range(0, structure.attributes.size()))
|
|
|
|
{
|
|
|
|
const DictionaryAttribute& attribute = structure.attributes[idx];
|
2017-05-26 16:08:56 +00:00
|
|
|
if (names.find(attribute.name) != names.end())
|
2017-04-27 17:16:24 +00:00
|
|
|
{
|
2017-05-04 18:14:23 +00:00
|
|
|
ColumnPtr column;
|
2017-05-26 16:08:56 +00:00
|
|
|
#define GET_COLUMN_FORM_ATTRIBUTE(TYPE) \
|
2017-05-04 18:14:23 +00:00
|
|
|
column = getColumnFromAttribute<TYPE, Getter<TYPE>>( \
|
2017-06-05 09:02:05 +00:00
|
|
|
&DictionaryType::get##TYPE, ids, keys, data_types, attribute, *dictionary)
|
2017-05-04 18:14:23 +00:00
|
|
|
switch (attribute.underlying_type)
|
|
|
|
{
|
2017-05-26 16:08:56 +00:00
|
|
|
case AttributeUnderlyingType::UInt8:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(UInt8);
|
|
|
|
break;
|
|
|
|
case AttributeUnderlyingType::UInt16:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(UInt16);
|
|
|
|
break;
|
|
|
|
case AttributeUnderlyingType::UInt32:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(UInt32);
|
|
|
|
break;
|
|
|
|
case AttributeUnderlyingType::UInt64:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(UInt64);
|
|
|
|
break;
|
2017-11-14 00:08:54 +00:00
|
|
|
case AttributeUnderlyingType::UInt128:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(UInt128);
|
|
|
|
break;
|
2017-05-26 16:08:56 +00:00
|
|
|
case AttributeUnderlyingType::Int8:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(Int8);
|
|
|
|
break;
|
|
|
|
case AttributeUnderlyingType::Int16:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(Int16);
|
|
|
|
break;
|
|
|
|
case AttributeUnderlyingType::Int32:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(Int32);
|
|
|
|
break;
|
|
|
|
case AttributeUnderlyingType::Int64:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(Int64);
|
|
|
|
break;
|
|
|
|
case AttributeUnderlyingType::Float32:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(Float32);
|
|
|
|
break;
|
|
|
|
case AttributeUnderlyingType::Float64:
|
|
|
|
GET_COLUMN_FORM_ATTRIBUTE(Float64);
|
|
|
|
break;
|
|
|
|
case AttributeUnderlyingType::String:
|
|
|
|
{
|
|
|
|
column = getColumnFromStringAttribute<StringGetter>(
|
2017-12-15 02:52:38 +00:00
|
|
|
&DictionaryType::getString, ids, keys, data_types, attribute, *dictionary);
|
2017-05-26 16:08:56 +00:00
|
|
|
break;
|
|
|
|
}
|
2017-05-04 18:14:23 +00:00
|
|
|
}
|
2017-04-27 17:16:24 +00:00
|
|
|
|
2017-06-05 09:02:05 +00:00
|
|
|
block_columns.emplace_back(column, attribute.type, attribute.name);
|
2017-05-04 18:14:23 +00:00
|
|
|
}
|
2017-04-27 17:16:24 +00:00
|
|
|
}
|
2017-06-05 09:02:05 +00:00
|
|
|
return Block(block_columns);
|
2017-04-27 17:16:24 +00:00
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
|
|
|
template <typename AttributeType, typename Getter>
|
2017-04-27 17:16:24 +00:00
|
|
|
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
|
2017-05-04 18:14:23 +00:00
|
|
|
Getter getter, const PaddedPODArray<Key> & ids,
|
2017-05-29 17:26:45 +00:00
|
|
|
const Columns & keys, const DataTypes & data_types,
|
2017-05-26 16:08:56 +00:00
|
|
|
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const
|
2017-04-27 17:16:24 +00:00
|
|
|
{
|
2017-05-04 18:14:23 +00:00
|
|
|
auto size = ids.size();
|
|
|
|
if (!keys.empty())
|
|
|
|
size = keys.front()->size();
|
2017-12-15 02:52:38 +00:00
|
|
|
auto column_vector = ColumnVector<AttributeType>::create(size);
|
2017-05-04 18:14:23 +00:00
|
|
|
callGetter(getter, ids, keys, data_types, column_vector->getData(), attribute, dictionary);
|
2017-12-15 02:52:38 +00:00
|
|
|
return std::move(column_vector);
|
2017-04-27 17:16:24 +00:00
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
|
|
|
template <typename Getter>
|
2017-05-04 18:14:23 +00:00
|
|
|
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromStringAttribute(
|
|
|
|
Getter getter, const PaddedPODArray<Key> & ids,
|
2017-05-29 17:26:45 +00:00
|
|
|
const Columns & keys, const DataTypes & data_types,
|
2017-05-26 16:08:56 +00:00
|
|
|
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const
|
2017-04-27 17:16:24 +00:00
|
|
|
{
|
2017-12-14 01:43:19 +00:00
|
|
|
auto column_string = ColumnString::create();
|
2017-05-04 18:14:23 +00:00
|
|
|
auto ptr = column_string.get();
|
|
|
|
callGetter(getter, ids, keys, data_types, ptr, attribute, dictionary);
|
2017-12-15 02:52:38 +00:00
|
|
|
return std::move(column_string);
|
2017-04-27 17:16:24 +00:00
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
2018-01-10 00:04:08 +00:00
|
|
|
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromIds(const PaddedPODArray<Key> & ids) const
|
2017-04-27 17:16:24 +00:00
|
|
|
{
|
2017-12-14 01:43:19 +00:00
|
|
|
auto column_vector = ColumnVector<UInt64>::create();
|
2017-04-27 17:16:24 +00:00
|
|
|
column_vector->getData().reserve(ids.size());
|
2017-05-26 16:08:56 +00:00
|
|
|
for (UInt64 id : ids)
|
2017-04-27 17:16:24 +00:00
|
|
|
column_vector->insert(id);
|
2017-12-15 02:52:38 +00:00
|
|
|
return std::move(column_vector);
|
2017-04-27 17:16:24 +00:00
|
|
|
}
|
|
|
|
|
2017-09-15 12:16:12 +00:00
|
|
|
template <typename DictionaryType, typename Key>
|
2017-05-04 18:14:23 +00:00
|
|
|
void DictionaryBlockInputStream<DictionaryType, Key>::fillKeyColumns(
|
2017-05-26 16:08:56 +00:00
|
|
|
const std::vector<StringRef> & keys, size_t start, size_t size,
|
2017-12-15 02:52:38 +00:00
|
|
|
const DictionaryStructure & dictionary_structure, ColumnsWithTypeAndName & res) const
|
2017-05-04 18:14:23 +00:00
|
|
|
{
|
2017-12-15 02:52:38 +00:00
|
|
|
MutableColumns columns;
|
|
|
|
columns.reserve(dictionary_structure.key->size());
|
|
|
|
|
2017-05-29 17:26:45 +00:00
|
|
|
for (const DictionaryAttribute & attribute : *dictionary_structure.key)
|
2017-12-15 02:52:38 +00:00
|
|
|
columns.emplace_back(attribute.type->createColumn());
|
|
|
|
|
2017-05-26 16:08:56 +00:00
|
|
|
for (auto idx : ext::range(start, size))
|
2017-05-04 18:14:23 +00:00
|
|
|
{
|
2017-05-26 16:08:56 +00:00
|
|
|
const auto & key = keys[idx];
|
2017-05-04 18:14:23 +00:00
|
|
|
auto ptr = key.data;
|
2017-12-15 02:52:38 +00:00
|
|
|
for (auto & column : columns)
|
|
|
|
ptr = column->deserializeAndInsertFromArena(ptr);
|
2017-05-04 18:14:23 +00:00
|
|
|
}
|
2017-12-15 02:52:38 +00:00
|
|
|
|
2018-06-03 22:11:50 +00:00
|
|
|
for (size_t i = 0, num_columns = columns.size(); i < num_columns; ++i)
|
2017-12-15 02:52:38 +00:00
|
|
|
res.emplace_back(ColumnWithTypeAndName{ std::move(columns[i]), (*dictionary_structure.key)[i].type, (*dictionary_structure.key)[i].name });
|
2017-05-04 18:14:23 +00:00
|
|
|
}
|
|
|
|
|
2017-05-26 16:08:56 +00:00
|
|
|
}
|