Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Ivan Blinkov 2019-02-06 20:52:05 +03:00
commit 40719edf38
84 changed files with 3001 additions and 1899 deletions

View File

@ -39,10 +39,20 @@ add_library(base64 ${LINK_MODE}
${LIBRARY_DIR}/lib/codecs.h ${LIBRARY_DIR}/lib/codecs.h
${CMAKE_CURRENT_BINARY_DIR}/config.h) ${CMAKE_CURRENT_BINARY_DIR}/config.h)
set_source_files_properties(${LIBRARY_DIR}/lib/arch/avx/codec.c PROPERTIES COMPILE_FLAGS -mavx) if(HAVE_AVX)
set_source_files_properties(${LIBRARY_DIR}/lib/arch/avx2/codec.c PROPERTIES COMPILE_FLAGS -mavx2) set_source_files_properties(${LIBRARY_DIR}/lib/arch/avx/codec.c PROPERTIES COMPILE_FLAGS -mavx)
set_source_files_properties(${LIBRARY_DIR}/lib/arch/sse41/codec.c PROPERTIES COMPILE_FLAGS -msse4.1) endif()
set_source_files_properties(${LIBRARY_DIR}/lib/arch/sse42/codec.c PROPERTIES COMPILE_FLAGS -msse4.2) if(HAVE_AVX2)
set_source_files_properties(${LIBRARY_DIR}/lib/arch/ssse3/codec.c PROPERTIES COMPILE_FLAGS -mssse3) set_source_files_properties(${LIBRARY_DIR}/lib/arch/avx2/codec.c PROPERTIES COMPILE_FLAGS -mavx2)
endif()
if(HAVE_SSE41)
set_source_files_properties(${LIBRARY_DIR}/lib/arch/sse41/codec.c PROPERTIES COMPILE_FLAGS -msse4.1)
endif()
if(HAVE_SSE42)
set_source_files_properties(${LIBRARY_DIR}/lib/arch/sse42/codec.c PROPERTIES COMPILE_FLAGS -msse4.2)
endif()
if(HAVE_SSSE3)
set_source_files_properties(${LIBRARY_DIR}/lib/arch/ssse3/codec.c PROPERTIES COMPILE_FLAGS -mssse3)
endif()
target_include_directories(base64 PRIVATE ${LIBRARY_DIR}/include ${CMAKE_CURRENT_BINARY_DIR}) target_include_directories(base64 PRIVATE ${LIBRARY_DIR}/include ${CMAKE_CURRENT_BINARY_DIR})

View File

@ -817,7 +817,7 @@ public:
try try
{ {
type->deserializeTextQuoted(*column_dummy, rb, FormatSettings()); type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings());
} }
catch (Exception & e) catch (Exception & e)
{ {
@ -1882,7 +1882,7 @@ protected:
for (size_t i = 0; i < column.column->size(); ++i) for (size_t i = 0; i < column.column->size(); ++i)
{ {
WriteBufferFromOwnString wb; WriteBufferFromOwnString wb;
column.type->serializeTextQuoted(*column.column, i, wb, FormatSettings()); column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings());
res.emplace(wb.str()); res.emplace(wb.str());
} }
} }

View File

@ -116,37 +116,38 @@ std::string ReportBuilder::buildFullReport(
/// in seconds /// in seconds
runJSON.set("min_time", statistics.min_time / double(1000)); runJSON.set("min_time", statistics.min_time / double(1000));
JSONString quantiles(4); /// here, 4 is the size of \t padding if (statistics.sampler.size() != 0)
for (double percent = 10; percent <= 90; percent += 10)
{ {
std::string quantile_key = std::to_string(percent / 100.0); JSONString quantiles(4); /// here, 4 is the size of \t padding
while (quantile_key.back() == '0') for (double percent = 10; percent <= 90; percent += 10)
quantile_key.pop_back(); {
std::string quantile_key = std::to_string(percent / 100.0);
while (quantile_key.back() == '0')
quantile_key.pop_back();
quantiles.set(quantile_key, quantiles.set(quantile_key,
statistics.sampler.quantileInterpolated(percent / 100.0)); statistics.sampler.quantileInterpolated(percent / 100.0));
}
quantiles.set("0.95",
statistics.sampler.quantileInterpolated(95 / 100.0));
quantiles.set("0.99",
statistics.sampler.quantileInterpolated(99 / 100.0));
quantiles.set("0.999",
statistics.sampler.quantileInterpolated(99.9 / 100.0));
quantiles.set("0.9999",
statistics.sampler.quantileInterpolated(99.99 / 100.0));
runJSON.set("quantiles", quantiles.asString());
} }
quantiles.set("0.95",
statistics.sampler.quantileInterpolated(95 / 100.0));
quantiles.set("0.99",
statistics.sampler.quantileInterpolated(99 / 100.0));
quantiles.set("0.999",
statistics.sampler.quantileInterpolated(99.9 / 100.0));
quantiles.set("0.9999",
statistics.sampler.quantileInterpolated(99.99 / 100.0));
runJSON.set("quantiles", quantiles.asString());
runJSON.set("total_time", statistics.total_time); runJSON.set("total_time", statistics.total_time);
runJSON.set("queries_per_second", if (statistics.total_time != 0)
static_cast<double>(statistics.queries) / statistics.total_time); {
runJSON.set("queries_per_second", static_cast<double>(statistics.queries) / statistics.total_time);
runJSON.set("rows_per_second", runJSON.set("rows_per_second", static_cast<double>(statistics.total_rows_read) / statistics.total_time);
static_cast<double>(statistics.total_rows_read) / statistics.total_time); runJSON.set("bytes_per_second", static_cast<double>(statistics.total_bytes_read) / statistics.total_time);
}
runJSON.set("bytes_per_second",
static_cast<double>(statistics.total_bytes_read) / statistics.total_time);
} }
else else
{ {

View File

@ -138,7 +138,7 @@ public:
StringRef getRawData() const override { return StringRef(chars.data(), chars.size()); } StringRef getRawData() const override { return StringRef(chars.data(), chars.size()); }
/// Specialized part of interface, not from IColumn. /// Specialized part of interface, not from IColumn.
void insertString(const String & string) { insertData(string.c_str(), string.size()); }
Chars & getChars() { return chars; } Chars & getChars() { return chars; }
const Chars & getChars() const { return chars; } const Chars & getChars() const { return chars; }

View File

@ -0,0 +1,557 @@
#pragma once
#include <Common/ColumnsHashingImpl.h>
#include <Common/Arena.h>
#include <Common/LRUCache.h>
#include <common/unaligned.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnLowCardinality.h>
#include <Core/Defines.h>
#include <memory>
namespace DB
{
namespace ColumnsHashing
{
/// For the case when there is one numeric key.
/// UInt8/16/32/64 for any type with corresponding bit width.
template <typename Value, typename Mapped, typename FieldType, bool use_cache = true>
struct HashMethodOneNumber
: public columns_hashing_impl::HashMethodBase<HashMethodOneNumber<Value, Mapped, FieldType, use_cache>, Value, Mapped, use_cache>
{
using Self = HashMethodOneNumber<Value, Mapped, FieldType, use_cache>;
using Base = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
const char * vec;
/// If the keys of a fixed length then key_sizes contains their lengths, empty otherwise.
HashMethodOneNumber(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &)
{
vec = key_columns[0]->getRawData().data;
}
/// Creates context. Method is called once and result context is used in all threads.
using Base::createContext; /// (const HashMethodContext::Settings &) -> HashMethodContextPtr
/// Emplace key into HashTable or HashMap. If Data is HashMap, returns ptr to value, otherwise nullptr.
/// Data is a HashTable where to insert key from column's row.
/// For Serialized method, key may be placed in pool.
using Base::emplaceKey; /// (Data & data, size_t row, Arena & pool) -> EmplaceResult
/// Find key into HashTable or HashMap. If Data is HashMap and key was found, returns ptr to value, otherwise nullptr.
using Base::findKey; /// (Data & data, size_t row, Arena & pool) -> FindResult
/// Get hash value of row.
using Base::getHash; /// (const Data & data, size_t row, Arena & pool) -> size_t
/// Is used for default implementation in HashMethodBase.
FieldType getKey(size_t row, Arena &) const { return unalignedLoad<FieldType>(vec + row * sizeof(FieldType)); }
/// Get StringRef from value which can be inserted into column.
static StringRef getValueRef(const Value & value)
{
return StringRef(reinterpret_cast<const char *>(&value.first), sizeof(value.first));
}
};
/// For the case when there is one string key.
template <typename Value, typename Mapped, bool place_string_to_arena = true, bool use_cache = true>
struct HashMethodString
: public columns_hashing_impl::HashMethodBase<HashMethodString<Value, Mapped, place_string_to_arena, use_cache>, Value, Mapped, use_cache>
{
using Self = HashMethodString<Value, Mapped, place_string_to_arena, use_cache>;
using Base = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
const IColumn::Offset * offsets;
const UInt8 * chars;
HashMethodString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &)
{
const IColumn & column = *key_columns[0];
const ColumnString & column_string = static_cast<const ColumnString &>(column);
offsets = column_string.getOffsets().data();
chars = column_string.getChars().data();
}
auto getKey(ssize_t row, Arena &) const
{
return StringRef(chars + offsets[row - 1], offsets[row] - offsets[row - 1] - 1);
}
static StringRef getValueRef(const Value & value) { return StringRef(value.first.data, value.first.size); }
protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
static ALWAYS_INLINE void onNewKey([[maybe_unused]] StringRef & key, [[maybe_unused]] Arena & pool)
{
if constexpr (place_string_to_arena)
{
if (key.size)
key.data = pool.insert(key.data, key.size);
}
}
};
/// For the case when there is one fixed-length string key.
template <typename Value, typename Mapped, bool place_string_to_arena = true, bool use_cache = true>
struct HashMethodFixedString
: public columns_hashing_impl::HashMethodBase<HashMethodFixedString<Value, Mapped, place_string_to_arena, use_cache>, Value, Mapped, use_cache>
{
using Self = HashMethodFixedString<Value, Mapped, place_string_to_arena, use_cache>;
using Base = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
size_t n;
const ColumnFixedString::Chars * chars;
HashMethodFixedString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &)
{
const IColumn & column = *key_columns[0];
const ColumnFixedString & column_string = static_cast<const ColumnFixedString &>(column);
n = column_string.getN();
chars = &column_string.getChars();
}
StringRef getKey(size_t row, Arena &) const { return StringRef(&(*chars)[row * n], n); }
static StringRef getValueRef(const Value & value) { return StringRef(value.first.data, value.first.size); }
protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
static ALWAYS_INLINE void onNewKey([[maybe_unused]] StringRef & key, [[maybe_unused]] Arena & pool)
{
if constexpr (place_string_to_arena)
key.data = pool.insert(key.data, key.size);
}
};
/// Cache stores dictionaries and saved_hash per dictionary key.
class LowCardinalityDictionaryCache : public HashMethodContext
{
public:
/// Will assume that dictionaries with same hash has the same keys.
/// Just in case, check that they have also the same size.
struct DictionaryKey
{
UInt128 hash;
UInt64 size;
bool operator== (const DictionaryKey & other) const { return hash == other.hash && size == other.size; }
};
struct DictionaryKeyHash
{
size_t operator()(const DictionaryKey & key) const
{
SipHash hash;
hash.update(key.hash.low);
hash.update(key.hash.high);
hash.update(key.size);
return hash.get64();
}
};
struct CachedValues
{
/// Store ptr to dictionary to be sure it won't be deleted.
ColumnPtr dictionary_holder;
/// Hashes for dictionary keys.
const UInt64 * saved_hash = nullptr;
};
using CachedValuesPtr = std::shared_ptr<CachedValues>;
explicit LowCardinalityDictionaryCache(const HashMethodContext::Settings & settings) : cache(settings.max_threads) {}
CachedValuesPtr get(const DictionaryKey & key) { return cache.get(key); }
void set(const DictionaryKey & key, const CachedValuesPtr & mapped) { cache.set(key, mapped); }
private:
using Cache = LRUCache<DictionaryKey, CachedValues, DictionaryKeyHash>;
Cache cache;
};
/// Single low cardinality column.
template <typename SingleColumnMethod, typename Mapped, bool use_cache>
struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
{
using Base = SingleColumnMethod;
enum class VisitValue
{
Empty = 0,
Found = 1,
NotFound = 2,
};
static constexpr bool has_mapped = !std::is_same<Mapped, void>::value;
using EmplaceResult = columns_hashing_impl::EmplaceResultImpl<Mapped>;
using FindResult = columns_hashing_impl::FindResultImpl<Mapped>;
static HashMethodContextPtr createContext(const HashMethodContext::Settings & settings)
{
return std::make_shared<LowCardinalityDictionaryCache>(settings);
}
ColumnRawPtrs key_columns;
const IColumn * positions = nullptr;
size_t size_of_index_type = 0;
/// saved hash is from current column or from cache.
const UInt64 * saved_hash = nullptr;
/// Hold dictionary in case saved_hash is from cache to be sure it won't be deleted.
ColumnPtr dictionary_holder;
/// Cache AggregateDataPtr for current column in order to decrease the number of hash table usages.
columns_hashing_impl::MappedCache<Mapped> mapped_cache;
PaddedPODArray<VisitValue> visit_cache;
/// If initialized column is nullable.
bool is_nullable = false;
static const ColumnLowCardinality & getLowCardinalityColumn(const IColumn * low_cardinality_column)
{
auto column = typeid_cast<const ColumnLowCardinality *>(low_cardinality_column);
if (!column)
throw Exception("Invalid aggregation key type for HashMethodSingleLowCardinalityColumn method. "
"Excepted LowCardinality, got " + column->getName(), ErrorCodes::LOGICAL_ERROR);
return *column;
}
HashMethodSingleLowCardinalityColumn(
const ColumnRawPtrs & key_columns_low_cardinality, const Sizes & key_sizes, const HashMethodContextPtr & context)
: Base({getLowCardinalityColumn(key_columns_low_cardinality[0]).getDictionary().getNestedNotNullableColumn().get()}, key_sizes, context)
{
auto column = &getLowCardinalityColumn(key_columns_low_cardinality[0]);
if (!context)
throw Exception("Cache wasn't created for HashMethodSingleLowCardinalityColumn",
ErrorCodes::LOGICAL_ERROR);
LowCardinalityDictionaryCache * cache;
if constexpr (use_cache)
{
cache = typeid_cast<LowCardinalityDictionaryCache *>(context.get());
if (!cache)
{
const auto & cached_val = *context;
throw Exception("Invalid type for HashMethodSingleLowCardinalityColumn cache: "
+ demangle(typeid(cached_val).name()), ErrorCodes::LOGICAL_ERROR);
}
}
auto * dict = column->getDictionary().getNestedNotNullableColumn().get();
is_nullable = column->getDictionary().nestedColumnIsNullable();
key_columns = {dict};
bool is_shared_dict = column->isSharedDictionary();
typename LowCardinalityDictionaryCache::DictionaryKey dictionary_key;
typename LowCardinalityDictionaryCache::CachedValuesPtr cached_values;
if (is_shared_dict)
{
dictionary_key = {column->getDictionary().getHash(), dict->size()};
if constexpr (use_cache)
cached_values = cache->get(dictionary_key);
}
if (cached_values)
{
saved_hash = cached_values->saved_hash;
dictionary_holder = cached_values->dictionary_holder;
}
else
{
saved_hash = column->getDictionary().tryGetSavedHash();
dictionary_holder = column->getDictionaryPtr();
if constexpr (use_cache)
{
if (is_shared_dict)
{
cached_values = std::make_shared<typename LowCardinalityDictionaryCache::CachedValues>();
cached_values->saved_hash = saved_hash;
cached_values->dictionary_holder = dictionary_holder;
cache->set(dictionary_key, cached_values);
}
}
}
if constexpr (has_mapped)
mapped_cache.resize(key_columns[0]->size());
VisitValue empty(VisitValue::Empty);
visit_cache.assign(key_columns[0]->size(), empty);
size_of_index_type = column->getSizeOfIndexType();
positions = column->getIndexesPtr().get();
}
ALWAYS_INLINE size_t getIndexAt(size_t row) const
{
switch (size_of_index_type)
{
case sizeof(UInt8): return static_cast<const ColumnUInt8 *>(positions)->getElement(row);
case sizeof(UInt16): return static_cast<const ColumnUInt16 *>(positions)->getElement(row);
case sizeof(UInt32): return static_cast<const ColumnUInt32 *>(positions)->getElement(row);
case sizeof(UInt64): return static_cast<const ColumnUInt64 *>(positions)->getElement(row);
default: throw Exception("Unexpected size of index type for low cardinality column.", ErrorCodes::LOGICAL_ERROR);
}
}
/// Get the key from the key columns for insertion into the hash table.
ALWAYS_INLINE auto getKey(size_t row, Arena & pool) const
{
return Base::getKey(getIndexAt(row), pool);
}
template <typename Data>
ALWAYS_INLINE EmplaceResult emplaceKey(Data & data, size_t row_, Arena & pool)
{
size_t row = getIndexAt(row_);
if (is_nullable && row == 0)
{
visit_cache[row] = VisitValue::Found;
bool has_null_key = data.hasNullKeyData();
data.hasNullKeyData() = true;
if constexpr (has_mapped)
return EmplaceResult(data.getNullKeyData(), mapped_cache[0], !has_null_key);
else
return EmplaceResult(!has_null_key);
}
if (visit_cache[row] == VisitValue::Found)
{
if constexpr (has_mapped)
return EmplaceResult(mapped_cache[row], mapped_cache[row], false);
else
return EmplaceResult(false);
}
auto key = getKey(row_, pool);
bool inserted = false;
typename Data::iterator it;
if (saved_hash)
data.emplace(key, it, inserted, saved_hash[row]);
else
data.emplace(key, it, inserted);
visit_cache[row] = VisitValue::Found;
if (inserted)
{
if constexpr (has_mapped)
{
new(&it->second) Mapped();
Base::onNewKey(it->first, pool);
}
else
Base::onNewKey(*it, pool);
}
if constexpr (has_mapped)
return EmplaceResult(it->second, mapped_cache[row], inserted);
else
return EmplaceResult(inserted);
}
ALWAYS_INLINE bool isNullAt(size_t i)
{
if (!is_nullable)
return false;
return getIndexAt(i) == 0;
}
template <typename Data>
ALWAYS_INLINE FindResult findFromRow(Data & data, size_t row_, Arena & pool)
{
size_t row = getIndexAt(row_);
if (is_nullable && row == 0)
{
if constexpr (has_mapped)
return FindResult(data.hasNullKeyData() ? &data.getNullKeyData() : nullptr, data.hasNullKeyData());
else
return FindResult(data.hasNullKeyData());
}
if (visit_cache[row] != VisitValue::Empty)
{
if constexpr (has_mapped)
return FindResult(&mapped_cache[row], visit_cache[row] == VisitValue::Found);
else
return FindResult(visit_cache[row] == VisitValue::Found);
}
auto key = getKey(row_, pool);
typename Data::iterator it;
if (saved_hash)
it = data.find(key, saved_hash[row]);
else
it = data.find(key);
bool found = it != data.end();
visit_cache[row] = found ? VisitValue::Found : VisitValue::NotFound;
if constexpr (has_mapped)
{
if (found)
mapped_cache[row] = it->second;
}
if constexpr (has_mapped)
return FindResult(&mapped_cache[row], found);
else
return FindResult(found);
}
template <typename Data>
ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & pool)
{
row = getIndexAt(row);
if (saved_hash)
return saved_hash[row];
return Base::getHash(data, row, pool);
}
};
// Optional mask for low cardinality columns.
template <bool has_low_cardinality>
struct LowCardinalityKeys
{
ColumnRawPtrs nested_columns;
ColumnRawPtrs positions;
Sizes position_sizes;
};
template <>
struct LowCardinalityKeys<false> {};
/// For the case when all keys are of fixed length, and they fit in N (for example, 128) bits.
template <typename Value, typename Key, typename Mapped, bool has_nullable_keys_ = false, bool has_low_cardinality_ = false, bool use_cache = true>
struct HashMethodKeysFixed
: private columns_hashing_impl::BaseStateKeysFixed<Key, has_nullable_keys_>
, public columns_hashing_impl::HashMethodBase<HashMethodKeysFixed<Value, Key, Mapped, has_nullable_keys_, has_low_cardinality_, use_cache>, Value, Mapped, use_cache>
{
using Self = HashMethodKeysFixed<Value, Key, Mapped, has_nullable_keys_, has_low_cardinality_, use_cache>;
using BaseHashed = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
using Base = columns_hashing_impl::BaseStateKeysFixed<Key, has_nullable_keys_>;
static constexpr bool has_nullable_keys = has_nullable_keys_;
static constexpr bool has_low_cardinality = has_low_cardinality_;
LowCardinalityKeys<has_low_cardinality> low_cardinality_keys;
Sizes key_sizes;
size_t keys_size;
HashMethodKeysFixed(const ColumnRawPtrs & key_columns, const Sizes & key_sizes, const HashMethodContextPtr &)
: Base(key_columns), key_sizes(std::move(key_sizes)), keys_size(key_columns.size())
{
if constexpr (has_low_cardinality)
{
low_cardinality_keys.nested_columns.resize(key_columns.size());
low_cardinality_keys.positions.assign(key_columns.size(), nullptr);
low_cardinality_keys.position_sizes.resize(key_columns.size());
for (size_t i = 0; i < key_columns.size(); ++i)
{
if (auto * low_cardinality_col = typeid_cast<const ColumnLowCardinality *>(key_columns[i]))
{
low_cardinality_keys.nested_columns[i] = low_cardinality_col->getDictionary().getNestedColumn().get();
low_cardinality_keys.positions[i] = &low_cardinality_col->getIndexes();
low_cardinality_keys.position_sizes[i] = low_cardinality_col->getSizeOfIndexType();
}
else
low_cardinality_keys.nested_columns[i] = key_columns[i];
}
}
}
ALWAYS_INLINE Key getKey(size_t row, Arena &) const
{
if constexpr (has_nullable_keys)
{
auto bitmap = Base::createBitmap(row);
return packFixed<Key>(row, keys_size, Base::getActualColumns(), key_sizes, bitmap);
}
else
{
if constexpr (has_low_cardinality)
return packFixed<Key, true>(row, keys_size, low_cardinality_keys.nested_columns, key_sizes,
&low_cardinality_keys.positions, &low_cardinality_keys.position_sizes);
return packFixed<Key>(row, keys_size, Base::getActualColumns(), key_sizes);
}
}
};
/** Hash by concatenating serialized key values.
* The serialized value differs in that it uniquely allows to deserialize it, having only the position with which it starts.
* That is, for example, for strings, it contains first the serialized length of the string, and then the bytes.
* Therefore, when aggregating by several strings, there is no ambiguity.
*/
template <typename Value, typename Mapped>
struct HashMethodSerialized
: public columns_hashing_impl::HashMethodBase<HashMethodSerialized<Value, Mapped>, Value, Mapped, false>
{
using Self = HashMethodSerialized<Value, Mapped>;
using Base = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, false>;
ColumnRawPtrs key_columns;
size_t keys_size;
HashMethodSerialized(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &)
: key_columns(key_columns), keys_size(key_columns.size()) {}
protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, false>;
ALWAYS_INLINE StringRef getKey(size_t row, Arena & pool) const
{
return serializeKeysToPoolContiguous(row, keys_size, key_columns, pool);
}
static ALWAYS_INLINE void onExistingKey(StringRef & key, Arena & pool) { pool.rollback(key.size); }
};
/// For the case when there is one string key.
template <typename Value, typename Mapped, bool use_cache = true>
struct HashMethodHashed
: public columns_hashing_impl::HashMethodBase<HashMethodHashed<Value, Mapped, use_cache>, Value, Mapped, use_cache>
{
using Key = UInt128;
using Self = HashMethodHashed<Value, Mapped, use_cache>;
using Base = columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
ColumnRawPtrs key_columns;
HashMethodHashed(ColumnRawPtrs key_columns, const Sizes &, const HashMethodContextPtr &)
: key_columns(std::move(key_columns)) {}
ALWAYS_INLINE Key getKey(size_t row, Arena &) const { return hash128(row, key_columns.size(), key_columns); }
static ALWAYS_INLINE StringRef getValueRef(const Value & value)
{
return StringRef(reinterpret_cast<const char *>(&value.first), sizeof(value.first));
}
};
}
}

View File

@ -0,0 +1,356 @@
#pragma once
#include <Columns/IColumn.h>
#include <Interpreters/AggregationCommon.h>
namespace DB
{
namespace ColumnsHashing
{
/// Generic context for HashMethod. Context is shared between multiple threads, all methods must be thread-safe.
/// Is used for caching.
class HashMethodContext
{
public:
virtual ~HashMethodContext() = default;
struct Settings
{
size_t max_threads;
};
};
using HashMethodContextPtr = std::shared_ptr<HashMethodContext>;
namespace columns_hashing_impl
{
template <typename Value, bool consecutive_keys_optimization_>
struct LastElementCache
{
static constexpr bool consecutive_keys_optimization = consecutive_keys_optimization_;
Value value;
bool empty = true;
bool found = false;
bool check(const Value & value_) { return !empty && value == value_; }
template <typename Key>
bool check(const Key & key) { return !empty && value.first == key; }
};
template <typename Data>
struct LastElementCache<Data, false>
{
static constexpr bool consecutive_keys_optimization = false;
};
template <typename Mapped>
class EmplaceResultImpl
{
Mapped & value;
Mapped & cached_value;
bool inserted;
public:
EmplaceResultImpl(Mapped & value, Mapped & cached_value, bool inserted)
: value(value), cached_value(cached_value), inserted(inserted) {}
bool isInserted() const { return inserted; }
auto & getMapped() const { return value; }
void setMapped(const Mapped & mapped)
{
cached_value = mapped;
value = mapped;
}
};
template <>
class EmplaceResultImpl<void>
{
bool inserted;
public:
explicit EmplaceResultImpl(bool inserted) : inserted(inserted) {}
bool isInserted() const { return inserted; }
};
template <typename Mapped>
class FindResultImpl
{
Mapped * value;
bool found;
public:
FindResultImpl(Mapped * value, bool found) : value(value), found(found) {}
bool isFound() const { return found; }
Mapped & getMapped() const { return *value; }
};
template <>
class FindResultImpl<void>
{
bool found;
public:
explicit FindResultImpl(bool found) : found(found) {}
bool isFound() const { return found; }
};
template <typename Derived, typename Value, typename Mapped, bool consecutive_keys_optimization>
class HashMethodBase
{
public:
using EmplaceResult = EmplaceResultImpl<Mapped>;
using FindResult = FindResultImpl<Mapped>;
static constexpr bool has_mapped = !std::is_same<Mapped, void>::value;
using Cache = LastElementCache<Value, consecutive_keys_optimization>;
static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; }
template <typename Data>
ALWAYS_INLINE EmplaceResult emplaceKey(Data & data, size_t row, Arena & pool)
{
auto key = static_cast<Derived &>(*this).getKey(row, pool);
return emplaceKeyImpl(key, data, pool);
}
template <typename Data>
ALWAYS_INLINE FindResult findKey(Data & data, size_t row, Arena & pool)
{
auto key = static_cast<Derived &>(*this).getKey(row, pool);
auto res = findKeyImpl(key, data);
static_cast<Derived &>(*this).onExistingKey(key, pool);
return res;
}
template <typename Data>
ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & pool)
{
auto key = static_cast<Derived &>(*this).getKey(row, pool);
auto res = data.hash(key);
static_cast<Derived &>(*this).onExistingKey(key, pool);
return res;
}
protected:
Cache cache;
HashMethodBase()
{
if constexpr (consecutive_keys_optimization)
{
if constexpr (has_mapped)
{
/// Init PairNoInit elements.
cache.value.second = Mapped();
using Key = decltype(cache.value.first);
cache.value.first = Key();
}
else
cache.value = Value();
}
}
template <typename Key>
static ALWAYS_INLINE void onNewKey(Key & /*key*/, Arena & /*pool*/) {}
template <typename Key>
static ALWAYS_INLINE void onExistingKey(Key & /*key*/, Arena & /*pool*/) {}
template <typename Data, typename Key>
ALWAYS_INLINE EmplaceResult emplaceKeyImpl(Key key, Data & data, Arena & pool)
{
if constexpr (Cache::consecutive_keys_optimization)
{
if (cache.found && cache.check(key))
{
static_cast<Derived &>(*this).onExistingKey(key, pool);
if constexpr (has_mapped)
return EmplaceResult(cache.value.second, cache.value.second, false);
else
return EmplaceResult(false);
}
}
typename Data::iterator it;
bool inserted = false;
data.emplace(key, it, inserted);
[[maybe_unused]] Mapped * cached = nullptr;
if constexpr (has_mapped)
cached = &it->second;
if (inserted)
{
if constexpr (has_mapped)
{
new(&it->second) Mapped();
static_cast<Derived &>(*this).onNewKey(it->first, pool);
}
else
static_cast<Derived &>(*this).onNewKey(*it, pool);
}
else
static_cast<Derived &>(*this).onExistingKey(key, pool);
if constexpr (consecutive_keys_optimization)
{
cache.value = *it;
cache.found = true;
cache.empty = false;
if constexpr (has_mapped)
cached = &cache.value.second;
}
if constexpr (has_mapped)
return EmplaceResult(it->second, *cached, inserted);
else
return EmplaceResult(inserted);
}
template <typename Data, typename Key>
ALWAYS_INLINE FindResult findKeyImpl(Key key, Data & data)
{
if constexpr (Cache::consecutive_keys_optimization)
{
if (cache.check(key))
{
if constexpr (has_mapped)
return FindResult(&cache.value.second, cache.found);
else
return FindResult(cache.found);
}
}
auto it = data.find(key);
bool found = it != data.end();
if constexpr (consecutive_keys_optimization)
{
cache.found = found;
cache.empty = false;
if (found)
cache.value = *it;
else
{
if constexpr (has_mapped)
cache.value.first = key;
else
cache.value = key;
}
}
if constexpr (has_mapped)
return FindResult(found ? &it->second : nullptr, found);
else
return FindResult(found);
}
};
template <typename T>
struct MappedCache : public PaddedPODArray<T> {};
template <>
struct MappedCache<void> {};
/// This class is designed to provide the functionality that is required for
/// supporting nullable keys in HashMethodKeysFixed. If there are
/// no nullable keys, this class is merely implemented as an empty shell.
template <typename Key, bool has_nullable_keys>
class BaseStateKeysFixed;
/// Case where nullable keys are supported.
template <typename Key>
class BaseStateKeysFixed<Key, true>
{
protected:
BaseStateKeysFixed(const ColumnRawPtrs & key_columns)
{
null_maps.reserve(key_columns.size());
actual_columns.reserve(key_columns.size());
for (const auto & col : key_columns)
{
if (col->isColumnNullable())
{
const auto & nullable_col = static_cast<const ColumnNullable &>(*col);
actual_columns.push_back(&nullable_col.getNestedColumn());
null_maps.push_back(&nullable_col.getNullMapColumn());
}
else
{
actual_columns.push_back(col);
null_maps.push_back(nullptr);
}
}
}
/// Return the columns which actually contain the values of the keys.
/// For a given key column, if it is nullable, we return its nested
/// column. Otherwise we return the key column itself.
inline const ColumnRawPtrs & getActualColumns() const
{
return actual_columns;
}
/// Create a bitmap that indicates whether, for a particular row,
/// a key column bears a null value or not.
KeysNullMap<Key> createBitmap(size_t row) const
{
KeysNullMap<Key> bitmap{};
for (size_t k = 0; k < null_maps.size(); ++k)
{
if (null_maps[k] != nullptr)
{
const auto & null_map = static_cast<const ColumnUInt8 &>(*null_maps[k]).getData();
if (null_map[row] == 1)
{
size_t bucket = k / 8;
size_t offset = k % 8;
bitmap[bucket] |= UInt8(1) << offset;
}
}
}
return bitmap;
}
private:
ColumnRawPtrs actual_columns;
ColumnRawPtrs null_maps;
};
/// Case where nullable keys are not supported.
template <typename Key>
class BaseStateKeysFixed<Key, false>
{
protected:
BaseStateKeysFixed(const ColumnRawPtrs & columns) : actual_columns(columns) {}
const ColumnRawPtrs & getActualColumns() const { return actual_columns; }
KeysNullMap<Key> createBitmap(size_t) const
{
throw Exception{"Internal error: calling createBitmap() for non-nullable keys"
" is forbidden", ErrorCodes::LOGICAL_ERROR};
}
private:
ColumnRawPtrs actual_columns;
};
}
}
}

