Merge pull request #9735 from ClickHouse/weak-hash

WeakHash32
This commit is contained in:
Nikolai Kochetov 2020-03-25 18:11:15 +03:00 committed by GitHub
commit a6c85ff97d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
28 changed files with 1276 additions and 0 deletions

View File

@ -10,6 +10,8 @@
#include <Common/AlignedBuffer.h>
#include <Common/typeid_cast.h>
#include <Common/Arena.h>
#include <Common/WeakHash.h>
#include <Common/HashTable/Hash.h>
#include <AggregateFunctions/AggregateFunctionMLMethod.h>
@ -282,6 +284,25 @@ void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) cons
hash.update(wbuf.str().c_str(), wbuf.str().size());
}
void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const
{
auto s = data.size();
if (hash.getData().size() != data.size())
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
auto & hash_data = hash.getData();
std::vector<UInt8> v;
for (size_t i = 0; i < s; ++i)
{
WriteBufferFromVector<std::vector<UInt8>> wbuf(v);
func->serialize(data[i], wbuf);
wbuf.finalize();
hash_data[i] = ::updateWeakHash32(v.data(), v.size(), hash_data[i]);
}
}
/// The returned size is less than real size. The reason is that some parts of
/// aggregate function data may be allocated on shared arenas. These arenas are
/// used for several blocks, and also may be updated concurrently from other

View File

@ -160,6 +160,8 @@ public:
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
size_t byteSize() const override;
size_t allocatedBytes() const override;

View File

@ -17,6 +17,8 @@
#include <Common/SipHash.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
#include <Common/HashTable/Hash.h>
namespace DB
@ -213,6 +215,36 @@ void ColumnArray::updateHashWithValue(size_t n, SipHash & hash) const
getData().updateHashWithValue(offset + i, hash);
}
void ColumnArray::updateWeakHash32(WeakHash32 & hash) const
{
auto s = offsets->size();
if (hash.getData().size() != s)
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
WeakHash32 internal_hash(data->size());
data->updateWeakHash32(internal_hash);
Offset prev_offset = 0;
auto & offsets_data = getOffsets();
auto & hash_data = hash.getData();
auto & internal_hash_data = internal_hash.getData();
for (size_t i = 0; i < s; ++i)
{
/// This row improves hash a little bit according to integration tests.
/// It is the same as to use previous hash value as the first element of array.
hash_data[i] = intHashCRC32(hash_data[i]);
for (size_t row = prev_offset; row < offsets_data[i]; ++row)
/// It is probably not the best way to combine hashes.
/// But much better then xor which lead to similar hash for arrays like [1], [1, 1, 1], [1, 1, 1, 1, 1], ...
/// Much better implementation - to add offsets as an optional argument to updateWeakHash32.
hash_data[i] = intHashCRC32(internal_hash_data[row], hash_data[i]);
prev_offset = offsets_data[i];
}
}
void ColumnArray::insert(const Field & x)
{

View File

@ -61,6 +61,7 @@ public:
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
void insert(const Field & x) override;
void insertFrom(const IColumn & src_, size_t n) override;

View File

@ -4,6 +4,8 @@
#include <Columns/ColumnsCommon.h>
#include <Common/PODArray.h>
#include <Common/typeid_cast.h>
#include <Common/WeakHash.h>
#include <Common/HashTable/Hash.h>
namespace DB
@ -12,6 +14,7 @@ namespace DB
namespace ErrorCodes
{
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int LOGICAL_ERROR;
}
ColumnConst::ColumnConst(const ColumnPtr & data_, size_t s_)
@ -103,4 +106,18 @@ void ColumnConst::getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_d
res[i] = i;
}
void ColumnConst::updateWeakHash32(WeakHash32 & hash) const
{
if (hash.getData().size() != s)
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
WeakHash32 element_hash(1);
data->updateWeakHash32(element_hash);
size_t data_hash = element_hash.getData()[0];
for (auto & value : hash.getData())
value = intHashCRC32(data_hash, value);
}
}

View File

@ -163,6 +163,8 @@ public:
data->updateHashWithValue(0, hash);
}
void updateWeakHash32(WeakHash32 & hash) const override;
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
ColumnPtr replicate(const Offsets & offsets) const override;
ColumnPtr permute(const Permutation & perm, size_t limit) const override;

View File

