Merge pull request #3154 from yandex/fix-hash-ranged-storage-dictionary

Fix hash ranged storage dictionary
This commit is contained in:
alexey-milovidov 2018-09-17 21:47:56 +03:00 committed by GitHub
commit 4bcc2f3600
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 103 additions and 44 deletions

View File

@ -18,7 +18,7 @@ namespace DB
* BlockInputStream implementation for external dictionaries
* read() returns single block consisting of the in-memory contents of the dictionaries
*/
template <typename DictionaryType, typename Key>
template <typename DictionaryType, typename RangeType, typename Key>
class RangeDictionaryBlockInputStream : public DictionaryBlockInputStreamBase
{
public:
@ -26,7 +26,7 @@ public:
RangeDictionaryBlockInputStream(
DictionaryPtr dictionary, size_t max_block_size, const Names & column_names, PaddedPODArray<Key> && ids_to_fill,
PaddedPODArray<Int64> && start_dates, PaddedPODArray<Int64> && end_dates);
PaddedPODArray<RangeType> && start_dates, PaddedPODArray<RangeType> && end_dates);
String getName() const override
{
@ -57,35 +57,35 @@ private:
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const;
Block fillBlock(const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<Int64> & block_start_dates, const PaddedPODArray<Int64> & block_end_dates) const;
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const;
PaddedPODArray<Int64> makeDateKey(
const PaddedPODArray<Int64> & block_start_dates, const PaddedPODArray<Int64> & block_end_dates) const;
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const;
DictionaryPtr dictionary;
Names column_names;
PaddedPODArray<Key> ids;
PaddedPODArray<Int64> start_dates;
PaddedPODArray<Int64> end_dates;
PaddedPODArray<RangeType> start_dates;
PaddedPODArray<RangeType> end_dates;
};
template <typename DictionaryType, typename Key>
RangeDictionaryBlockInputStream<DictionaryType, Key>::RangeDictionaryBlockInputStream(
template <typename DictionaryType, typename RangeType, typename Key>
RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::RangeDictionaryBlockInputStream(
DictionaryPtr dictionary, size_t max_column_size, const Names & column_names, PaddedPODArray<Key> && ids,
PaddedPODArray<Int64> && block_start_dates, PaddedPODArray<Int64> && block_end_dates)
PaddedPODArray<RangeType> && block_start_dates, PaddedPODArray<RangeType> && block_end_dates)
: DictionaryBlockInputStreamBase(ids.size(), max_column_size),
dictionary(dictionary), column_names(column_names),
ids(std::move(ids)), start_dates(std::move(block_start_dates)), end_dates(std::move(block_end_dates))
{
}
template <typename DictionaryType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, size_t length) const
template <typename DictionaryType, typename RangeType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getBlock(size_t start, size_t length) const
{
PaddedPODArray<Key> block_ids;
PaddedPODArray<Int64> block_start_dates;
PaddedPODArray<Int64> block_end_dates;
PaddedPODArray<RangeType> block_start_dates;
PaddedPODArray<RangeType> block_end_dates;
block_ids.reserve(length);
block_start_dates.reserve(length);
block_end_dates.reserve(length);
@ -100,9 +100,9 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t star
return fillBlock(block_ids, block_start_dates, block_end_dates);
}
template <typename DictionaryType, typename Key>
template <typename DictionaryType, typename RangeType, typename Key>
template <typename AttributeType>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getColumnFromAttribute(
DictionaryGetter<AttributeType> getter, const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<Int64> & dates, const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const
{
@ -111,8 +111,8 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAtt
return column_vector;
}
template <typename DictionaryType, typename Key>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttributeString(
template <typename DictionaryType, typename RangeType, typename Key>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getColumnFromAttributeString(
const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<Int64> & dates,
const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const
{
@ -121,9 +121,9 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAtt
return column_string;
}
template <typename DictionaryType, typename Key>
template <typename DictionaryType, typename RangeType, typename Key>
template <typename T>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromPODArray(const PaddedPODArray<T> & array) const
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getColumnFromPODArray(const PaddedPODArray<T> & array) const
{
auto column_vector = ColumnVector<T>::create();
column_vector->getData().reserve(array.size());
@ -133,9 +133,9 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromPOD
}
template <typename DictionaryType, typename Key>
template <typename DictionaryType, typename RangeType, typename Key>
template <typename DictionarySpecialAttributeType, typename T>
void RangeDictionaryBlockInputStream<DictionaryType, Key>::addSpecialColumn(
void RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::addSpecialColumn(
const std::optional<DictionarySpecialAttributeType> & attribute, DataTypePtr type,
const std::string & default_name, const std::unordered_set<std::string> & column_names_set,
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const
@ -148,9 +148,9 @@ void RangeDictionaryBlockInputStream<DictionaryType, Key>::addSpecialColumn(
columns.emplace_back(getColumnFromPODArray(values), type, name);
}
template <typename DictionaryType, typename Key>
PaddedPODArray<Int64> RangeDictionaryBlockInputStream<DictionaryType, Key>::makeDateKey(
const PaddedPODArray<Int64> & block_start_dates, const PaddedPODArray<Int64> & block_end_dates) const
template <typename DictionaryType, typename RangeType, typename Key>
PaddedPODArray<Int64> RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::makeDateKey(
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const
{
PaddedPODArray<Int64> key(block_start_dates.size());
for (size_t i = 0; i < key.size(); ++i)
@ -165,10 +165,10 @@ PaddedPODArray<Int64> RangeDictionaryBlockInputStream<DictionaryType, Key>::make
}
template <typename DictionaryType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
template <typename DictionaryType, typename RangeType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::fillBlock(
const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<Int64> & block_start_dates, const PaddedPODArray<Int64> & block_end_dates) const
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const
{
ColumnsWithTypeAndName columns;
const DictionaryStructure & structure = dictionary->getStructure();

View File

@ -2,6 +2,7 @@
#include <Dictionaries/RangeDictionaryBlockInputStream.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnNullable.h>
#include <Common/TypeList.h>
#include <ext/range.h>
@ -14,7 +15,7 @@ using RangeStorageType = DB::RangeHashedDictionary::RangeStorageType;
// To simplify comparison, null value of min bound should be bigger than any other value,
// and null value of maxbound - less than any value.
const RangeStorageType RANGE_MIN_NULL_VALUE = std::numeric_limits<RangeStorageType>::max();
const RangeStorageType RANGE_MAX_NULL_VALUE = std::numeric_limits<RangeStorageType>::min();
const RangeStorageType RANGE_MAX_NULL_VALUE = std::numeric_limits<RangeStorageType>::lowest();
// Handle both kinds of null values: explicit nulls of NullableColumn and 'implicit' nulls of Date type.
RangeStorageType getColumnIntValueOrDefault(const DB::IColumn & column, size_t index, bool isDate, const RangeStorageType & default_value)
@ -170,7 +171,7 @@ void RangeHashedDictionary::loadData()
const auto & id_column = *block.safeGetByPosition(0).column;
// Support old behaviour, where invalid date means 'open range'.
const bool is_date = isDate(block.safeGetByPosition(1).type);
const bool is_date = isDate(dict_struct.range_min->type);
const auto & min_range_column = unwrapNullableColumn(*block.safeGetByPosition(1).column);
const auto & max_range_column = unwrapNullableColumn(*block.safeGetByPosition(2).column);
@ -184,11 +185,22 @@ void RangeHashedDictionary::loadData()
for (const auto row_idx : ext::range(0, id_column.size()))
{
const auto min = getColumnIntValueOrDefault(min_range_column, row_idx, is_date, RANGE_MIN_NULL_VALUE);
const auto max = getColumnIntValueOrDefault(max_range_column, row_idx, is_date, RANGE_MAX_NULL_VALUE);
RangeStorageType lower_bound;
RangeStorageType upper_bound;
if (is_date)
{
lower_bound = getColumnIntValueOrDefault(min_range_column, row_idx, is_date, 0);
upper_bound = getColumnIntValueOrDefault(max_range_column, row_idx, is_date, DATE_LUT_MAX_DAY_NUM + 1);
}
else
{
lower_bound = getColumnIntValueOrDefault(min_range_column, row_idx, is_date, RANGE_MIN_NULL_VALUE);
upper_bound = getColumnIntValueOrDefault(max_range_column, row_idx, is_date, RANGE_MAX_NULL_VALUE);
}
setAttributeValue(attribute, id_column.getUInt(row_idx),
Range{min, max},
Range{lower_bound, upper_bound},
attribute_column[row_idx]);
}
}
@ -416,9 +428,10 @@ const RangeHashedDictionary::Attribute & RangeHashedDictionary::getAttributeWith
return attribute;
}
template <typename RangeType>
void RangeHashedDictionary::getIdsAndDates(PaddedPODArray<Key> & ids,
PaddedPODArray<RangeStorageType> & start_dates,
PaddedPODArray<RangeStorageType> & end_dates) const
PaddedPODArray<RangeType> & start_dates,
PaddedPODArray<RangeType> & end_dates) const
{
const auto & attribute = attributes.front();
@ -439,10 +452,10 @@ void RangeHashedDictionary::getIdsAndDates(PaddedPODArray<Key> & ids,
}
}
template <typename T>
template <typename T, typename RangeType>
void RangeHashedDictionary::getIdsAndDates(const Attribute & attribute, PaddedPODArray<Key> & ids,
PaddedPODArray<RangeStorageType> & start_dates,
PaddedPODArray<RangeStorageType> & end_dates) const
PaddedPODArray<RangeType> & start_dates,
PaddedPODArray<RangeType> & end_dates) const
{
const HashMap<UInt64, Values<T>> & attr = *std::get<Ptr<T>>(attribute.maps);
@ -450,6 +463,8 @@ void RangeHashedDictionary::getIdsAndDates(const Attribute & attribute, PaddedPO
start_dates.reserve(attr.size());
end_dates.reserve(attr.size());
const bool is_date = isDate(dict_struct.range_min->type);
for (const auto & key : attr)
{
for (const auto & value : key.second)
@ -457,22 +472,60 @@ void RangeHashedDictionary::getIdsAndDates(const Attribute & attribute, PaddedPO
ids.push_back(key.first);
start_dates.push_back(value.range.left);
end_dates.push_back(value.range.right);
if (is_date && static_cast<UInt64>(end_dates.back()) > DATE_LUT_MAX_DAY_NUM)
end_dates.back() = 0;
}
}
}
BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const
template <typename RangeType>
BlockInputStreamPtr RangeHashedDictionary::getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const
{
PaddedPODArray<Key> ids;
PaddedPODArray<RangeStorageType> start_dates;
PaddedPODArray<RangeStorageType> end_dates;
PaddedPODArray<RangeType> start_dates;
PaddedPODArray<RangeType> end_dates;
getIdsAndDates(ids, start_dates, end_dates);
using BlockInputStreamType = RangeDictionaryBlockInputStream<RangeHashedDictionary, Key>;
using BlockInputStreamType = RangeDictionaryBlockInputStream<RangeHashedDictionary, RangeType, Key>;
auto dict_ptr = std::static_pointer_cast<const RangeHashedDictionary>(shared_from_this());
return std::make_shared<BlockInputStreamType>(
dict_ptr, max_block_size, column_names, std::move(ids), std::move(start_dates), std::move(end_dates));
}
struct RangeHashedDIctionaryCallGetBlockInputStreamImpl
{
BlockInputStreamPtr stream;
const RangeHashedDictionary * dict;
const Names * column_names;
size_t max_block_size;
template <typename RangeType, size_t>
void operator()()
{
auto & type = dict->dict_struct.range_min->type;
if (!stream && dynamic_cast<const DataTypeNumberBase<RangeType> *>(type.get()))
stream = dict->getBlockInputStreamImpl<RangeType>(*column_names, max_block_size);
}
};
BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const
{
using ListType = TypeList<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
RangeHashedDIctionaryCallGetBlockInputStreamImpl callable;
callable.dict = this;
callable.column_names = &column_names;
callable.max_block_size = max_block_size;
ListType::forEach(callable);
if (!callable.stream)
throw Exception("Unexpected range type for RangeHashed dictionary: " + dict_struct.range_min->type->getName(),
ErrorCodes::LOGICAL_ERROR);
return callable.stream;
}
}

View File

@ -162,12 +162,18 @@ private:
const Attribute & getAttributeWithType(const std::string & name, const AttributeUnderlyingType type) const;
template <typename RangeType>
void getIdsAndDates(PaddedPODArray<Key> & ids,
PaddedPODArray<RangeStorageType> & start_dates, PaddedPODArray<RangeStorageType> & end_dates) const;
PaddedPODArray<RangeType> & start_dates, PaddedPODArray<RangeType> & end_dates) const;
template <typename T>
template <typename T, typename RangeType>
void getIdsAndDates(const Attribute & attribute, PaddedPODArray<Key> & ids,
PaddedPODArray<RangeStorageType> & start_dates, PaddedPODArray<RangeStorageType> & end_dates) const;
PaddedPODArray<RangeType> & start_dates, PaddedPODArray<RangeType> & end_dates) const;
template <typename RangeType>
BlockInputStreamPtr getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const;
friend struct RangeHashedDIctionaryCallGetBlockInputStreamImpl;
const std::string dictionary_name;
const DictionaryStructure dict_struct;