View File

@ -414,6 +414,7 @@ namespace ErrorCodes
extern const int PROTOBUF_FIELD_NOT_REPEATED = 437; extern const int PROTOBUF_FIELD_NOT_REPEATED = 437;
extern const int DATA_TYPE_CANNOT_BE_PROMOTED = 438; extern const int DATA_TYPE_CANNOT_BE_PROMOTED = 438;
extern const int CANNOT_SCHEDULE_TASK = 439; extern const int CANNOT_SCHEDULE_TASK = 439;
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING = 440;
extern const int KEEPER_EXCEPTION = 999; extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000; extern const int POCO_EXCEPTION = 1000;

View File

@ -17,6 +17,7 @@
#include <common/unaligned.h> #include <common/unaligned.h>
#include <string> #include <string>
#include <type_traits> #include <type_traits>
#include <Core/Defines.h>
#define ROTL(x, b) static_cast<UInt64>(((x) << (b)) | ((x) >> (64 - (b)))) #define ROTL(x, b) static_cast<UInt64>(((x) << (b)) | ((x) >> (64 - (b))))
@ -49,7 +50,7 @@ private:
UInt8 current_bytes[8]; UInt8 current_bytes[8];
}; };
void finalize() ALWAYS_INLINE void finalize()
{ {
/// In the last free byte, we write the remainder of the division by 256. /// In the last free byte, we write the remainder of the division by 256.
current_bytes[7] = cnt; current_bytes[7] = cnt;
@ -156,7 +157,7 @@ public:
/// template for avoiding 'unsigned long long' vs 'unsigned long' problem on old poco in macos /// template for avoiding 'unsigned long long' vs 'unsigned long' problem on old poco in macos
template <typename T> template <typename T>
void get128(T & lo, T & hi) ALWAYS_INLINE void get128(T & lo, T & hi)
{ {
static_assert(sizeof(T) == 8); static_assert(sizeof(T) == 8);
finalize(); finalize();

View File

@ -1,12 +1,44 @@
#include <Common/formatIPv6.h> #include <Common/formatIPv6.h>
#include <Common/hex.h> #include <Common/hex.h>
#include <Common/StringUtils/StringUtils.h>
#include <ext/range.h> #include <ext/range.h>
#include <array> #include <array>
#include <algorithm>
namespace DB namespace DB
{ {
// To be used in formatIPv4, maps a byte to it's string form prefixed with length (so save strlen call).
extern const char one_byte_to_string_lookup_table[256][4] = {
{1, '0'}, {1, '1'}, {1, '2'}, {1, '3'}, {1, '4'}, {1, '5'}, {1, '6'}, {1, '7'}, {1, '8'}, {1, '9'},
{2, '1', '0'}, {2, '1', '1'}, {2, '1', '2'}, {2, '1', '3'}, {2, '1', '4'}, {2, '1', '5'}, {2, '1', '6'}, {2, '1', '7'}, {2, '1', '8'}, {2, '1', '9'},
{2, '2', '0'}, {2, '2', '1'}, {2, '2', '2'}, {2, '2', '3'}, {2, '2', '4'}, {2, '2', '5'}, {2, '2', '6'}, {2, '2', '7'}, {2, '2', '8'}, {2, '2', '9'},
{2, '3', '0'}, {2, '3', '1'}, {2, '3', '2'}, {2, '3', '3'}, {2, '3', '4'}, {2, '3', '5'}, {2, '3', '6'}, {2, '3', '7'}, {2, '3', '8'}, {2, '3', '9'},
{2, '4', '0'}, {2, '4', '1'}, {2, '4', '2'}, {2, '4', '3'}, {2, '4', '4'}, {2, '4', '5'}, {2, '4', '6'}, {2, '4', '7'}, {2, '4', '8'}, {2, '4', '9'},
{2, '5', '0'}, {2, '5', '1'}, {2, '5', '2'}, {2, '5', '3'}, {2, '5', '4'}, {2, '5', '5'}, {2, '5', '6'}, {2, '5', '7'}, {2, '5', '8'}, {2, '5', '9'},
{2, '6', '0'}, {2, '6', '1'}, {2, '6', '2'}, {2, '6', '3'}, {2, '6', '4'}, {2, '6', '5'}, {2, '6', '6'}, {2, '6', '7'}, {2, '6', '8'}, {2, '6', '9'},
{2, '7', '0'}, {2, '7', '1'}, {2, '7', '2'}, {2, '7', '3'}, {2, '7', '4'}, {2, '7', '5'}, {2, '7', '6'}, {2, '7', '7'}, {2, '7', '8'}, {2, '7', '9'},
{2, '8', '0'}, {2, '8', '1'}, {2, '8', '2'}, {2, '8', '3'}, {2, '8', '4'}, {2, '8', '5'}, {2, '8', '6'}, {2, '8', '7'}, {2, '8', '8'}, {2, '8', '9'},
{2, '9', '0'}, {2, '9', '1'}, {2, '9', '2'}, {2, '9', '3'}, {2, '9', '4'}, {2, '9', '5'}, {2, '9', '6'}, {2, '9', '7'}, {2, '9', '8'}, {2, '9', '9'},
{3, '1', '0', '0'}, {3, '1', '0', '1'}, {3, '1', '0', '2'}, {3, '1', '0', '3'}, {3, '1', '0', '4'}, {3, '1', '0', '5'}, {3, '1', '0', '6'}, {3, '1', '0', '7'}, {3, '1', '0', '8'}, {3, '1', '0', '9'},
{3, '1', '1', '0'}, {3, '1', '1', '1'}, {3, '1', '1', '2'}, {3, '1', '1', '3'}, {3, '1', '1', '4'}, {3, '1', '1', '5'}, {3, '1', '1', '6'}, {3, '1', '1', '7'}, {3, '1', '1', '8'}, {3, '1', '1', '9'},
{3, '1', '2', '0'}, {3, '1', '2', '1'}, {3, '1', '2', '2'}, {3, '1', '2', '3'}, {3, '1', '2', '4'}, {3, '1', '2', '5'}, {3, '1', '2', '6'}, {3, '1', '2', '7'}, {3, '1', '2', '8'}, {3, '1', '2', '9'},
{3, '1', '3', '0'}, {3, '1', '3', '1'}, {3, '1', '3', '2'}, {3, '1', '3', '3'}, {3, '1', '3', '4'}, {3, '1', '3', '5'}, {3, '1', '3', '6'}, {3, '1', '3', '7'}, {3, '1', '3', '8'}, {3, '1', '3', '9'},
{3, '1', '4', '0'}, {3, '1', '4', '1'}, {3, '1', '4', '2'}, {3, '1', '4', '3'}, {3, '1', '4', '4'}, {3, '1', '4', '5'}, {3, '1', '4', '6'}, {3, '1', '4', '7'}, {3, '1', '4', '8'}, {3, '1', '4', '9'},
{3, '1', '5', '0'}, {3, '1', '5', '1'}, {3, '1', '5', '2'}, {3, '1', '5', '3'}, {3, '1', '5', '4'}, {3, '1', '5', '5'}, {3, '1', '5', '6'}, {3, '1', '5', '7'}, {3, '1', '5', '8'}, {3, '1', '5', '9'},
{3, '1', '6', '0'}, {3, '1', '6', '1'}, {3, '1', '6', '2'}, {3, '1', '6', '3'}, {3, '1', '6', '4'}, {3, '1', '6', '5'}, {3, '1', '6', '6'}, {3, '1', '6', '7'}, {3, '1', '6', '8'}, {3, '1', '6', '9'},
{3, '1', '7', '0'}, {3, '1', '7', '1'}, {3, '1', '7', '2'}, {3, '1', '7', '3'}, {3, '1', '7', '4'}, {3, '1', '7', '5'}, {3, '1', '7', '6'}, {3, '1', '7', '7'}, {3, '1', '7', '8'}, {3, '1', '7', '9'},
{3, '1', '8', '0'}, {3, '1', '8', '1'}, {3, '1', '8', '2'}, {3, '1', '8', '3'}, {3, '1', '8', '4'}, {3, '1', '8', '5'}, {3, '1', '8', '6'}, {3, '1', '8', '7'}, {3, '1', '8', '8'}, {3, '1', '8', '9'},
{3, '1', '9', '0'}, {3, '1', '9', '1'}, {3, '1', '9', '2'}, {3, '1', '9', '3'}, {3, '1', '9', '4'}, {3, '1', '9', '5'}, {3, '1', '9', '6'}, {3, '1', '9', '7'}, {3, '1', '9', '8'}, {3, '1', '9', '9'},
{3, '2', '0', '0'}, {3, '2', '0', '1'}, {3, '2', '0', '2'}, {3, '2', '0', '3'}, {3, '2', '0', '4'}, {3, '2', '0', '5'}, {3, '2', '0', '6'}, {3, '2', '0', '7'}, {3, '2', '0', '8'}, {3, '2', '0', '9'},
{3, '2', '1', '0'}, {3, '2', '1', '1'}, {3, '2', '1', '2'}, {3, '2', '1', '3'}, {3, '2', '1', '4'}, {3, '2', '1', '5'}, {3, '2', '1', '6'}, {3, '2', '1', '7'}, {3, '2', '1', '8'}, {3, '2', '1', '9'},
{3, '2', '2', '0'}, {3, '2', '2', '1'}, {3, '2', '2', '2'}, {3, '2', '2', '3'}, {3, '2', '2', '4'}, {3, '2', '2', '5'}, {3, '2', '2', '6'}, {3, '2', '2', '7'}, {3, '2', '2', '8'}, {3, '2', '2', '9'},
{3, '2', '3', '0'}, {3, '2', '3', '1'}, {3, '2', '3', '2'}, {3, '2', '3', '3'}, {3, '2', '3', '4'}, {3, '2', '3', '5'}, {3, '2', '3', '6'}, {3, '2', '3', '7'}, {3, '2', '3', '8'}, {3, '2', '3', '9'},
{3, '2', '4', '0'}, {3, '2', '4', '1'}, {3, '2', '4', '2'}, {3, '2', '4', '3'}, {3, '2', '4', '4'}, {3, '2', '4', '5'}, {3, '2', '4', '6'}, {3, '2', '4', '7'}, {3, '2', '4', '8'}, {3, '2', '4', '9'},
{3, '2', '5', '0'}, {3, '2', '5', '1'}, {3, '2', '5', '2'}, {3, '2', '5', '3'}, {3, '2', '5', '4'}, {3, '2', '5', '5'},
};
/// integer logarithm, return ceil(log(value, base)) (the smallest integer greater or equal than log(value, base) /// integer logarithm, return ceil(log(value, base)) (the smallest integer greater or equal than log(value, base)
static constexpr UInt32 intLog(const UInt32 value, const UInt32 base, const bool carry) static constexpr UInt32 intLog(const UInt32 value, const UInt32 base, const bool carry)
{ {
@ -45,22 +77,6 @@ static void printInteger(char *& out, T value)
} }
} }
/// print IPv4 address as %u.%u.%u.%u
static void formatIPv4(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count)
{
const auto limit = IPV4_BINARY_LENGTH - zeroed_tail_bytes_count;
for (const auto i : ext::range(0, IPV4_BINARY_LENGTH))
{
UInt8 byte = (i < limit) ? src[i] : 0;
printInteger<10, UInt8>(dst, byte);
if (i != IPV4_BINARY_LENGTH - 1)
*dst++ = '.';
}
}
void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count) void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count)
{ {
struct { int base, len; } best{-1, 0}, cur{-1, 0}; struct { int base, len; } best{-1, 0}, cur{-1, 0};
@ -122,8 +138,14 @@ void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_
/// Is this address an encapsulated IPv4? /// Is this address an encapsulated IPv4?
if (i == 6 && best.base == 0 && (best.len == 6 || (best.len == 5 && words[5] == 0xffffu))) if (i == 6 && best.base == 0 && (best.len == 6 || (best.len == 5 && words[5] == 0xffffu)))
{ {
formatIPv4(src + 12, dst, std::min(zeroed_tail_bytes_count, static_cast<UInt8>(IPV4_BINARY_LENGTH))); UInt8 ipv4_buffer[IPV4_BINARY_LENGTH] = {0};
break; memcpy(ipv4_buffer, src + 12, IPV4_BINARY_LENGTH);
// Due to historical reasons formatIPv4() takes ipv4 in BE format, but inside ipv6 we store it in LE-format.
std::reverse(std::begin(ipv4_buffer), std::end(ipv4_buffer));
formatIPv4(ipv4_buffer, dst, std::min(zeroed_tail_bytes_count, static_cast<UInt8>(IPV4_BINARY_LENGTH)), "0");
// formatIPv4 has already added a null-terminator for us.
return;
} }
printInteger<16>(dst, words[i]); printInteger<16>(dst, words[i]);

View File

@ -1,12 +1,17 @@
#pragma once #pragma once
#include <common/Types.h> #include <common/Types.h>
#include <string.h>
#include <algorithm>
#include <utility>
#include <ext/range.h>
#include <Common/hex.h>
#include <Common/StringUtils/StringUtils.h>
#define IPV4_BINARY_LENGTH 4 constexpr size_t IPV4_BINARY_LENGTH = 4;
#define IPV6_BINARY_LENGTH 16 constexpr size_t IPV6_BINARY_LENGTH = 16;
#define IPV4_MAX_TEXT_LENGTH 15 /// Does not count tail zero byte. constexpr size_t IPV4_MAX_TEXT_LENGTH = 15; /// Does not count tail zero byte.
#define IPV6_MAX_TEXT_LENGTH 39 constexpr size_t IPV6_MAX_TEXT_LENGTH = 39;
namespace DB namespace DB
{ {
@ -18,4 +23,205 @@ namespace DB
*/ */
void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count = 0); void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count = 0);
/** Unsafe (no bounds-checking for src nor dst), optimized version of parsing IPv4 string.
*
* Parses the input string `src` and stores binary BE value into buffer pointed by `dst`,
* which should be long enough.
* That is "127.0.0.1" becomes 0x7f000001.
*
* In case of failure returns false and doesn't modify buffer pointed by `dst`.
*
* @param src - input string, expected to be non-null and null-terminated right after the IPv4 string value.
* @param dst - where to put output bytes, expected to be non-null and atleast IPV4_BINARY_LENGTH-long.
* @return false if parsing failed, true otherwise.
*/
inline bool parseIPv4(const char * src, unsigned char * dst)
{
UInt32 result = 0;
for (int offset = 24; offset >= 0; offset -= 8)
{
UInt32 value = 0;
size_t len = 0;
while (isNumericASCII(*src) && len <= 3)
{
value = value * 10 + (*src - '0');
++len;
++src;
}
if (len == 0 || value > 255 || (offset > 0 && *src != '.'))
return false;
result |= value << offset;
++src;
}
if (*(src - 1) != '\0')
return false;
memcpy(dst, &result, sizeof(result));
return true;
}
/** Unsafe (no bounds-checking for src nor dst), optimized version of parsing IPv6 string.
*
* Slightly altered implementation from http://svn.apache.org/repos/asf/apr/apr/trunk/network_io/unix/inet_pton.c
* Parses the input string `src` and stores binary LE value into buffer pointed by `dst`,
* which should be long enough. In case of failure zeroes
* IPV6_BINARY_LENGTH bytes of buffer pointed by `dst`.
*
* @param src - input string, expected to be non-null and null-terminated right after the IPv6 string value.
* @param dst - where to put output bytes, expected to be non-null and atleast IPV6_BINARY_LENGTH-long.
* @return false if parsing failed, true otherwise.
*/
inline bool parseIPv6(const char * src, unsigned char * dst)
{
const auto clear_dst = [dst]()
{
memset(dst, '\0', IPV6_BINARY_LENGTH);
return false;
};
/// Leading :: requires some special handling.
if (*src == ':')
if (*++src != ':')
return clear_dst();
unsigned char tmp[IPV6_BINARY_LENGTH]{};
auto tp = tmp;
auto endp = tp + IPV6_BINARY_LENGTH;
auto curtok = src;
auto saw_xdigit = false;
UInt32 val{};
unsigned char * colonp = nullptr;
/// Assuming zero-terminated string.
while (const auto ch = *src++)
{
const auto num = unhex(ch);
if (num != '\xff')
{
val <<= 4;
val |= num;
if (val > 0xffffu)
return clear_dst();
saw_xdigit = 1;
continue;
}
if (ch == ':')
{
curtok = src;
if (!saw_xdigit)
{
if (colonp)
return clear_dst();
colonp = tp;
continue;
}
if (tp + sizeof(UInt16) > endp)
return clear_dst();
*tp++ = static_cast<unsigned char>((val >> 8) & 0xffu);
*tp++ = static_cast<unsigned char>(val & 0xffu);
saw_xdigit = false;
val = 0;
continue;
}
if (ch == '.' && (tp + IPV4_BINARY_LENGTH) <= endp)
{
if (!parseIPv4(curtok, tp))
return clear_dst();
std::reverse(tp, tp + IPV4_BINARY_LENGTH);
tp += IPV4_BINARY_LENGTH;
saw_xdigit = false;
break; /* '\0' was seen by ipv4_scan(). */
}
return clear_dst();
}
if (saw_xdigit)
{
if (tp + sizeof(UInt16) > endp)
return clear_dst();
*tp++ = static_cast<unsigned char>((val >> 8) & 0xffu);
*tp++ = static_cast<unsigned char>(val & 0xffu);
}
if (colonp)
{
/*
* Since some memmove()'s erroneously fail to handle
* overlapping regions, we'll do the shift by hand.
*/
const auto n = tp - colonp;
for (int i = 1; i <= n; ++i)
{
endp[- i] = colonp[n - i];
colonp[n - i] = 0;
}
tp = endp;
}
if (tp != endp)
return clear_dst();
memcpy(dst, tmp, sizeof(tmp));
return true;
}
/** Format 4-byte binary sequesnce as IPv4 text: 'aaa.bbb.ccc.ddd',
* expects inout to be in BE-format, that is 0x7f000001 => "127.0.0.1".
*
* Any number of the tail bytes can be masked with given mask string.
*
* Assumptions:
* src is IPV4_BINARY_LENGTH long,
* dst is IPV4_MAX_TEXT_LENGTH long,
* mask_tail_octets <= IPV4_BINARY_LENGTH
* mask_string is NON-NULL, if mask_tail_octets > 0.
*
* Examples:
* formatIPv4(&0x7f000001, dst, mask_tail_octets = 0, nullptr);
* > dst == "127.0.0.1"
* formatIPv4(&0x7f000001, dst, mask_tail_octets = 1, "xxx");
* > dst == "127.0.0.xxx"
* formatIPv4(&0x7f000001, dst, mask_tail_octets = 1, "0");
* > dst == "127.0.0.0"
*/
inline void formatIPv4(const unsigned char * src, char *& dst, UInt8 mask_tail_octets = 0, const char * mask_string = "xxx")
{
extern const char one_byte_to_string_lookup_table[256][4];
const size_t mask_length = mask_string ? strlen(mask_string) : 0;
const size_t limit = std::min(IPV4_BINARY_LENGTH, IPV4_BINARY_LENGTH - mask_tail_octets);
for (size_t octet = 0; octet < limit; ++octet)
{
const UInt8 value = static_cast<UInt8>(src[IPV4_BINARY_LENGTH - octet - 1]);
auto rep = one_byte_to_string_lookup_table[value];
const UInt8 len = rep[0];
const char* str = rep + 1;
memcpy(dst, str, len);
dst += len;
*dst++ = '.';
}
for (size_t mask = 0; mask < mask_tail_octets; ++mask)
{
memcpy(dst, mask_string, mask_length);
dst += mask_length;
*dst++ = '.';
}
dst[-1] = '\0';
}
} }

View File

@ -85,24 +85,15 @@ void DistinctBlockInputStream::buildFilter(
size_t rows, size_t rows,
SetVariants & variants) const SetVariants & variants) const
{ {
typename Method::State state; typename Method::State state(columns, key_sizes, nullptr);
state.init(columns);
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < rows; ++i)
{ {
/// Make a key. auto emplace_result = state.emplaceKey(method.data, i, variants.string_pool);
typename Method::Key key = state.getKey(columns, columns.size(), i, key_sizes);
typename Method::Data::iterator it;
bool inserted;
method.data.emplace(key, it, inserted);
if (inserted)
method.onNewKey(*it, columns.size(), variants.string_pool);
/// Emit the record if there is no such key in the current set yet. /// Emit the record if there is no such key in the current set yet.
/// Skip it otherwise. /// Skip it otherwise.
filter[i] = inserted; filter[i] = emplace_result.isInserted();
} }
} }

View File

@ -85,8 +85,7 @@ bool DistinctSortedBlockInputStream::buildFilter(
size_t rows, size_t rows,
ClearableSetVariants & variants) const ClearableSetVariants & variants) const
{ {
typename Method::State state; typename Method::State state(columns, key_sizes, nullptr);
state.init(columns);
/// Compare last row of previous block and first row of current block, /// Compare last row of previous block and first row of current block,
/// If rows not equal, we can clear HashSet, /// If rows not equal, we can clear HashSet,
@ -106,21 +105,14 @@ bool DistinctSortedBlockInputStream::buildFilter(
if (i > 0 && !clearing_hint_columns.empty() && !rowsEqual(clearing_hint_columns, i, clearing_hint_columns, i - 1)) if (i > 0 && !clearing_hint_columns.empty() && !rowsEqual(clearing_hint_columns, i, clearing_hint_columns, i - 1))
method.data.clear(); method.data.clear();
/// Make a key. auto emplace_result = state.emplaceKey(method.data, i, variants.string_pool);
typename Method::Key key = state.getKey(columns, columns.size(), i, key_sizes);
typename Method::Data::iterator it = method.data.find(key);
bool inserted;
method.data.emplace(key, it, inserted);
if (inserted) if (emplace_result.isInserted())
{
method.onNewKey(*it, columns.size(), variants.string_pool);
has_new_data = true; has_new_data = true;
}
/// Emit the record if there is no such key in the current set yet. /// Emit the record if there is no such key in the current set yet.
/// Skip it otherwise. /// Skip it otherwise.
filter[i] = inserted; filter[i] = emplace_result.isInserted();
} }
return has_new_data; return has_new_data;
} }

View File

@ -32,7 +32,7 @@ namespace ErrorCodes
} }
std::string DataTypeAggregateFunction::getName() const std::string DataTypeAggregateFunction::doGetName() const
{ {
std::stringstream stream; std::stringstream stream;
stream << "AggregateFunction(" << function->getName(); stream << "AggregateFunction(" << function->getName();

View File

@ -29,7 +29,7 @@ public:
std::string getFunctionName() const { return function->getName(); } std::string getFunctionName() const { return function->getName(); }
AggregateFunctionPtr getFunction() const { return function; } AggregateFunctionPtr getFunction() const { return function; }
std::string getName() const override; std::string doGetName() const override;
const char * getFamilyName() const override { return "AggregateFunction"; } const char * getFamilyName() const override { return "AggregateFunction"; }
TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; } TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; }

View File