@ -2,6 +2,8 @@
#include <Common/Arena.h>
#include <Common/SipHash.h>
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
#include <Common/HashTable/Hash.h>
#include <common/unaligned.h>
@ -22,6 +24,7 @@ namespace ErrorCodes
extern const int PARAMETER_OUT_OF_BOUND;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
}
template <typename T>
@ -65,6 +68,27 @@ void ColumnDecimal<T>::updateHashWithValue(size_t n, SipHash & hash) const
hash.update(data[n]);
}
template <typename T>
void ColumnDecimal<T>::updateWeakHash32(WeakHash32 & hash) const
{
auto s = data.size();
if (hash.getData().size() != s)
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
const T * begin = data.data();
const T * end = begin + s;
UInt32 * hash_data = hash.getData().data();
while (begin < end)
{
*hash_data = intHashCRC32(*begin, *hash_data);
++begin;
++hash_data;
}
}
template <typename T>
void ColumnDecimal<T>::getPermutation(bool reverse, size_t limit, int , IColumn::Permutation & res) const
{

View File

@ -105,6 +105,7 @@ public:
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override;
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override;

View File

@ -6,6 +6,8 @@
#include <Common/memcpySmall.h>
#include <Common/memcmpSmall.h>
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
#include <Common/HashTable/Hash.h>
#include <DataStreams/ColumnGathererStream.h>
@ -25,6 +27,7 @@ namespace ErrorCodes
extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int PARAMETER_OUT_OF_BOUND;
extern const int LOGICAL_ERROR;
}
@ -101,6 +104,26 @@ void ColumnFixedString::updateHashWithValue(size_t index, SipHash & hash) const
hash.update(reinterpret_cast<const char *>(&chars[n * index]), n);
}
void ColumnFixedString::updateWeakHash32(WeakHash32 & hash) const
{
auto s = size();
if (hash.getData().size() != s)
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
const UInt8 * pos = chars.data();
UInt32 * hash_data = hash.getData().data();
for (size_t row = 0; row < s; ++row)
{
*hash_data = ::updateWeakHash32(pos, n, *hash_data);
pos += n;
++hash_data;
}
}
template <bool positive>
struct ColumnFixedString::less
{

View File

@ -108,6 +108,8 @@ public:
void updateHashWithValue(size_t index, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override
{
const ColumnFixedString & rhs = assert_cast<const ColumnFixedString &>(rhs_);

View File

@ -101,6 +101,11 @@ public:
throw Exception("updateHashWithValue is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void updateWeakHash32(WeakHash32 &) const override
{
throw Exception("updateWeakHash32 is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void popBack(size_t) override
{
throw Exception("popBack is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);

View File

@ -4,6 +4,7 @@
#include <DataTypes/NumberTraits.h>
#include <Common/HashTable/HashMap.h>
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
namespace DB
@ -241,6 +242,21 @@ const char * ColumnLowCardinality::deserializeAndInsertFromArena(const char * po
return new_pos;
}
void ColumnLowCardinality::updateWeakHash32(WeakHash32 & hash) const
{
auto s = size();
if (hash.getData().size() != s)
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
auto & dict = getDictionary().getNestedColumn();
WeakHash32 dict_hash(dict->size());
dict->updateWeakHash32(dict_hash);
idx.updateWeakHash(hash, dict_hash);
}
void ColumnLowCardinality::gather(ColumnGathererStream & gatherer)
{
gatherer.gather(*this);
@ -645,6 +661,24 @@ bool ColumnLowCardinality::Index::containsDefault() const
return contains;
}
void ColumnLowCardinality::Index::updateWeakHash(WeakHash32 & hash, WeakHash32 & dict_hash) const
{
auto & hash_data = hash.getData();
auto & dict_hash_data = dict_hash.getData();
auto update_weak_hash = [&](auto x)
{
using CurIndexType = decltype(x);
auto & data = getPositionsData<CurIndexType>();
auto size = data.size();
for (size_t i = 0; i < size; ++i)
hash_data[i] = intHashCRC32(dict_hash_data[data[i]], hash_data[i]);
};
callForType(std::move(update_weak_hash), size_of_type);
}
ColumnLowCardinality::Dictionary::Dictionary(MutableColumnPtr && column_unique_, bool is_shared)
: column_unique(std::move(column_unique_)), shared(is_shared)

View File

@ -90,6 +90,8 @@ public:
return getDictionary().updateHashWithValue(getIndexes().getUInt(n), hash);
}
void updateWeakHash32(WeakHash32 & hash) const override;
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override
{
return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().filter(filt, result_size_hint));
@ -230,6 +232,8 @@ public:
bool containsDefault() const;
void updateWeakHash(WeakHash32 & hash, WeakHash32 & dict_hash) const;
private:
WrappedPtr positions;
size_t size_of_type = 0;

View File

@ -3,6 +3,7 @@
#include <Common/NaNUtils.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnConst.h>
#include <DataStreams/ColumnGathererStream.h>
@ -42,6 +43,26 @@ void ColumnNullable::updateHashWithValue(size_t n, SipHash & hash) const
getNestedColumn().updateHashWithValue(n, hash);
}
void ColumnNullable::updateWeakHash32(WeakHash32 & hash) const
{
auto s = size();
if (hash.getData().size() != s)
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
WeakHash32 old_hash = hash;
nested_column->updateWeakHash32(hash);
auto & null_map_data = getNullMapData();
auto & hash_data = hash.getData();
auto & old_hash_data = old_hash.getData();
/// Use old data for nulls.
for (size_t row = 0; row < s; ++row)
if (null_map_data[row])
hash_data[row] = old_hash_data[row];
}
MutableColumnPtr ColumnNullable::cloneResized(size_t new_size) const
{

View File

@ -84,6 +84,7 @@ public:
void protect() override;
ColumnPtr replicate(const Offsets & replicate_offsets) const override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void getExtremes(Field & min, Field & max) const override;
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override

View File

@ -2,6 +2,8 @@
#include <Common/Arena.h>
#include <Common/memcmpSmall.h>
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
#include <Common/HashTable/Hash.h>
#include <Columns/Collator.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsCommon.h>
@ -17,6 +19,7 @@ namespace ErrorCodes
{
extern const int PARAMETER_OUT_OF_BOUND;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int LOGICAL_ERROR;
}
@ -63,6 +66,30 @@ MutableColumnPtr ColumnString::cloneResized(size_t to_size) const
return res;
}
void ColumnString::updateWeakHash32(WeakHash32 & hash) const
{
auto s = offsets.size();
if (hash.getData().size() != s)
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
const UInt8 * pos = chars.data();
UInt32 * hash_data = hash.getData().data();
Offset prev_offset = 0;
for (auto & offset : offsets)
{
auto str_size = offset - prev_offset;
/// Skip last zero byte.
*hash_data = ::updateWeakHash32(pos, str_size - 1, *hash_data);
pos += str_size;
prev_offset = offset;
++hash_data;
}
}
void ColumnString::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{

View File

@ -188,6 +188,8 @@ public:
hash.update(reinterpret_cast<const char *>(&chars[offset]), string_size);
}
void updateWeakHash32(WeakHash32 & hash) const override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;

View File

@ -6,6 +6,7 @@
#include <ext/range.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
#include <Core/Field.h>
@ -17,6 +18,7 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
extern const int NOT_IMPLEMENTED;
extern const int CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE;
extern const int LOGICAL_ERROR;
}
@ -180,6 +182,18 @@ void ColumnTuple::updateHashWithValue(size_t n, SipHash & hash) const
column->updateHashWithValue(n, hash);
}
void ColumnTuple::updateWeakHash32(WeakHash32 & hash) const
{
auto s = size();
if (hash.getData().size() != s)
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
for (auto & column : columns)
column->updateWeakHash32(hash);
}
void ColumnTuple::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{
const size_t tuple_size = columns.size();

View File

@ -61,6 +61,7 @@ public:
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
ColumnPtr permute(const Permutation & perm, size_t limit) const override;

View File

@ -9,6 +9,8 @@
#include <Common/NaNUtils.h>
#include <Common/RadixSort.h>
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
#include <Common/HashTable/Hash.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnsCommon.h>
@ -27,6 +29,7 @@ namespace ErrorCodes
{
extern const int PARAMETER_OUT_OF_BOUND;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int LOGICAL_ERROR;
}
@ -51,6 +54,27 @@ void ColumnVector<T>::updateHashWithValue(size_t n, SipHash & hash) const
hash.update(data[n]);
}
template <typename T>
void ColumnVector<T>::updateWeakHash32(WeakHash32 & hash) const
{
auto s = data.size();
if (hash.getData().size() != s)
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
const T * begin = data.data();
const T * end = begin + s;
UInt32 * hash_data = hash.getData().data();
while (begin < end)
{
*hash_data = intHashCRC32(*begin, *hash_data);
++begin;
++hash_data;
}
}
template <typename T>
struct ColumnVector<T>::less
{

View File

@ -160,6 +160,8 @@ public:
void updateHashWithValue(size_t n, SipHash & hash) const override;
void updateWeakHash32(WeakHash32 & hash) const override;
size_t byteSize() const override
{
return data.size() * sizeof(data[0]);

View File

@ -23,6 +23,7 @@ namespace ErrorCodes
class Arena;
class ColumnGathererStream;
class Field;
class WeakHash32;
/// Declares interface to store columns in memory.
class IColumn : public COW<IColumn>
@ -200,6 +201,11 @@ public:
/// passed bytes to hash must identify sequence of values unambiguously.
virtual void updateHashWithValue(size_t n, SipHash & hash) const = 0;
/// Update hash function value. Hash is calculated for each element.
/// It's a fast weak hash function. Mainly need to scatter data between threads.
/// WeakHash32 must have the same size as column.
virtual void updateWeakHash32(WeakHash32 & hash) const = 0;
/** Removes elements that don't match the filter.
* Is used in WHERE and HAVING operations.
* If result_size_hint > 0, then makes advance reserve(result_size_hint) for the result column;

View File

@ -65,6 +65,10 @@ public:
{
}
void updateWeakHash32(WeakHash32 & /*hash*/) const override
{
}
void insertFrom(const IColumn &, size_t) override
{
++s;

View File

@ -136,6 +136,11 @@ public:
{
throw Exception("Method scatter is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
}
void updateWeakHash32(WeakHash32 &) const override
{
throw Exception("Method updateWeakHash32 is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
}
};
using ColumnUniquePtr = IColumnUnique::ColumnUniquePtr;

View File

@ -0,0 +1,868 @@
#include <gtest/gtest.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypesNumber.h>
#include <Common/WeakHash.h>
#include <unordered_map>
#include <iostream>
#include <Common/hex.h>
using namespace DB;
template <typename T>
void checkColumn(
const WeakHash32::Container & hash,
const PaddedPODArray<T> & eq_class,
const std::function<std::string(size_t)> & print_function,
size_t allowed_collisions = 0,
size_t max_collisions_to_print = 10)
{
ASSERT_EQ(hash.size(), eq_class.size());
auto print_for_row = [&](size_t row)
{
std::string res = "row: " + std::to_string(row);
res += " hash: " + std::to_string(hash[row]);
res += " value: " + print_function(row);
return res;
};
/// Check equal rows has equal hash.
{
std::unordered_map<T, UInt32> map;
for (size_t i = 0; i < eq_class.size(); ++i)
{
auto & val = eq_class[i];
auto it = map.find(val);
if (it == map.end())
map[val] = hash[i];
else
{
if (it->second != hash[i])
{
std::cout << "Different hashes for the same equivalent class (" << val << "):\n";
std::cout << print_for_row(it->first) << '\n';
std::cout << print_for_row(i) << std::endl;
}
ASSERT_EQ(it->second, hash[i]);
}
}
}
/// Check have not many collisions.
{
std::unordered_map<UInt32, T> map;
size_t num_collisions = 0;
for (size_t i = 0; i < eq_class.size(); ++i)
{
auto & val = eq_class[i];
auto it = map.find(hash[i]);
if (it == map.end())
map[hash[i]] = val;
else if (it->second != val)
{
++num_collisions;
if (num_collisions <= max_collisions_to_print)
{
std::cout << "Collision:\n";
std::cout << print_for_row(it->second) << '\n';
std::cout << print_for_row(i) << std::endl;
}
else if (num_collisions > allowed_collisions)
break;
}
}
ASSERT_LE(num_collisions, allowed_collisions);
}
}
TEST(WeakHash32, ColumnVectorU8)
{
auto col = ColumnUInt8::create();
auto & data = col->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (size_t i = 0; i < 265; ++i)
data.push_back(i);
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), col->getData(), [&](size_t row) { return std::to_string(col->getElement(row)); });
}
TEST(WeakHash32, ColumnVectorI8)
{
auto col = ColumnInt8::create();
auto & data = col->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (int i = -128; i < 128; ++i)
data.push_back(i);
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), col->getData(), [&](size_t row) { return std::to_string(col->getElement(row)); });
}
TEST(WeakHash32, ColumnVectorU16)
{
auto col = ColumnUInt16::create();
auto & data = col->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (size_t i = 0; i < 65536; ++i)
data.push_back(i);
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), col->getData(), [&](size_t row) { return std::to_string(col->getElement(row)); });
}
TEST(WeakHash32, ColumnVectorI16)
{
auto col = ColumnInt16::create();
auto & data = col->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (int i = -32768; i < 32768; ++i)
data.push_back(i);
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), col->getData(), [&](size_t row) { return std::to_string(col->getElement(row)); });
}
TEST(WeakHash32, ColumnVectorU32)
{
auto col = ColumnUInt32::create();
auto & data = col->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (uint64_t i = 0; i < 65536; ++i)
data.push_back(i << 16u);
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), col->getData(), [&](size_t row) { return std::to_string(col->getElement(row)); });
}
TEST(WeakHash32, ColumnVectorI32)
{
auto col = ColumnInt32::create();
auto & data = col->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (int64_t i = -32768; i < 32768; ++i)
data.push_back(i << 16); //-V610
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), col->getData(), [&](size_t row) { return std::to_string(col->getElement(row)); });
}
TEST(WeakHash32, ColumnVectorU64)
{
auto col = ColumnUInt64::create();
auto & data = col->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (uint64_t i = 0; i < 65536; ++i)
data.push_back(i << 32u);
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), col->getData(), [&](size_t row) { return std::to_string(col->getElement(row)); });
}
TEST(WeakHash32, ColumnVectorI64)
{
auto col = ColumnInt64::create();
auto & data = col->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (int64_t i = -32768; i < 32768; ++i)
data.push_back(i << 32); //-V610
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), col->getData(), [&](size_t row) { return std::to_string(col->getElement(row)); });
}
TEST(WeakHash32, ColumnVectorU128)
{
auto col = ColumnUInt128::create();
auto & data = col->getData();
auto eq = ColumnUInt32::create();
auto & eq_data = eq->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (uint64_t i = 0; i < 65536; ++i)
{
UInt128 val(i << 32u, i << 32u);
data.push_back(val);
eq_data.push_back(i);
}
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), eq_data, [&](size_t row) { return col->getElement(row).toHexString(); });
}
TEST(WeakHash32, ColumnDecimal32)
{
auto col = ColumnDecimal<Decimal32>::create(0, 0);
auto & data = col->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (int64_t i = -32768; i < 32768; ++i)
data.push_back(i << 16); //-V610
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), col->getData(), [&](size_t row) { return std::to_string(col->getElement(row)); });
}
TEST(WeakHash32, ColumnDecimal64)
{
auto col = ColumnDecimal<Decimal64>::create(0, 0);
auto & data = col->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (int64_t i = -32768; i < 32768; ++i)
data.push_back(i << 32); //-V610
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), col->getData(), [&](size_t row) { return std::to_string(col->getElement(row)); });
}
TEST(WeakHash32, ColumnDecimal128)
{
auto col = ColumnDecimal<Decimal128>::create(0, 0);
auto & data = col->getData();
auto eq = ColumnUInt32::create();
auto & eq_data = eq->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (uint64_t i = 0; i < 65536; ++i)
{
UInt128 val(i << 32u, i << 32u);
data.push_back(val);
eq_data.push_back(i);
}
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), eq_data, [&](size_t row) { return getHexUIntLowercase(col->getElement(row)); });
}
TEST(WeakHash32, ColumnString_1)
{
auto col = ColumnString::create();
auto eq = ColumnUInt32::create();
auto & data = eq->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (int64_t i = 0; i < 65536; ++i)
{
data.push_back(i);
auto str = std::to_string(i);
col->insertData(str.data(), str.size());
}
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), data, [&](size_t row) { return col->getDataAt(row).toString(); });
}
TEST(WeakHash32, ColumnString_2)
{
auto col = ColumnString::create();
auto eq = ColumnUInt32::create();
auto & data = eq->getData();
/*
* a
* aa
* aaa
* ...
* b
* bb
* bbb
*/
for (int _i [[maybe_unused]] : {1, 2})
{
size_t max_size = 3000;
char letter = 'a';
for (int64_t i = 0; i < 65536; ++i)
{
data.push_back(i);
size_t s = (i % max_size) + 1;
std::string str(s, letter);
col->insertData(str.data(), str.size());
if (s == max_size)
++letter;
}
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
/// Now there is single collision between 'k' * 544 and 'q' * 2512 (which is calculated twice)
size_t allowed_collisions = 4;
checkColumn(hash.getData(), data, [&](size_t row) { return col->getDataAt(row).toString(); }, allowed_collisions);
}
TEST(WeakHash32, ColumnString_3)
{
auto col = ColumnString::create();
auto eq = ColumnUInt32::create();
auto & data = eq->getData();
/*
* a
* a\0
* a\0\0
* ...
* b
* b\0
* b\0\0
*/
for (int _i [[maybe_unused]] : {1, 2})
{
size_t max_size = 3000;
char letter = 'a';
for (int64_t i = 0; i < 65536; ++i)
{
data.push_back(i);
size_t s = (i % max_size) + 1;
std::string str(s,'\0');
str[0] = letter;
col->insertData(str.data(), str.size());
if (s == max_size)
++letter;
}
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), data, [&](size_t row) { return col->getDataAt(row).toString(); });
}
TEST(WeakHash32, ColumnFixedString)
{
size_t max_size = 3000;
auto col = ColumnFixedString::create(max_size);
auto eq = ColumnUInt32::create();
auto & data = eq->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
char letter = 'a';
for (int64_t i = 0; i < 65536; ++i)
{
data.push_back(i);
size_t s = (i % max_size) + 1;
std::string str(s, letter);
col->insertData(str.data(), str.size());
if (s == max_size)
++letter;
}
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), data, [&](size_t row) { return col->getDataAt(row).toString(); });
}
TEST(WeakHash32, ColumnArray)
{
size_t max_size = 3000;
auto val = ColumnUInt32::create();
auto off = ColumnUInt64::create();
auto eq = ColumnUInt32::create();
auto & eq_data = eq->getData();
auto & val_data = val->getData();
auto & off_data = off->getData();
/* [1]
* [1, 1]
* [1, 1, 1]
* ...
* [2]
* [2, 2]
* [2, 2, 2]
* ...
*/
UInt64 cur_off = 0;
for (int _i [[maybe_unused]] : {1, 2})
{
UInt32 cur = 0;
for (int64_t i = 0; i < 65536; ++i)
{
eq_data.push_back(i);
size_t s = (i % max_size) + 1;
cur_off += s;
off_data.push_back(cur_off);
for (size_t j = 0; j < s; ++j)
val_data.push_back(cur);
if (s == max_size)
++cur;
}
}
auto col_arr = ColumnArray::create(std::move(val), std::move(off));
WeakHash32 hash(col_arr->size());
col_arr->updateWeakHash32(hash);
auto print_function = [&col_arr](size_t row)
{
auto & offsets = col_arr->getOffsets();
size_t s = offsets[row] - offsets[row - 1];
auto value = col_arr->getData().getUInt(offsets[row]);
return std::string("[array of size ") + std::to_string(s) + " with values " + std::to_string(value) + "]";
};
checkColumn(hash.getData(), eq_data, print_function);
}
TEST(WeakHash32, ColumnArray_2)
{
auto val = ColumnUInt32::create();
auto off = ColumnUInt64::create();
auto eq = ColumnUInt32::create();
auto & eq_data = eq->getData();
auto & val_data = val->getData();
auto & off_data = off->getData();
UInt64 cur_off = 0;
for (int _i [[maybe_unused]] : {1, 2})
{
for (int64_t i = 0; i < 1000; ++i)
{
for (size_t j = 0; j < 1000; ++j)
{
eq_data.push_back(i * 1000 + j);
cur_off += 2;
off_data.push_back(cur_off);
val_data.push_back(i);
val_data.push_back(j);
}
}
}
auto col_arr = ColumnArray::create(std::move(val), std::move(off));
WeakHash32 hash(col_arr->size());
col_arr->updateWeakHash32(hash);
auto print_function = [&col_arr](size_t row)
{
auto & offsets = col_arr->getOffsets();
auto value1 = col_arr->getData().getUInt(offsets[row]);
auto value2 = col_arr->getData().getUInt(offsets[row] + 1);
return std::string("[") + std::to_string(value1) + ", " + std::to_string(value2) + "]";
};
checkColumn(hash.getData(), eq_data, print_function);
}
TEST(WeakHash32, ColumnArrayArray)
{
size_t max_size = 3000;
auto val = ColumnUInt32::create();
auto off = ColumnUInt64::create();
auto off2 = ColumnUInt64::create();
auto eq = ColumnUInt32::create();
auto & eq_data = eq->getData();
auto & val_data = val->getData();
auto & off_data = off->getData();
auto & off2_data = off2->getData();
/* [[0]]
* [[0], [0]]
* [[0], [0], [0]]
* ...
* [[0, 0]]
* [[0, 0], [0, 0]]
* [[0, 0], [0, 0], [0, 0]]
* ...
*/
UInt64 cur_off = 0;
UInt64 cur_off2 = 0;
for (int _i [[maybe_unused]] : {1, 2})
{
UInt32 cur = 1;
for (int64_t i = 0; i < 3000; ++i)
{
eq_data.push_back(i);
size_t s = (i % max_size) + 1;
cur_off2 += s;
off2_data.push_back(cur_off2);
for (size_t j = 0; j < s; ++j)
{
for (size_t k = 0; k < cur; ++k)
val_data.push_back(0);
cur_off += cur;
off_data.push_back(cur_off);
}
if (s == max_size)
++cur;
}
}
auto col_arr = ColumnArray::create(std::move(val), std::move(off));
auto col_arr_arr = ColumnArray::create(std::move(col_arr), std::move(off2));
WeakHash32 hash(col_arr_arr->size());
col_arr_arr->updateWeakHash32(hash);
auto print_function = [&col_arr_arr](size_t row2)
{
auto & offsets2 = col_arr_arr->getOffsets();
size_t s2 = offsets2[row2] - offsets2[row2 - 1];
auto & arr2 = typeid_cast<const ColumnArray &>(col_arr_arr->getData());
auto & offsets = arr2.getOffsets();
size_t row = offsets2[row2];
size_t s = offsets[row] - offsets[row - 1];
auto value = arr2.getData().getUInt(offsets[row]);
return std::string("[array of size ") + std::to_string(s2) + " with values ["
"[[array of size " + std::to_string(s) + " with values " + std::to_string(value) + "]]";
};
checkColumn(hash.getData(), eq_data, print_function);
}
TEST(WeakHash32, ColumnConst)
{
auto inner_col = ColumnUInt8::create();
inner_col->insert(0);
auto cls = ColumnUInt8::create();
auto & data = cls->getData();
for (size_t i = 0; i < 256; ++i)
data.push_back(0);
auto col_const = ColumnConst::create(std::move(inner_col), 256);
WeakHash32 hash(col_const->size());
col_const->updateWeakHash32(hash);
checkColumn(hash.getData(), data, [&](size_t) { return std::to_string(0); });
}
TEST(WeakHash32, ColumnLowcardinality)
{
auto col = DataTypeLowCardinality(std::make_shared<DataTypeUInt8>()).createColumn();
auto eq = ColumnUInt8::create();
auto & data = eq->getData();
for (int _i [[maybe_unused]] : {1, 2})
{
for (size_t i = 0; i < 65536; ++i)
{
data.push_back(i);
col->insert(i);
}
}
WeakHash32 hash(col->size());
col->updateWeakHash32(hash);
checkColumn(hash.getData(), data, [&](size_t row) { return std::to_string(col->getUInt(row)); });
}
TEST(WeakHash32, ColumnNullable)
{
auto col = ColumnUInt64::create();
auto & data = col->getData();
auto mask = ColumnUInt8::create();
auto & mask_data = mask->getData();
PaddedPODArray<Int64> eq;
for (int _i [[maybe_unused]] : {1, 2})
{
for (uint64_t i = 0; i < 65536; ++i)
{
data.push_back(i << 32u);
mask_data.push_back(i % 7 == 0 ? 1 : 0);
eq.push_back(i % 7 == 0 ? -1 : (i << 32u));
}
}
auto col_null = ColumnNullable::create(std::move(col), std::move(mask));
WeakHash32 hash(col_null->size());
col_null->updateWeakHash32(hash);
checkColumn(hash.getData(), eq, [&](size_t row) { return eq[row] == -1 ? "Null" : std::to_string(eq[row]); });
}
TEST(WeakHash32, ColumnTuple_UInt64_UInt64)
{
auto col1 = ColumnUInt64::create();
auto col2 = ColumnUInt64::create();
auto & data1 = col1->getData();
auto & data2 = col2->getData();
PaddedPODArray<Int32> eq;
for (int _i [[maybe_unused]] : {0, 1, 2, 3})
{
auto l = _i % 2;
for (uint64_t i = 0; i < 65536; ++i)
{
data1.push_back(l);
data2.push_back(i << 32u);
eq.push_back(l * 65536 + i);
}
}
Columns columns;
columns.emplace_back(std::move(col1));
columns.emplace_back(std::move(col2));
auto col_tuple = ColumnTuple::create(std::move(columns));
WeakHash32 hash(col_tuple->size());
col_tuple->updateWeakHash32(hash);
auto print_func = [&](size_t row)
{
std::string l = std::to_string(col_tuple->getColumn(0).getUInt(row));
std::string r = std::to_string(col_tuple->getColumn(1).getUInt(row));
return "(" + l + ", " + r + ")";
};
checkColumn(hash.getData(), eq, print_func);
}
TEST(WeakHash32, ColumnTuple_UInt64_String)
{
auto col1 = ColumnUInt64::create();
auto col2 = ColumnString::create();
auto & data1 = col1->getData();
PaddedPODArray<Int32> eq;
for (int _i [[maybe_unused]] : {0, 1, 2, 3})
{
auto l = _i % 2;
size_t max_size = 3000;
char letter = 'a';
for (int64_t i = 0; i < 65536; ++i)
{
data1.push_back(l);
eq.push_back(l * 65536 + i);
size_t s = (i % max_size) + 1;
std::string str(s, letter);
col2->insertData(str.data(), str.size());
if (s == max_size)
++letter;
}
}
Columns columns;
columns.emplace_back(std::move(col1));
columns.emplace_back(std::move(col2));
auto col_tuple = ColumnTuple::create(std::move(columns));
WeakHash32 hash(col_tuple->size());
col_tuple->updateWeakHash32(hash);
auto print_func = [&](size_t row)
{
std::string l = std::to_string(col_tuple->getColumn(0).getUInt(row));
std::string r = col_tuple->getColumn(1).getDataAt(row).toString();
return "(" + l + ", " + r + ")";
};
size_t allowed_collisions = 8;
checkColumn(hash.getData(), eq, print_func, allowed_collisions);
}
TEST(WeakHash32, ColumnTuple_UInt64_FixedString)
{
size_t max_size = 3000;
auto col1 = ColumnUInt64::create();
auto col2 = ColumnFixedString::create(max_size);
auto & data1 = col1->getData();
PaddedPODArray<Int32> eq;
for (int _i [[maybe_unused]] : {0, 1, 2, 3})
{
auto l = _i % 2;
char letter = 'a';
for (int64_t i = 0; i < 65536; ++i)
{
data1.push_back(l);
eq.push_back(l * 65536 + i);
size_t s = (i % max_size) + 1;
std::string str(s, letter);
col2->insertData(str.data(), str.size());
if (s == max_size)
++letter;
}
}
Columns columns;
columns.emplace_back(std::move(col1));
columns.emplace_back(std::move(col2));
auto col_tuple = ColumnTuple::create(std::move(columns));
WeakHash32 hash(col_tuple->size());
col_tuple->updateWeakHash32(hash);
auto print_func = [&](size_t row)
{
std::string l = std::to_string(col_tuple->getColumn(0).getUInt(row));
std::string r = col_tuple->getColumn(1).getDataAt(row).toString();
return "(" + l + ", " + r + ")";
};
checkColumn(hash.getData(), eq, print_func);
}
TEST(WeakHash32, ColumnTuple_UInt64_Array)
{
size_t max_size = 3000;
auto val = ColumnUInt32::create();
auto off = ColumnUInt64::create();
auto eq = ColumnUInt32::create();
auto & eq_data = eq->getData();
auto & val_data = val->getData();
auto & off_data = off->getData();
auto col1 = ColumnUInt64::create();
auto & data1 = col1->getData();
UInt64 cur_off = 0;
for (int _i [[maybe_unused]] : {0, 1, 2, 3})
{
auto l = _i % 2;
UInt32 cur = 0;
for (int64_t i = 0; i < 65536; ++i)
{
data1.push_back(l);
eq_data.push_back(l * 65536 + i);
size_t s = (i % max_size) + 1;
cur_off += s;
off_data.push_back(cur_off);
for (size_t j = 0; j < s; ++j)
val_data.push_back(cur);
if (s == max_size)
++cur;
}
}
Columns columns;
columns.emplace_back(std::move(col1));
columns.emplace_back(ColumnArray::create(std::move(val), std::move(off)));
auto col_tuple = ColumnTuple::create(std::move(columns));
WeakHash32 hash(col_tuple->size());
col_tuple->updateWeakHash32(hash);
auto print_func = [&](size_t row)
{
std::string l = std::to_string(col_tuple->getColumn(0).getUInt(row));
auto * col_arr = typeid_cast<const ColumnArray *>(col_tuple->getColumnPtr(1).get());
auto & offsets = col_arr->getOffsets();
size_t s = offsets[row] - offsets[row - 1];
auto value = col_arr->getData().getUInt(offsets[row]);
auto r = std::string("[array of size ") + std::to_string(s) + " with values " + std::to_string(value) + "]";
return "(" + l + ", " + r + ")";
};
/// There are 2 collisions right now (repeated 2 times each):
/// (0, [array of size 1212 with values 7]) vs (0, [array of size 2265 with values 17])
/// (0, [array of size 558 with values 5]) vs (1, [array of size 879 with values 21])
size_t allowed_collisions = 8;
checkColumn(hash.getData(), eq_data, print_func, allowed_collisions);
}