@ -350,7 +350,7 @@ void DataTypeArray::serializeText(const IColumn & column, size_t row_num, WriteB
serializeTextImpl(column, row_num, ostr, serializeTextImpl(column, row_num, ostr,
[&](const IColumn & nested_column, size_t i) [&](const IColumn & nested_column, size_t i)
{ {
nested->serializeTextQuoted(nested_column, i, ostr, settings); nested->serializeAsTextQuoted(nested_column, i, ostr, settings);
}); });
} }
@ -360,7 +360,7 @@ void DataTypeArray::deserializeText(IColumn & column, ReadBuffer & istr, const F
deserializeTextImpl(column, istr, deserializeTextImpl(column, istr,
[&](IColumn & nested_column) [&](IColumn & nested_column)
{ {
nested->deserializeTextQuoted(nested_column, istr, settings); nested->deserializeAsTextQuoted(nested_column, istr, settings);
}); });
} }
@ -379,7 +379,7 @@ void DataTypeArray::serializeTextJSON(const IColumn & column, size_t row_num, Wr
{ {
if (i != offset) if (i != offset)
writeChar(',', ostr); writeChar(',', ostr);
nested->serializeTextJSON(nested_column, i, ostr, settings); nested->serializeAsTextJSON(nested_column, i, ostr, settings);
} }
writeChar(']', ostr); writeChar(']', ostr);
} }
@ -387,7 +387,7 @@ void DataTypeArray::serializeTextJSON(const IColumn & column, size_t row_num, Wr
void DataTypeArray::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const void DataTypeArray::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
deserializeTextImpl(column, istr, [&](IColumn & nested_column) { nested->deserializeTextJSON(nested_column, istr, settings); }); deserializeTextImpl(column, istr, [&](IColumn & nested_column) { nested->deserializeAsTextJSON(nested_column, istr, settings); });
} }
@ -405,7 +405,7 @@ void DataTypeArray::serializeTextXML(const IColumn & column, size_t row_num, Wri
for (size_t i = offset; i < next_offset; ++i) for (size_t i = offset; i < next_offset; ++i)
{ {
writeCString("<elem>", ostr); writeCString("<elem>", ostr);
nested->serializeTextXML(nested_column, i, ostr, settings); nested->serializeAsTextXML(nested_column, i, ostr, settings);
writeCString("</elem>", ostr); writeCString("</elem>", ostr);
} }
writeCString("</array>", ostr); writeCString("</array>", ostr);

View File

@ -20,7 +20,7 @@ public:
TypeIndex getTypeId() const override { return TypeIndex::Array; } TypeIndex getTypeId() const override { return TypeIndex::Array; }
std::string getName() const override std::string doGetName() const override
{ {
return "Array(" + nested->getName() + ")"; return "Array(" + nested->getName() + ")";
} }

View File

@ -26,7 +26,7 @@ DataTypeDateTime::DataTypeDateTime(const std::string & time_zone_name)
{ {
} }
std::string DataTypeDateTime::getName() const std::string DataTypeDateTime::doGetName() const
{ {
if (!has_explicit_time_zone) if (!has_explicit_time_zone)
return "DateTime"; return "DateTime";

View File

@ -34,7 +34,7 @@ public:
DataTypeDateTime(const std::string & time_zone_name = ""); DataTypeDateTime(const std::string & time_zone_name = "");
const char * getFamilyName() const override { return "DateTime"; } const char * getFamilyName() const override { return "DateTime"; }
std::string getName() const override; std::string doGetName() const override;
TypeIndex getTypeId() const override { return TypeIndex::DateTime; } TypeIndex getTypeId() const override { return TypeIndex::DateTime; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -0,0 +1,118 @@
#include <Columns/ColumnsNumber.h>
#include <Common/Exception.h>
#include <Common/formatIPv6.h>
#include <DataTypes/DataTypeDomainWithSimpleSerialization.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/IDataTypeDomain.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionsCoding.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int UNSUPPORTED_METHOD;
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
}
namespace
{
class DataTypeDomanIPv4 : public DataTypeDomainWithSimpleSerialization
{
public:
const char * getName() const override
{
return "IPv4";
}
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override
{
const auto col = checkAndGetColumn<ColumnUInt32>(&column);
if (!col)
{
throw Exception(String(getName()) + " domain can only serialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'};
char * ptr = buffer;
formatIPv4(reinterpret_cast<const unsigned char *>(&col->getData()[row_num]), ptr);
ostr.write(buffer, strlen(buffer));
}
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override
{
ColumnUInt32 * col = typeid_cast<ColumnUInt32 *>(&column);
if (!col)
{
throw Exception(String(getName()) + " domain can only deserialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'};
istr.read(buffer, sizeof(buffer) - 1);
UInt32 ipv4_value = 0;
if (!parseIPv4(buffer, reinterpret_cast<unsigned char *>(&ipv4_value)))
{
throw Exception("Invalid IPv4 value.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
}
col->insert(ipv4_value);
}
};
class DataTypeDomanIPv6 : public DataTypeDomainWithSimpleSerialization
{
public:
const char * getName() const override
{
return "IPv6";
}
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override
{
const auto col = checkAndGetColumn<ColumnFixedString>(&column);
if (!col)
{
throw Exception(String(getName()) + " domain can only serialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'};
char * ptr = buffer;
formatIPv6(reinterpret_cast<const unsigned char *>(col->getDataAt(row_num).data), ptr);
ostr.write(buffer, strlen(buffer));
}
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override
{
ColumnFixedString * col = typeid_cast<ColumnFixedString *>(&column);
if (!col)
{
throw Exception(String(getName()) + " domain can only deserialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
}
char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'};
istr.read(buffer, sizeof(buffer) - 1);
std::string ipv6_value(IPV6_BINARY_LENGTH, '\0');
if (!parseIPv6(buffer, reinterpret_cast<unsigned char *>(ipv6_value.data())))
{
throw Exception(String("Invalid ") + getName() + " value.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
}
col->insertString(ipv6_value);
}
};
} // namespace
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
{
factory.registerDataTypeDomain("UInt32", std::make_unique<DataTypeDomanIPv4>());
factory.registerDataTypeDomain("FixedString(16)", std::make_unique<DataTypeDomanIPv6>());
}
} // namespace DB

View File

@ -0,0 +1,88 @@
#include <DataTypes/DataTypeDomainWithSimpleSerialization.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
namespace
{
using namespace DB;
static String serializeToString(const DataTypeDomainWithSimpleSerialization & domain, const IColumn & column, size_t row_num, const FormatSettings & settings)
{
WriteBufferFromOwnString buffer;
domain.serializeText(column, row_num, buffer, settings);
return buffer.str();
}
static void deserializeFromString(const DataTypeDomainWithSimpleSerialization & domain, IColumn & column, const String & s, const FormatSettings & settings)
{
ReadBufferFromString istr(s);
domain.deserializeText(column, istr, settings);
}
} // namespace
namespace DB
{
DataTypeDomainWithSimpleSerialization::~DataTypeDomainWithSimpleSerialization()
{
}
void DataTypeDomainWithSimpleSerialization::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeEscapedString(serializeToString(*this, column, row_num, settings), ostr);
}
void DataTypeDomainWithSimpleSerialization::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String str;
readEscapedString(str, istr);
deserializeFromString(*this, column, str, settings);
}
void DataTypeDomainWithSimpleSerialization::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeQuotedString(serializeToString(*this, column, row_num, settings), ostr);
}
void DataTypeDomainWithSimpleSerialization::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String str;
readQuotedString(str, istr);
deserializeFromString(*this, column, str, settings);
}
void DataTypeDomainWithSimpleSerialization::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeCSVString(serializeToString(*this, column, row_num, settings), ostr);
}
void DataTypeDomainWithSimpleSerialization::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String str;
readCSVString(str, istr, settings.csv);
deserializeFromString(*this, column, str, settings);
}
void DataTypeDomainWithSimpleSerialization::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeJSONString(serializeToString(*this, column, row_num, settings), ostr, settings);
}
void DataTypeDomainWithSimpleSerialization::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String str;
readJSONString(str, istr);
deserializeFromString(*this, column, str, settings);
}
void DataTypeDomainWithSimpleSerialization::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeXMLString(serializeToString(*this, column, row_num, settings), ostr);
}
} // namespace DB

View File

@ -0,0 +1,53 @@
#pragma once
#include <DataTypes/IDataTypeDomain.h>
namespace DB
{
class ReadBuffer;
class WriteBuffer;
struct FormatSettings;
class IColumn;
/** Simple DataTypeDomain that uses serializeText/deserializeText
* for all serialization and deserialization. */
class DataTypeDomainWithSimpleSerialization : public IDataTypeDomain
{
public:
virtual ~DataTypeDomainWithSimpleSerialization() override;
// Methods that subclasses must override in order to get full serialization/deserialization support.
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override = 0;
virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
/** Text serialization with escaping but without quoting.
*/
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
/** Text serialization as a literal that may be inserted into a query.
*/
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
/** Text serialization for the CSV format.
*/
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
/** delimiter - the delimiter we expect when reading a string value that is not double-quoted
* (the delimiter is not consumed).
*/
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
/** Text serialization intended for using in JSON format.
* force_quoting_64bit_integers parameter forces to brace UInt64 and Int64 types into quotes.
*/
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
/** Text serialization for putting into the XML format.
*/
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
};
} // namespace DB

View File

@ -61,7 +61,7 @@ public:
explicit DataTypeEnum(const Values & values_); explicit DataTypeEnum(const Values & values_);
const Values & getValues() const { return values; } const Values & getValues() const { return values; }
std::string getName() const override { return type_name; } std::string doGetName() const override { return type_name; }
const char * getFamilyName() const override; const char * getFamilyName() const override;
TypeIndex getTypeId() const override { return sizeof(FieldType) == 1 ? TypeIndex::Enum8 : TypeIndex::Enum16; } TypeIndex getTypeId() const override { return sizeof(FieldType) == 1 ? TypeIndex::Enum8 : TypeIndex::Enum16; }

View File

@ -1,4 +1,5 @@
#include <DataTypes/DataTypeFactory.h> #include <DataTypes/DataTypeFactory.h>
#include <DataTypes/IDataTypeDomain.h>
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h> #include <Parsers/ParserCreateQuery.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
@ -73,25 +74,7 @@ DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr
return get("LowCardinality", low_cardinality_params); return get("LowCardinality", low_cardinality_params);
} }
{ return findCreatorByName(family_name)(parameters);
DataTypesDictionary::const_iterator it = data_types.find(family_name);
if (data_types.end() != it)
return it->second(parameters);
}
String family_name_lowercase = Poco::toLower(family_name);
{
DataTypesDictionary::const_iterator it = case_insensitive_data_types.find(family_name_lowercase);
if (case_insensitive_data_types.end() != it)
return it->second(parameters);
}
auto hints = this->getHints(family_name);
if (!hints.empty())
throw Exception("Unknown data type family: " + family_name + ". Maybe you meant: " + toString(hints), ErrorCodes::UNKNOWN_TYPE);
else
throw Exception("Unknown data type family: " + family_name, ErrorCodes::UNKNOWN_TYPE);
} }
@ -132,6 +115,49 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator
}, case_sensitiveness); }, case_sensitiveness);
} }
void DataTypeFactory::registerDataTypeDomain(const String & type_name, DataTypeDomainPtr domain, CaseSensitiveness case_sensitiveness)
{
all_domains.reserve(all_domains.size() + 1);
auto data_type = get(type_name);
setDataTypeDomain(*data_type, *domain);
registerDataType(domain->getName(), [data_type](const ASTPtr & /*ast*/)
{
return data_type;
}, case_sensitiveness);
all_domains.emplace_back(std::move(domain));
}
const DataTypeFactory::Creator& DataTypeFactory::findCreatorByName(const String & family_name) const
{
{
DataTypesDictionary::const_iterator it = data_types.find(family_name);
if (data_types.end() != it)
return it->second;
}
String family_name_lowercase = Poco::toLower(family_name);
{
DataTypesDictionary::const_iterator it = case_insensitive_data_types.find(family_name_lowercase);
if (case_insensitive_data_types.end() != it)
return it->second;
}
auto hints = this->getHints(family_name);
if (!hints.empty())
throw Exception("Unknown data type family: " + family_name + ". Maybe you meant: " + toString(hints), ErrorCodes::UNKNOWN_TYPE);
else
throw Exception("Unknown data type family: " + family_name, ErrorCodes::UNKNOWN_TYPE);
}
void DataTypeFactory::setDataTypeDomain(const IDataType & data_type, const IDataTypeDomain & domain)
{
data_type.setDomain(&domain);
}
void registerDataTypeNumbers(DataTypeFactory & factory); void registerDataTypeNumbers(DataTypeFactory & factory);
void registerDataTypeDecimal(DataTypeFactory & factory); void registerDataTypeDecimal(DataTypeFactory & factory);
void registerDataTypeDate(DataTypeFactory & factory); void registerDataTypeDate(DataTypeFactory & factory);
@ -148,6 +174,7 @@ void registerDataTypeAggregateFunction(DataTypeFactory & factory);
void registerDataTypeNested(DataTypeFactory & factory); void registerDataTypeNested(DataTypeFactory & factory);
void registerDataTypeInterval(DataTypeFactory & factory); void registerDataTypeInterval(DataTypeFactory & factory);
void registerDataTypeLowCardinality(DataTypeFactory & factory); void registerDataTypeLowCardinality(DataTypeFactory & factory);
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory);
DataTypeFactory::DataTypeFactory() DataTypeFactory::DataTypeFactory()
@ -168,6 +195,10 @@ DataTypeFactory::DataTypeFactory()
registerDataTypeNested(*this); registerDataTypeNested(*this);
registerDataTypeInterval(*this); registerDataTypeInterval(*this);
registerDataTypeLowCardinality(*this); registerDataTypeLowCardinality(*this);
registerDataTypeDomainIPv4AndIPv6(*this);
} }
DataTypeFactory::~DataTypeFactory()
{}
} }

View File

@ -14,6 +14,9 @@ namespace DB
class IDataType; class IDataType;
using DataTypePtr = std::shared_ptr<const IDataType>; using DataTypePtr = std::shared_ptr<const IDataType>;
class IDataTypeDomain;
using DataTypeDomainPtr = std::unique_ptr<const IDataTypeDomain>;
class IAST; class IAST;
using ASTPtr = std::shared_ptr<IAST>; using ASTPtr = std::shared_ptr<IAST>;
@ -37,13 +40,24 @@ public:
/// Register a simple data type, that have no parameters. /// Register a simple data type, that have no parameters.
void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive); void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
// Register a domain - a refinement of existing type.
void registerDataTypeDomain(const String & type_name, DataTypeDomainPtr domain, CaseSensitiveness case_sensitiveness = CaseSensitive);
private:
static void setDataTypeDomain(const IDataType & data_type, const IDataTypeDomain & domain);
const Creator& findCreatorByName(const String & family_name) const;
private: private:
DataTypesDictionary data_types; DataTypesDictionary data_types;
/// Case insensitive data types will be additionally added here with lowercased name. /// Case insensitive data types will be additionally added here with lowercased name.
DataTypesDictionary case_insensitive_data_types; DataTypesDictionary case_insensitive_data_types;
// All domains are owned by factory and shared amongst DataType instances.
std::vector<DataTypeDomainPtr> all_domains;
DataTypeFactory(); DataTypeFactory();
~DataTypeFactory() override;
const DataTypesDictionary & getCreatorMap() const override { return data_types; } const DataTypesDictionary & getCreatorMap() const override { return data_types; }

View File

@ -32,7 +32,7 @@ namespace ErrorCodes
} }
std::string DataTypeFixedString::getName() const std::string DataTypeFixedString::doGetName() const
{ {
return "FixedString(" + toString(n) + ")"; return "FixedString(" + toString(n) + ")";
} }

View File

@ -30,7 +30,7 @@ public:
throw Exception("FixedString size is too large", ErrorCodes::ARGUMENT_OUT_OF_BOUND); throw Exception("FixedString size is too large", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
} }
std::string getName() const override; std::string doGetName() const override;
TypeIndex getTypeId() const override { return TypeIndex::FixedString; } TypeIndex getTypeId() const override { return TypeIndex::FixedString; }
const char * getFamilyName() const override { return "FixedString"; } const char * getFamilyName() const override { return "FixedString"; }

View File

@ -6,7 +6,7 @@
namespace DB namespace DB
{ {
std::string DataTypeFunction::getName() const std::string DataTypeFunction::doGetName() const
{ {
WriteBufferFromOwnString res; WriteBufferFromOwnString res;

View File

@ -22,7 +22,7 @@ public:
DataTypeFunction(const DataTypes & argument_types_ = DataTypes(), const DataTypePtr & return_type_ = nullptr) DataTypeFunction(const DataTypes & argument_types_ = DataTypes(), const DataTypePtr & return_type_ = nullptr)
: argument_types(argument_types_), return_type(return_type_) {} : argument_types(argument_types_), return_type(return_type_) {}
std::string getName() const override; std::string doGetName() const override;
const char * getFamilyName() const override { return "Function"; } const char * getFamilyName() const override { return "Function"; }
TypeIndex getTypeId() const override { return TypeIndex::Function; } TypeIndex getTypeId() const override { return TypeIndex::Function; }

View File

@ -56,7 +56,7 @@ public:
DataTypeInterval(Kind kind) : kind(kind) {} DataTypeInterval(Kind kind) : kind(kind) {}
std::string getName() const override { return std::string("Interval") + kindToString(); } std::string doGetName() const override { return std::string("Interval") + kindToString(); }
const char * getFamilyName() const override { return "Interval"; } const char * getFamilyName() const override { return "Interval"; }
TypeIndex getTypeId() const override { return TypeIndex::Interval; } TypeIndex getTypeId() const override { return TypeIndex::Interval; }

View File

@ -15,7 +15,7 @@ public:
const DataTypePtr & getDictionaryType() const { return dictionary_type; } const DataTypePtr & getDictionaryType() const { return dictionary_type; }
String getName() const override String doGetName() const override
{ {
return "LowCardinality(" + dictionary_type->getName() + ")"; return "LowCardinality(" + dictionary_type->getName() + ")";
} }
@ -63,51 +63,51 @@ public:
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{ {
serializeImpl(column, row_num, &IDataType::serializeTextEscaped, ostr, settings); serializeImpl(column, row_num, &IDataType::serializeAsTextEscaped, ostr, settings);
} }
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{ {
deserializeImpl(column, &IDataType::deserializeTextEscaped, istr, settings); deserializeImpl(column, &IDataType::deserializeAsTextEscaped, istr, settings);
} }
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{ {
serializeImpl(column, row_num, &IDataType::serializeTextQuoted, ostr, settings); serializeImpl(column, row_num, &IDataType::serializeAsTextQuoted, ostr, settings);
} }
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{ {
deserializeImpl(column, &IDataType::deserializeTextQuoted, istr, settings); deserializeImpl(column, &IDataType::deserializeAsTextQuoted, istr, settings);
} }
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{ {
serializeImpl(column, row_num, &IDataType::serializeTextCSV, ostr, settings); serializeImpl(column, row_num, &IDataType::serializeAsTextCSV, ostr, settings);
} }
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{ {
deserializeImpl(column, &IDataType::deserializeTextCSV, istr, settings); deserializeImpl(column, &IDataType::deserializeAsTextCSV, istr, settings);
} }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{ {
serializeImpl(column, row_num, &IDataType::serializeText, ostr, settings); serializeImpl(column, row_num, &IDataType::serializeAsText, ostr, settings);
} }
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{ {
serializeImpl(column, row_num, &IDataType::serializeTextJSON, ostr, settings); serializeImpl(column, row_num, &IDataType::serializeAsTextJSON, ostr, settings);
} }
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{ {
deserializeImpl(column, &IDataType::deserializeTextJSON, istr, settings); deserializeImpl(column, &IDataType::deserializeAsTextJSON, istr, settings);
} }
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{ {
serializeImpl(column, row_num, &IDataType::serializeTextXML, ostr, settings); serializeImpl(column, row_num, &IDataType::serializeAsTextXML, ostr, settings);
} }
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override

View File

@ -172,7 +172,7 @@ void DataTypeNullable::serializeTextEscaped(const IColumn & column, size_t row_n
if (col.isNullAt(row_num)) if (col.isNullAt(row_num))
writeCString("\\N", ostr); writeCString("\\N", ostr);
else else
nested_data_type->serializeTextEscaped(col.getNestedColumn(), row_num, ostr, settings); nested_data_type->serializeAsTextEscaped(col.getNestedColumn(), row_num, ostr, settings);
} }
@ -188,7 +188,7 @@ void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & ist
{ {
safeDeserialize(column, safeDeserialize(column,
[] { return false; }, [] { return false; },
[this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeTextEscaped(nested, istr, settings); }); [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsTextEscaped(nested, istr, settings); });
} }
else else
{ {
@ -214,7 +214,7 @@ void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & ist
{ {
/// We could step back to consume backslash again. /// We could step back to consume backslash again.
--istr.position(); --istr.position();
nested_data_type->deserializeTextEscaped(nested, istr, settings); nested_data_type->deserializeAsTextEscaped(nested, istr, settings);
} }
else else
{ {
@ -222,7 +222,7 @@ void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & ist
ReadBufferFromMemory prefix("\\", 1); ReadBufferFromMemory prefix("\\", 1);
ConcatReadBuffer prepended_istr(prefix, istr); ConcatReadBuffer prepended_istr(prefix, istr);
nested_data_type->deserializeTextEscaped(nested, prepended_istr, settings); nested_data_type->deserializeAsTextEscaped(nested, prepended_istr, settings);
/// Synchronise cursor position in original buffer. /// Synchronise cursor position in original buffer.
@ -240,7 +240,7 @@ void DataTypeNullable::serializeTextQuoted(const IColumn & column, size_t row_nu
if (col.isNullAt(row_num)) if (col.isNullAt(row_num))
writeCString("NULL", ostr); writeCString("NULL", ostr);
else else
nested_data_type->serializeTextQuoted(col.getNestedColumn(), row_num, ostr, settings); nested_data_type->serializeAsTextQuoted(col.getNestedColumn(), row_num, ostr, settings);
} }
@ -248,7 +248,7 @@ void DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr
{ {
safeDeserialize(column, safeDeserialize(column,
[&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); }, [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); },
[this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeTextQuoted(nested, istr, settings); }); [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsTextQuoted(nested, istr, settings); });
} }
void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -258,14 +258,14 @@ void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num,
if (col.isNullAt(row_num)) if (col.isNullAt(row_num))
writeCString("\\N", ostr); writeCString("\\N", ostr);
else else
nested_data_type->serializeTextCSV(col.getNestedColumn(), row_num, ostr, settings); nested_data_type->serializeAsTextCSV(col.getNestedColumn(), row_num, ostr, settings);
} }
void DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const void DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
safeDeserialize(column, safeDeserialize(column,
[&istr] { return checkStringByFirstCharacterAndAssertTheRest("\\N", istr); }, [&istr] { return checkStringByFirstCharacterAndAssertTheRest("\\N", istr); },
[this, &settings, &istr] (IColumn & nested) { nested_data_type->deserializeTextCSV(nested, istr, settings); }); [this, &settings, &istr] (IColumn & nested) { nested_data_type->deserializeAsTextCSV(nested, istr, settings); });
} }
void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -281,7 +281,7 @@ void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, Wri
if (col.isNullAt(row_num)) if (col.isNullAt(row_num))
writeCString("ᴺᵁᴸᴸ", ostr); writeCString("ᴺᵁᴸᴸ", ostr);
else else
nested_data_type->serializeText(col.getNestedColumn(), row_num, ostr, settings); nested_data_type->serializeAsText(col.getNestedColumn(), row_num, ostr, settings);
} }
void DataTypeNullable::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const void DataTypeNullable::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -291,14 +291,14 @@ void DataTypeNullable::serializeTextJSON(const IColumn & column, size_t row_num,
if (col.isNullAt(row_num)) if (col.isNullAt(row_num))
writeCString("null", ostr); writeCString("null", ostr);
else else
nested_data_type->serializeTextJSON(col.getNestedColumn(), row_num, ostr, settings); nested_data_type->serializeAsTextJSON(col.getNestedColumn(), row_num, ostr, settings);
} }
void DataTypeNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const void DataTypeNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{ {
safeDeserialize(column, safeDeserialize(column,
[&istr] { return checkStringByFirstCharacterAndAssertTheRest("null", istr); }, [&istr] { return checkStringByFirstCharacterAndAssertTheRest("null", istr); },
[this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeTextJSON(nested, istr, settings); }); [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsTextJSON(nested, istr, settings); });
} }
void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -308,7 +308,7 @@ void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num,
if (col.isNullAt(row_num)) if (col.isNullAt(row_num))
writeCString("\\N", ostr); writeCString("\\N", ostr);
else else
nested_data_type->serializeTextXML(col.getNestedColumn(), row_num, ostr, settings); nested_data_type->serializeAsTextXML(col.getNestedColumn(), row_num, ostr, settings);
} }
void DataTypeNullable::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const void DataTypeNullable::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const

View File

@ -14,7 +14,7 @@ public:
static constexpr bool is_parametric = true; static constexpr bool is_parametric = true;
explicit DataTypeNullable(const DataTypePtr & nested_data_type_); explicit DataTypeNullable(const DataTypePtr & nested_data_type_);
std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } std::string doGetName() const override { return "Nullable(" + nested_data_type->getName() + ")"; }
const char * getFamilyName() const override { return "Nullable"; } const char * getFamilyName() const override { return "Nullable"; }
TypeIndex getTypeId() const override { return TypeIndex::Nullable; } TypeIndex getTypeId() const override { return TypeIndex::Nullable; }

View File

@ -64,7 +64,7 @@ DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_)
std::string DataTypeTuple::getName() const std::string DataTypeTuple::doGetName() const
{ {
size_t size = elems.size(); size_t size = elems.size();
WriteBufferFromOwnString s; WriteBufferFromOwnString s;
@ -160,7 +160,7 @@ void DataTypeTuple::serializeText(const IColumn & column, size_t row_num, WriteB
{ {
if (i != 0) if (i != 0)
writeChar(',', ostr); writeChar(',', ostr);
elems[i]->serializeTextQuoted(extractElementColumn(column, i), row_num, ostr, settings); elems[i]->serializeAsTextQuoted(extractElementColumn(column, i), row_num, ostr, settings);
} }
writeChar(')', ostr); writeChar(')', ostr);
} }
@ -180,7 +180,7 @@ void DataTypeTuple::deserializeText(IColumn & column, ReadBuffer & istr, const F
assertChar(',', istr); assertChar(',', istr);
skipWhitespaceIfAny(istr); skipWhitespaceIfAny(istr);
} }
elems[i]->deserializeTextQuoted(extractElementColumn(column, i), istr, settings); elems[i]->deserializeAsTextQuoted(extractElementColumn(column, i), istr, settings);
} }
}); });
@ -195,7 +195,7 @@ void DataTypeTuple::serializeTextJSON(const IColumn & column, size_t row_num, Wr
{ {
if (i != 0) if (i != 0)
writeChar(',', ostr); writeChar(',', ostr);
elems[i]->serializeTextJSON(extractElementColumn(column, i), row_num, ostr, settings); elems[i]->serializeAsTextJSON(extractElementColumn(column, i), row_num, ostr, settings);
} }
writeChar(']', ostr); writeChar(']', ostr);
} }
@ -215,7 +215,7 @@ void DataTypeTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr, con
assertChar(',', istr); assertChar(',', istr);
skipWhitespaceIfAny(istr); skipWhitespaceIfAny(istr);
} }
elems[i]->deserializeTextJSON(extractElementColumn(column, i), istr, settings); elems[i]->deserializeAsTextJSON(extractElementColumn(column, i), istr, settings);
} }
}); });
@ -229,7 +229,7 @@ void DataTypeTuple::serializeTextXML(const IColumn & column, size_t row_num, Wri
for (const auto i : ext::range(0, ext::size(elems))) for (const auto i : ext::range(0, ext::size(elems)))
{ {
writeCString("<elem>", ostr); writeCString("<elem>", ostr);
elems[i]->serializeTextXML(extractElementColumn(column, i), row_num, ostr, settings); elems[i]->serializeAsTextXML(extractElementColumn(column, i), row_num, ostr, settings);
writeCString("</elem>", ostr); writeCString("</elem>", ostr);
} }
writeCString("</tuple>", ostr); writeCString("</tuple>", ostr);
@ -241,7 +241,7 @@ void DataTypeTuple::serializeTextCSV(const IColumn & column, size_t row_num, Wri
{ {
if (i != 0) if (i != 0)
writeChar(',', ostr); writeChar(',', ostr);
elems[i]->serializeTextCSV(extractElementColumn(column, i), row_num, ostr, settings); elems[i]->serializeAsTextCSV(extractElementColumn(column, i), row_num, ostr, settings);
} }
} }
@ -258,7 +258,7 @@ void DataTypeTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons
assertChar(settings.csv.delimiter, istr); assertChar(settings.csv.delimiter, istr);
skipWhitespaceIfAny(istr); skipWhitespaceIfAny(istr);
} }
elems[i]->deserializeTextCSV(extractElementColumn(column, i), istr, settings); elems[i]->deserializeAsTextCSV(extractElementColumn(column, i), istr, settings);
} }
}); });
} }

View File

@ -29,7 +29,7 @@ public:
DataTypeTuple(const DataTypes & elems, const Strings & names); DataTypeTuple(const DataTypes & elems, const Strings & names);
TypeIndex getTypeId() const override { return TypeIndex::Tuple; } TypeIndex getTypeId() const override { return TypeIndex::Tuple; }
std::string getName() const override; std::string doGetName() const override;
const char * getFamilyName() const override { return "Tuple"; } const char * getFamilyName() const override { return "Tuple"; }
bool canBeInsideNullable() const override { return false; } bool canBeInsideNullable() const override { return false; }

View File

@ -28,7 +28,7 @@ bool decimalCheckArithmeticOverflow(const Context & context) { return context.ge
// //
template <typename T> template <typename T>
std::string DataTypeDecimal<T>::getName() const std::string DataTypeDecimal<T>::doGetName() const
{ {
std::stringstream ss; std::stringstream ss;
ss << "Decimal(" << precision << ", " << scale << ")"; ss << "Decimal(" << precision << ", " << scale << ")";

View File

@ -86,7 +86,7 @@ public:
} }
const char * getFamilyName() const override { return "Decimal"; } const char * getFamilyName() const override { return "Decimal"; }
std::string getName() const override; std::string doGetName() const override;
TypeIndex getTypeId() const override { return TypeId<T>::value; } TypeIndex getTypeId() const override { return TypeId<T>::value; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -9,6 +9,7 @@
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <DataTypes/IDataType.h> #include <DataTypes/IDataType.h>
#include <DataTypes/IDataTypeDomain.h>
#include <DataTypes/NestedUtils.h> #include <DataTypes/NestedUtils.h>
@ -22,6 +23,31 @@ namespace ErrorCodes
extern const int DATA_TYPE_CANNOT_BE_PROMOTED; extern const int DATA_TYPE_CANNOT_BE_PROMOTED;
} }
IDataType::IDataType()
: domain(nullptr)
{
}
IDataType::~IDataType()
{
}
String IDataType::getName() const
{
if (domain)
{
return domain->getName();
}
else
{
return doGetName();
}
}
String IDataType::doGetName() const
{
return getFamilyName();
}
void IDataType::updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint) void IDataType::updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint)
{ {
@ -114,4 +140,133 @@ void IDataType::insertDefaultInto(IColumn & column) const
column.insertDefault(); column.insertDefault();
} }
void IDataType::serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (domain)
{
domain->serializeTextEscaped(column, row_num, ostr, settings);
}
else
{
serializeTextEscaped(column, row_num, ostr, settings);
}
}
void IDataType::deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (domain)
{
domain->deserializeTextEscaped(column, istr, settings);
}
else
{
deserializeTextEscaped(column, istr, settings);
}
}
void IDataType::serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (domain)
{
domain->serializeTextQuoted(column, row_num, ostr, settings);
}
else
{
serializeTextQuoted(column, row_num, ostr, settings);
}
}
void IDataType::deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (domain)
{
domain->deserializeTextQuoted(column, istr, settings);
}
else
{
deserializeTextQuoted(column, istr, settings);
}
}
void IDataType::serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (domain)
{
domain->serializeTextCSV(column, row_num, ostr, settings);
}
else
{
serializeTextCSV(column, row_num, ostr, settings);
}
}
void IDataType::deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (domain)
{
domain->deserializeTextCSV(column, istr, settings);
}
else
{
deserializeTextCSV(column, istr, settings);
}
}
void IDataType::serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (domain)
{
domain->serializeText(column, row_num, ostr, settings);
}
else
{
serializeText(column, row_num, ostr, settings);
}
}
void IDataType::serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (domain)
{
domain->serializeTextJSON(column, row_num, ostr, settings);
}
else
{
serializeTextJSON(column, row_num, ostr, settings);
}
}
void IDataType::deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (domain)
{
domain->deserializeTextJSON(column, istr, settings);
}
else
{
deserializeTextJSON(column, istr, settings);
}
}
void IDataType::serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (domain)
{
domain->serializeTextXML(column, row_num, ostr, settings);
}
else
{
serializeTextXML(column, row_num, ostr, settings);
}
}
void IDataType::setDomain(const IDataTypeDomain* const new_domain) const
{
if (domain != nullptr)
{
throw Exception("Type " + getName() + " already has a domain.", ErrorCodes::LOGICAL_ERROR);
}
domain = new_domain;
}
} }

View File

@ -12,6 +12,7 @@ namespace DB
class ReadBuffer; class ReadBuffer;
class WriteBuffer; class WriteBuffer;
class IDataTypeDomain;
class IDataType; class IDataType;
struct FormatSettings; struct FormatSettings;
@ -35,6 +36,9 @@ class ProtobufWriter;
class IDataType : private boost::noncopyable class IDataType : private boost::noncopyable
{ {
public: public:
IDataType();
virtual ~IDataType();
/// Compile time flag. If false, then if C++ types are the same, then SQL types are also the same. /// Compile time flag. If false, then if C++ types are the same, then SQL types are also the same.
/// Example: DataTypeString is not parametric: thus all instances of DataTypeString are the same SQL type. /// Example: DataTypeString is not parametric: thus all instances of DataTypeString are the same SQL type.
/// Example: DataTypeFixedString is parametric: different instances of DataTypeFixedString may be different SQL types. /// Example: DataTypeFixedString is parametric: different instances of DataTypeFixedString may be different SQL types.
@ -42,7 +46,7 @@ public:
/// static constexpr bool is_parametric = false; /// static constexpr bool is_parametric = false;
/// Name of data type (examples: UInt64, Array(String)). /// Name of data type (examples: UInt64, Array(String)).
virtual String getName() const { return getFamilyName(); } String getName() const;
/// Name of data type family (example: FixedString, Array). /// Name of data type family (example: FixedString, Array).
virtual const char * getFamilyName() const = 0; virtual const char * getFamilyName() const = 0;
@ -217,6 +221,43 @@ public:
/// If method will throw an exception, then column will be in same state as before call to method. /// If method will throw an exception, then column will be in same state as before call to method.
virtual void deserializeBinary(IColumn & column, ReadBuffer & istr) const = 0; virtual void deserializeBinary(IColumn & column, ReadBuffer & istr) const = 0;
/** Text serialization with escaping but without quoting.
*/
virtual void serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
virtual void deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization as a literal that may be inserted into a query.
*/
virtual void serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
virtual void deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization for the CSV format.
*/
virtual void serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
virtual void deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization for displaying on a terminal or saving into a text file, and the like.
* Without escaping or quoting.
*/
virtual void serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
/** Text serialization intended for using in JSON format.
*/
virtual void serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
virtual void deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization for putting into the XML format.
*/
virtual void serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
/** Serialize to a protobuf. */
virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const = 0;
protected:
virtual String doGetName() const;
/** Text serialization with escaping but without quoting. /** Text serialization with escaping but without quoting.
*/ */
virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0; virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
@ -232,10 +273,6 @@ public:
/** Text serialization for the CSV format. /** Text serialization for the CSV format.
*/ */
virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0; virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
/** delimiter - the delimiter we expect when reading a string value that is not double-quoted
* (the delimiter is not consumed).
*/
virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0; virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
/** Text serialization for displaying on a terminal or saving into a text file, and the like. /** Text serialization for displaying on a terminal or saving into a text file, and the like.
@ -256,9 +293,7 @@ public:
serializeText(column, row_num, ostr, settings); serializeText(column, row_num, ostr, settings);
} }
/** Serialize to a protobuf. */ public:
virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const = 0;
/** Create empty column for corresponding type. /** Create empty column for corresponding type.
*/ */
virtual MutableColumnPtr createColumn() const = 0; virtual MutableColumnPtr createColumn() const = 0;
@ -290,8 +325,6 @@ public:
/// Checks that two instances belong to the same type /// Checks that two instances belong to the same type
virtual bool equals(const IDataType & rhs) const = 0; virtual bool equals(const IDataType & rhs) const = 0;
virtual ~IDataType() {}
/// Various properties on behaviour of data type. /// Various properties on behaviour of data type.
@ -419,6 +452,21 @@ public:
static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint);
static String getFileNameForStream(const String & column_name, const SubstreamPath & path); static String getFileNameForStream(const String & column_name, const SubstreamPath & path);
private:
friend class DataTypeFactory;
/** Sets domain on existing DataType, can be considered as second phase
* of construction explicitly done by DataTypeFactory.
* Will throw an exception if domain is already set.
*/
void setDomain(const IDataTypeDomain* newDomain) const;
private:
/** This is mutable to allow setting domain on `const IDataType` post construction,
* simplifying creation of domains for all types, without them even knowing
* of domain existence.
*/
mutable IDataTypeDomain const* domain;
}; };