View File

@ -2,6 +2,7 @@
#include <Core/Types.h>
#include <Common/UInt128.h>
#include <common/unaligned.h>
#include <type_traits>
@ -58,6 +59,109 @@ inline DB::UInt64 intHashCRC32(DB::UInt64 x)
#endif
}
inline DB::UInt64 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value)
{
#ifdef __SSE4_2__
return _mm_crc32_u64(updated_value, x);
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
return __crc32cd(updated_value, x);
#else
/// On other platforms we do not have CRC32. NOTE This can be confusing.
return intHash64(x) ^ updated_value;
#endif
}
template <typename T>
inline typename std::enable_if<(sizeof(T) > sizeof(DB::UInt64)), DB::UInt64>::type
intHashCRC32(const T & x, DB::UInt64 updated_value)
{
auto * begin = reinterpret_cast<const char *>(&x);
for (size_t i = 0; i < sizeof(T); i += sizeof(UInt64))
{
updated_value = intHashCRC32(unalignedLoad<DB::UInt64>(begin), updated_value);
begin += sizeof(DB::UInt64);
}
return updated_value;
}
inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 updated_value)
{
if (size < 8)
{
DB::UInt64 value = 0;
auto * value_ptr = reinterpret_cast<unsigned char *>(&value);
typedef __attribute__((__aligned__(1))) uint16_t uint16_unaligned_t;
typedef __attribute__((__aligned__(1))) uint32_t uint32_unaligned_t;
/// Adopted code from FastMemcpy.h (memcpy_tiny)
switch (size)
{
case 0:
break;
case 1:
value_ptr[0] = pos[0];
break;
case 2:
*reinterpret_cast<uint16_t *>(value_ptr) = *reinterpret_cast<const uint16_unaligned_t *>(pos);
break;
case 3:
*reinterpret_cast<uint16_t *>(value_ptr) = *reinterpret_cast<const uint16_unaligned_t *>(pos);
value_ptr[2] = pos[2];
break;
case 4:
*reinterpret_cast<uint32_t *>(value_ptr) = *reinterpret_cast<const uint32_unaligned_t *>(pos);
break;
case 5:
*reinterpret_cast<uint32_t *>(value_ptr) = *reinterpret_cast<const uint32_unaligned_t *>(pos);
value_ptr[4] = pos[4];
break;
case 6:
*reinterpret_cast<uint32_t *>(value_ptr) = *reinterpret_cast<const uint32_unaligned_t *>(pos);
*reinterpret_cast<uint16_unaligned_t *>(value_ptr + 4) =
*reinterpret_cast<const uint16_unaligned_t *>(pos + 4);
break;
case 7:
*reinterpret_cast<uint32_t *>(value_ptr) = *reinterpret_cast<const uint32_unaligned_t *>(pos);
*reinterpret_cast<uint32_unaligned_t *>(value_ptr + 3) =
*reinterpret_cast<const uint32_unaligned_t *>(pos + 3);
break;
default:
__builtin_unreachable();
}
value_ptr[7] = size;
return intHashCRC32(value, updated_value);
}
const auto * end = pos + size;
while (pos + 8 <= end)
{
auto word = unalignedLoad<UInt64>(pos);
updated_value = intHashCRC32(word, updated_value);
pos += 8;
}
if (pos < end)
{
/// If string size is not divisible by 8.
/// Lets' assume the string was 'abcdefghXYZ', so it's tail is 'XYZ'.
DB::UInt8 tail_size = end - pos;
/// Load tailing 8 bytes. Word is 'defghXYZ'.
auto word = unalignedLoad<UInt64>(end - 8);
/// Prepare mask which will set other 5 bytes to 0. It is 0xFFFFFFFFFFFFFFFF << 5 = 0xFFFFFF0000000000.
/// word & mask = '\0\0\0\0\0XYZ' (bytes are reversed because of little ending)
word &= (~UInt64(0)) << DB::UInt8(8 * (8 - tail_size));
/// Use least byte to store tail length.
word |= tail_size;
/// Now word is '\3\0\0\0\0XYZ'
updated_value = intHashCRC32(word, updated_value);
}
return updated_value;
}
template <typename T>
inline size_t DefaultHash64(T key)

View File

@ -0,0 +1,2 @@
#include <Common/WeakHash.h>

View File

@ -0,0 +1,27 @@
#pragma once
#include <Common/PODArray.h>
namespace DB
{
/// It's a class which represents the result of weak and fast hash function per row in column.
/// It's usually hardware accelerated CRC32-C.
/// Has function result may be combined to calculate hash for tuples.
///
/// The main purpose why this class needed is to support data initialization. Initially, every bit is 1.
class WeakHash32
{
public:
using Container = PaddedPODArray<UInt32>;
explicit WeakHash32(size_t size) : data(size, ~UInt32(0)) {}
WeakHash32(const WeakHash32 & other) { data.assign(other.data); }
const Container & getData() const { return data; }
Container & getData() { return data; }
private:
PaddedPODArray<UInt32> data;
};
}