View File

@ -0,0 +1,59 @@
#pragma once
#include <cstddef>
namespace DB
{
class ReadBuffer;
class WriteBuffer;
struct FormatSettings;
class IColumn;
/** Further refinment of the properties of data type.
*
* Contains methods for serialization/deserialization.
* Implementations of this interface represent a data type domain (example: IPv4)
* which is a refinement of the exsitgin type with a name and specific text
* representation.
*
* IDataTypeDomain is totally immutable object. You can always share them.
*/
class IDataTypeDomain
{
public:
virtual ~IDataTypeDomain() {}
virtual const char* getName() const = 0;
/** Text serialization for displaying on a terminal or saving into a text file, and the like.
* Without escaping or quoting.
*/
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
/** Text serialization with escaping but without quoting.
*/
virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
/** Text serialization as a literal that may be inserted into a query.
*/
virtual void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
/** Text serialization for the CSV format.
*/
virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
/** Text serialization intended for using in JSON format.
*/
virtual void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
/** Text serialization for putting into the XML format.
*/
virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const = 0;
};
} // namespace DB

View File

@ -355,7 +355,7 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
{ {
ReadBufferFromString null_value_buffer{null_value_string}; ReadBufferFromString null_value_buffer{null_value_string};
auto column_with_null_value = type->createColumn(); auto column_with_null_value = type->createColumn();
type->deserializeTextEscaped(*column_with_null_value, null_value_buffer, format_settings); type->deserializeAsTextEscaped(*column_with_null_value, null_value_buffer, format_settings);
null_value = (*column_with_null_value)[0]; null_value = (*column_with_null_value)[0];
} }
} }

View File

@ -345,7 +345,7 @@ void ExternalQueryBuilder::composeKeyCondition(const Columns & key_columns, cons
/// key_i=value_i /// key_i=value_i
writeString(key_description.name, out); writeString(key_description.name, out);
writeString("=", out); writeString("=", out);
key_description.type->serializeTextQuoted(*key_columns[i], row, out, format_settings); key_description.type->serializeAsTextQuoted(*key_columns[i], row, out, format_settings);
} }
writeString(")", out); writeString(")", out);
@ -387,7 +387,7 @@ void ExternalQueryBuilder::composeKeyTuple(const Columns & key_columns, const si
writeString(", ", out); writeString(", ", out);
first = false; first = false;
(*dict_struct.key)[i].type->serializeTextQuoted(*key_columns[i], row, out, format_settings); (*dict_struct.key)[i].type->serializeAsTextQuoted(*key_columns[i], row, out, format_settings);
} }
writeString(")", out); writeString(")", out);

View File

@ -123,7 +123,7 @@ bool CSVRowInputStream::read(MutableColumns & columns, RowReadExtension &)
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
skipWhitespacesAndTabs(istr); skipWhitespacesAndTabs(istr);
data_types[i]->deserializeTextCSV(*columns[i], istr, format_settings); data_types[i]->deserializeAsTextCSV(*columns[i], istr, format_settings);
skipWhitespacesAndTabs(istr); skipWhitespacesAndTabs(istr);
skipDelimiter(istr, format_settings.csv.delimiter, i + 1 == size); skipDelimiter(istr, format_settings.csv.delimiter, i + 1 == size);
@ -215,7 +215,7 @@ bool CSVRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
{ {
skipWhitespacesAndTabs(istr); skipWhitespacesAndTabs(istr);
prev_position = istr.position(); prev_position = istr.position();
data_types[i]->deserializeTextCSV(*columns[i], istr, format_settings); data_types[i]->deserializeAsTextCSV(*columns[i], istr, format_settings);
curr_position = istr.position(); curr_position = istr.position();
skipWhitespacesAndTabs(istr); skipWhitespacesAndTabs(istr);
} }

View File

@ -42,7 +42,7 @@ void CSVRowOutputStream::writePrefix()
void CSVRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num) void CSVRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{ {
type.serializeTextCSV(column, row_num, ostr, format_settings); type.serializeAsTextCSV(column, row_num, ostr, format_settings);
} }

View File

@ -16,7 +16,7 @@ JSONCompactRowOutputStream::JSONCompactRowOutputStream(WriteBuffer & ostr_, cons
void JSONCompactRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num) void JSONCompactRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{ {
type.serializeTextJSON(column, row_num, *ostr, settings); type.serializeAsTextJSON(column, row_num, *ostr, settings);
++field_number; ++field_number;
} }
@ -58,7 +58,7 @@ void JSONCompactRowOutputStream::writeTotals()
writeChar(',', *ostr); writeChar(',', *ostr);
const ColumnWithTypeAndName & column = totals.safeGetByPosition(i); const ColumnWithTypeAndName & column = totals.safeGetByPosition(i);
column.type->serializeTextJSON(*column.column.get(), 0, *ostr, settings); column.type->serializeAsTextJSON(*column.column.get(), 0, *ostr, settings);
} }
writeChar(']', *ostr); writeChar(']', *ostr);
@ -79,7 +79,7 @@ static void writeExtremesElement(const char * title, const Block & extremes, siz
writeChar(',', ostr); writeChar(',', ostr);
const ColumnWithTypeAndName & column = extremes.safeGetByPosition(i); const ColumnWithTypeAndName & column = extremes.safeGetByPosition(i);
column.type->serializeTextJSON(*column.column.get(), row_num, ostr, settings); column.type->serializeAsTextJSON(*column.column.get(), row_num, ostr, settings);
} }
writeChar(']', ostr); writeChar(']', ostr);

View File

@ -134,7 +134,7 @@ void JSONEachRowRowInputStream::readField(size_t index, MutableColumns & columns
try try
{ {
header.getByPosition(index).type->deserializeTextJSON(*columns[index], istr, format_settings); header.getByPosition(index).type->deserializeAsTextJSON(*columns[index], istr, format_settings);
} }
catch (Exception & e) catch (Exception & e)
{ {

View File

@ -27,7 +27,7 @@ void JSONEachRowRowOutputStream::writeField(const IColumn & column, const IDataT
{ {
writeString(fields[field_number], ostr); writeString(fields[field_number], ostr);
writeChar(':', ostr); writeChar(':', ostr);
type.serializeTextJSON(column, row_num, ostr, settings); type.serializeAsTextJSON(column, row_num, ostr, settings);
++field_number; ++field_number;
} }

View File

@ -71,7 +71,7 @@ void JSONRowOutputStream::writeField(const IColumn & column, const IDataType & t
writeCString("\t\t\t", *ostr); writeCString("\t\t\t", *ostr);
writeString(fields[field_number].name, *ostr); writeString(fields[field_number].name, *ostr);
writeCString(": ", *ostr); writeCString(": ", *ostr);
type.serializeTextJSON(column, row_num, *ostr, settings); type.serializeAsTextJSON(column, row_num, *ostr, settings);
++field_number; ++field_number;
} }
@ -151,7 +151,7 @@ void JSONRowOutputStream::writeTotals()
writeCString("\t\t", *ostr); writeCString("\t\t", *ostr);
writeJSONString(column.name, *ostr, settings); writeJSONString(column.name, *ostr, settings);
writeCString(": ", *ostr); writeCString(": ", *ostr);
column.type->serializeTextJSON(*column.column.get(), 0, *ostr, settings); column.type->serializeAsTextJSON(*column.column.get(), 0, *ostr, settings);
} }
writeChar('\n', *ostr); writeChar('\n', *ostr);
@ -178,7 +178,7 @@ static void writeExtremesElement(const char * title, const Block & extremes, siz
writeCString("\t\t\t", ostr); writeCString("\t\t\t", ostr);
writeJSONString(column.name, ostr, settings); writeJSONString(column.name, ostr, settings);
writeCString(": ", ostr); writeCString(": ", ostr);
column.type->serializeTextJSON(*column.column.get(), row_num, ostr, settings); column.type->serializeAsTextJSON(*column.column.get(), row_num, ostr, settings);
} }
writeChar('\n', ostr); writeChar('\n', ostr);

View File

@ -43,7 +43,7 @@ static void writeRow(const Block & block, size_t row_idx, WriteBuffer & out, con
{ {
{ {
WriteBufferFromString text_out(buffer); WriteBufferFromString text_out(buffer);
col.type->serializeText(*col.column, row_idx, text_out, format_settings); col.type->serializeAsText(*col.column, row_idx, text_out, format_settings);
} }
writeODBCString(out, buffer); writeODBCString(out, buffer);
} }

View File

@ -33,7 +33,7 @@ void ODBCDriverBlockOutputStream::write(const Block & block)
{ {
WriteBufferFromString text_out(text_value); WriteBufferFromString text_out(text_value);
col.type->serializeText(*col.column, i, text_out, format_settings); col.type->serializeAsText(*col.column, i, text_out, format_settings);
} }
writeStringBinary(text_value, out); writeStringBinary(text_value, out);

View File

@ -58,7 +58,7 @@ void PrettyBlockOutputStream::calculateWidths(
{ {
{ {
WriteBufferFromString out(serialized_value); WriteBufferFromString out(serialized_value);
elem.type->serializeText(*elem.column, j, out, format_settings); elem.type->serializeAsText(*elem.column, j, out, format_settings);
} }
widths[i][j] = std::min<UInt64>(format_settings.pretty.max_column_pad_width, widths[i][j] = std::min<UInt64>(format_settings.pretty.max_column_pad_width,
@ -206,11 +206,11 @@ void PrettyBlockOutputStream::writeValueWithPadding(const ColumnWithTypeAndName
if (elem.type->shouldAlignRightInPrettyFormats()) if (elem.type->shouldAlignRightInPrettyFormats())
{ {
writePadding(); writePadding();
elem.type->serializeText(*elem.column.get(), row_num, ostr, format_settings); elem.type->serializeAsText(*elem.column.get(), row_num, ostr, format_settings);
} }
else else
{ {
elem.type->serializeText(*elem.column.get(), row_num, ostr, format_settings); elem.type->serializeAsText(*elem.column.get(), row_num, ostr, format_settings);
writePadding(); writePadding();
} }
} }

View File

@ -135,7 +135,7 @@ bool TSKVRowInputStream::read(MutableColumns & columns, RowReadExtension & ext)
read_columns[index] = true; read_columns[index] = true;
header.getByPosition(index).type->deserializeTextEscaped(*columns[index], istr, format_settings); header.getByPosition(index).type->deserializeAsTextEscaped(*columns[index], istr, format_settings);
} }
} }
else else

View File

@ -28,7 +28,7 @@ TSKVRowOutputStream::TSKVRowOutputStream(WriteBuffer & ostr_, const Block & samp
void TSKVRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num) void TSKVRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{ {
writeString(fields[field_number].name, ostr); writeString(fields[field_number].name, ostr);
type.serializeTextEscaped(column, row_num, ostr, format_settings); type.serializeAsTextEscaped(column, row_num, ostr, format_settings);
++field_number; ++field_number;
} }

View File

@ -18,7 +18,7 @@ public:
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override void writeField(const IColumn & column, const IDataType & type, size_t row_num) override
{ {
type.serializeText(column, row_num, ostr, format_settings); type.serializeAsText(column, row_num, ostr, format_settings);
} }
}; };

View File

@ -86,7 +86,7 @@ bool TabSeparatedRowInputStream::read(MutableColumns & columns, RowReadExtension
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
data_types[i]->deserializeTextEscaped(*columns[i], istr, format_settings); data_types[i]->deserializeAsTextEscaped(*columns[i], istr, format_settings);
/// skip separators /// skip separators
if (i + 1 == size) if (i + 1 == size)
@ -183,7 +183,7 @@ bool TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns &
try try
{ {
data_types[i]->deserializeTextEscaped(*columns[i], istr, format_settings); data_types[i]->deserializeAsTextEscaped(*columns[i], istr, format_settings);
} }
catch (...) catch (...)
{ {

View File

@ -48,7 +48,7 @@ void TabSeparatedRowOutputStream::writePrefix()
void TabSeparatedRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num) void TabSeparatedRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{ {
type.serializeTextEscaped(column, row_num, ostr, format_settings); type.serializeAsTextEscaped(column, row_num, ostr, format_settings);
} }

View File

@ -64,7 +64,7 @@ bool ValuesRowInputStream::read(MutableColumns & columns, RowReadExtension &)
bool rollback_on_exception = false; bool rollback_on_exception = false;
try try
{ {
header.getByPosition(i).type->deserializeTextQuoted(*columns[i], istr, format_settings); header.getByPosition(i).type->deserializeAsTextQuoted(*columns[i], istr, format_settings);
rollback_on_exception = true; rollback_on_exception = true;
skipWhitespaceIfAny(istr); skipWhitespaceIfAny(istr);

View File

@ -23,7 +23,7 @@ void ValuesRowOutputStream::flush()
void ValuesRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num) void ValuesRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{ {
type.serializeTextQuoted(column, row_num, ostr, format_settings); type.serializeAsTextQuoted(column, row_num, ostr, format_settings);
} }
void ValuesRowOutputStream::writeFieldDelimiter() void ValuesRowOutputStream::writeFieldDelimiter()

View File

@ -71,7 +71,7 @@ void VerticalRowOutputStream::writeField(const IColumn & column, const IDataType
void VerticalRowOutputStream::writeValue(const IColumn & column, const IDataType & type, size_t row_num) const void VerticalRowOutputStream::writeValue(const IColumn & column, const IDataType & type, size_t row_num) const
{ {
type.serializeText(column, row_num, ostr, format_settings); type.serializeAsText(column, row_num, ostr, format_settings);
} }

View File

@ -87,7 +87,7 @@ void XMLRowOutputStream::writeField(const IColumn & column, const IDataType & ty
writeCString("\t\t\t<", *ostr); writeCString("\t\t\t<", *ostr);
writeString(field_tag_names[field_number], *ostr); writeString(field_tag_names[field_number], *ostr);
writeCString(">", *ostr); writeCString(">", *ostr);
type.serializeTextXML(column, row_num, *ostr, format_settings); type.serializeAsTextXML(column, row_num, *ostr, format_settings);
writeCString("</", *ostr); writeCString("</", *ostr);
writeString(field_tag_names[field_number], *ostr); writeString(field_tag_names[field_number], *ostr);
writeCString(">\n", *ostr); writeCString(">\n", *ostr);
@ -153,7 +153,7 @@ void XMLRowOutputStream::writeTotals()
writeCString("\t\t<", *ostr); writeCString("\t\t<", *ostr);
writeString(field_tag_names[i], *ostr); writeString(field_tag_names[i], *ostr);
writeCString(">", *ostr); writeCString(">", *ostr);
column.type->serializeTextXML(*column.column.get(), 0, *ostr, format_settings); column.type->serializeAsTextXML(*column.column.get(), 0, *ostr, format_settings);
writeCString("</", *ostr); writeCString("</", *ostr);
writeString(field_tag_names[i], *ostr); writeString(field_tag_names[i], *ostr);
writeCString(">\n", *ostr); writeCString(">\n", *ostr);
@ -179,7 +179,7 @@ static void writeExtremesElement(
writeCString("\t\t\t<", ostr); writeCString("\t\t\t<", ostr);
writeString(field_tag_names[i], ostr); writeString(field_tag_names[i], ostr);
writeCString(">", ostr); writeCString(">", ostr);
column.type->serializeTextXML(*column.column.get(), row_num, ostr, format_settings); column.type->serializeAsTextXML(*column.column.get(), row_num, ostr, format_settings);
writeCString("</", ostr); writeCString("</", ostr);
writeString(field_tag_names[i], ostr); writeString(field_tag_names[i], ostr);
writeCString(">\n", ostr); writeCString(">\n", ostr);

View File

@ -26,6 +26,8 @@ void registerFunctionsCoding(FunctionFactory & factory)
factory.registerFunction<FunctionHex>(); factory.registerFunction<FunctionHex>();
factory.registerFunction<FunctionUnhex>(); factory.registerFunction<FunctionUnhex>();
factory.registerFunction<FunctionBitmaskToArray>(); factory.registerFunction<FunctionBitmaskToArray>();
factory.registerFunction<FunctionToIPv4>();
factory.registerFunction<FunctionToIPv6>();
} }
} }

View File

@ -4,6 +4,7 @@
#include <Common/formatIPv6.h> #include <Common/formatIPv6.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h> #include <DataTypes/DataTypeFixedString.h>
@ -54,8 +55,6 @@ namespace ErrorCodes
*/ */
constexpr size_t ipv4_bytes_length = 4;
constexpr size_t ipv6_bytes_length = 16;
constexpr size_t uuid_bytes_length = 16; constexpr size_t uuid_bytes_length = 16;
constexpr size_t uuid_text_length = 36; constexpr size_t uuid_text_length = 36;
@ -74,10 +73,10 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{ {
const auto ptr = checkAndGetDataType<DataTypeFixedString>(arguments[0].get()); const auto ptr = checkAndGetDataType<DataTypeFixedString>(arguments[0].get());
if (!ptr || ptr->getN() != ipv6_bytes_length) if (!ptr || ptr->getN() != IPV6_BINARY_LENGTH)
throw Exception("Illegal type " + arguments[0]->getName() + throw Exception("Illegal type " + arguments[0]->getName() +
" of argument of function " + getName() + " of argument of function " + getName() +
", expected FixedString(" + toString(ipv6_bytes_length) + ")", ", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>(); return std::make_shared<DataTypeString>();
@ -92,11 +91,11 @@ public:
if (const auto col_in = checkAndGetColumn<ColumnFixedString>(column.get())) if (const auto col_in = checkAndGetColumn<ColumnFixedString>(column.get()))
{ {
if (col_in->getN() != ipv6_bytes_length) if (col_in->getN() != IPV6_BINARY_LENGTH)
throw Exception("Illegal type " + col_type_name.type->getName() + throw Exception("Illegal type " + col_type_name.type->getName() +
" of column " + col_in->getName() + " of column " + col_in->getName() +
" argument of function " + getName() + " argument of function " + getName() +
", expected FixedString(" + toString(ipv6_bytes_length) + ")", ", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto size = col_in->size(); const auto size = col_in->size();
@ -112,7 +111,7 @@ public:
auto begin = reinterpret_cast<char *>(vec_res.data()); auto begin = reinterpret_cast<char *>(vec_res.data());
auto pos = begin; auto pos = begin;
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i) for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += IPV6_BINARY_LENGTH, ++i)
{ {
formatIPv6(&vec_in[offset], pos); formatIPv6(&vec_in[offset], pos);
offsets_res[i] = pos - begin; offsets_res[i] = pos - begin;
@ -143,10 +142,10 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{ {
const auto ptr = checkAndGetDataType<DataTypeFixedString>(arguments[0].get()); const auto ptr = checkAndGetDataType<DataTypeFixedString>(arguments[0].get());
if (!ptr || ptr->getN() != ipv6_bytes_length) if (!ptr || ptr->getN() != IPV6_BINARY_LENGTH)
throw Exception("Illegal type " + arguments[0]->getName() + throw Exception("Illegal type " + arguments[0]->getName() +
" of argument 1 of function " + getName() + " of argument 1 of function " + getName() +
", expected FixedString(" + toString(ipv6_bytes_length) + ")", ", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!WhichDataType(arguments[1]).isUInt8()) if (!WhichDataType(arguments[1]).isUInt8())
@ -177,11 +176,11 @@ public:
if (const auto col_in = checkAndGetColumn<ColumnFixedString>(column.get())) if (const auto col_in = checkAndGetColumn<ColumnFixedString>(column.get()))
{ {
if (col_in->getN() != ipv6_bytes_length) if (col_in->getN() != IPV6_BINARY_LENGTH)
throw Exception("Illegal type " + col_type_name.type->getName() + throw Exception("Illegal type " + col_type_name.type->getName() +
" of column " + col_in->getName() + " of column " + col_in->getName() +
" argument of function " + getName() + " argument of function " + getName() +
", expected FixedString(" + toString(ipv6_bytes_length) + ")", ", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto ipv6_zeroed_tail_bytes = checkAndGetColumnConst<ColumnVector<UInt8>>(col_ipv6_zeroed_tail_bytes.get()); const auto ipv6_zeroed_tail_bytes = checkAndGetColumnConst<ColumnVector<UInt8>>(col_ipv6_zeroed_tail_bytes.get());
@ -191,7 +190,7 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
UInt8 ipv6_zeroed_tail_bytes_count = ipv6_zeroed_tail_bytes->getValue<UInt8>(); UInt8 ipv6_zeroed_tail_bytes_count = ipv6_zeroed_tail_bytes->getValue<UInt8>();
if (ipv6_zeroed_tail_bytes_count > ipv6_bytes_length) if (ipv6_zeroed_tail_bytes_count > IPV6_BINARY_LENGTH)
throw Exception("Illegal value for argument 2 " + col_ipv6_zeroed_tail_bytes_type.type->getName() + throw Exception("Illegal value for argument 2 " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
" of function " + getName(), " of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -203,7 +202,7 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
UInt8 ipv4_zeroed_tail_bytes_count = ipv4_zeroed_tail_bytes->getValue<UInt8>(); UInt8 ipv4_zeroed_tail_bytes_count = ipv4_zeroed_tail_bytes->getValue<UInt8>();
if (ipv4_zeroed_tail_bytes_count > ipv6_bytes_length) if (ipv4_zeroed_tail_bytes_count > IPV6_BINARY_LENGTH)
throw Exception("Illegal value for argument 3 " + col_ipv4_zeroed_tail_bytes_type.type->getName() + throw Exception("Illegal value for argument 3 " + col_ipv4_zeroed_tail_bytes_type.type->getName() +
" of function " + getName(), " of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -221,7 +220,7 @@ public:
auto begin = reinterpret_cast<char *>(vec_res.data()); auto begin = reinterpret_cast<char *>(vec_res.data());
auto pos = begin; auto pos = begin;
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i) for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += IPV6_BINARY_LENGTH, ++i)
{ {
const auto address = &vec_in[offset]; const auto address = &vec_in[offset];
UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count; UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count;
@ -269,146 +268,7 @@ public:
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeFixedString>(ipv6_bytes_length); return std::make_shared<DataTypeFixedString>(IPV6_BINARY_LENGTH);
}
static bool ipv4_scan(const char * src, unsigned char * dst)
{
constexpr auto size = sizeof(UInt32);
char bytes[size]{};
for (const auto i : ext::range(0, size))
{
UInt32 value = 0;
size_t len = 0;
while (isNumericASCII(*src) && len <= 3)
{
value = value * 10 + (*src - '0');
++len;
++src;
}
if (len == 0 || value > 255 || (i < size - 1 && *src != '.'))
{
memset(dst, 0, size);
return false;
}
bytes[i] = value;
++src;
}
if (src[-1] != '\0')
{
memset(dst, 0, size);
return false;
}
memcpy(dst, bytes, sizeof(bytes));
return true;
}
/// slightly altered implementation from http://svn.apache.org/repos/asf/apr/apr/trunk/network_io/unix/inet_pton.c
static void ipv6_scan(const char * src, unsigned char * dst)
{
const auto clear_dst = [dst]
{
memset(dst, '\0', ipv6_bytes_length);
};
/// Leading :: requires some special handling.
if (*src == ':')
if (*++src != ':')
return clear_dst();
unsigned char tmp[ipv6_bytes_length]{};
auto tp = tmp;
auto endp = tp + ipv6_bytes_length;
auto curtok = src;
auto saw_xdigit = false;
UInt32 val{};
unsigned char * colonp = nullptr;
/// Assuming zero-terminated string.
while (const auto ch = *src++)
{
const auto num = unhex(ch);
if (num != '\xff')
{
val <<= 4;
val |= num;
if (val > 0xffffu)
return clear_dst();
saw_xdigit = 1;
continue;
}
if (ch == ':')
{
curtok = src;
if (!saw_xdigit)
{
if (colonp)
return clear_dst();
colonp = tp;
continue;
}
if (tp + sizeof(UInt16) > endp)
return clear_dst();
*tp++ = static_cast<unsigned char>((val >> 8) & 0xffu);
*tp++ = static_cast<unsigned char>(val & 0xffu);
saw_xdigit = false;
val = 0;
continue;
}
if (ch == '.' && (tp + ipv4_bytes_length) <= endp)
{
if (!ipv4_scan(curtok, tp))
return clear_dst();
tp += ipv4_bytes_length;
saw_xdigit = false;
break; /* '\0' was seen by ipv4_scan(). */
}
return clear_dst();
}
if (saw_xdigit)
{
if (tp + sizeof(UInt16) > endp)
return clear_dst();
*tp++ = static_cast<unsigned char>((val >> 8) & 0xffu);
*tp++ = static_cast<unsigned char>(val & 0xffu);
}
if (colonp)
{
/*
* Since some memmove()'s erroneously fail to handle
* overlapping regions, we'll do the shift by hand.
*/
const auto n = tp - colonp;
for (int i = 1; i <= n; ++i)
{
endp[- i] = colonp[n - i];
colonp[n - i] = 0;
}
tp = endp;
}
if (tp != endp)
return clear_dst();
memcpy(dst, tmp, sizeof(tmp));
} }
bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; }
@ -419,10 +279,10 @@ public:
if (const auto col_in = checkAndGetColumn<ColumnString>(column.get())) if (const auto col_in = checkAndGetColumn<ColumnString>(column.get()))
{ {
auto col_res = ColumnFixedString::create(ipv6_bytes_length); auto col_res = ColumnFixedString::create(IPV6_BINARY_LENGTH);
auto & vec_res = col_res->getChars(); auto & vec_res = col_res->getChars();
vec_res.resize(col_in->size() * ipv6_bytes_length); vec_res.resize(col_in->size() * IPV6_BINARY_LENGTH);
const ColumnString::Chars & vec_src = col_in->getChars(); const ColumnString::Chars & vec_src = col_in->getChars();
const ColumnString::Offsets & offsets_src = col_in->getOffsets(); const ColumnString::Offsets & offsets_src = col_in->getOffsets();
@ -430,9 +290,10 @@ public:
for (size_t out_offset = 0, i = 0; for (size_t out_offset = 0, i = 0;
out_offset < vec_res.size(); out_offset < vec_res.size();
out_offset += ipv6_bytes_length, ++i) out_offset += IPV6_BINARY_LENGTH, ++i)
{ {
ipv6_scan(reinterpret_cast<const char * >(&vec_src[src_offset]), &vec_res[out_offset]); //TODO(nemkov): handle failure ?
parseIPv6(reinterpret_cast<const char *>(&vec_src[src_offset]), &vec_res[out_offset]);
src_offset = offsets_src[i]; src_offset = offsets_src[i];
} }
@ -451,59 +312,6 @@ public:
template <size_t mask_tail_octets, typename Name> template <size_t mask_tail_octets, typename Name>
class FunctionIPv4NumToString : public IFunction class FunctionIPv4NumToString : public IFunction
{ {
private:
static void formatIP(UInt32 ip, char *& out)
{
char * begin = out;
for (size_t octet = 0; octet < mask_tail_octets; ++octet)
{
if (octet > 0)
{
*out = '.';
++out;
}
memcpy(out, "xxx", 3); /// Strange choice, but meets the specification.
out += 3;
}
/// Write everything backwards. NOTE The loop is unrolled.
for (size_t octet = mask_tail_octets; octet < 4; ++octet)
{
if (octet > 0)
{
*out = '.';
++out;
}
/// Get the next byte.
UInt32 value = (ip >> (octet * 8)) & static_cast<UInt32>(0xFF);
/// Faster than sprintf. NOTE Actually not good enough. LUT will be better.
if (value == 0)
{
*out = '0';
++out;
}
else
{
while (value > 0)
{
*out = '0' + value % 10;
++out;
value /= 10;
}
}
}
/// And reverse.
std::reverse(begin, out);
*out = '\0';
++out;
}
public: public:
static constexpr auto name = Name::name; static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv4NumToString<mask_tail_octets, Name>>(); } static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv4NumToString<mask_tail_octets, Name>>(); }
@ -547,7 +355,7 @@ public:
for (size_t i = 0; i < vec_in.size(); ++i) for (size_t i = 0; i < vec_in.size(); ++i)
{ {
formatIP(vec_in[i], pos); DB::formatIPv4(reinterpret_cast<const unsigned char*>(&vec_in[i]), pos, mask_tail_octets, "xxx");
offsets_res[i] = pos - begin; offsets_res[i] = pos - begin;
} }
@ -585,27 +393,12 @@ public:
return std::make_shared<DataTypeUInt32>(); return std::make_shared<DataTypeUInt32>();
} }
static UInt32 parseIPv4(const char * pos) static inline UInt32 parseIPv4(const char * pos)
{ {
UInt32 res = 0; UInt32 result = 0;
for (int offset = 24; offset >= 0; offset -= 8) DB::parseIPv4(pos, reinterpret_cast<unsigned char*>(&result));
{
UInt32 value = 0; return result;
size_t len = 0;
while (isNumericASCII(*pos) && len <= 3)
{
value = value * 10 + (*pos - '0');
++len;
++pos;
}
if (len == 0 || value > 255 || (offset > 0 && *pos != '.'))
return 0;
res |= value << offset;
++pos;
}
if (*(pos - 1) != '\0')
return 0;
return res;
} }
bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; }
@ -670,14 +463,14 @@ public:
if (const auto col_in = typeid_cast<const ColumnUInt32 *>(column.get())) if (const auto col_in = typeid_cast<const ColumnUInt32 *>(column.get()))
{ {
auto col_res = ColumnFixedString::create(ipv6_bytes_length); auto col_res = ColumnFixedString::create(IPV6_BINARY_LENGTH);
auto & vec_res = col_res->getChars(); auto & vec_res = col_res->getChars();
vec_res.resize(col_in->size() * ipv6_bytes_length); vec_res.resize(col_in->size() * IPV6_BINARY_LENGTH);
const auto & vec_in = col_in->getData(); const auto & vec_in = col_in->getData();
for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); out_offset += ipv6_bytes_length, ++i) for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); out_offset += IPV6_BINARY_LENGTH, ++i)
mapIPv4ToIPv6(vec_in[i], &vec_res[out_offset]); mapIPv4ToIPv6(vec_in[i], &vec_res[out_offset]);
block.getByPosition(result).column = std::move(col_res); block.getByPosition(result).column = std::move(col_res);
@ -696,6 +489,46 @@ private:
} }
}; };
class FunctionToIPv4 : public FunctionIPv4StringToNum
{
public:
static constexpr auto name = "toIPv4";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToIPv4>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return DataTypeFactory::instance().get("IPv4");
}
};
class FunctionToIPv6 : public FunctionIPv6StringToNum
{
public:
static constexpr auto name = "toIPv6";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToIPv6>(); }
String getName() const override { return name; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return DataTypeFactory::instance().get("IPv6");
}
};
class FunctionMACNumToString : public IFunction class FunctionMACNumToString : public IFunction
{ {

View File

@ -347,7 +347,7 @@ struct ConvertImplGenericToString
FormatSettings format_settings; FormatSettings format_settings;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
type.serializeText(col_from, i, write_buffer, format_settings); type.serializeAsText(col_from, i, write_buffer, format_settings);
writeChar(0, write_buffer); writeChar(0, write_buffer);
offsets_to[i] = write_buffer.count(); offsets_to[i] = write_buffer.count();
} }
@ -631,7 +631,7 @@ struct ConvertImplGenericFromString
{ {
ReadBufferFromMemory read_buffer(&chars[current_offset], offsets[i] - current_offset - 1); ReadBufferFromMemory read_buffer(&chars[current_offset], offsets[i] - current_offset - 1);
data_type_to.deserializeTextEscaped(column_to, read_buffer, format_settings); data_type_to.deserializeAsTextEscaped(column_to, read_buffer, format_settings);
if (!read_buffer.eof()) if (!read_buffer.eof())
throwExceptionForIncompletelyParsedValue(read_buffer, block, result); throwExceptionForIncompletelyParsedValue(read_buffer, block, result);

View File

@ -204,8 +204,6 @@ inline float roundWithMode(float x, RoundingMode mode)
case RoundingMode::Floor: return floorf(x); case RoundingMode::Floor: return floorf(x);
case RoundingMode::Ceil: return ceilf(x); case RoundingMode::Ceil: return ceilf(x);
case RoundingMode::Trunc: return truncf(x); case RoundingMode::Trunc: return truncf(x);
default:
throw Exception("Logical error: unexpected 'mode' parameter passed to function roundWithMode", ErrorCodes::LOGICAL_ERROR);
} }
} }
@ -217,8 +215,6 @@ inline double roundWithMode(double x, RoundingMode mode)
case RoundingMode::Floor: return floor(x); case RoundingMode::Floor: return floor(x);
case RoundingMode::Ceil: return ceil(x); case RoundingMode::Ceil: return ceil(x);
case RoundingMode::Trunc: return trunc(x); case RoundingMode::Trunc: return trunc(x);
default:
throw Exception("Logical error: unexpected 'mode' parameter passed to function roundWithMode", ErrorCodes::LOGICAL_ERROR);
} }
} }

View File

@ -8,6 +8,7 @@
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Interpreters/AggregationCommon.h> #include <Interpreters/AggregationCommon.h>
#include <Common/HashTable/ClearableHashMap.h> #include <Common/HashTable/ClearableHashMap.h>
#include <Common/ColumnsHashing.h>
namespace DB namespace DB
@ -60,11 +61,56 @@ private:
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
static constexpr size_t INITIAL_SIZE_DEGREE = 9; static constexpr size_t INITIAL_SIZE_DEGREE = 9;
template <typename T>
struct MethodOneNumber
{
using Set = ClearableHashMap<T, UInt32, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
using Method = ColumnsHashing::HashMethodOneNumber<typename Set::value_type, UInt32, T, false>;
};
struct MethodString
{
using Set = ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
using Method = ColumnsHashing::HashMethodString<typename Set::value_type, UInt32, false, false>;
};
struct MethodFixedString
{
using Set = ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
using Method = ColumnsHashing::HashMethodFixedString<typename Set::value_type, UInt32, false, false>;
};
struct MethodFixed
{
using Set = ClearableHashMap<UInt128, UInt32, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
using Method = ColumnsHashing::HashMethodKeysFixed<typename Set::value_type, UInt128, UInt32, false, false, false>;
};
struct MethodHashed
{
using Set = ClearableHashMap<UInt128, UInt32, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
using Method = ColumnsHashing::HashMethodHashed<typename Set::value_type, UInt32, false>;
};
template <typename Method>
void executeMethod(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, const Sizes & key_sizes,
const NullMap * null_map, ColumnUInt32::Container & res_values);
template <typename Method, bool has_null_map>
void executeMethodImpl(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, const Sizes & key_sizes,
const NullMap * null_map, ColumnUInt32::Container & res_values);
template <typename T> template <typename T>
bool executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values); bool executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values);
bool executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values); bool executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values);
bool executeFixedString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values);
bool execute128bit(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values); bool execute128bit(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values);
bool executeHashed(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values); void executeHashed(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values);
}; };
@ -131,7 +177,7 @@ void FunctionArrayEnumerateExtended<Derived>::executeImpl(Block & block, const C
if (num_arguments == 1) if (num_arguments == 1)
{ {
executeNumber<UInt8>(*offsets, *data_columns[0], null_map, res_values) if (!(executeNumber<UInt8>(*offsets, *data_columns[0], null_map, res_values)
|| executeNumber<UInt16>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<UInt16>(*offsets, *data_columns[0], null_map, res_values)
|| executeNumber<UInt32>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<UInt32>(*offsets, *data_columns[0], null_map, res_values)
|| executeNumber<UInt64>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<UInt64>(*offsets, *data_columns[0], null_map, res_values)
@ -142,47 +188,56 @@ void FunctionArrayEnumerateExtended<Derived>::executeImpl(Block & block, const C
|| executeNumber<Float32>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<Float32>(*offsets, *data_columns[0], null_map, res_values)
|| executeNumber<Float64>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<Float64>(*offsets, *data_columns[0], null_map, res_values)
|| executeString(*offsets, *data_columns[0], null_map, res_values) || executeString(*offsets, *data_columns[0], null_map, res_values)
|| executeHashed(*offsets, data_columns, res_values); || executeFixedString(*offsets, *data_columns[0], null_map, res_values)))
executeHashed(*offsets, data_columns, res_values);
} }
else else
{ {
execute128bit(*offsets, data_columns, res_values) if (!execute128bit(*offsets, data_columns, res_values))
|| executeHashed(*offsets, data_columns, res_values); executeHashed(*offsets, data_columns, res_values);
} }
block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), offsets_column); block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), offsets_column);
} }
template <typename Derived> template <typename Derived>
template <typename T> template <typename Method, bool has_null_map>
bool FunctionArrayEnumerateExtended<Derived>::executeNumber( void FunctionArrayEnumerateExtended<Derived>::executeMethodImpl(
const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) const ColumnArray::Offsets & offsets,
const ColumnRawPtrs & columns,
const Sizes & key_sizes,
[[maybe_unused]] const NullMap * null_map,
ColumnUInt32::Container & res_values)
{ {
const ColumnVector<T> * data_concrete = checkAndGetColumn<ColumnVector<T>>(&data); typename Method::Set indices;
if (!data_concrete) typename Method::Method method(columns, key_sizes, nullptr);
return false; Arena pool; /// Won't use it;
const auto & values = data_concrete->getData();
using ValuesToIndices = ClearableHashMap<T, UInt32, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>, ColumnArray::Offset prev_off = 0;
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
ValuesToIndices indices;
size_t prev_off = 0;
if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>) if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>)
{ {
// Unique // Unique
for (size_t i = 0; i < offsets.size(); ++i) for (size_t off : offsets)
{ {
indices.clear(); indices.clear();
UInt32 null_count = 0; UInt32 null_count = 0;
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j) for (size_t j = prev_off; j < off; ++j)
{ {
if (null_map && (*null_map)[j]) if constexpr (has_null_map)
res_values[j] = ++null_count; {
else if ((*null_map)[j])
res_values[j] = ++indices[values[j]]; {
res_values[j] = ++null_count;
continue;
}
}
auto emplace_result = method.emplaceKey(indices, j, pool);
auto idx = emplace_result.getMapped() + 1;
emplace_result.setMapped(idx);
res_values[j] = idx;
} }
prev_off = off; prev_off = off;
} }
@ -190,31 +245,67 @@ bool FunctionArrayEnumerateExtended<Derived>::executeNumber(
else else
{ {
// Dense // Dense
for (size_t i = 0; i < offsets.size(); ++i) for (size_t off : offsets)
{ {
indices.clear(); indices.clear();
size_t rank = 0; UInt32 rank = 0;
UInt32 null_index = 0; [[maybe_unused]] UInt32 null_index = 0;
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j) for (size_t j = prev_off; j < off; ++j)
{ {
if (null_map && (*null_map)[j]) if constexpr (has_null_map)
{ {
if (!null_index) if ((*null_map)[j])
null_index = ++rank; {
res_values[j] = null_index; if (!null_index)
null_index = ++rank;
res_values[j] = null_index;
continue;
}
} }
else
auto emplace_result = method.emplaceKey(indices, j, pool);
auto idx = emplace_result.getMapped();
if (!idx)
{ {
auto & idx = indices[values[j]]; idx = ++rank;
if (!idx) emplace_result.setMapped(idx);
idx = ++rank;
res_values[j] = idx;
} }
res_values[j] = idx;
} }
prev_off = off; prev_off = off;
} }
} }
}
template <typename Derived>
template <typename Method>
void FunctionArrayEnumerateExtended<Derived>::executeMethod(
const ColumnArray::Offsets & offsets,
const ColumnRawPtrs & columns,
const Sizes & key_sizes,
const NullMap * null_map,
ColumnUInt32::Container & res_values)
{
if (null_map)
executeMethodImpl<Method, true>(offsets, columns, key_sizes, null_map, res_values);
else
executeMethodImpl<Method, false>(offsets, columns, key_sizes, null_map, res_values);
}
template <typename Derived>
template <typename T>
bool FunctionArrayEnumerateExtended<Derived>::executeNumber(
const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values)
{
const auto * nested = checkAndGetColumn<ColumnVector<T>>(&data);
if (!nested)
return false;
executeMethod<MethodOneNumber<T>>(offsets, {nested}, {}, null_map, res_values);
return true; return true;
} }
@ -222,62 +313,22 @@ template <typename Derived>
bool FunctionArrayEnumerateExtended<Derived>::executeString( bool FunctionArrayEnumerateExtended<Derived>::executeString(
const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values)
{ {
const ColumnString * values = checkAndGetColumn<ColumnString>(&data); const auto * nested = checkAndGetColumn<ColumnString>(&data);
if (!values) if (nested)
return false; executeMethod<MethodString>(offsets, {nested}, {}, null_map, res_values);
size_t prev_off = 0; return nested;
using ValuesToIndices = ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>, }
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
ValuesToIndices indices; template <typename Derived>
if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>) bool FunctionArrayEnumerateExtended<Derived>::executeFixedString(
{ const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values)
// Unique {
for (size_t i = 0; i < offsets.size(); ++i) const auto * nested = checkAndGetColumn<ColumnString>(&data);
{ if (nested)
indices.clear(); executeMethod<MethodFixedString>(offsets, {nested}, {}, null_map, res_values);
UInt32 null_count = 0;
size_t off = offsets[i]; return nested;
for (size_t j = prev_off; j < off; ++j)
{
if (null_map && (*null_map)[j])
res_values[j] = ++null_count;
else
res_values[j] = ++indices[values->getDataAt(j)];
}
prev_off = off;
}
}
else
{
// Dense
for (size_t i = 0; i < offsets.size(); ++i)
{
indices.clear();
size_t rank = 0;
UInt32 null_index = 0;
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j)
{
if (null_map && (*null_map)[j])
{
if (!null_index)
null_index = ++rank;
res_values[j] = null_index;
}
else
{
auto & idx = indices[values->getDataAt(j)];
if (!idx)
idx = ++rank;
res_values[j] = idx;
}
}
prev_off = off;
}
}
return true;
} }
template <typename Derived> template <typename Derived>
@ -298,95 +349,17 @@ bool FunctionArrayEnumerateExtended<Derived>::execute128bit(
keys_bytes += key_sizes[j]; keys_bytes += key_sizes[j];
} }
if (keys_bytes > 16) executeMethod<MethodFixed>(offsets, columns, key_sizes, nullptr, res_values);
return false;
using ValuesToIndices = ClearableHashMap<UInt128, UInt32, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
ValuesToIndices indices;
size_t prev_off = 0;
if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>)
{
// Unique
for (size_t i = 0; i < offsets.size(); ++i)
{
indices.clear();
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j)
res_values[j] = ++indices[packFixed<UInt128>(j, count, columns, key_sizes)];
prev_off = off;
}
}
else
{
// Dense
for (size_t i = 0; i < offsets.size(); ++i)
{
indices.clear();
size_t off = offsets[i];
size_t rank = 0;
for (size_t j = prev_off; j < off; ++j)
{
auto & idx = indices[packFixed<UInt128>(j, count, columns, key_sizes)];
if (!idx)
idx = ++rank;
res_values[j] = idx;
}
prev_off = off;
}
}
return true; return true;
} }
template <typename Derived> template <typename Derived>
bool FunctionArrayEnumerateExtended<Derived>::executeHashed( void FunctionArrayEnumerateExtended<Derived>::executeHashed(
const ColumnArray::Offsets & offsets, const ColumnArray::Offsets & offsets,
const ColumnRawPtrs & columns, const ColumnRawPtrs & columns,
ColumnUInt32::Container & res_values) ColumnUInt32::Container & res_values)
{ {
size_t count = columns.size(); executeMethod<MethodHashed>(offsets, columns, {}, nullptr, res_values);
using ValuesToIndices = ClearableHashMap<UInt128, UInt32, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
ValuesToIndices indices;
size_t prev_off = 0;
if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>)
{
// Unique
for (size_t i = 0; i < offsets.size(); ++i)
{
indices.clear();
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j)
{
res_values[j] = ++indices[hash128(j, count, columns)];
}
prev_off = off;
}
}
else
{
// Dense
for (size_t i = 0; i < offsets.size(); ++i)
{
indices.clear();
size_t off = offsets[i];
size_t rank = 0;
for (size_t j = prev_off; j < off; ++j)
{
auto & idx = indices[hash128(j, count, columns)];
if (!idx)
idx = ++rank;
res_values[j] = idx;
}
prev_off = off;
}
}
return true;
} }
} }

View File

@ -10,6 +10,7 @@
#include <Common/HashTable/ClearableHashSet.h> #include <Common/HashTable/ClearableHashSet.h>
#include <Interpreters/AggregationCommon.h> #include <Interpreters/AggregationCommon.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Common/ColumnsHashing.h>
namespace DB namespace DB
@ -62,11 +63,56 @@ private:
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
static constexpr size_t INITIAL_SIZE_DEGREE = 9; static constexpr size_t INITIAL_SIZE_DEGREE = 9;
template <typename T>
struct MethodOneNumber
{
using Set = ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
using Method = ColumnsHashing::HashMethodOneNumber<typename Set::value_type, void, T, false>;
};
struct MethodString
{
using Set = ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
using Method = ColumnsHashing::HashMethodString<typename Set::value_type, void, false, false>;
};
struct MethodFixedString
{
using Set = ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
using Method = ColumnsHashing::HashMethodFixedString<typename Set::value_type, void, false, false>;
};
struct MethodFixed
{
using Set = ClearableHashSet<UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
using Method = ColumnsHashing::HashMethodKeysFixed<typename Set::value_type, UInt128, void, false, false, false>;
};
struct MethodHashed
{
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
using Method = ColumnsHashing::HashMethodHashed<typename Set::value_type, void, false>;
};
template <typename Method>
void executeMethod(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, const Sizes & key_sizes,
const NullMap * null_map, ColumnUInt32::Container & res_values);
template <typename Method, bool has_null_map>
void executeMethodImpl(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, const Sizes & key_sizes,
const NullMap * null_map, ColumnUInt32::Container & res_values);
template <typename T> template <typename T>
bool executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values); bool executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values);
bool executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values); bool executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values);
bool executeFixedString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values);
bool execute128bit(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values); bool execute128bit(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values);
bool executeHashed(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values); void executeHashed(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values);
}; };
@ -126,7 +172,7 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen
if (num_arguments == 1) if (num_arguments == 1)
{ {
executeNumber<UInt8>(*offsets, *data_columns[0], null_map, res_values) if (!(executeNumber<UInt8>(*offsets, *data_columns[0], null_map, res_values)
|| executeNumber<UInt16>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<UInt16>(*offsets, *data_columns[0], null_map, res_values)
|| executeNumber<UInt32>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<UInt32>(*offsets, *data_columns[0], null_map, res_values)
|| executeNumber<UInt64>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<UInt64>(*offsets, *data_columns[0], null_map, res_values)
@ -136,30 +182,31 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen
|| executeNumber<Int64>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<Int64>(*offsets, *data_columns[0], null_map, res_values)
|| executeNumber<Float32>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<Float32>(*offsets, *data_columns[0], null_map, res_values)
|| executeNumber<Float64>(*offsets, *data_columns[0], null_map, res_values) || executeNumber<Float64>(*offsets, *data_columns[0], null_map, res_values)
|| executeString(*offsets, *data_columns[0], null_map, res_values) || executeFixedString(*offsets, *data_columns[0], null_map, res_values)
|| executeHashed(*offsets, data_columns, res_values); || executeString(*offsets, *data_columns[0], null_map, res_values)))
executeHashed(*offsets, data_columns, res_values);
} }
else else
{ {
execute128bit(*offsets, data_columns, res_values) if (!execute128bit(*offsets, data_columns, res_values))
|| executeHashed(*offsets, data_columns, res_values); executeHashed(*offsets, data_columns, res_values);
} }
block.getByPosition(result).column = std::move(res); block.getByPosition(result).column = std::move(res);
} }
template <typename T> template <typename Method, bool has_null_map>
bool FunctionArrayUniq::executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) void FunctionArrayUniq::executeMethodImpl(
const ColumnArray::Offsets & offsets,
const ColumnRawPtrs & columns,
const Sizes & key_sizes,
[[maybe_unused]] const NullMap * null_map,
ColumnUInt32::Container & res_values)
{ {
const ColumnVector<T> * nested = checkAndGetColumn<ColumnVector<T>>(&data); typename Method::Set set;
if (!nested) typename Method::Method method(columns, key_sizes, nullptr);
return false; Arena pool; /// Won't use it;
const auto & values = nested->getData();
using Set = ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
Set set;
ColumnArray::Offset prev_off = 0; ColumnArray::Offset prev_off = 0;
for (size_t i = 0; i < offsets.size(); ++i) for (size_t i = 0; i < offsets.size(); ++i)
{ {
@ -168,53 +215,71 @@ bool FunctionArrayUniq::executeNumber(const ColumnArray::Offsets & offsets, cons
ColumnArray::Offset off = offsets[i]; ColumnArray::Offset off = offsets[i];
for (ColumnArray::Offset j = prev_off; j < off; ++j) for (ColumnArray::Offset j = prev_off; j < off; ++j)
{ {
if (null_map && (*null_map)[j]) if constexpr (has_null_map)
found_null = true; {
else if ((*null_map)[j])
set.insert(values[j]); {
found_null = true;
continue;
}
}
method.emplaceKey(set, j, pool);
} }
res_values[i] = set.size() + found_null; res_values[i] = set.size() + found_null;
prev_off = off; prev_off = off;
} }
}
template <typename Method>
void FunctionArrayUniq::executeMethod(
const ColumnArray::Offsets & offsets,
const ColumnRawPtrs & columns,
const Sizes & key_sizes,
const NullMap * null_map,
ColumnUInt32::Container & res_values)
{
if (null_map)
executeMethodImpl<Method, true>(offsets, columns, key_sizes, null_map, res_values);
else
executeMethodImpl<Method, false>(offsets, columns, key_sizes, null_map, res_values);
}
template <typename T>
bool FunctionArrayUniq::executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values)
{
const auto * nested = checkAndGetColumn<ColumnVector<T>>(&data);
if (!nested)
return false;
executeMethod<MethodOneNumber<T>>(offsets, {nested}, {}, null_map, res_values);
return true; return true;
} }
bool FunctionArrayUniq::executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) bool FunctionArrayUniq::executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values)
{ {
const ColumnString * nested = checkAndGetColumn<ColumnString>(&data); const auto * nested = checkAndGetColumn<ColumnString>(&data);
if (!nested) if (nested)
return false; executeMethod<MethodString>(offsets, {nested}, {}, null_map, res_values);
using Set = ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>, return nested;
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
Set set;
ColumnArray::Offset prev_off = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
set.clear();
bool found_null = false;
ColumnArray::Offset off = offsets[i];
for (ColumnArray::Offset j = prev_off; j < off; ++j)
{
if (null_map && (*null_map)[j])
found_null = true;
else
set.insert(nested->getDataAt(j));
}
res_values[i] = set.size() + found_null;
prev_off = off;
}
return true;
} }
bool FunctionArrayUniq::executeFixedString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values)
{
const auto * nested = checkAndGetColumn<ColumnFixedString>(&data);
if (nested)
executeMethod<MethodFixedString>(offsets, {nested}, {}, null_map, res_values);
return nested;
}
bool FunctionArrayUniq::execute128bit( bool FunctionArrayUniq::execute128bit(
const ColumnArray::Offsets & offsets, const ColumnArray::Offsets & offsets,
const ColumnRawPtrs & columns, const ColumnRawPtrs & columns,
ColumnUInt32::Container & res_values) ColumnUInt32::Container & res_values)
{ {
size_t count = columns.size(); size_t count = columns.size();
size_t keys_bytes = 0; size_t keys_bytes = 0;
@ -231,49 +296,16 @@ bool FunctionArrayUniq::execute128bit(
if (keys_bytes > 16) if (keys_bytes > 16)
return false; return false;
using Set = ClearableHashSet<UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>, executeMethod<MethodFixed>(offsets, columns, key_sizes, nullptr, res_values);
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
Set set;
ColumnArray::Offset prev_off = 0;
for (ColumnArray::Offset i = 0; i < offsets.size(); ++i)
{
set.clear();
ColumnArray::Offset off = offsets[i];
for (ColumnArray::Offset j = prev_off; j < off; ++j)
set.insert(packFixed<UInt128>(j, count, columns, key_sizes));
res_values[i] = set.size();
prev_off = off;
}
return true; return true;
} }
bool FunctionArrayUniq::executeHashed( void FunctionArrayUniq::executeHashed(
const ColumnArray::Offsets & offsets, const ColumnArray::Offsets & offsets,
const ColumnRawPtrs & columns, const ColumnRawPtrs & columns,
ColumnUInt32::Container & res_values) ColumnUInt32::Container & res_values)
{ {
size_t count = columns.size(); executeMethod<MethodHashed>(offsets, columns, {}, nullptr, res_values);
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
Set set;
ColumnArray::Offset prev_off = 0;
for (ColumnArray::Offset i = 0; i < offsets.size(); ++i)
{
set.clear();
ColumnArray::Offset off = offsets[i];
for (ColumnArray::Offset j = prev_off; j < off; ++j)
set.insert(hash128(j, count, columns));
res_values[i] = set.size();
prev_off = off;
}
return true;
} }

View File

@ -60,7 +60,7 @@ public:
{ {
{ {
WriteBufferFromString out(tmp); WriteBufferFromString out(tmp);
src.type->serializeText(*src.column, i, out, format_settings); src.type->serializeAsText(*src.column, i, out, format_settings);
} }
res_data[i] = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(tmp.data()), tmp.size()); res_data[i] = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(tmp.data()), tmp.size());

View File

@ -6,9 +6,11 @@
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <DataTypes/DataTypeAggregateFunction.h> #include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeNullable.h> #include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <Columns/ColumnArray.h> #include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h> #include <Columns/ColumnTuple.h>
#include <Columns/ColumnLowCardinality.h>
#include <AggregateFunctions/AggregateFunctionCount.h> #include <AggregateFunctions/AggregateFunctionCount.h>
#include <DataStreams/IBlockInputStream.h> #include <DataStreams/IBlockInputStream.h>
#include <DataStreams/NativeBlockOutputStream.h> #include <DataStreams/NativeBlockOutputStream.h>
@ -22,11 +24,9 @@
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <common/demangle.h> #include <common/demangle.h>
#if __has_include(<Interpreters/config_compile.h>) #if __has_include(<Interpreters/config_compile.h>)
#include <Interpreters/config_compile.h> #include <Interpreters/config_compile.h>
#include <Columns/ColumnLowCardinality.h>
#include <DataTypes/DataTypeLowCardinality.h>
#endif #endif
@ -188,7 +188,7 @@ Aggregator::Aggregator(const Params & params_)
} }
method_chosen = chooseAggregationMethod(); method_chosen = chooseAggregationMethod();
AggregationStateCache::Settings cache_settings; HashMethodContext::Settings cache_settings;
cache_settings.max_threads = params.max_threads; cache_settings.max_threads = params.max_threads;
aggregation_state_cache = AggregatedDataVariants::createCache(method_chosen, cache_settings); aggregation_state_cache = AggregatedDataVariants::createCache(method_chosen, cache_settings);
} }
@ -586,11 +586,7 @@ void NO_INLINE Aggregator::executeImpl(
bool no_more_keys, bool no_more_keys,
AggregateDataPtr overflow_row) const AggregateDataPtr overflow_row) const
{ {
typename Method::State state; typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
if constexpr (Method::low_cardinality_optimization)
state.init(key_columns, aggregation_state_cache);
else
state.init(key_columns);
if (!no_more_keys) if (!no_more_keys)
executeImplCase<false>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, keys, overflow_row); executeImplCase<false>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, keys, overflow_row);
@ -605,97 +601,51 @@ void NO_INLINE Aggregator::executeImplCase(
typename Method::State & state, typename Method::State & state,
Arena * aggregates_pool, Arena * aggregates_pool,
size_t rows, size_t rows,
ColumnRawPtrs & key_columns, ColumnRawPtrs & /*key_columns*/,
AggregateFunctionInstruction * aggregate_instructions, AggregateFunctionInstruction * aggregate_instructions,
StringRefs & keys, StringRefs & /*keys*/,
AggregateDataPtr overflow_row) const AggregateDataPtr overflow_row) const
{ {
/// NOTE When editing this code, also pay attention to SpecializedAggregator.h. /// NOTE When editing this code, also pay attention to SpecializedAggregator.h.
/// For all rows. /// For all rows.
typename Method::Key prev_key{};
AggregateDataPtr value = nullptr;
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < rows; ++i)
{ {
bool inserted = false; /// Inserted a new key, or was this key already? AggregateDataPtr aggregate_data = nullptr;
/// Get the key to insert into the hash table. if constexpr (!no_more_keys) /// Insert.
typename Method::Key key;
if constexpr (!Method::low_cardinality_optimization)
key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
AggregateDataPtr * aggregate_data = nullptr;
typename Method::iterator it; /// Is not used if Method::low_cardinality_optimization
if (!no_more_keys) /// Insert.
{ {
/// Optimization for consecutive identical keys. auto emplace_result = state.emplaceKey(method.data, i, *aggregates_pool);
if (!Method::no_consecutive_keys_optimization)
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
if (emplace_result.isInserted())
{ {
if (i != 0 && key == prev_key) /// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
{ emplace_result.setMapped(nullptr);
/// Add values to the aggregate functions.
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
(*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i, aggregates_pool);
method.onExistingKey(key, keys, *aggregates_pool); aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
continue; createAggregateStates(aggregate_data);
}
else emplace_result.setMapped(aggregate_data);
prev_key = key;
} }
if constexpr (Method::low_cardinality_optimization)
aggregate_data = state.emplaceKeyFromRow(method.data, i, inserted, params.keys_size, keys, *aggregates_pool);
else else
{ aggregate_data = emplace_result.getMapped();
method.data.emplace(key, it, inserted);
aggregate_data = &Method::getAggregateData(it->second);
}
} }
else else
{ {
/// Add only if the key already exists. /// Add only if the key already exists.
auto find_result = state.findKey(method.data, i, *aggregates_pool);
if constexpr (Method::low_cardinality_optimization) if (find_result.isFound())
aggregate_data = state.findFromRow(method.data, i); aggregate_data = find_result.getMapped();
else
{
it = method.data.find(key);
if (method.data.end() != it)
aggregate_data = &Method::getAggregateData(it->second);
}
} }
/// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys. /// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys.
/// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do. /// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do.
if (!aggregate_data && !overflow_row) if (!aggregate_data && !overflow_row)
{
method.onExistingKey(key, keys, *aggregates_pool);
continue; continue;
}
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key. AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row;
if (inserted)
{
/// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
*aggregate_data = nullptr;
if constexpr (!Method::low_cardinality_optimization)
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
AggregateDataPtr place = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(place);
*aggregate_data = place;
if constexpr (Method::low_cardinality_optimization)
state.cacheAggregateData(i, place);
}
else
method.onExistingKey(key, keys, *aggregates_pool);
value = aggregate_data ? *aggregate_data : overflow_row;
/// Add values to the aggregate functions. /// Add values to the aggregate functions.
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
@ -1174,7 +1124,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->insertResultInto( aggregate_functions[i]->insertResultInto(
Method::getAggregateData(value.second) + offsets_of_aggregate_states[i], value.second + offsets_of_aggregate_states[i],
*final_aggregate_columns[i]); *final_aggregate_columns[i]);
} }
@ -1205,9 +1155,9 @@ void NO_INLINE Aggregator::convertToBlockImplNotFinal(
/// reserved, so push_back does not throw exceptions /// reserved, so push_back does not throw exceptions
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_columns[i]->push_back(Method::getAggregateData(value.second) + offsets_of_aggregate_states[i]); aggregate_columns[i]->push_back(value.second + offsets_of_aggregate_states[i]);
Method::getAggregateData(value.second) = nullptr; value.second = nullptr;
} }
} }
@ -1551,20 +1501,20 @@ void NO_INLINE Aggregator::mergeDataImpl(
{ {
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge( aggregate_functions[i]->merge(
Method::getAggregateData(res_it->second) + offsets_of_aggregate_states[i], res_it->second + offsets_of_aggregate_states[i],
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i], it->second + offsets_of_aggregate_states[i],
arena); arena);
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy( aggregate_functions[i]->destroy(
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); it->second + offsets_of_aggregate_states[i]);
} }
else else
{ {
res_it->second = it->second; res_it->second = it->second;
} }
Method::getAggregateData(it->second) = nullptr; it->second = nullptr;
} }
table_src.clearAndShrink(); table_src.clearAndShrink();
@ -1588,19 +1538,18 @@ void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl(
AggregateDataPtr res_data = table_dst.end() == res_it AggregateDataPtr res_data = table_dst.end() == res_it
? overflows ? overflows
: Method::getAggregateData(res_it->second); : res_it->second;
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge( aggregate_functions[i]->merge(
res_data + offsets_of_aggregate_states[i], res_data + offsets_of_aggregate_states[i],
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i], it->second + offsets_of_aggregate_states[i],
arena); arena);
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy( aggregate_functions[i]->destroy(it->second + offsets_of_aggregate_states[i]);
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
Method::getAggregateData(it->second) = nullptr; it->second = nullptr;
} }
table_src.clearAndShrink(); table_src.clearAndShrink();
@ -1623,19 +1572,18 @@ void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl(
if (table_dst.end() == res_it) if (table_dst.end() == res_it)
continue; continue;
AggregateDataPtr res_data = Method::getAggregateData(res_it->second); AggregateDataPtr res_data = res_it->second;
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge( aggregate_functions[i]->merge(
res_data + offsets_of_aggregate_states[i], res_data + offsets_of_aggregate_states[i],
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i], it->second + offsets_of_aggregate_states[i],
arena); arena);
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy( aggregate_functions[i]->destroy(it->second + offsets_of_aggregate_states[i]);
Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]);
Method::getAggregateData(it->second) = nullptr; it->second = nullptr;
} }
table_src.clearAndShrink(); table_src.clearAndShrink();
@ -1986,7 +1934,7 @@ template <bool no_more_keys, typename Method, typename Table>
void NO_INLINE Aggregator::mergeStreamsImplCase( void NO_INLINE Aggregator::mergeStreamsImplCase(
Block & block, Block & block,
Arena * aggregates_pool, Arena * aggregates_pool,
Method & method, Method & method [[maybe_unused]],
Table & data, Table & data,
AggregateDataPtr overflow_row) const AggregateDataPtr overflow_row) const
{ {
@ -2000,77 +1948,43 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
for (size_t i = 0; i < params.aggregates_size; ++i) for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_columns[i] = &typeid_cast<const ColumnAggregateFunction &>(*block.safeGetByPosition(params.keys_size + i).column).getData(); aggregate_columns[i] = &typeid_cast<const ColumnAggregateFunction &>(*block.safeGetByPosition(params.keys_size + i).column).getData();
typename Method::State state; typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
if constexpr (Method::low_cardinality_optimization)
state.init(key_columns, aggregation_state_cache);
else
state.init(key_columns);
/// For all rows. /// For all rows.
StringRefs keys(params.keys_size);
size_t rows = block.rows(); size_t rows = block.rows();
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < rows; ++i)
{ {
typename Table::iterator it; AggregateDataPtr aggregate_data = nullptr;
AggregateDataPtr * aggregate_data = nullptr;
bool inserted = false; /// Inserted a new key, or was this key already?
/// Get the key to insert into the hash table.
typename Method::Key key;
if constexpr (!Method::low_cardinality_optimization)
key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
if (!no_more_keys) if (!no_more_keys)
{ {
if constexpr (Method::low_cardinality_optimization) auto emplace_result = state.emplaceKey(data, i, *aggregates_pool);
aggregate_data = state.emplaceKeyFromRow(data, i, inserted, params.keys_size, keys, *aggregates_pool); if (emplace_result.isInserted())
else
{ {
data.emplace(key, it, inserted); emplace_result.setMapped(nullptr);
aggregate_data = &Method::getAggregateData(it->second);
aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(aggregate_data);
emplace_result.setMapped(aggregate_data);
} }
else
aggregate_data = emplace_result.getMapped();
} }
else else
{ {
if constexpr (Method::low_cardinality_optimization) auto find_result = state.findKey(data, i, *aggregates_pool);
aggregate_data = state.findFromRow(data, i); if (find_result.isFound())
else aggregate_data = find_result.getMapped();
{
it = data.find(key);
if (data.end() != it)
aggregate_data = &Method::getAggregateData(it->second);
}
} }
/// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys. /// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys.
/// If the key does not fit, and the data does not need to be aggregated into a separate row, then there's nothing to do. /// If the key does not fit, and the data does not need to be aggregated into a separate row, then there's nothing to do.
if (!aggregate_data && !overflow_row) if (!aggregate_data && !overflow_row)
{
method.onExistingKey(key, keys, *aggregates_pool);
continue; continue;
}
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key. AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row;
if (inserted)
{
*aggregate_data = nullptr;
if constexpr (!Method::low_cardinality_optimization)
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
AggregateDataPtr place = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(place);
*aggregate_data = place;
if constexpr (Method::low_cardinality_optimization)
state.cacheAggregateData(i, place);
}
else
method.onExistingKey(key, keys, *aggregates_pool);
AggregateDataPtr value = aggregate_data ? *aggregate_data : overflow_row;
/// Merge state of aggregate functions. /// Merge state of aggregate functions.
for (size_t j = 0; j < params.aggregates_size; ++j) for (size_t j = 0; j < params.aggregates_size; ++j)
@ -2165,7 +2079,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
* If there is at least one block with a bucket number greater or equal than zero, then there was a two-level aggregation. * If there is at least one block with a bucket number greater or equal than zero, then there was a two-level aggregation.
*/ */
auto max_bucket = bucket_to_blocks.rbegin()->first; auto max_bucket = bucket_to_blocks.rbegin()->first;
size_t has_two_level = max_bucket >= 0; bool has_two_level = max_bucket >= 0;
if (has_two_level) if (has_two_level)
{ {
@ -2395,15 +2309,11 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl(
Method & method, Method & method,
Arena * pool, Arena * pool,
ColumnRawPtrs & key_columns, ColumnRawPtrs & key_columns,
StringRefs & keys, StringRefs & keys [[maybe_unused]],
const Block & source, const Block & source,
std::vector<Block> & destinations) const std::vector<Block> & destinations) const
{ {
typename Method::State state; typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
if constexpr (Method::low_cardinality_optimization)
state.init(key_columns, aggregation_state_cache);
else
state.init(key_columns);
size_t rows = source.rows(); size_t rows = source.rows();
size_t columns = source.columns(); size_t columns = source.columns();
@ -2423,16 +2333,11 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl(
} }
} }
/// Obtain a key. Calculate bucket number from it. /// Calculate bucket number from row hash.
typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *pool); auto hash = state.getHash(method.data, i, *pool);
auto hash = method.data.hash(key);
auto bucket = method.data.getBucketFromHash(hash); auto bucket = method.data.getBucketFromHash(hash);
selector[i] = bucket; selector[i] = bucket;
/// We don't need to store this key in pool.
method.onExistingKey(key, keys, *pool);
} }
size_t num_buckets = destinations.size(); size_t num_buckets = destinations.size();
@ -2523,7 +2428,7 @@ void NO_INLINE Aggregator::destroyImpl(Table & table) const
{ {
for (auto elem : table) for (auto elem : table)
{ {
AggregateDataPtr & data = Method::getAggregateData(elem.second); AggregateDataPtr & data = elem.second;
/** If an exception (usually a lack of memory, the MemoryTracker throws) arose /** If an exception (usually a lack of memory, the MemoryTracker throws) arose
* after inserting the key into a hash table, but before creating all states of aggregate functions, * after inserting the key into a hash table, but before creating all states of aggregate functions,

View File

@ -15,6 +15,7 @@
#include <Common/ThreadPool.h> #include <Common/ThreadPool.h>
#include <Common/UInt128.h> #include <Common/UInt128.h>
#include <Common/LRUCache.h> #include <Common/LRUCache.h>
#include <Common/ColumnsHashing.h>
#include <DataStreams/IBlockInputStream.h> #include <DataStreams/IBlockInputStream.h>
#include <DataStreams/SizeLimits.h> #include <DataStreams/SizeLimits.h>
@ -138,18 +139,6 @@ using AggregatedDataWithNullableStringKeyTwoLevel = AggregationDataWithNullKeyTw
TwoLevelHashMapWithSavedHash<StringRef, AggregateDataPtr, DefaultHash<StringRef>, TwoLevelHashMapWithSavedHash<StringRef, AggregateDataPtr, DefaultHash<StringRef>,
TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>; TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>;
/// Cache which can be used by aggregations method's states. Object is shared in all threads.
struct AggregationStateCache
{
virtual ~AggregationStateCache() = default;
struct Settings
{
size_t max_threads;
};
};
using AggregationStateCachePtr = std::shared_ptr<AggregationStateCache>;
/// For the case where there is one numeric key. /// For the case where there is one numeric key.
template <typename FieldType, typename TData> /// UInt8/16/32/64 for any type with corresponding bit width. template <typename FieldType, typename TData> /// UInt8/16/32/64 for any type with corresponding bit width.
@ -169,65 +158,16 @@ struct AggregationMethodOneNumber
AggregationMethodOneNumber(const Other & other) : data(other.data) {} AggregationMethodOneNumber(const Other & other) : data(other.data) {}
/// To use one `Method` in different threads, use different `State`. /// To use one `Method` in different threads, use different `State`.
struct State using State = ColumnsHashing::HashMethodOneNumber<typename Data::value_type, Mapped, FieldType>;
{
const char * vec;
/** Called at the start of each block processing.
* Sets the variables needed for the other methods called in inner loops.
*/
void init(ColumnRawPtrs & key_columns)
{
vec = key_columns[0]->getRawData().data;
}
/// Get the key from the key columns for insertion into the hash table.
ALWAYS_INLINE Key getKey(
const ColumnRawPtrs & /*key_columns*/,
size_t /*keys_size*/, /// Number of key columns.
size_t i, /// From which row of the block, get the key.
const Sizes & /*key_sizes*/, /// If the keys of a fixed length - their lengths. It is not used in aggregation methods for variable length keys.
StringRefs & /*keys*/, /// Here references to key data in columns can be written. They can be used in the future.
Arena & /*pool*/) const
{
return unalignedLoad<FieldType>(vec + i * sizeof(FieldType));
}
};
/// From the value in the hash table, get AggregateDataPtr.
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
/** Place additional data, if necessary, in case a new key was inserted into the hash table.
*/
static ALWAYS_INLINE void onNewKey(typename Data::value_type & /*value*/, size_t /*keys_size*/, StringRefs & /*keys*/, Arena & /*pool*/)
{
}
/** The action to be taken if the key is not new. For example, roll back the memory allocation in the pool.
*/
static ALWAYS_INLINE void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {}
/** Do not use optimization for consecutive keys.
*/
static const bool no_consecutive_keys_optimization = false;
/// Use optimization for low cardinality. /// Use optimization for low cardinality.
static const bool low_cardinality_optimization = false; static const bool low_cardinality_optimization = false;
/** Insert the key from the hash table into columns. // Insert the key from the hash table into columns.
*/
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/)
{ {
static_cast<ColumnVectorHelper *>(key_columns[0].get())->insertRawData<sizeof(FieldType)>(reinterpret_cast<const char *>(&value.first)); static_cast<ColumnVectorHelper *>(key_columns[0].get())->insertRawData<sizeof(FieldType)>(reinterpret_cast<const char *>(&value.first));
} }
/// Get StringRef from value which can be inserted into column.
static StringRef getValueRef(const typename Data::value_type & value)
{
return StringRef(reinterpret_cast<const char *>(&value.first), sizeof(value.first));
}
static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; }
}; };
@ -248,58 +188,14 @@ struct AggregationMethodString
template <typename Other> template <typename Other>
AggregationMethodString(const Other & other) : data(other.data) {} AggregationMethodString(const Other & other) : data(other.data) {}
struct State using State = ColumnsHashing::HashMethodString<typename Data::value_type, Mapped>;
{
const IColumn::Offset * offsets;
const UInt8 * chars;
void init(ColumnRawPtrs & key_columns)
{
const IColumn & column = *key_columns[0];
const ColumnString & column_string = static_cast<const ColumnString &>(column);
offsets = column_string.getOffsets().data();
chars = column_string.getChars().data();
}
ALWAYS_INLINE Key getKey(
const ColumnRawPtrs & /*key_columns*/,
size_t /*keys_size*/,
ssize_t i,
const Sizes & /*key_sizes*/,
StringRefs & /*keys*/,
Arena & /*pool*/) const
{
return StringRef(
chars + offsets[i - 1],
offsets[i] - offsets[i - 1] - 1);
}
};
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
static ALWAYS_INLINE void onNewKey(typename Data::value_type & value, size_t /*keys_size*/, StringRefs & /*keys*/, Arena & pool)
{
if (value.first.size)
value.first.data = pool.insert(value.first.data, value.first.size);
}
static ALWAYS_INLINE void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {}
static const bool no_consecutive_keys_optimization = false;
static const bool low_cardinality_optimization = false; static const bool low_cardinality_optimization = false;
static StringRef getValueRef(const typename Data::value_type & value)
{
return StringRef(value.first.data, value.first.size);
}
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &) static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &)
{ {
key_columns[0]->insertData(value.first.data, value.first.size); key_columns[0]->insertData(value.first.data, value.first.size);
} }
static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; }
}; };
@ -320,101 +216,14 @@ struct AggregationMethodFixedString
template <typename Other> template <typename Other>
AggregationMethodFixedString(const Other & other) : data(other.data) {} AggregationMethodFixedString(const Other & other) : data(other.data) {}
struct State using State = ColumnsHashing::HashMethodFixedString<typename Data::value_type, Mapped>;
{
size_t n;
const ColumnFixedString::Chars * chars;
void init(ColumnRawPtrs & key_columns)
{
const IColumn & column = *key_columns[0];
const ColumnFixedString & column_string = static_cast<const ColumnFixedString &>(column);
n = column_string.getN();
chars = &column_string.getChars();
}
ALWAYS_INLINE Key getKey(
const ColumnRawPtrs &,
size_t,
size_t i,
const Sizes &,
StringRefs &,
Arena &) const
{
return StringRef(&(*chars)[i * n], n);
}
};
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
static ALWAYS_INLINE void onNewKey(typename Data::value_type & value, size_t, StringRefs &, Arena & pool)
{
value.first.data = pool.insert(value.first.data, value.first.size);
}
static ALWAYS_INLINE void onExistingKey(const Key &, StringRefs &, Arena &) {}
static const bool no_consecutive_keys_optimization = false;
static const bool low_cardinality_optimization = false; static const bool low_cardinality_optimization = false;
static StringRef getValueRef(const typename Data::value_type & value)
{
return StringRef(value.first.data, value.first.size);
}
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &) static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &)
{ {
key_columns[0]->insertData(value.first.data, value.first.size); key_columns[0]->insertData(value.first.data, value.first.size);
} }
static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; }
};
/// Cache stores dictionaries and saved_hash per dictionary key.
class LowCardinalityDictionaryCache : public AggregationStateCache
{
public:
/// Will assume that dictionaries with same hash has the same keys.
/// Just in case, check that they have also the same size.
struct DictionaryKey
{
UInt128 hash;
UInt64 size;
bool operator== (const DictionaryKey & other) const { return hash == other.hash && size == other.size; }
};
struct DictionaryKeyHash
{
size_t operator()(const DictionaryKey & key) const
{
SipHash hash;
hash.update(key.hash.low);
hash.update(key.hash.high);
hash.update(key.size);
return hash.get64();
}
};
struct CachedValues
{
/// Store ptr to dictionary to be sure it won't be deleted.
ColumnPtr dictionary_holder;
/// Hashes for dictionary keys.
const UInt64 * saved_hash = nullptr;
};
using CachedValuesPtr = std::shared_ptr<CachedValues>;
explicit LowCardinalityDictionaryCache(const AggregationStateCache::Settings & settings) : cache(settings.max_threads) {}
CachedValuesPtr get(const DictionaryKey & key) { return cache.get(key); }
void set(const DictionaryKey & key, const CachedValuesPtr & mapped) { cache.set(key, mapped); }
private:
using Cache = LRUCache<DictionaryKey, CachedValues, DictionaryKeyHash>;
Cache cache;
}; };
/// Single low cardinality column. /// Single low cardinality column.
@ -432,342 +241,23 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod
using Base::data; using Base::data;
static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & settings)
{
return std::make_shared<LowCardinalityDictionaryCache>(settings);
}
AggregationMethodSingleLowCardinalityColumn() = default; AggregationMethodSingleLowCardinalityColumn() = default;
template <typename Other> template <typename Other>
explicit AggregationMethodSingleLowCardinalityColumn(const Other & other) : Base(other) {} explicit AggregationMethodSingleLowCardinalityColumn(const Other & other) : Base(other) {}
struct State : public BaseState using State = ColumnsHashing::HashMethodSingleLowCardinalityColumn<BaseState, Mapped, true>;
{
ColumnRawPtrs key_columns;
const IColumn * positions = nullptr;
size_t size_of_index_type = 0;
/// saved hash is from current column or from cache.
const UInt64 * saved_hash = nullptr;
/// Hold dictionary in case saved_hash is from cache to be sure it won't be deleted.
ColumnPtr dictionary_holder;
/// Cache AggregateDataPtr for current column in order to decrease the number of hash table usages.
PaddedPODArray<AggregateDataPtr> aggregate_data_cache;
/// If initialized column is nullable.
bool is_nullable = false;
void init(ColumnRawPtrs &)
{
throw Exception("Expected cache for AggregationMethodSingleLowCardinalityColumn::init", ErrorCodes::LOGICAL_ERROR);
}
void init(ColumnRawPtrs & key_columns_low_cardinality, const AggregationStateCachePtr & cache_ptr)
{
auto column = typeid_cast<const ColumnLowCardinality *>(key_columns_low_cardinality[0]);
if (!column)
throw Exception("Invalid aggregation key type for AggregationMethodSingleLowCardinalityColumn method. "
"Excepted LowCardinality, got " + key_columns_low_cardinality[0]->getName(), ErrorCodes::LOGICAL_ERROR);
if (!cache_ptr)
throw Exception("Cache wasn't created for AggregationMethodSingleLowCardinalityColumn", ErrorCodes::LOGICAL_ERROR);
auto cache = typeid_cast<LowCardinalityDictionaryCache *>(cache_ptr.get());
if (!cache)
{
const auto & cached_val = *cache_ptr;
throw Exception("Invalid type for AggregationMethodSingleLowCardinalityColumn cache: "
+ demangle(typeid(cached_val).name()), ErrorCodes::LOGICAL_ERROR);
}
auto * dict = column->getDictionary().getNestedNotNullableColumn().get();
is_nullable = column->getDictionary().nestedColumnIsNullable();
key_columns = {dict};
bool is_shared_dict = column->isSharedDictionary();
typename LowCardinalityDictionaryCache::DictionaryKey dictionary_key;
typename LowCardinalityDictionaryCache::CachedValuesPtr cached_values;
if (is_shared_dict)
{
dictionary_key = {column->getDictionary().getHash(), dict->size()};
cached_values = cache->get(dictionary_key);
}
if (cached_values)
{
saved_hash = cached_values->saved_hash;
dictionary_holder = cached_values->dictionary_holder;
}
else
{
saved_hash = column->getDictionary().tryGetSavedHash();
dictionary_holder = column->getDictionaryPtr();
if (is_shared_dict)
{
cached_values = std::make_shared<typename LowCardinalityDictionaryCache::CachedValues>();
cached_values->saved_hash = saved_hash;
cached_values->dictionary_holder = dictionary_holder;
cache->set(dictionary_key, cached_values);
}
}
AggregateDataPtr default_data = nullptr;
aggregate_data_cache.assign(key_columns[0]->size(), default_data);
size_of_index_type = column->getSizeOfIndexType();
positions = column->getIndexesPtr().get();
BaseState::init(key_columns);
}
ALWAYS_INLINE size_t getIndexAt(size_t row) const
{
switch (size_of_index_type)
{
case sizeof(UInt8): return static_cast<const ColumnUInt8 *>(positions)->getElement(row);
case sizeof(UInt16): return static_cast<const ColumnUInt16 *>(positions)->getElement(row);
case sizeof(UInt32): return static_cast<const ColumnUInt32 *>(positions)->getElement(row);
case sizeof(UInt64): return static_cast<const ColumnUInt64 *>(positions)->getElement(row);
default: throw Exception("Unexpected size of index type for low cardinality column.", ErrorCodes::LOGICAL_ERROR);
}
}
/// Get the key from the key columns for insertion into the hash table.
ALWAYS_INLINE Key getKey(
const ColumnRawPtrs & /*key_columns*/,
size_t /*keys_size*/,
size_t i,
const Sizes & key_sizes,
StringRefs & keys,
Arena & pool) const
{
size_t row = getIndexAt(i);
return BaseState::getKey(key_columns, 1, row, key_sizes, keys, pool);
}
template <typename D>
ALWAYS_INLINE AggregateDataPtr * emplaceKeyFromRow(
D & data,
size_t i,
bool & inserted,
size_t keys_size,
StringRefs & keys,
Arena & pool)
{
size_t row = getIndexAt(i);
if (is_nullable && row == 0)
{
inserted = !data.hasNullKeyData();
data.hasNullKeyData() = true;
return &data.getNullKeyData();
}
if (aggregate_data_cache[row])
{
inserted = false;
return &aggregate_data_cache[row];
}
else
{
Sizes key_sizes;
auto key = getKey({}, 0, i, key_sizes, keys, pool);
typename D::iterator it;
if (saved_hash)
data.emplace(key, it, inserted, saved_hash[row]);
else
data.emplace(key, it, inserted);
if (inserted)
Base::onNewKey(*it, keys_size, keys, pool);
else
aggregate_data_cache[row] = Base::getAggregateData(it->second);
return &Base::getAggregateData(it->second);
}
}
ALWAYS_INLINE bool isNullAt(size_t i)
{
if (!is_nullable)
return false;
return getIndexAt(i) == 0;
}
ALWAYS_INLINE void cacheAggregateData(size_t i, AggregateDataPtr data)
{
size_t row = getIndexAt(i);
aggregate_data_cache[row] = data;
}
template <typename D>
ALWAYS_INLINE AggregateDataPtr * findFromRow(D & data, size_t i)
{
size_t row = getIndexAt(i);
if (is_nullable && row == 0)
return data.hasNullKeyData() ? &data.getNullKeyData() : nullptr;
if (!aggregate_data_cache[row])
{
Sizes key_sizes;
StringRefs keys;
Arena pool;
auto key = getKey({}, 0, i, key_sizes, keys, pool);
typename D::iterator it;
if (saved_hash)
it = data.find(key, saved_hash[row]);
else
it = data.find(key);
if (it != data.end())
aggregate_data_cache[row] = Base::getAggregateData(it->second);
}
return &aggregate_data_cache[row];
}
};
static AggregateDataPtr & getAggregateData(Mapped & value) { return Base::getAggregateData(value); }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return Base::getAggregateData(value); }
static void onNewKey(typename Data::value_type & value, size_t keys_size, StringRefs & keys, Arena & pool)
{
return Base::onNewKey(value, keys_size, keys, pool);
}
static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool)
{
return Base::onExistingKey(key, keys, pool);
}
static const bool no_consecutive_keys_optimization = true;
static const bool low_cardinality_optimization = true; static const bool low_cardinality_optimization = true;
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns_low_cardinality, const Sizes & /*key_sizes*/) static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns_low_cardinality, const Sizes & /*key_sizes*/)
{ {
auto ref = Base::getValueRef(value); auto ref = BaseState::getValueRef(value);
static_cast<ColumnLowCardinality *>(key_columns_low_cardinality[0].get())->insertData(ref.data, ref.size); static_cast<ColumnLowCardinality *>(key_columns_low_cardinality[0].get())->insertData(ref.data, ref.size);
} }
}; };
namespace aggregator_impl
{
/// This class is designed to provide the functionality that is required for
/// supporting nullable keys in AggregationMethodKeysFixed. If there are
/// no nullable keys, this class is merely implemented as an empty shell.
template <typename Key, bool has_nullable_keys>
class BaseStateKeysFixed;
/// Case where nullable keys are supported.
template <typename Key>
class BaseStateKeysFixed<Key, true>
{
protected:
void init(const ColumnRawPtrs & key_columns)
{
null_maps.reserve(key_columns.size());
actual_columns.reserve(key_columns.size());
for (const auto & col : key_columns)
{
if (col->isColumnNullable())
{
const auto & nullable_col = static_cast<const ColumnNullable &>(*col);
actual_columns.push_back(&nullable_col.getNestedColumn());
null_maps.push_back(&nullable_col.getNullMapColumn());
}
else
{
actual_columns.push_back(col);
null_maps.push_back(nullptr);
}
}
}
/// Return the columns which actually contain the values of the keys.
/// For a given key column, if it is nullable, we return its nested
/// column. Otherwise we return the key column itself.
inline const ColumnRawPtrs & getActualColumns() const
{
return actual_columns;
}
/// Create a bitmap that indicates whether, for a particular row,
/// a key column bears a null value or not.
KeysNullMap<Key> createBitmap(size_t row) const
{
KeysNullMap<Key> bitmap{};
for (size_t k = 0; k < null_maps.size(); ++k)
{
if (null_maps[k] != nullptr)
{
const auto & null_map = static_cast<const ColumnUInt8 &>(*null_maps[k]).getData();
if (null_map[row] == 1)
{
size_t bucket = k / 8;
size_t offset = k % 8;
bitmap[bucket] |= UInt8(1) << offset;
}
}
}
return bitmap;
}
private:
ColumnRawPtrs actual_columns;
ColumnRawPtrs null_maps;
};
/// Case where nullable keys are not supported.
template <typename Key>
class BaseStateKeysFixed<Key, false>
{
protected:
void init(const ColumnRawPtrs &)
{
throw Exception{"Internal error: calling init() for non-nullable"
" keys is forbidden", ErrorCodes::LOGICAL_ERROR};
}
const ColumnRawPtrs & getActualColumns() const
{
throw Exception{"Internal error: calling getActualColumns() for non-nullable"
" keys is forbidden", ErrorCodes::LOGICAL_ERROR};
}
KeysNullMap<Key> createBitmap(size_t) const
{
throw Exception{"Internal error: calling createBitmap() for non-nullable keys"
" is forbidden", ErrorCodes::LOGICAL_ERROR};
}
};
}
// Oprional mask for low cardinality columns.
template <bool has_low_cardinality>
struct LowCardinalityKeys
{
ColumnRawPtrs nested_columns;
ColumnRawPtrs positions;
Sizes position_sizes;
};
template <>
struct LowCardinalityKeys<false> {};
/// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits. /// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits.
template <typename TData, bool has_nullable_keys_ = false, bool has_low_cardinality_ = false> template <typename TData, bool has_nullable_keys_ = false, bool has_low_cardinality_ = false>
struct AggregationMethodKeysFixed struct AggregationMethodKeysFixed
@ -787,71 +277,8 @@ struct AggregationMethodKeysFixed
template <typename Other> template <typename Other>
AggregationMethodKeysFixed(const Other & other) : data(other.data) {} AggregationMethodKeysFixed(const Other & other) : data(other.data) {}
class State final : private aggregator_impl::BaseStateKeysFixed<Key, has_nullable_keys> using State = ColumnsHashing::HashMethodKeysFixed<typename Data::value_type, Key, Mapped, has_nullable_keys, has_low_cardinality>;
{
LowCardinalityKeys<has_low_cardinality> low_cardinality_keys;
public:
using Base = aggregator_impl::BaseStateKeysFixed<Key, has_nullable_keys>;
void init(ColumnRawPtrs & key_columns)
{
if constexpr (has_low_cardinality)
{
low_cardinality_keys.nested_columns.resize(key_columns.size());
low_cardinality_keys.positions.assign(key_columns.size(), nullptr);
low_cardinality_keys.position_sizes.resize(key_columns.size());
for (size_t i = 0; i < key_columns.size(); ++i)
{
if (auto * low_cardinality_col = typeid_cast<const ColumnLowCardinality *>(key_columns[i]))
{
low_cardinality_keys.nested_columns[i] = low_cardinality_col->getDictionary().getNestedColumn().get();
low_cardinality_keys.positions[i] = &low_cardinality_col->getIndexes();
low_cardinality_keys.position_sizes[i] = low_cardinality_col->getSizeOfIndexType();
}
else
low_cardinality_keys.nested_columns[i] = key_columns[i];
}
}
if (has_nullable_keys)
Base::init(key_columns);
}
ALWAYS_INLINE Key getKey(
const ColumnRawPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes,
StringRefs &,
Arena &) const
{
if (has_nullable_keys)
{
auto bitmap = Base::createBitmap(i);
return packFixed<Key>(i, keys_size, Base::getActualColumns(), key_sizes, bitmap);
}
else
{
if constexpr (has_low_cardinality)
return packFixed<Key, true>(i, keys_size, low_cardinality_keys.nested_columns, key_sizes,
&low_cardinality_keys.positions, &low_cardinality_keys.position_sizes);
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
}
}
};
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
static ALWAYS_INLINE void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &)
{
}
static ALWAYS_INLINE void onExistingKey(const Key &, StringRefs &, Arena &) {}
static const bool no_consecutive_keys_optimization = false;
static const bool low_cardinality_optimization = false; static const bool low_cardinality_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & key_sizes) static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & key_sizes)
@ -904,8 +331,6 @@ struct AggregationMethodKeysFixed
} }
} }
} }
static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; }
}; };
@ -930,53 +355,24 @@ struct AggregationMethodSerialized
template <typename Other> template <typename Other>
AggregationMethodSerialized(const Other & other) : data(other.data) {} AggregationMethodSerialized(const Other & other) : data(other.data) {}
struct State using State = ColumnsHashing::HashMethodSerialized<typename Data::value_type, Mapped>;
{
void init(ColumnRawPtrs &)
{
}
ALWAYS_INLINE Key getKey(
const ColumnRawPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes &,
StringRefs &,
Arena & pool) const
{
return serializeKeysToPoolContiguous(i, keys_size, key_columns, pool);
}
};
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
static ALWAYS_INLINE void onNewKey(typename Data::value_type &, size_t, StringRefs &, Arena &)
{
}
static ALWAYS_INLINE void onExistingKey(const Key & key, StringRefs &, Arena & pool)
{
pool.rollback(key.size);
}
/// If the key already was, it is removed from the pool (overwritten), and the next key can not be compared with it.
static const bool no_consecutive_keys_optimization = true;
static const bool low_cardinality_optimization = false; static const bool low_cardinality_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &) static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &)
{ {
auto pos = value.first.data; auto pos = value.first.data;
for (size_t i = 0; i < key_columns.size(); ++i) for (auto & column : key_columns)
pos = key_columns[i]->deserializeAndInsertFromArena(pos); pos = column->deserializeAndInsertFromArena(pos);
} }
static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; }
}; };
class Aggregator; class Aggregator;
using ColumnsHashing::HashMethodContext;
using ColumnsHashing::HashMethodContextPtr;
struct AggregatedDataVariants : private boost::noncopyable struct AggregatedDataVariants : private boost::noncopyable
{ {
/** Working with states of aggregate functions in the pool is arranged in the following (inconvenient) way: /** Working with states of aggregate functions in the pool is arranged in the following (inconvenient) way:
@ -1298,7 +694,7 @@ struct AggregatedDataVariants : private boost::noncopyable
} }
} }
static AggregationStateCachePtr createCache(Type type, const AggregationStateCache::Settings & settings) static HashMethodContextPtr createCache(Type type, const HashMethodContext::Settings & settings)
{ {
switch (type) switch (type)
{ {
@ -1309,7 +705,7 @@ struct AggregatedDataVariants : private boost::noncopyable
{ \ { \
using TPtr ## NAME = decltype(AggregatedDataVariants::NAME); \ using TPtr ## NAME = decltype(AggregatedDataVariants::NAME); \
using T ## NAME = typename TPtr ## NAME ::element_type; \ using T ## NAME = typename TPtr ## NAME ::element_type; \
return T ## NAME ::createCache(settings); \ return T ## NAME ::State::createContext(settings); \
} }
APPLY_FOR_AGGREGATED_VARIANTS(M) APPLY_FOR_AGGREGATED_VARIANTS(M)
@ -1496,7 +892,7 @@ protected:
AggregatedDataVariants::Type method_chosen; AggregatedDataVariants::Type method_chosen;
Sizes key_sizes; Sizes key_sizes;
AggregationStateCachePtr aggregation_state_cache; HashMethodContextPtr aggregation_state_cache;
AggregateFunctionsPlainPtrs aggregate_functions; AggregateFunctionsPlainPtrs aggregate_functions;

View File

@ -170,18 +170,54 @@ static size_t getTotalByteCountImpl(const Maps & maps, Join::Type type)
} }
template <Join::Type type> template <Join::Type type, typename Value, typename Mapped>
struct KeyGetterForType; struct KeyGetterForTypeImpl;
template <> struct KeyGetterForType<Join::Type::key8> { using Type = JoinKeyGetterOneNumber<UInt8>; }; template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<Join::Type::key8, Value, Mapped>
template <> struct KeyGetterForType<Join::Type::key16> { using Type = JoinKeyGetterOneNumber<UInt16>; }; {
template <> struct KeyGetterForType<Join::Type::key32> { using Type = JoinKeyGetterOneNumber<UInt32>; }; using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt8, false>;
template <> struct KeyGetterForType<Join::Type::key64> { using Type = JoinKeyGetterOneNumber<UInt64>; }; };
template <> struct KeyGetterForType<Join::Type::key_string> { using Type = JoinKeyGetterString; }; template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<Join::Type::key16, Value, Mapped>
template <> struct KeyGetterForType<Join::Type::key_fixed_string> { using Type = JoinKeyGetterFixedString; }; {
template <> struct KeyGetterForType<Join::Type::keys128> { using Type = JoinKeyGetterFixed<UInt128>; }; using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt16, false>;
template <> struct KeyGetterForType<Join::Type::keys256> { using Type = JoinKeyGetterFixed<UInt256>; }; };
template <> struct KeyGetterForType<Join::Type::hashed> { using Type = JoinKeyGetterHashed; }; template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<Join::Type::key32, Value, Mapped>
{
using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt32, false>;
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<Join::Type::key64, Value, Mapped>
{
using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt64, false>;
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<Join::Type::key_string, Value, Mapped>
{
using Type = ColumnsHashing::HashMethodString<Value, Mapped, true, false>;
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<Join::Type::key_fixed_string, Value, Mapped>
{
using Type = ColumnsHashing::HashMethodFixedString<Value, Mapped, true, false>;
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<Join::Type::keys128, Value, Mapped>
{
using Type = ColumnsHashing::HashMethodKeysFixed<Value, UInt128, Mapped, false, false, false>;
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<Join::Type::keys256, Value, Mapped>
{
using Type = ColumnsHashing::HashMethodKeysFixed<Value, UInt256, Mapped, false, false, false>;
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<Join::Type::hashed, Value, Mapped>
{
using Type = ColumnsHashing::HashMethodHashed<Value, Mapped, false>;
};
template <Join::Type type, typename Data>
struct KeyGetterForType
{
using Value = typename Data::value_type;
using Mapped_t = typename Data::mapped_type;
using Mapped = std::conditional_t<std::is_const_v<Data>, const Mapped_t, Mapped_t>;
using Type = typename KeyGetterForTypeImpl<type, Value, Mapped>::Type;
};
/// Do I need to use the hash table maps_*_full, in which we remember whether the row was joined. /// Do I need to use the hash table maps_*_full, in which we remember whether the row was joined.
@ -316,42 +352,30 @@ namespace
template <ASTTableJoin::Strictness STRICTNESS, typename Map, typename KeyGetter> template <ASTTableJoin::Strictness STRICTNESS, typename Map, typename KeyGetter>
struct Inserter struct Inserter
{ {
static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool); static void insert(Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool);
}; };
template <typename Map, typename KeyGetter> template <typename Map, typename KeyGetter>
struct Inserter<ASTTableJoin::Strictness::Any, Map, KeyGetter> struct Inserter<ASTTableJoin::Strictness::Any, Map, KeyGetter>
{ {
static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool) static ALWAYS_INLINE void insert(Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool)
{ {
typename Map::iterator it; auto emplace_result = key_getter.emplaceKey(map, i, pool);
bool inserted;
map.emplace(key, it, inserted);
if (inserted) if (emplace_result.isInserted() || emplace_result.getMapped().overwrite)
{ new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i);
KeyGetter::onNewKey(it->first, pool);
new (&it->second) typename Map::mapped_type(stored_block, i);
}
else if (it->second.overwrite)
new (&it->second) typename Map::mapped_type(stored_block, i);
} }
}; };
template <typename Map, typename KeyGetter> template <typename Map, typename KeyGetter>
struct Inserter<ASTTableJoin::Strictness::All, Map, KeyGetter> struct Inserter<ASTTableJoin::Strictness::All, Map, KeyGetter>
{ {
static void insert(Map & map, const typename Map::key_type & key, Block * stored_block, size_t i, Arena & pool) static ALWAYS_INLINE void insert(Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool)
{ {
typename Map::iterator it; auto emplace_result = key_getter.emplaceKey(map, i, pool);
bool inserted;
map.emplace(key, it, inserted);
if (inserted) if (emplace_result.isInserted())
{ new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i);
KeyGetter::onNewKey(it->first, pool);
new (&it->second) typename Map::mapped_type(stored_block, i);
}
else else
{ {
/** The first element of the list is stored in the value of the hash table, the rest in the pool. /** The first element of the list is stored in the value of the hash table, the rest in the pool.
@ -359,9 +383,10 @@ namespace
* That is, the former second element, if it was, will be the third, and so on. * That is, the former second element, if it was, will be the third, and so on.
*/ */
auto elem = pool.alloc<typename Map::mapped_type>(); auto elem = pool.alloc<typename Map::mapped_type>();
auto & mapped = emplace_result.getMapped();
elem->next = it->second.next; elem->next = mapped.next;
it->second.next = elem; mapped.next = elem;
elem->block = stored_block; elem->block = stored_block;
elem->row_num = i; elem->row_num = i;
} }
@ -372,17 +397,16 @@ namespace
template <ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map, bool has_null_map> template <ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map, bool has_null_map>
void NO_INLINE insertFromBlockImplTypeCase( void NO_INLINE insertFromBlockImplTypeCase(
Map & map, size_t rows, const ColumnRawPtrs & key_columns, Map & map, size_t rows, const ColumnRawPtrs & key_columns,
size_t keys_size, const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool) const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool)
{ {
KeyGetter key_getter(key_columns); KeyGetter key_getter(key_columns, key_sizes, nullptr);
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < rows; ++i)
{ {
if (has_null_map && (*null_map)[i]) if (has_null_map && (*null_map)[i])
continue; continue;
auto key = key_getter.getKey(key_columns, keys_size, i, key_sizes); Inserter<STRICTNESS, Map, KeyGetter>::insert(map, key_getter, stored_block, i, pool);
Inserter<STRICTNESS, Map, KeyGetter>::insert(map, key, stored_block, i, pool);
} }
} }
@ -390,19 +414,19 @@ namespace
template <ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map> template <ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map>
void insertFromBlockImplType( void insertFromBlockImplType(
Map & map, size_t rows, const ColumnRawPtrs & key_columns, Map & map, size_t rows, const ColumnRawPtrs & key_columns,
size_t keys_size, const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool) const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool)
{ {
if (null_map) if (null_map)
insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, true>(map, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, true>(map, rows, key_columns, key_sizes, stored_block, null_map, pool);
else else
insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, false>(map, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, false>(map, rows, key_columns, key_sizes, stored_block, null_map, pool);
} }
template <ASTTableJoin::Strictness STRICTNESS, typename Maps> template <ASTTableJoin::Strictness STRICTNESS, typename Maps>
void insertFromBlockImpl( void insertFromBlockImpl(
Join::Type type, Maps & maps, size_t rows, const ColumnRawPtrs & key_columns, Join::Type type, Maps & maps, size_t rows, const ColumnRawPtrs & key_columns,
size_t keys_size, const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool) const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool)
{ {
switch (type) switch (type)
{ {
@ -411,8 +435,8 @@ namespace
#define M(TYPE) \ #define M(TYPE) \
case Join::Type::TYPE: \ case Join::Type::TYPE: \
insertFromBlockImplType<STRICTNESS, typename KeyGetterForType<Join::Type::TYPE>::Type>(\ insertFromBlockImplType<STRICTNESS, typename KeyGetterForType<Join::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>(\
*maps.TYPE, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); \ *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, pool); \
break; break;
APPLY_FOR_JOIN_VARIANTS(M) APPLY_FOR_JOIN_VARIANTS(M)
#undef M #undef M
@ -499,7 +523,7 @@ bool Join::insertFromBlock(const Block & block)
{ {
dispatch([&](auto, auto strictness_, auto & map) dispatch([&](auto, auto strictness_, auto & map)
{ {
insertFromBlockImpl<strictness_>(type, map, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); insertFromBlockImpl<strictness_>(type, map, rows, key_columns, key_sizes, stored_block, null_map, pool);
}); });
} }
@ -515,14 +539,14 @@ namespace
template <typename Map> template <typename Map>
struct Adder<true, ASTTableJoin::Strictness::Any, Map> struct Adder<true, ASTTableJoin::Strictness::Any, Map>
{ {
static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, static void addFound(const typename Map::mapped_type & mapped, size_t num_columns_to_add, MutableColumns & added_columns,
size_t i, IColumn::Filter & filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, size_t i, IColumn::Filter & filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/,
const std::vector<size_t> & right_indexes) const std::vector<size_t> & right_indexes)
{ {
filter[i] = 1; filter[i] = 1;
for (size_t j = 0; j < num_columns_to_add; ++j) for (size_t j = 0; j < num_columns_to_add; ++j)
added_columns[j]->insertFrom(*it->second.block->getByPosition(right_indexes[j]).column.get(), it->second.row_num); added_columns[j]->insertFrom(*mapped.block->getByPosition(right_indexes[j]).column, mapped.row_num);
} }
static void addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, static void addNotFound(size_t num_columns_to_add, MutableColumns & added_columns,
@ -538,14 +562,14 @@ namespace
template <typename Map> template <typename Map>
struct Adder<false, ASTTableJoin::Strictness::Any, Map> struct Adder<false, ASTTableJoin::Strictness::Any, Map>
{ {
static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, static void addFound(const typename Map::mapped_type & mapped, size_t num_columns_to_add, MutableColumns & added_columns,
size_t i, IColumn::Filter & filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, size_t i, IColumn::Filter & filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/,
const std::vector<size_t> & right_indexes) const std::vector<size_t> & right_indexes)
{ {
filter[i] = 1; filter[i] = 1;
for (size_t j = 0; j < num_columns_to_add; ++j) for (size_t j = 0; j < num_columns_to_add; ++j)
added_columns[j]->insertFrom(*it->second.block->getByPosition(right_indexes[j]).column.get(), it->second.row_num); added_columns[j]->insertFrom(*mapped.block->getByPosition(right_indexes[j]).column, mapped.row_num);
} }
static void addNotFound(size_t /*num_columns_to_add*/, MutableColumns & /*added_columns*/, static void addNotFound(size_t /*num_columns_to_add*/, MutableColumns & /*added_columns*/,
@ -558,14 +582,14 @@ namespace
template <bool fill_left, typename Map> template <bool fill_left, typename Map>
struct Adder<fill_left, ASTTableJoin::Strictness::All, Map> struct Adder<fill_left, ASTTableJoin::Strictness::All, Map>
{ {
static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, static void addFound(const typename Map::mapped_type & mapped, size_t num_columns_to_add, MutableColumns & added_columns,
size_t i, IColumn::Filter & filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets, size_t i, IColumn::Filter & filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets,
const std::vector<size_t> & right_indexes) const std::vector<size_t> & right_indexes)
{ {
filter[i] = 1; filter[i] = 1;
size_t rows_joined = 0; size_t rows_joined = 0;
for (auto current = &static_cast<const typename Map::mapped_type::Base_t &>(it->second); current != nullptr; current = current->next) for (auto current = &static_cast<const typename Map::mapped_type::Base_t &>(mapped); current != nullptr; current = current->next)
{ {
for (size_t j = 0; j < num_columns_to_add; ++j) for (size_t j = 0; j < num_columns_to_add; ++j)
added_columns[j]->insertFrom(*current->block->getByPosition(right_indexes[j]).column.get(), current->row_num); added_columns[j]->insertFrom(*current->block->getByPosition(right_indexes[j]).column.get(), current->row_num);
@ -605,10 +629,10 @@ namespace
const std::vector<size_t> & right_indexes) const std::vector<size_t> & right_indexes)
{ {
IColumn::Offset current_offset = 0; IColumn::Offset current_offset = 0;
size_t keys_size = key_columns.size();
size_t num_columns_to_add = right_indexes.size(); size_t num_columns_to_add = right_indexes.size();
KeyGetter key_getter(key_columns); Arena pool;
KeyGetter key_getter(key_columns, key_sizes, nullptr);
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < rows; ++i)
{ {
@ -619,14 +643,14 @@ namespace
} }
else else
{ {
auto key = key_getter.getKey(key_columns, keys_size, i, key_sizes); auto find_result = key_getter.findKey(map, i, pool);
typename Map::const_iterator it = map.find(key);
if (it != map.end()) if (find_result.isFound())
{ {
it->second.setUsed(); auto & mapped = find_result.getMapped();
mapped.setUsed();
Adder<Join::KindTrait<KIND>::fill_left, STRICTNESS, Map>::addFound( Adder<Join::KindTrait<KIND>::fill_left, STRICTNESS, Map>::addFound(
it, num_columns_to_add, added_columns, i, filter, current_offset, offsets_to_replicate.get(), right_indexes); mapped, num_columns_to_add, added_columns, i, filter, current_offset, offsets_to_replicate.get(), right_indexes);
} }
else else
Adder<Join::KindTrait<KIND>::fill_left, STRICTNESS, Map>::addNotFound( Adder<Join::KindTrait<KIND>::fill_left, STRICTNESS, Map>::addNotFound(
@ -753,7 +777,7 @@ void Join::joinBlockImpl(
#define M(TYPE) \ #define M(TYPE) \
case Join::Type::TYPE: \ case Join::Type::TYPE: \
std::tie(filter, offsets_to_replicate) = \ std::tie(filter, offsets_to_replicate) = \
joinBlockImplType<KIND, STRICTNESS, typename KeyGetterForType<Join::Type::TYPE>::Type>(\ joinBlockImplType<KIND, STRICTNESS, typename KeyGetterForType<Join::Type::TYPE, const std::remove_reference_t<decltype(*maps_.TYPE)>>::Type>(\
*maps_.TYPE, block.rows(), key_columns, key_sizes, added_columns, null_map, right_indexes); \ *maps_.TYPE, block.rows(), key_columns, key_sizes, added_columns, null_map, right_indexes); \
break; break;
APPLY_FOR_JOIN_VARIANTS(M) APPLY_FOR_JOIN_VARIANTS(M)

View File

@ -8,6 +8,7 @@
#include <Interpreters/SettingsCommon.h> #include <Interpreters/SettingsCommon.h>
#include <Common/Arena.h> #include <Common/Arena.h>
#include <Common/ColumnsHashing.h>
#include <Common/HashTable/HashMap.h> #include <Common/HashTable/HashMap.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
@ -21,148 +22,6 @@
namespace DB namespace DB
{ {
/// Helpers to obtain keys (to use in a hash table or similar data structure) for various equi-JOINs.
/// UInt8/16/32/64 or another types with same number of bits.
template <typename FieldType>
struct JoinKeyGetterOneNumber
{
using Key = FieldType;
const char * vec;
/** Created before processing of each block.
* Initialize some members, used in another methods, called in inner loops.
*/
JoinKeyGetterOneNumber(const ColumnRawPtrs & key_columns)
{
vec = key_columns[0]->getRawData().data;
}
Key getKey(
const ColumnRawPtrs & /*key_columns*/,
size_t /*keys_size*/, /// number of key columns.
size_t i, /// row number to get key from.
const Sizes & /*key_sizes*/) const /// If keys are of fixed size - their sizes. Not used for methods with variable-length keys.
{
return unalignedLoad<FieldType>(vec + i * sizeof(FieldType));
}
/// Place additional data into memory pool, if needed, when new key was inserted into hash table.
static void onNewKey(Key & /*key*/, Arena & /*pool*/) {}
};
/// For single String key.
struct JoinKeyGetterString
{
using Key = StringRef;
const IColumn::Offset * offsets;
const UInt8 * chars;
JoinKeyGetterString(const ColumnRawPtrs & key_columns)
{
const IColumn & column = *key_columns[0];
const ColumnString & column_string = static_cast<const ColumnString &>(column);
offsets = column_string.getOffsets().data();
chars = column_string.getChars().data();
}
Key getKey(
const ColumnRawPtrs &,
size_t,
ssize_t i,
const Sizes &) const
{
return StringRef(
chars + offsets[i - 1],
offsets[i] - offsets[i - 1] - 1);
}
static void onNewKey(Key & key, Arena & pool)
{
if (key.size)
key.data = pool.insert(key.data, key.size);
}
};
/// For single FixedString key.
struct JoinKeyGetterFixedString
{
using Key = StringRef;
size_t n;
const ColumnFixedString::Chars * chars;
JoinKeyGetterFixedString(const ColumnRawPtrs & key_columns)
{
const IColumn & column = *key_columns[0];
const ColumnFixedString & column_string = static_cast<const ColumnFixedString &>(column);
n = column_string.getN();
chars = &column_string.getChars();
}
Key getKey(
const ColumnRawPtrs &,
size_t,
size_t i,
const Sizes &) const
{
return StringRef(&(*chars)[i * n], n);
}
static void onNewKey(Key & key, Arena & pool)
{
key.data = pool.insert(key.data, key.size);
}
};
/// For keys of fixed size, that could be packed in sizeof TKey width.
template <typename TKey>
struct JoinKeyGetterFixed
{
using Key = TKey;
JoinKeyGetterFixed(const ColumnRawPtrs &)
{
}
Key getKey(
const ColumnRawPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
{
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
}
static void onNewKey(Key &, Arena &) {}
};
/// Generic method, use crypto hash function.
struct JoinKeyGetterHashed
{
using Key = UInt128;
JoinKeyGetterHashed(const ColumnRawPtrs &)
{
}
Key getKey(
const ColumnRawPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes &) const
{
return hash128(i, keys_size, key_columns);
}
static void onNewKey(Key &, Arena &) {}
};
/** Data structure for implementation of JOIN. /** Data structure for implementation of JOIN.
* It is just a hash table: keys -> rows of joined ("right") table. * It is just a hash table: keys -> rows of joined ("right") table.
* Additionally, CROSS JOIN is supported: instead of hash table, it use just set of blocks without keys. * Additionally, CROSS JOIN is supported: instead of hash table, it use just set of blocks without keys.

View File

@ -75,30 +75,22 @@ void NO_INLINE Set::insertFromBlockImplCase(
const ColumnRawPtrs & key_columns, const ColumnRawPtrs & key_columns,
size_t rows, size_t rows,
SetVariants & variants, SetVariants & variants,
ConstNullMapPtr null_map, [[maybe_unused]] ConstNullMapPtr null_map,
ColumnUInt8::Container * out_filter) [[maybe_unused]] ColumnUInt8::Container * out_filter)
{ {
typename Method::State state; typename Method::State state(key_columns, key_sizes, nullptr);
state.init(key_columns);
/// For all rows /// For all rows
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < rows; ++i)
{ {
if (has_null_map && (*null_map)[i]) if constexpr (has_null_map)
continue; if ((*null_map)[i])
continue;
/// Obtain a key to insert to the set [[maybe_unused]] auto emplace_result = state.emplaceKey(method.data, i, variants.string_pool);
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes);
typename Method::Data::iterator it; if constexpr (build_filter)
bool inserted; (*out_filter)[i] = emplace_result.isInserted();
method.data.emplace(key, it, inserted);
if (inserted)
method.onNewKey(*it, keys_size, variants.string_pool);
if (build_filter)
(*out_filter)[i] = inserted;
} }
} }
@ -392,10 +384,10 @@ void NO_INLINE Set::executeImplCase(
size_t rows, size_t rows,
ConstNullMapPtr null_map) const ConstNullMapPtr null_map) const
{ {
typename Method::State state; Arena pool;
state.init(key_columns); typename Method::State state(key_columns, key_sizes, nullptr);
/// NOTE Optimization is not used for consecutive identical values. /// NOTE Optimization is not used for consecutive identical strings.
/// For all rows /// For all rows
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < rows; ++i)
@ -404,9 +396,8 @@ void NO_INLINE Set::executeImplCase(
vec_res[i] = negative; vec_res[i] = negative;
else else
{ {
/// Build the key auto find_result = state.findKey(method.data, i, pool);
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes); vec_res[i] = negative ^ find_result.isFound();
vec_res[i] = negative ^ method.data.has(key);
} }
} }
} }

View File

@ -3,6 +3,7 @@
#include <Columns/ColumnNullable.h> #include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Interpreters/AggregationCommon.h> #include <Interpreters/AggregationCommon.h>
#include <Common/ColumnsHashing.h>
#include <Common/Arena.h> #include <Common/Arena.h>
#include <Common/HashTable/HashSet.h> #include <Common/HashTable/HashSet.h>
@ -27,33 +28,7 @@ struct SetMethodOneNumber
Data data; Data data;
/// To use one `Method` in different threads, use different `State`. using State = ColumnsHashing::HashMethodOneNumber<typename Data::value_type, void, FieldType>;
struct State
{
const char * vec;
/** Called at the start of each block processing.
* Sets the variables required for the other methods called in inner loops.
*/
void init(const ColumnRawPtrs & key_columns)
{
vec = key_columns[0]->getRawData().data;
}
/// Get key from key columns for insertion into hash table.
Key getKey(
const ColumnRawPtrs & /*key_columns*/,
size_t /*keys_size*/, /// Number of key columns.
size_t i, /// From what row of the block I get the key.
const Sizes & /*key_sizes*/) const /// If keys of a fixed length - their lengths. Not used in methods for variable length keys.
{
return unalignedLoad<FieldType>(vec + i * sizeof(FieldType));
}
};
/** Place additional data, if necessary, in case a new key was inserted into the hash table.
*/
static void onNewKey(typename Data::value_type & /*value*/, size_t /*keys_size*/, Arena & /*pool*/) {}
}; };
/// For the case where there is one string key. /// For the case where there is one string key.
@ -65,36 +40,7 @@ struct SetMethodString
Data data; Data data;
struct State using State = ColumnsHashing::HashMethodString<typename Data::value_type, void, true, false>;
{
const IColumn::Offset * offsets;
const UInt8 * chars;
void init(const ColumnRawPtrs & key_columns)
{
const IColumn & column = *key_columns[0];
const ColumnString & column_string = static_cast<const ColumnString &>(column);
offsets = column_string.getOffsets().data();
chars = column_string.getChars().data();
}
Key getKey(
const ColumnRawPtrs &,
size_t,
ssize_t i,
const Sizes &) const
{
return StringRef(
chars + offsets[i - 1],
offsets[i] - offsets[i - 1] - 1);
}
};
static void onNewKey(typename Data::value_type & value, size_t, Arena & pool)
{
if (value.size)
value.data = pool.insert(value.data, value.size);
}
}; };
/// For the case when there is one fixed-length string key. /// For the case when there is one fixed-length string key.
@ -106,33 +52,7 @@ struct SetMethodFixedString
Data data; Data data;
struct State using State = ColumnsHashing::HashMethodFixedString<typename Data::value_type, void, true, false>;
{
size_t n;
const ColumnFixedString::Chars * chars;
void init(const ColumnRawPtrs & key_columns)
{
const IColumn & column = *key_columns[0];
const ColumnFixedString & column_string = static_cast<const ColumnFixedString &>(column);
n = column_string.getN();
chars = &column_string.getChars();
}
Key getKey(
const ColumnRawPtrs &,
size_t,
size_t i,
const Sizes &) const
{
return StringRef(&(*chars)[i * n], n);
}
};
static void onNewKey(typename Data::value_type & value, size_t, Arena & pool)
{
value.data = pool.insert(value.data, value.size);
}
}; };
namespace set_impl namespace set_impl
@ -242,34 +162,7 @@ struct SetMethodKeysFixed
Data data; Data data;
class State : private set_impl::BaseStateKeysFixed<Key, has_nullable_keys> using State = ColumnsHashing::HashMethodKeysFixed<typename Data::value_type, Key, void, has_nullable_keys, false>;
{
public:
using Base = set_impl::BaseStateKeysFixed<Key, has_nullable_keys>;
void init(const ColumnRawPtrs & key_columns)
{
if (has_nullable_keys)
Base::init(key_columns);
}
Key getKey(
const ColumnRawPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes & key_sizes) const
{
if (has_nullable_keys)
{
auto bitmap = Base::createBitmap(i);
return packFixed<Key>(i, keys_size, Base::getActualColumns(), key_sizes, bitmap);
}
else
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
}
};
static void onNewKey(typename Data::value_type &, size_t, Arena &) {}
}; };
/// For other cases. 128 bit hash from the key. /// For other cases. 128 bit hash from the key.
@ -281,23 +174,7 @@ struct SetMethodHashed
Data data; Data data;
struct State using State = ColumnsHashing::HashMethodHashed<typename Data::value_type, void>;
{
void init(const ColumnRawPtrs &)
{
}
Key getKey(
const ColumnRawPtrs & key_columns,
size_t keys_size,
size_t i,
const Sizes &) const
{
return hash128(i, keys_size, key_columns);
}
};
static void onNewKey(typename Data::value_type &, size_t, Arena &) {}
}; };

View File

@ -107,11 +107,7 @@ void NO_INLINE Aggregator::executeSpecialized(
bool no_more_keys, bool no_more_keys,
AggregateDataPtr overflow_row) const AggregateDataPtr overflow_row) const
{ {
typename Method::State state; typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
if constexpr (Method::low_cardinality_optimization)
state.init(key_columns, aggregation_state_cache);
else
state.init(key_columns);
if (!no_more_keys) if (!no_more_keys)
executeSpecializedCase<false, Method, AggregateFunctionsList>( executeSpecializedCase<false, Method, AggregateFunctionsList>(
@ -130,94 +126,48 @@ void NO_INLINE Aggregator::executeSpecializedCase(
typename Method::State & state, typename Method::State & state,
Arena * aggregates_pool, Arena * aggregates_pool,
size_t rows, size_t rows,
ColumnRawPtrs & key_columns, ColumnRawPtrs & /*key_columns*/,
AggregateColumns & aggregate_columns, AggregateColumns & aggregate_columns,
StringRefs & keys, StringRefs & /*keys*/,
AggregateDataPtr overflow_row) const AggregateDataPtr overflow_row) const
{ {
/// For all rows. /// For all rows.
typename Method::Key prev_key{};
AggregateDataPtr value = nullptr;
for (size_t i = 0; i < rows; ++i) for (size_t i = 0; i < rows; ++i)
{ {
bool inserted = false; /// Inserted a new key, or was this key already? AggregateDataPtr aggregate_data = nullptr;
/// Get the key to insert into the hash table.
typename Method::Key key;
if constexpr (!Method::low_cardinality_optimization)
key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
AggregateDataPtr * aggregate_data = nullptr;
typename Method::iterator it; /// Is not used if Method::low_cardinality_optimization
if (!no_more_keys) /// Insert. if (!no_more_keys) /// Insert.
{ {
/// Optimization for frequently repeating keys. auto emplace_result = state.emplaceKey(method.data, i, *aggregates_pool);
if (!Method::no_consecutive_keys_optimization)
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
if (emplace_result.isInserted())
{ {
if (i != 0 && key == prev_key) /// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
{ emplace_result.setMapped(nullptr);
/// Add values into aggregate functions.
AggregateFunctionsList::forEach(AggregateFunctionsUpdater(
aggregate_functions, offsets_of_aggregate_states, aggregate_columns, value, i, aggregates_pool));
method.onExistingKey(key, keys, *aggregates_pool); aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
continue; AggregateFunctionsList::forEach(AggregateFunctionsCreator(
} aggregate_functions, offsets_of_aggregate_states, aggregate_data));
else
prev_key = key; emplace_result.setMapped(aggregate_data);
} }
if constexpr (Method::low_cardinality_optimization)
aggregate_data = state.emplaceKeyFromRow(method.data, i, inserted, params.keys_size, keys, *aggregates_pool);
else else
{ aggregate_data = emplace_result.getMapped();
method.data.emplace(key, it, inserted);
aggregate_data = &Method::getAggregateData(it->second);
}
} }
else else
{ {
/// Add only if the key already exists. /// Add only if the key already exists.
if constexpr (Method::low_cardinality_optimization) auto find_result = state.findKey(method.data, i, *aggregates_pool);
aggregate_data = state.findFromRow(method.data, i); if (find_result.isFound())
else aggregate_data = find_result.getMapped();
{
it = method.data.find(key);
if (method.data.end() != it)
aggregate_data = &Method::getAggregateData(it->second);
}
} }
/// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do. /// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do.
if (!aggregate_data && !overflow_row) if (!aggregate_data && !overflow_row)
{
method.onExistingKey(key, keys, *aggregates_pool);
continue; continue;
}
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly some stuff related to the key. auto value = aggregate_data ? aggregate_data : overflow_row;
if (inserted)
{
*aggregate_data = nullptr;
if constexpr (!Method::low_cardinality_optimization)
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
AggregateDataPtr place = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
AggregateFunctionsList::forEach(AggregateFunctionsCreator(
aggregate_functions, offsets_of_aggregate_states, place));
*aggregate_data = place;
if constexpr (Method::low_cardinality_optimization)
state.cacheAggregateData(i, place);
}
else
method.onExistingKey(key, keys, *aggregates_pool);
value = aggregate_data ? *aggregate_data : overflow_row;
/// Add values into the aggregate functions. /// Add values into the aggregate functions.
AggregateFunctionsList::forEach(AggregateFunctionsUpdater( AggregateFunctionsList::forEach(AggregateFunctionsUpdater(

View File

@ -249,6 +249,15 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
} }
} }
if (src.getType() == Field::Types::String)
{
const auto col = type.createColumn();
ReadBufferFromString buffer(src.get<String>());
type.deserializeAsTextEscaped(*col, buffer, FormatSettings{});
return (*col)[0];
}
// TODO (nemkov): should we attempt to parse value using or `type.deserializeAsTextEscaped()` type.deserializeAsTextEscaped() ?
throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: " throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: "
+ Field::Types::toString(src.getType()), ErrorCodes::TYPE_MISMATCH); + Field::Types::toString(src.getType()), ErrorCodes::TYPE_MISMATCH);
} }

View File

@ -98,7 +98,7 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe
const DataTypePtr & type = storage.partition_key_sample.getByPosition(0).type; const DataTypePtr & type = storage.partition_key_sample.getByPosition(0).type;
auto column = type->createColumn(); auto column = type->createColumn();
column->insert(value[0]); column->insert(value[0]);
type->serializeText(*column, 0, out, format_settings); type->serializeAsText(*column, 0, out, format_settings);
} }
else else
{ {

View File

@ -0,0 +1,48 @@
<test>
<name>Benchmark</name>
<tags>
<tag>columns_hashing</tag>
</tags>
<preconditions>
<table_exists>hits_100m_single</table_exists>
<table_exists>hits_1000m_single</table_exists>
</preconditions>
<type>loop</type>
<stop_conditions>
<all_of>
<iterations>5</iterations>
<min_time_not_changing_for_ms>60000</min_time_not_changing_for_ms>
</all_of>
<any_of>
<iterations>10</iterations>
<total_time_ms>150000</total_time_ms>
</any_of>
</stop_conditions>
<!--
<query><![CDATA[select count() from hits_100m_single any left join hits_100m_single using (UserID, RegionID)]]></query>
<query><![CDATA[select count() from hits_100m_single any left join hits_100m_single using (UserID)]]></query>
<query><![CDATA[select count() from hits_100m_single any left join hits_100m_single using URL where URL != '']]></query>
<query><![CDATA[select count() from hits_1000m_single any left join hits_1000m_single using MobilePhoneModel where MobilePhoneModel != '']]></query>
<query><![CDATA[select count() from hits_100m_single any left join hits_100m_single using (MobilePhoneModel, UserID) where MobilePhoneModel != '']]></query>
<query><![CDATA[select count() from (select count() from hits_1000m_single group by (UserID))]]></query>
<query><![CDATA[select count() from (select count() from hits_100m_single group by (UserID, RegionID))]]></query>
<query><![CDATA[select count() from (select count() from hits_100m_single where URL != '' group by URL)]]></query>
<query><![CDATA[select count() from (select count() from hits_1000m_single where MobilePhoneModel != '' group by MobilePhoneModel)]]></query>
<query><![CDATA[select count() from (select count() from hits_1000m_single where MobilePhoneModel != '' group by (MobilePhoneModel, UserID))]]></query>
-->
<query><![CDATA[select sum(UserID + 1 in (select UserID from hits_100m_single)) from hits_100m_single]]></query>
<query><![CDATA[select sum((UserID + 1, RegionID) in (select UserID, RegionID from hits_100m_single)) from hits_100m_single]]></query>
<query><![CDATA[select sum(URL in (select URL from hits_100m where URL != '')) from hits_100m_single]]></query>
<query><![CDATA[select sum(MobilePhoneModel in (select MobilePhoneModel from hits_1000m where MobilePhoneModel != '')) from hits_1000m_single]]></query>
<query><![CDATA[select sum((MobilePhoneModel, UserID + 1) in (select MobilePhoneModel, UserID from hits_1000m where MobilePhoneModel != '')) from hits_1000m_single]]></query>
<main_metric>
<min_time/>
</main_metric>
</test>

View File

@ -0,0 +1,152 @@
<test>
<name>IPv4 Functions</name>
<type>once</type>
<tags>
</tags>
<stop_conditions>
<any_of>
<average_speed_not_changing_for_ms>2000</average_speed_not_changing_for_ms>
<total_time_ms>10000</total_time_ms>
</any_of>
</stop_conditions>
<metrics>
<max_rows_per_second />
<max_bytes_per_second />
<avg_rows_per_second />
<avg_bytes_per_second />
</metrics>
<substitutions>
<substitution>
<name>ipv4_string</name>
<values>
<!-- The CAIDA UCSD IPv4 Routed /24 DNS Names Dataset 20171130,
http://www.caida.org/data/active/ipv4_dnsnames_dataset.xml.
Randomly selected entries from first 50000 rows of dataset. -->
<value>116.253.40.133</value>
<value>183.247.232.58</value>
<value>116.106.34.242</value>
<value>111.56.27.171</value>
<value>183.245.137.140</value>
<value>183.212.25.70</value>
<value>162.144.2.57</value>
<value>111.4.229.190</value>
<value>59.52.3.168</value>
<value>115.11.21.200</value>
<value>121.28.97.113</value>
<value>111.46.39.248</value>
<value>120.192.122.34</value>
<value>113.56.44.105</value>
<value>116.66.238.92</value>
<value>67.22.254.206</value>
<value>115.0.24.191</value>
<value>182.30.107.86</value>
<value>223.73.153.243</value>
<value>115.159.103.38</value>
<value>36.186.75.121</value>
<value>111.56.188.125</value>
<value>115.14.93.25</value>
<value>211.97.110.141</value>
<value>61.58.96.173</value>
<value>203.126.212.37</value>
<value>192.220.125.142</value>
<value>115.22.20.223</value>
<value>121.25.160.80</value>
<value>117.150.98.199</value>
<value>183.211.172.143</value>
<value>180.244.18.143</value>
<value>209.131.3.252</value>
<value>220.200.1.22</value>
<value>171.225.130.45</value>
<value>115.4.78.200</value>
<value>36.183.59.29</value>
<value>218.42.159.17</value>
<value>115.13.39.164</value>
<value>142.254.161.133</value>
<value>116.2.211.43</value>
<value>36.183.126.25</value>
<value>66.150.171.196</value>
<value>104.149.148.137</value>
<value>120.239.82.212</value>
<value>111.14.182.156</value>
<value>115.6.63.224</value>
<value>153.35.83.233</value>
<value>113.142.1.1</value>
<value>121.25.82.29</value>
<value>62.151.203.189</value>
<value>104.27.46.146</value>
<value>36.189.46.88</value>
<value>116.252.54.207</value>
<value>64.77.240.1</value>
<value>142.252.102.78</value>
<value>36.82.224.170</value>
<value>117.33.191.217</value>
<value>144.12.164.251</value>
<value>122.10.93.66</value>
<value>104.25.84.59</value>
<value>111.4.242.106</value>
<value>222.216.51.186</value>
<value>112.33.13.212</value>
<value>115.9.240.116</value>
<value>171.228.0.153</value>
<value>45.3.47.158</value>
<value>69.57.193.230</value>
<value>115.6.104.199</value>
<value>104.24.237.140</value>
<value>199.17.84.108</value>
<value>120.193.17.57</value>
<value>112.40.38.145</value>
<value>67.55.90.43</value>
<value>180.253.57.249</value>
<value>14.204.253.158</value>
<value>1.83.241.116</value>
<value>202.198.37.147</value>
<value>115.6.31.95</value>
<value>117.32.14.179</value>
<value>23.238.237.26</value>
<value>116.97.76.104</value>
<value>1.80.2.248</value>
<value>59.50.185.152</value>
<value>42.117.228.166</value>
<value>119.36.22.147</value>
<value>210.66.18.184</value>
<value>115.19.192.159</value>
<value>112.15.128.113</value>
<value>1.55.138.211</value>
<value>210.183.19.113</value>
<value>42.115.43.114</value>
<value>58.16.171.31</value>
<value>171.234.78.185</value>
<value>113.56.43.134</value>
<value>111.53.182.225</value>
<value>107.160.215.141</value>
<value>171.229.231.90</value>
<value>58.19.84.138</value>
<value>36.79.88.107</value>
<!-- invalid values -->
<value tag="error"></value>
<value tag="error"> </value>
<value tag="error">1</value>
<value tag="error">1.</value>
<value tag="error">1.2.</value>
<value tag="error">.2.</value>
<value tag="error">abc</value>
<value tag="error">127.0.0.1/24</value>
<value tag="error"> 127.0.0.1</value>
<value tag="error">127.0.0.1 </value>
<value tag="error">127.0.0.1?</value>
<value tag="error">999.999.999.999</value>
</values>
</substitution>
</substitutions>
<query tag='IPv4StringToNum'>SELECT count() FROM system.numbers WHERE NOT ignore(IPv4StringToNum(materialize('{ipv4_string}')))</query>
<query tag='IPv4NumToString+IPv4StringToNum'>SELECT count() FROM system.numbers WHERE NOT ignore(IPv4NumToString(IPv4StringToNum(materialize('{ipv4_string}'))))</query>
<query tag='IPv4NumToStringClassC+IPv4StringToNum'>SELECT count() FROM system.numbers WHERE NOT ignore(IPv4NumToStringClassC(IPv4StringToNum(materialize('{ipv4_string}'))))</query>
<query tag='IPv4ToIPv6+IPv4StringToNum'>SELECT count() FROM system.numbers WHERE NOT ignore(IPv4ToIPv6(IPv4StringToNum(materialize('{ipv4_string}'))))</query>
</test>

View File

@ -0,0 +1,257 @@
<test>
<name>IPv6 Functions</name>
<type>once</type>
<tags>
</tags>
<stop_conditions>
<any_of>
<average_speed_not_changing_for_ms>2000</average_speed_not_changing_for_ms>
<total_time_ms>10000</total_time_ms>
</any_of>
</stop_conditions>
<metrics>
<max_rows_per_second />
<max_bytes_per_second />
<avg_rows_per_second />
<avg_bytes_per_second />
</metrics>
<substitutions>
<substitution>
<name>ipv6_string</name>
<values>
<!-- The CAIDA UCSD IPv4 Routed /24 DNS Names Dataset - 20181130,
http://www.caida.org/data/active/ipv4_dnsnames_dataset.xml.
Randomly selected entries from first 50000 rows of dataset. -->
<value>2606:2b00::1</value>
<value>2001:2000:3080:1351::2</value>
<value>2a01:8840:16::1</value>
<value>2001:550:0:1000::9a36:2a61</value>
<value>2001:578:400:4:2000::19</value>
<value>2607:f290::1</value>
<value>2a02:23f0:ffff:8::5</value>
<value>2400:c700:0:158::</value>
<value>2001:67c:24e4:4::250</value>
<value>2a02:2a38:37:5::2</value>
<value>2001:41a8:400:2::13a</value>
<value>2405:9800:9800:66::2</value>
<value>2a07:a343:f210::1</value>
<value>2403:5000:171:46::2</value>
<value>2800:c20:1141::8</value>
<value>2402:7800:40:2::62</value>
<value>2a00:de00::1</value>
<value>2001:688:0:2:1::9e</value>
<value>2001:2000:3080:80::2</value>
<value>2001:428::205:171:200:230</value>
<value>2001:fb1:fe0:9::8</value>
<value>2001:2000:3080:10ca::2</value>
<value>2400:dd0b:1003::2</value>
<value>2001:1a98:6677::9d9d:140a</value>
<value>2001:2000:3018:3b::1</value>
<value>2607:fa70:3:33::2</value>
<value>2001:5b0:23ff:fffa::113</value>
<value>2001:450:2001:1000:0:40:6924:23</value>
<value>2001:418:0:5000::c2d</value>
<value>2a01:b740:a09::1</value>
<value>2607:f0d0:2:2::243</value>
<value>2a01:348::e:1:1</value>
<value>2405:4800::3221:3621:2</value>
<value>2a02:aa08:e000:3100::2</value>
<value>2001:44c8:129:2632:33:0:252:2</value>
<value>2a02:e980:1e::1</value>
<value>2a0a:6f40:2::1</value>
<value>2001:550:2:29::2c9:1</value>
<value>2001:c20:4800::175</value>
<value>2c0f:feb0:1:2::d1</value>
<value>2a0b:7086:fff0::1</value>
<value>2a04:2dc0::16d</value>
<value>2604:7e00::105d</value>
<value>2001:470:1:946::2</value>
<value>2a0c:3240::1</value>
<value>2800:630:4010:8::2</value>
<value>2001:1af8:4040::12</value>
<value>2c0f:fc98:1200::2</value>
<value>2001:470:1:248::2</value>
<value>2620:44:a000::1</value>
<value>2402:800:63ff:40::1</value>
<value>2a02:b000:fff::524</value>
<value>2001:470:0:327::1</value>
<value>2401:7400:8888:2::8</value>
<value>2001:500:55::1</value>
<value>2001:668:0:3::f000:c2</value>
<value>2400:bf40::1</value>
<value>2001:67c:754::1</value>
<value>2402:28c0:100:ffff:ffff:ffff:ffff:ffff</value>
<value>2001:470:0:1fa::2</value>
<value>2001:550:0:1000::9a18:292a</value>
<value>2001:470:1:89e::2</value>
<value>2001:579:6f05:500:9934:5b3e:b7fe:1447</value>
<value>2804:158c::1</value>
<value>2600:140e:6::1</value>
<value>2a00:18e0:0:bb04::82</value>
<value>2a02:2698:5000::1e06</value>
<value>2402:800:63ff:10::7:2</value>
<value>2a02:e980:19::1</value>
<value>2001:4888::342:1:0:0</value>
<value>2607:fc68:0:4:0:2:2711:21</value>
<value>2606:2800:602a::1</value>
<value>2404:c600:1000:2::1d1</value>
<value>2001:578:1400:4::9d</value>
<value>2804:64:0:25::1</value>
<value>2605:3e00::1:2:2</value>
<value>2c0f:fa18:0:4::b</value>
<value>2606:2800:602c:b::d004</value>
<value>2610:18:181:4000::66</value>
<value>2001:48f8:1000:1::16</value>
<value>2408:8000:c000::1</value>
<value>2a03:4200:441:2::4e</value>
<value>2400:dd00:1:200a::2</value>
<value>2a02:e980:83:5b09:ecb8:c669:b336:650e</value>
<value>2001:16a0:2:200a::2</value>
<value>2001:4888:1f:e891:161:26::</value>
<value>2a0c:f743::1</value>
<value>2a02:e980:b::1</value>
<value>2001:578:201:1::601:9</value>
<value>2001:438:ffff::407d:1bc1</value>
<value>2001:920:1833::1</value>
<value>2001:1b70:a1:610::b102:2</value>
<value>2001:13c7:6014::1</value>
<value>2003:0:1203:4001::1</value>
<value>2804:a8:2:c8::d6</value>
<value>2a02:2e00:2080:f000:0:261:1:11</value>
<value>2001:578:20::d</value>
<value>2001:550:2:48::34:1</value>
<value>2a03:9d40:fe00:5::</value>
<value>2403:e800:200:102::2</value>
<!-- The CAIDA UCSD IPv4 Routed /24 DNS Names Dataset 20171130,
http://www.caida.org/data/active/ipv4_dnsnames_dataset.xml.
Randomly selected entries from first 50000 rows of dataset.
IPv4 addresses from dataset are represented in IPv6 form. -->
<value tag="mapped">::ffff:116.253.40.133</value>
<value tag="mapped">::ffff:183.247.232.58</value>
<value tag="mapped">::ffff:116.106.34.242</value>
<value tag="mapped">::ffff:111.56.27.171</value>
<value tag="mapped">::ffff:183.245.137.140</value>
<value tag="mapped">::ffff:183.212.25.70</value>
<value tag="mapped">::ffff:162.144.2.57</value>
<value tag="mapped">::ffff:111.4.229.190</value>
<value tag="mapped">::ffff:59.52.3.168</value>
<value tag="mapped">::ffff:115.11.21.200</value>
<value tag="mapped">::ffff:121.28.97.113</value>
<value tag="mapped">::ffff:111.46.39.248</value>
<value tag="mapped">::ffff:120.192.122.34</value>
<value tag="mapped">::ffff:113.56.44.105</value>
<value tag="mapped">::ffff:116.66.238.92</value>
<value tag="mapped">::ffff:67.22.254.206</value>
<value tag="mapped">::ffff:115.0.24.191</value>
<value tag="mapped">::ffff:182.30.107.86</value>
<value tag="mapped">::ffff:223.73.153.243</value>
<value tag="mapped">::ffff:115.159.103.38</value>
<value tag="mapped">::ffff:36.186.75.121</value>
<value tag="mapped">::ffff:111.56.188.125</value>
<value tag="mapped">::ffff:115.14.93.25</value>
<value tag="mapped">::ffff:211.97.110.141</value>
<value tag="mapped">::ffff:61.58.96.173</value>
<value tag="mapped">::ffff:203.126.212.37</value>
<value tag="mapped">::ffff:192.220.125.142</value>
<value tag="mapped">::ffff:115.22.20.223</value>
<value tag="mapped">::ffff:121.25.160.80</value>
<value tag="mapped">::ffff:117.150.98.199</value>
<value tag="mapped">::ffff:183.211.172.143</value>
<value tag="mapped">::ffff:180.244.18.143</value>
<value tag="mapped">::ffff:209.131.3.252</value>
<value tag="mapped">::ffff:220.200.1.22</value>
<value tag="mapped">::ffff:171.225.130.45</value>
<value tag="mapped">::ffff:115.4.78.200</value>
<value tag="mapped">::ffff:36.183.59.29</value>
<value tag="mapped">::ffff:218.42.159.17</value>
<value tag="mapped">::ffff:115.13.39.164</value>
<value tag="mapped">::ffff:142.254.161.133</value>
<value tag="mapped">::ffff:116.2.211.43</value>
<value tag="mapped">::ffff:36.183.126.25</value>
<value tag="mapped">::ffff:66.150.171.196</value>
<value tag="mapped">::ffff:104.149.148.137</value>
<value tag="mapped">::ffff:120.239.82.212</value>
<value tag="mapped">::ffff:111.14.182.156</value>
<value tag="mapped">::ffff:115.6.63.224</value>
<value tag="mapped">::ffff:153.35.83.233</value>
<value tag="mapped">::ffff:113.142.1.1</value>
<value tag="mapped">::ffff:121.25.82.29</value>
<value tag="mapped">::ffff:62.151.203.189</value>
<value tag="mapped">::ffff:104.27.46.146</value>
<value tag="mapped">::ffff:36.189.46.88</value>
<value tag="mapped">::ffff:116.252.54.207</value>
<value tag="mapped">::ffff:64.77.240.1</value>
<value tag="mapped">::ffff:142.252.102.78</value>
<value tag="mapped">::ffff:36.82.224.170</value>
<value tag="mapped">::ffff:117.33.191.217</value>
<value tag="mapped">::ffff:144.12.164.251</value>
<value tag="mapped">::ffff:122.10.93.66</value>
<value tag="mapped">::ffff:104.25.84.59</value>
<value tag="mapped">::ffff:111.4.242.106</value>
<value tag="mapped">::ffff:222.216.51.186</value>
<value tag="mapped">::ffff:112.33.13.212</value>
<value tag="mapped">::ffff:115.9.240.116</value>
<value tag="mapped">::ffff:171.228.0.153</value>
<value tag="mapped">::ffff:45.3.47.158</value>
<value tag="mapped">::ffff:69.57.193.230</value>
<value tag="mapped">::ffff:115.6.104.199</value>
<value tag="mapped">::ffff:104.24.237.140</value>
<value tag="mapped">::ffff:199.17.84.108</value>
<value tag="mapped">::ffff:120.193.17.57</value>
<value tag="mapped">::ffff:112.40.38.145</value>
<value tag="mapped">::ffff:67.55.90.43</value>
<value tag="mapped">::ffff:180.253.57.249</value>
<value tag="mapped">::ffff:14.204.253.158</value>
<value tag="mapped">::ffff:1.83.241.116</value>
<value tag="mapped">::ffff:202.198.37.147</value>
<value tag="mapped">::ffff:115.6.31.95</value>
<value tag="mapped">::ffff:117.32.14.179</value>
<value tag="mapped">::ffff:23.238.237.26</value>
<value tag="mapped">::ffff:116.97.76.104</value>
<value tag="mapped">::ffff:1.80.2.248</value>
<value tag="mapped">::ffff:59.50.185.152</value>
<value tag="mapped">::ffff:42.117.228.166</value>
<value tag="mapped">::ffff:119.36.22.147</value>
<value tag="mapped">::ffff:210.66.18.184</value>
<value tag="mapped">::ffff:115.19.192.159</value>
<value tag="mapped">::ffff:112.15.128.113</value>
<value tag="mapped">::ffff:1.55.138.211</value>
<value tag="mapped">::ffff:210.183.19.113</value>
<value tag="mapped">::ffff:42.115.43.114</value>
<value tag="mapped">::ffff:58.16.171.31</value>
<value tag="mapped">::ffff:171.234.78.185</value>
<value tag="mapped">::ffff:113.56.43.134</value>
<value tag="mapped">::ffff:111.53.182.225</value>
<value tag="mapped">::ffff:107.160.215.141</value>
<value tag="mapped">::ffff:171.229.231.90</value>
<value tag="mapped">::ffff:58.19.84.138</value>
<value tag="mapped">::ffff:36.79.88.107</value>
<!-- invalid values -->
<value tag="error"></value>
<value tag="error"> </value>
<value tag="error">1</value>
<value tag="error">1.</value>
<value tag="error">1.2.</value>
<value tag="error">.2.</value>
<value tag="error">abc</value>
<value tag="error">ab:cd:ef:gh:ij:kl:mn</value>
<value tag="error">ffffffffffffff</value>
<value tag="error">abcdefghijklmn</value>
<value tag="error">::::::::::::::</value>
<value tag="error">::ffff:127.0.0.1 </value>
<value tag="error"> ::ffff:127.0.0.1</value>
<value tag="error">::ffff:999.999.999.999</value>
</values>
</substitution>
</substitutions>
<query tag="IPv6StringToNum">SELECT count() FROM system.numbers WHERE NOT ignore(IPv6StringToNum(materialize('{ipv6_string}')))</query>
<query tag="IPv6NumToString+IPv6StringToNum">SELECT count() FROM system.numbers WHERE NOT ignore(IPv6NumToString(IPv6StringToNum(materialize('{ipv6_string}'))))</query>
</test>

View File

@ -16,6 +16,8 @@
1 1
1 1
1 1
1
1
00000000000000000000FFFF4D583737 00000000000000000000FFFF4D583737
00000000000000000000FFFF4D583737 00000000000000000000FFFF4D583737
00000000000000000000FFFF7F000001 00000000000000000000FFFF7F000001

View File

@ -6,6 +6,8 @@ select IPv4StringToNum('127.0.0.1' as p) == (0x7f000001 as n), IPv4NumToString(n
select IPv4StringToNum(materialize('127.0.0.1') as p) == (materialize(0x7f000001) as n), IPv4NumToString(n) == p; select IPv4StringToNum(materialize('127.0.0.1') as p) == (materialize(0x7f000001) as n), IPv4NumToString(n) == p;
select IPv4NumToString(toUInt32(0)) == '0.0.0.0'; select IPv4NumToString(toUInt32(0)) == '0.0.0.0';
select IPv4NumToString(materialize(toUInt32(0))) == materialize('0.0.0.0'); select IPv4NumToString(materialize(toUInt32(0))) == materialize('0.0.0.0');
select IPv4NumToString(toUInt32(0x7f000001)) == '127.0.0.1';
select IPv4NumToString(materialize(toUInt32(0x7f000001))) == materialize('127.0.0.1');
select IPv6NumToString(toFixedString('', 16)) == '::'; select IPv6NumToString(toFixedString('', 16)) == '::';
select IPv6NumToString(toFixedString(materialize(''), 16)) == materialize('::'); select IPv6NumToString(toFixedString(materialize(''), 16)) == materialize('::');

View File

@ -0,0 +1,27 @@
CREATE TABLE test.ipv4_test ( ipv4_ IPv4) ENGINE = Memory
0.0.0.0 00
8.8.8.8 08080808
127.0.0.1 7F000001
192.168.0.91 C0A8005B
255.255.255.255 FFFFFFFF
< 127.0.0.1 0.0.0.0
< 127.0.0.1 8.8.8.8
> 127.0.0.1 192.168.0.91
> 127.0.0.1 255.255.255.255
= 127.0.0.1 127.0.0.1
euqality of IPv4-mapped IPv6 value and IPv4 promoted to IPv6 with function: 1
CREATE TABLE test.ipv6_test ( ipv6_ IPv6) ENGINE = Memory
:: 00000000000000000000000000000000
:: 00000000000000000000000000000000
::ffff:8.8.8.8 00000000000000000000FFFF08080808
::ffff:127.0.0.1 00000000000000000000FFFF7F000001
::ffff:193.252.17.10 00000000000000000000FFFFC1FC110A
2001:db8:ac10:fe01:feed:babe:cafe:f00d 20010DB8AC10FE01FEEDBABECAFEF00D
ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF
< 127.0.0.1 ::
< 127.0.0.1 ::
< 127.0.0.1 ::ffff:8.8.8.8
> 127.0.0.1 ::ffff:193.252.17.10
> 127.0.0.1 2001:db8:ac10:fe01:feed:babe:cafe:f00d
> 127.0.0.1 ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
= 127.0.0.1 ::ffff:127.0.0.1

View File

@ -0,0 +1,59 @@
DROP TABLE IF EXISTS test.ipv4_test;
-- Only valid values for IPv4
CREATE TABLE test.ipv4_test (ipv4_ IPv4) ENGINE = Memory;
-- ipv4_ column shoud have type 'IPv4'
SHOW CREATE TABLE test.ipv4_test;
INSERT INTO test.ipv4_test (ipv4_) VALUES ('0.0.0.0'), ('255.255.255.255'), ('192.168.0.91'), ('127.0.0.1'), ('8.8.8.8');
SELECT ipv4_, hex(ipv4_) FROM test.ipv4_test ORDER BY ipv4_;
SELECT '< 127.0.0.1', ipv4_ FROM test.ipv4_test
WHERE ipv4_ < toIPv4('127.0.0.1')
ORDER BY ipv4_;
SELECT '> 127.0.0.1', ipv4_ FROM test.ipv4_test
WHERE ipv4_ > toIPv4('127.0.0.1')
ORDER BY ipv4_;
SELECT '= 127.0.0.1', ipv4_ FROM test.ipv4_test
WHERE ipv4_ = toIPv4('127.0.0.1')
ORDER BY ipv4_;
-- TODO: Assert that invalid values can't be inserted into IPv4 column.
DROP TABLE IF EXISTS test.ipv4_test;
select 'euqality of IPv4-mapped IPv6 value and IPv4 promoted to IPv6 with function:', toIPv6('::ffff:127.0.0.1') = IPv4ToIPv6(toIPv4('127.0.0.1'));
DROP TABLE IF EXISTS test.ipv6_test;
-- Only valid values for IPv6
CREATE TABLE test.ipv6_test (ipv6_ IPv6) ENGINE = Memory;
-- ipv6_ column shoud have type 'IPv6'
SHOW CREATE TABLE test.ipv6_test;
INSERT INTO test.ipv6_test VALUES ('::'), ('0:0:0:0:0:0:0:0'), ('FFFF:FFFF:FFFF:FFFF:FFFF:FFFF:FFFF:FFFF'), ('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), ('0000:0000:0000:0000:0000:FFFF:C1FC:110A'), ('::ffff:127.0.0.1'), ('::ffff:8.8.8.8');
SELECT ipv6_, hex(ipv6_) FROM test.ipv6_test ORDER BY ipv6_;
SELECT '< 127.0.0.1', ipv6_ FROM test.ipv6_test
WHERE ipv6_ < IPv4ToIPv6(toIPv4('127.0.0.1'))
ORDER BY ipv6_;
SELECT '> 127.0.0.1', ipv6_ FROM test.ipv6_test
WHERE ipv6_ > IPv4ToIPv6(toIPv4('127.0.0.1'))
ORDER BY ipv6_;
SELECT '= 127.0.0.1', ipv6_ FROM test.ipv6_test
WHERE ipv6_ = IPv4ToIPv6(toIPv4('127.0.0.1'))
ORDER BY ipv6_;
-- TODO: Assert that invalid values can't be inserted into IPv6 column.
DROP TABLE IF EXISTS test.ipv6_test;

View File

@ -52,7 +52,7 @@ def run_image_with_env(image_name, output, env_variables, ch_root):
subprocess.check_call(cmd, shell=True) subprocess.check_call(cmd, shell=True)
def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, distcc_hosts, unbundled): def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, distcc_hosts, unbundled, split_binary):
result = [] result = []
if package_type == "deb": if package_type == "deb":
result.append("DEB_CC={}".format(compiler)) result.append("DEB_CC={}".format(compiler))
@ -77,6 +77,10 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, di
if unbundled: if unbundled:
result.append('CMAKE_FLAGS="-DUNBUNDLED=1 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0 $CMAKE_FLAGS"') result.append('CMAKE_FLAGS="-DUNBUNDLED=1 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0 $CMAKE_FLAGS"')
if split_binary:
result.append('CMAKE_FLAGS="-DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 $CMAKE_FLAGS"')
return result return result
if __name__ == "__main__": if __name__ == "__main__":
@ -89,6 +93,7 @@ if __name__ == "__main__":
parser.add_argument("--compiler", choices=("clang-6.0", "clang-7", "gcc-7", "gcc-8"), default="gcc-7") parser.add_argument("--compiler", choices=("clang-6.0", "clang-7", "gcc-7", "gcc-8"), default="gcc-7")
parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="") parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="")
parser.add_argument("--unbundled", action="store_true") parser.add_argument("--unbundled", action="store_true")
parser.add_argument("--split-binary", action="store_true")
parser.add_argument("--cache", choices=("", "ccache", "distcc"), default="") parser.add_argument("--cache", choices=("", "ccache", "distcc"), default="")
parser.add_argument("--distcc-hosts", nargs="+") parser.add_argument("--distcc-hosts", nargs="+")
parser.add_argument("--force-build-image", action="store_true") parser.add_argument("--force-build-image", action="store_true")
@ -108,6 +113,6 @@ if __name__ == "__main__":
if not check_image_exists_locally(image_name) or args.force_build_image: if not check_image_exists_locally(image_name) or args.force_build_image:
if not pull_image(image_name) or args.force_build_image: if not pull_image(image_name) or args.force_build_image:
build_image(image_name, dockerfile) build_image(image_name, dockerfile)
env_prepared = parse_env_variables(args.build_type, args.compiler, args.sanitizer, args.package_type, args.cache, args.distcc_hosts, args.unbundled) env_prepared = parse_env_variables(args.build_type, args.compiler, args.sanitizer, args.package_type, args.cache, args.distcc_hosts, args.unbundled, args.split_binary)
run_image_with_env(image_name, args.output_dir, env_prepared, ch_root) run_image_with_env(image_name, args.output_dir, env_prepared, ch_root)
logging.info("Output placed into {}".format(args.output_dir)) logging.info("Output placed into {}".format(args.output_dir))

View File

@ -1,2 +1,2 @@
add_executable (convert-month-partitioned-parts main.cpp) add_executable (convert-month-partitioned-parts main.cpp)
target_link_libraries(convert-month-partitioned-parts PRIVATE dbms ${Boost_PROGRAM_OPTIONS_LIBRARY}) target_link_libraries(convert-month-partitioned-parts PRIVATE dbms clickhouse_parsers ${Boost_PROGRAM_OPTIONS_LIBRARY})