diff --git a/contrib/base64-cmake/CMakeLists.txt b/contrib/base64-cmake/CMakeLists.txt index 8ec83201109..12379c190e1 100644 --- a/contrib/base64-cmake/CMakeLists.txt +++ b/contrib/base64-cmake/CMakeLists.txt @@ -39,10 +39,20 @@ add_library(base64 ${LINK_MODE} ${LIBRARY_DIR}/lib/codecs.h ${CMAKE_CURRENT_BINARY_DIR}/config.h) -set_source_files_properties(${LIBRARY_DIR}/lib/arch/avx/codec.c PROPERTIES COMPILE_FLAGS -mavx) -set_source_files_properties(${LIBRARY_DIR}/lib/arch/avx2/codec.c PROPERTIES COMPILE_FLAGS -mavx2) -set_source_files_properties(${LIBRARY_DIR}/lib/arch/sse41/codec.c PROPERTIES COMPILE_FLAGS -msse4.1) -set_source_files_properties(${LIBRARY_DIR}/lib/arch/sse42/codec.c PROPERTIES COMPILE_FLAGS -msse4.2) -set_source_files_properties(${LIBRARY_DIR}/lib/arch/ssse3/codec.c PROPERTIES COMPILE_FLAGS -mssse3) +if(HAVE_AVX) + set_source_files_properties(${LIBRARY_DIR}/lib/arch/avx/codec.c PROPERTIES COMPILE_FLAGS -mavx) +endif() +if(HAVE_AVX2) + 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}) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 7cdebd5b043..633c6ef9b04 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -817,7 +817,7 @@ public: try { - type->deserializeTextQuoted(*column_dummy, rb, FormatSettings()); + type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings()); } catch (Exception & e) { @@ -1882,7 +1882,7 @@ protected: for (size_t i = 0; i < column.column->size(); ++i) { WriteBufferFromOwnString wb; - column.type->serializeTextQuoted(*column.column, i, wb, FormatSettings()); + column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings()); res.emplace(wb.str()); } } diff --git a/dbms/programs/performance-test/ReportBuilder.cpp b/dbms/programs/performance-test/ReportBuilder.cpp index a74721a7faf..97d4874ca5d 100644 --- a/dbms/programs/performance-test/ReportBuilder.cpp +++ b/dbms/programs/performance-test/ReportBuilder.cpp @@ -116,37 +116,38 @@ std::string ReportBuilder::buildFullReport( /// in seconds runJSON.set("min_time", statistics.min_time / double(1000)); - JSONString quantiles(4); /// here, 4 is the size of \t padding - for (double percent = 10; percent <= 90; percent += 10) + if (statistics.sampler.size() != 0) { - std::string quantile_key = std::to_string(percent / 100.0); - while (quantile_key.back() == '0') - quantile_key.pop_back(); + JSONString quantiles(4); /// here, 4 is the size of \t padding + for (double percent = 10; percent <= 90; percent += 10) + { + std::string quantile_key = std::to_string(percent / 100.0); + while (quantile_key.back() == '0') + quantile_key.pop_back(); - quantiles.set(quantile_key, - statistics.sampler.quantileInterpolated(percent / 100.0)); + quantiles.set(quantile_key, + 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("queries_per_second", - static_cast(statistics.queries) / statistics.total_time); - - runJSON.set("rows_per_second", - static_cast(statistics.total_rows_read) / statistics.total_time); - - runJSON.set("bytes_per_second", - static_cast(statistics.total_bytes_read) / statistics.total_time); + if (statistics.total_time != 0) + { + runJSON.set("queries_per_second", static_cast(statistics.queries) / statistics.total_time); + runJSON.set("rows_per_second", static_cast(statistics.total_rows_read) / statistics.total_time); + runJSON.set("bytes_per_second", static_cast(statistics.total_bytes_read) / statistics.total_time); + } } else { diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index b963e074df2..96f9b05189c 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -138,7 +138,7 @@ public: StringRef getRawData() const override { return StringRef(chars.data(), chars.size()); } /// Specialized part of interface, not from IColumn. - + void insertString(const String & string) { insertData(string.c_str(), string.size()); } Chars & getChars() { return chars; } const Chars & getChars() const { return chars; } diff --git a/dbms/src/Common/ColumnsHashing.h b/dbms/src/Common/ColumnsHashing.h new file mode 100644 index 00000000000..4f6708ae2f1 --- /dev/null +++ b/dbms/src/Common/ColumnsHashing.h @@ -0,0 +1,557 @@ +#pragma once + + +#include +#include +#include +#include + +#include +#include +#include + +#include +#include + +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 +struct HashMethodOneNumber + : public columns_hashing_impl::HashMethodBase, Value, Mapped, use_cache> +{ + using Self = HashMethodOneNumber; + using Base = columns_hashing_impl::HashMethodBase; + + 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(vec + row * sizeof(FieldType)); } + + /// Get StringRef from value which can be inserted into column. + static StringRef getValueRef(const Value & value) + { + return StringRef(reinterpret_cast(&value.first), sizeof(value.first)); + } +}; + + +/// For the case when there is one string key. +template +struct HashMethodString + : public columns_hashing_impl::HashMethodBase, Value, Mapped, use_cache> +{ + using Self = HashMethodString; + using Base = columns_hashing_impl::HashMethodBase; + + 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(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; + + 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 +struct HashMethodFixedString + : public columns_hashing_impl::HashMethodBase, Value, Mapped, use_cache> +{ + using Self = HashMethodFixedString; + using Base = columns_hashing_impl::HashMethodBase; + + 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(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; + 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; + + 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; + Cache cache; +}; + + +/// Single low cardinality column. +template +struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod +{ + using Base = SingleColumnMethod; + + enum class VisitValue + { + Empty = 0, + Found = 1, + NotFound = 2, + }; + + static constexpr bool has_mapped = !std::is_same::value; + using EmplaceResult = columns_hashing_impl::EmplaceResultImpl; + using FindResult = columns_hashing_impl::FindResultImpl; + + static HashMethodContextPtr createContext(const HashMethodContext::Settings & settings) + { + return std::make_shared(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_cache; + PaddedPODArray visit_cache; + + /// If initialized column is nullable. + bool is_nullable = false; + + static const ColumnLowCardinality & getLowCardinalityColumn(const IColumn * low_cardinality_column) + { + auto column = typeid_cast(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(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(); + 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(positions)->getElement(row); + case sizeof(UInt16): return static_cast(positions)->getElement(row); + case sizeof(UInt32): return static_cast(positions)->getElement(row); + case sizeof(UInt64): return static_cast(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 + 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 + 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 + 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 +struct LowCardinalityKeys +{ + ColumnRawPtrs nested_columns; + ColumnRawPtrs positions; + Sizes position_sizes; +}; + +template <> +struct LowCardinalityKeys {}; + +/// For the case when all keys are of fixed length, and they fit in N (for example, 128) bits. +template +struct HashMethodKeysFixed + : private columns_hashing_impl::BaseStateKeysFixed + , public columns_hashing_impl::HashMethodBase, Value, Mapped, use_cache> +{ + using Self = HashMethodKeysFixed; + using BaseHashed = columns_hashing_impl::HashMethodBase; + using Base = columns_hashing_impl::BaseStateKeysFixed; + + static constexpr bool has_nullable_keys = has_nullable_keys_; + static constexpr bool has_low_cardinality = has_low_cardinality_; + + LowCardinalityKeys 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(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(row, keys_size, Base::getActualColumns(), key_sizes, bitmap); + } + else + { + if constexpr (has_low_cardinality) + return packFixed(row, keys_size, low_cardinality_keys.nested_columns, key_sizes, + &low_cardinality_keys.positions, &low_cardinality_keys.position_sizes); + + return packFixed(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 +struct HashMethodSerialized + : public columns_hashing_impl::HashMethodBase, Value, Mapped, false> +{ + using Self = HashMethodSerialized; + using Base = columns_hashing_impl::HashMethodBase; + + 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; + + 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 +struct HashMethodHashed + : public columns_hashing_impl::HashMethodBase, Value, Mapped, use_cache> +{ + using Key = UInt128; + using Self = HashMethodHashed; + using Base = columns_hashing_impl::HashMethodBase; + + 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(&value.first), sizeof(value.first)); + } +}; + +} +} diff --git a/dbms/src/Common/ColumnsHashingImpl.h b/dbms/src/Common/ColumnsHashingImpl.h new file mode 100644 index 00000000000..e853ff66e64 --- /dev/null +++ b/dbms/src/Common/ColumnsHashingImpl.h @@ -0,0 +1,356 @@ +#pragma once + +#include +#include + +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; + + +namespace columns_hashing_impl +{ + +template +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 + bool check(const Key & key) { return !empty && value.first == key; } +}; + +template +struct LastElementCache +{ + static constexpr bool consecutive_keys_optimization = false; +}; + +template +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 +{ + bool inserted; + +public: + explicit EmplaceResultImpl(bool inserted) : inserted(inserted) {} + bool isInserted() const { return inserted; } +}; + +template +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 +{ + bool found; + +public: + explicit FindResultImpl(bool found) : found(found) {} + bool isFound() const { return found; } +}; + +template +class HashMethodBase +{ +public: + using EmplaceResult = EmplaceResultImpl; + using FindResult = FindResultImpl; + static constexpr bool has_mapped = !std::is_same::value; + using Cache = LastElementCache; + + static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; } + + template + ALWAYS_INLINE EmplaceResult emplaceKey(Data & data, size_t row, Arena & pool) + { + auto key = static_cast(*this).getKey(row, pool); + return emplaceKeyImpl(key, data, pool); + } + + template + ALWAYS_INLINE FindResult findKey(Data & data, size_t row, Arena & pool) + { + auto key = static_cast(*this).getKey(row, pool); + auto res = findKeyImpl(key, data); + static_cast(*this).onExistingKey(key, pool); + return res; + } + + template + ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & pool) + { + auto key = static_cast(*this).getKey(row, pool); + auto res = data.hash(key); + static_cast(*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 + static ALWAYS_INLINE void onNewKey(Key & /*key*/, Arena & /*pool*/) {} + template + static ALWAYS_INLINE void onExistingKey(Key & /*key*/, Arena & /*pool*/) {} + + template + ALWAYS_INLINE EmplaceResult emplaceKeyImpl(Key key, Data & data, Arena & pool) + { + if constexpr (Cache::consecutive_keys_optimization) + { + if (cache.found && cache.check(key)) + { + static_cast(*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(*this).onNewKey(it->first, pool); + } + else + static_cast(*this).onNewKey(*it, pool); + } + else + static_cast(*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 + 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 +struct MappedCache : public PaddedPODArray {}; + +template <> +struct MappedCache {}; + + +/// 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 +class BaseStateKeysFixed; + +/// Case where nullable keys are supported. +template +class BaseStateKeysFixed +{ +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(*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 createBitmap(size_t row) const + { + KeysNullMap bitmap{}; + + for (size_t k = 0; k < null_maps.size(); ++k) + { + if (null_maps[k] != nullptr) + { + const auto & null_map = static_cast(*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 +class BaseStateKeysFixed +{ +protected: + BaseStateKeysFixed(const ColumnRawPtrs & columns) : actual_columns(columns) {} + + const ColumnRawPtrs & getActualColumns() const { return actual_columns; } + + KeysNullMap createBitmap(size_t) const + { + throw Exception{"Internal error: calling createBitmap() for non-nullable keys" + " is forbidden", ErrorCodes::LOGICAL_ERROR}; + } + +private: + ColumnRawPtrs actual_columns; +}; + +} + +} + +} diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index ed0d127cfb9..5a89283e624 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -414,6 +414,7 @@ namespace ErrorCodes extern const int PROTOBUF_FIELD_NOT_REPEATED = 437; extern const int DATA_TYPE_CANNOT_BE_PROMOTED = 438; 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 POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/SipHash.h b/dbms/src/Common/SipHash.h index a800bceedf0..8bb82e52ff9 100644 --- a/dbms/src/Common/SipHash.h +++ b/dbms/src/Common/SipHash.h @@ -17,6 +17,7 @@ #include #include #include +#include #define ROTL(x, b) static_cast(((x) << (b)) | ((x) >> (64 - (b)))) @@ -49,7 +50,7 @@ private: UInt8 current_bytes[8]; }; - void finalize() + ALWAYS_INLINE void finalize() { /// In the last free byte, we write the remainder of the division by 256. 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 - void get128(T & lo, T & hi) + ALWAYS_INLINE void get128(T & lo, T & hi) { static_assert(sizeof(T) == 8); finalize(); diff --git a/dbms/src/Common/formatIPv6.cpp b/dbms/src/Common/formatIPv6.cpp index 1b09a7e0e1d..71f6c934a15 100644 --- a/dbms/src/Common/formatIPv6.cpp +++ b/dbms/src/Common/formatIPv6.cpp @@ -1,12 +1,44 @@ #include #include +#include + #include #include - +#include 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) 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) { 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? 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(IPV4_BINARY_LENGTH))); - break; + UInt8 ipv4_buffer[IPV4_BINARY_LENGTH] = {0}; + 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(IPV4_BINARY_LENGTH)), "0"); + // formatIPv4 has already added a null-terminator for us. + return; } printInteger<16>(dst, words[i]); diff --git a/dbms/src/Common/formatIPv6.h b/dbms/src/Common/formatIPv6.h index 6baf5544f02..66bc0912f88 100644 --- a/dbms/src/Common/formatIPv6.h +++ b/dbms/src/Common/formatIPv6.h @@ -1,12 +1,17 @@ #pragma once #include +#include +#include +#include +#include +#include +#include -#define IPV4_BINARY_LENGTH 4 -#define IPV6_BINARY_LENGTH 16 -#define IPV4_MAX_TEXT_LENGTH 15 /// Does not count tail zero byte. -#define IPV6_MAX_TEXT_LENGTH 39 - +constexpr size_t IPV4_BINARY_LENGTH = 4; +constexpr size_t IPV6_BINARY_LENGTH = 16; +constexpr size_t IPV4_MAX_TEXT_LENGTH = 15; /// Does not count tail zero byte. +constexpr size_t IPV6_MAX_TEXT_LENGTH = 39; namespace DB { @@ -18,4 +23,205 @@ namespace DB */ 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((val >> 8) & 0xffu); + *tp++ = static_cast(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((val >> 8) & 0xffu); + *tp++ = static_cast(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(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'; +} + } diff --git a/dbms/src/DataStreams/DistinctBlockInputStream.cpp b/dbms/src/DataStreams/DistinctBlockInputStream.cpp index 848fe923db4..2b9bcc490b2 100644 --- a/dbms/src/DataStreams/DistinctBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctBlockInputStream.cpp @@ -85,24 +85,15 @@ void DistinctBlockInputStream::buildFilter( size_t rows, SetVariants & variants) const { - typename Method::State state; - state.init(columns); + typename Method::State state(columns, key_sizes, nullptr); for (size_t i = 0; i < rows; ++i) { - /// Make a key. - 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); + auto emplace_result = state.emplaceKey(method.data, i, variants.string_pool); /// Emit the record if there is no such key in the current set yet. /// Skip it otherwise. - filter[i] = inserted; + filter[i] = emplace_result.isInserted(); } } diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp index 651b9738f73..ed52f63dc86 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp @@ -85,8 +85,7 @@ bool DistinctSortedBlockInputStream::buildFilter( size_t rows, ClearableSetVariants & variants) const { - typename Method::State state; - state.init(columns); + typename Method::State state(columns, key_sizes, nullptr); /// Compare last row of previous block and first row of current block, /// 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)) method.data.clear(); - /// Make a key. - 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); + auto emplace_result = state.emplaceKey(method.data, i, variants.string_pool); - if (inserted) - { - method.onNewKey(*it, columns.size(), variants.string_pool); + if (emplace_result.isInserted()) has_new_data = true; - } /// Emit the record if there is no such key in the current set yet. /// Skip it otherwise. - filter[i] = inserted; + filter[i] = emplace_result.isInserted(); } return has_new_data; } diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp index 19800db3b05..108bd70112c 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes } -std::string DataTypeAggregateFunction::getName() const +std::string DataTypeAggregateFunction::doGetName() const { std::stringstream stream; stream << "AggregateFunction(" << function->getName(); diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.h b/dbms/src/DataTypes/DataTypeAggregateFunction.h index a21f5d65125..1bd5269ffe0 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.h +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.h @@ -29,7 +29,7 @@ public: std::string getFunctionName() const { return function->getName(); } AggregateFunctionPtr getFunction() const { return function; } - std::string getName() const override; + std::string doGetName() const override; const char * getFamilyName() const override { return "AggregateFunction"; } TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; } diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index a1677d9bdbd..2325b692cab 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -350,7 +350,7 @@ void DataTypeArray::serializeText(const IColumn & column, size_t row_num, WriteB serializeTextImpl(column, row_num, ostr, [&](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, [&](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) writeChar(',', ostr); - nested->serializeTextJSON(nested_column, i, ostr, settings); + nested->serializeAsTextJSON(nested_column, i, ostr, settings); } 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 { - 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) { writeCString("", ostr); - nested->serializeTextXML(nested_column, i, ostr, settings); + nested->serializeAsTextXML(nested_column, i, ostr, settings); writeCString("", ostr); } writeCString("", ostr); diff --git a/dbms/src/DataTypes/DataTypeArray.h b/dbms/src/DataTypes/DataTypeArray.h index 4c04a2624f5..802f5922342 100644 --- a/dbms/src/DataTypes/DataTypeArray.h +++ b/dbms/src/DataTypes/DataTypeArray.h @@ -20,7 +20,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::Array; } - std::string getName() const override + std::string doGetName() const override { return "Array(" + nested->getName() + ")"; } diff --git a/dbms/src/DataTypes/DataTypeDateTime.cpp b/dbms/src/DataTypes/DataTypeDateTime.cpp index e894eb79252..5081b68e9f0 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.cpp +++ b/dbms/src/DataTypes/DataTypeDateTime.cpp @@ -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) return "DateTime"; diff --git a/dbms/src/DataTypes/DataTypeDateTime.h b/dbms/src/DataTypes/DataTypeDateTime.h index 10fe80a8213..eba05814550 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.h +++ b/dbms/src/DataTypes/DataTypeDateTime.h @@ -34,7 +34,7 @@ public: DataTypeDateTime(const std::string & time_zone_name = ""); const char * getFamilyName() const override { return "DateTime"; } - std::string getName() const override; + std::string doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::DateTime; } void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/dbms/src/DataTypes/DataTypeDomainIPv4AndIPv6.cpp b/dbms/src/DataTypes/DataTypeDomainIPv4AndIPv6.cpp new file mode 100644 index 00000000000..339409caf24 --- /dev/null +++ b/dbms/src/DataTypes/DataTypeDomainIPv4AndIPv6.cpp @@ -0,0 +1,118 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +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(&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(&col->getData()[row_num]), ptr); + + ostr.write(buffer, strlen(buffer)); + } + + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override + { + ColumnUInt32 * col = typeid_cast(&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(&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(&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(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(&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(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()); + factory.registerDataTypeDomain("FixedString(16)", std::make_unique()); +} + +} // namespace DB diff --git a/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.cpp b/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.cpp new file mode 100644 index 00000000000..12b1837be1f --- /dev/null +++ b/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.cpp @@ -0,0 +1,88 @@ +#include + +#include +#include +#include +#include + +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 diff --git a/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.h b/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.h new file mode 100644 index 00000000000..7834e9235d2 --- /dev/null +++ b/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.h @@ -0,0 +1,53 @@ +#pragma once + +#include + +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 diff --git a/dbms/src/DataTypes/DataTypeEnum.h b/dbms/src/DataTypes/DataTypeEnum.h index f25d387c818..61104e70505 100644 --- a/dbms/src/DataTypes/DataTypeEnum.h +++ b/dbms/src/DataTypes/DataTypeEnum.h @@ -61,7 +61,7 @@ public: explicit DataTypeEnum(const Values & 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; TypeIndex getTypeId() const override { return sizeof(FieldType) == 1 ? TypeIndex::Enum8 : TypeIndex::Enum16; } diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index 1ca74a69608..85df6bee260 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -73,25 +74,7 @@ DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr return get("LowCardinality", low_cardinality_params); } - { - 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); + return findCreatorByName(family_name)(parameters); } @@ -132,6 +115,49 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator }, 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 registerDataTypeDecimal(DataTypeFactory & factory); void registerDataTypeDate(DataTypeFactory & factory); @@ -148,6 +174,7 @@ void registerDataTypeAggregateFunction(DataTypeFactory & factory); void registerDataTypeNested(DataTypeFactory & factory); void registerDataTypeInterval(DataTypeFactory & factory); void registerDataTypeLowCardinality(DataTypeFactory & factory); +void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory); DataTypeFactory::DataTypeFactory() @@ -168,6 +195,10 @@ DataTypeFactory::DataTypeFactory() registerDataTypeNested(*this); registerDataTypeInterval(*this); registerDataTypeLowCardinality(*this); + registerDataTypeDomainIPv4AndIPv6(*this); } +DataTypeFactory::~DataTypeFactory() +{} + } diff --git a/dbms/src/DataTypes/DataTypeFactory.h b/dbms/src/DataTypes/DataTypeFactory.h index 21d22cf932e..95cda9002f4 100644 --- a/dbms/src/DataTypes/DataTypeFactory.h +++ b/dbms/src/DataTypes/DataTypeFactory.h @@ -14,6 +14,9 @@ namespace DB class IDataType; using DataTypePtr = std::shared_ptr; +class IDataTypeDomain; +using DataTypeDomainPtr = std::unique_ptr; + class IAST; using ASTPtr = std::shared_ptr; @@ -37,13 +40,24 @@ public: /// Register a simple data type, that have no parameters. 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: DataTypesDictionary data_types; /// Case insensitive data types will be additionally added here with lowercased name. DataTypesDictionary case_insensitive_data_types; + // All domains are owned by factory and shared amongst DataType instances. + std::vector all_domains; + DataTypeFactory(); + ~DataTypeFactory() override; const DataTypesDictionary & getCreatorMap() const override { return data_types; } diff --git a/dbms/src/DataTypes/DataTypeFixedString.cpp b/dbms/src/DataTypes/DataTypeFixedString.cpp index 6fbea93efbd..6edb9d2af4c 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.cpp +++ b/dbms/src/DataTypes/DataTypeFixedString.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes } -std::string DataTypeFixedString::getName() const +std::string DataTypeFixedString::doGetName() const { return "FixedString(" + toString(n) + ")"; } diff --git a/dbms/src/DataTypes/DataTypeFixedString.h b/dbms/src/DataTypes/DataTypeFixedString.h index c69d4859584..f929e09a7ee 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.h +++ b/dbms/src/DataTypes/DataTypeFixedString.h @@ -30,7 +30,7 @@ public: 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; } const char * getFamilyName() const override { return "FixedString"; } diff --git a/dbms/src/DataTypes/DataTypeFunction.cpp b/dbms/src/DataTypes/DataTypeFunction.cpp index 76b433e8de6..82f3d7ee515 100644 --- a/dbms/src/DataTypes/DataTypeFunction.cpp +++ b/dbms/src/DataTypes/DataTypeFunction.cpp @@ -6,7 +6,7 @@ namespace DB { -std::string DataTypeFunction::getName() const +std::string DataTypeFunction::doGetName() const { WriteBufferFromOwnString res; diff --git a/dbms/src/DataTypes/DataTypeFunction.h b/dbms/src/DataTypes/DataTypeFunction.h index 73089400a18..489ed4545f4 100644 --- a/dbms/src/DataTypes/DataTypeFunction.h +++ b/dbms/src/DataTypes/DataTypeFunction.h @@ -22,7 +22,7 @@ public: DataTypeFunction(const DataTypes & argument_types_ = DataTypes(), const DataTypePtr & return_type_ = nullptr) : 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"; } TypeIndex getTypeId() const override { return TypeIndex::Function; } diff --git a/dbms/src/DataTypes/DataTypeInterval.h b/dbms/src/DataTypes/DataTypeInterval.h index 5c9c15c48c8..22f088c01f8 100644 --- a/dbms/src/DataTypes/DataTypeInterval.h +++ b/dbms/src/DataTypes/DataTypeInterval.h @@ -56,7 +56,7 @@ public: 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"; } TypeIndex getTypeId() const override { return TypeIndex::Interval; } diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.h b/dbms/src/DataTypes/DataTypeLowCardinality.h index a099ce72fa8..8ba2a4d1c4d 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.h +++ b/dbms/src/DataTypes/DataTypeLowCardinality.h @@ -15,7 +15,7 @@ public: const DataTypePtr & getDictionaryType() const { return dictionary_type; } - String getName() const override + String doGetName() const override { 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 { - 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 { - 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 { - 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 { - 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 { - 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 { - 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 { - 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 { - 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 { - 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 { - 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 diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index f8c19b73bbe..3cf798724ac 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -172,7 +172,7 @@ void DataTypeNullable::serializeTextEscaped(const IColumn & column, size_t row_n if (col.isNullAt(row_num)) writeCString("\\N", ostr); 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, [] { 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 { @@ -214,7 +214,7 @@ void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & ist { /// We could step back to consume backslash again. --istr.position(); - nested_data_type->deserializeTextEscaped(nested, istr, settings); + nested_data_type->deserializeAsTextEscaped(nested, istr, settings); } else { @@ -222,7 +222,7 @@ void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & ist ReadBufferFromMemory prefix("\\", 1); 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. @@ -240,7 +240,7 @@ void DataTypeNullable::serializeTextQuoted(const IColumn & column, size_t row_nu if (col.isNullAt(row_num)) writeCString("NULL", ostr); 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, [&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 @@ -258,14 +258,14 @@ void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num, if (col.isNullAt(row_num)) writeCString("\\N", ostr); 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 { safeDeserialize(column, [&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 @@ -281,7 +281,7 @@ void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, Wri if (col.isNullAt(row_num)) writeCString("ᴺᵁᴸᴸ", ostr); 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 @@ -291,14 +291,14 @@ void DataTypeNullable::serializeTextJSON(const IColumn & column, size_t row_num, if (col.isNullAt(row_num)) writeCString("null", ostr); 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 { safeDeserialize(column, [&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 @@ -308,7 +308,7 @@ void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, if (col.isNullAt(row_num)) writeCString("\\N", ostr); 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 diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index c45736d06a4..20a6b195c84 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -14,7 +14,7 @@ public: static constexpr bool is_parametric = true; 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"; } TypeIndex getTypeId() const override { return TypeIndex::Nullable; } diff --git a/dbms/src/DataTypes/DataTypeTuple.cpp b/dbms/src/DataTypes/DataTypeTuple.cpp index 18ae891d576..1940c44134f 100644 --- a/dbms/src/DataTypes/DataTypeTuple.cpp +++ b/dbms/src/DataTypes/DataTypeTuple.cpp @@ -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(); WriteBufferFromOwnString s; @@ -160,7 +160,7 @@ void DataTypeTuple::serializeText(const IColumn & column, size_t row_num, WriteB { if (i != 0) writeChar(',', ostr); - elems[i]->serializeTextQuoted(extractElementColumn(column, i), row_num, ostr, settings); + elems[i]->serializeAsTextQuoted(extractElementColumn(column, i), row_num, ostr, settings); } writeChar(')', ostr); } @@ -180,7 +180,7 @@ void DataTypeTuple::deserializeText(IColumn & column, ReadBuffer & istr, const F assertChar(',', 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) writeChar(',', ostr); - elems[i]->serializeTextJSON(extractElementColumn(column, i), row_num, ostr, settings); + elems[i]->serializeAsTextJSON(extractElementColumn(column, i), row_num, ostr, settings); } writeChar(']', ostr); } @@ -215,7 +215,7 @@ void DataTypeTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr, con assertChar(',', 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))) { writeCString("", ostr); - elems[i]->serializeTextXML(extractElementColumn(column, i), row_num, ostr, settings); + elems[i]->serializeAsTextXML(extractElementColumn(column, i), row_num, ostr, settings); writeCString("", ostr); } writeCString("", ostr); @@ -241,7 +241,7 @@ void DataTypeTuple::serializeTextCSV(const IColumn & column, size_t row_num, Wri { if (i != 0) 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); skipWhitespaceIfAny(istr); } - elems[i]->deserializeTextCSV(extractElementColumn(column, i), istr, settings); + elems[i]->deserializeAsTextCSV(extractElementColumn(column, i), istr, settings); } }); } diff --git a/dbms/src/DataTypes/DataTypeTuple.h b/dbms/src/DataTypes/DataTypeTuple.h index c3422be70db..d489ae4ba7f 100644 --- a/dbms/src/DataTypes/DataTypeTuple.h +++ b/dbms/src/DataTypes/DataTypeTuple.h @@ -29,7 +29,7 @@ public: DataTypeTuple(const DataTypes & elems, const Strings & names); TypeIndex getTypeId() const override { return TypeIndex::Tuple; } - std::string getName() const override; + std::string doGetName() const override; const char * getFamilyName() const override { return "Tuple"; } bool canBeInsideNullable() const override { return false; } diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp index e6fe1bbd7f2..b9ccb41af3d 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ b/dbms/src/DataTypes/DataTypesDecimal.cpp @@ -28,7 +28,7 @@ bool decimalCheckArithmeticOverflow(const Context & context) { return context.ge // template -std::string DataTypeDecimal::getName() const +std::string DataTypeDecimal::doGetName() const { std::stringstream ss; ss << "Decimal(" << precision << ", " << scale << ")"; diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h index c287742928a..89841b762c1 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ b/dbms/src/DataTypes/DataTypesDecimal.h @@ -86,7 +86,7 @@ public: } const char * getFamilyName() const override { return "Decimal"; } - std::string getName() const override; + std::string doGetName() const override; TypeIndex getTypeId() const override { return TypeId::value; } void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/dbms/src/DataTypes/IDataType.cpp b/dbms/src/DataTypes/IDataType.cpp index a8eec08a7c8..679871dba71 100644 --- a/dbms/src/DataTypes/IDataType.cpp +++ b/dbms/src/DataTypes/IDataType.cpp @@ -9,6 +9,7 @@ #include #include +#include #include @@ -22,6 +23,31 @@ namespace ErrorCodes 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) { @@ -114,4 +140,133 @@ void IDataType::insertDefaultInto(IColumn & column) const 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; +} + } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index ae14fe1aa36..6bb59106d10 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -12,6 +12,7 @@ namespace DB class ReadBuffer; class WriteBuffer; +class IDataTypeDomain; class IDataType; struct FormatSettings; @@ -35,6 +36,9 @@ class ProtobufWriter; class IDataType : private boost::noncopyable { public: + IDataType(); + virtual ~IDataType(); + /// 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: DataTypeFixedString is parametric: different instances of DataTypeFixedString may be different SQL types. @@ -42,7 +46,7 @@ public: /// static constexpr bool is_parametric = false; /// 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). 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. 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. */ 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. */ 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; /** 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); } - /** Serialize to a protobuf. */ - virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const = 0; - +public: /** Create empty column for corresponding type. */ virtual MutableColumnPtr createColumn() const = 0; @@ -290,8 +325,6 @@ public: /// Checks that two instances belong to the same type virtual bool equals(const IDataType & rhs) const = 0; - virtual ~IDataType() {} - /// Various properties on behaviour of data type. @@ -419,6 +452,21 @@ public: static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); 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; }; diff --git a/dbms/src/DataTypes/IDataTypeDomain.h b/dbms/src/DataTypes/IDataTypeDomain.h new file mode 100644 index 00000000000..ad38e88a213 --- /dev/null +++ b/dbms/src/DataTypes/IDataTypeDomain.h @@ -0,0 +1,59 @@ +#pragma once + +#include + +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 diff --git a/dbms/src/Dictionaries/DictionaryStructure.cpp b/dbms/src/Dictionaries/DictionaryStructure.cpp index 49d6eb770f0..95fce1cda19 100644 --- a/dbms/src/Dictionaries/DictionaryStructure.cpp +++ b/dbms/src/Dictionaries/DictionaryStructure.cpp @@ -355,7 +355,7 @@ std::vector DictionaryStructure::getAttributes( { ReadBufferFromString null_value_buffer{null_value_string}; 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]; } } diff --git a/dbms/src/Dictionaries/ExternalQueryBuilder.cpp b/dbms/src/Dictionaries/ExternalQueryBuilder.cpp index 2f822c76ee2..029229cf9a4 100644 --- a/dbms/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/dbms/src/Dictionaries/ExternalQueryBuilder.cpp @@ -345,7 +345,7 @@ void ExternalQueryBuilder::composeKeyCondition(const Columns & key_columns, cons /// key_i=value_i writeString(key_description.name, 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); @@ -387,7 +387,7 @@ void ExternalQueryBuilder::composeKeyTuple(const Columns & key_columns, const si writeString(", ", out); 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); diff --git a/dbms/src/Formats/CSVRowInputStream.cpp b/dbms/src/Formats/CSVRowInputStream.cpp index d7f8c6eea28..1057e633445 100644 --- a/dbms/src/Formats/CSVRowInputStream.cpp +++ b/dbms/src/Formats/CSVRowInputStream.cpp @@ -123,7 +123,7 @@ bool CSVRowInputStream::read(MutableColumns & columns, RowReadExtension &) for (size_t i = 0; i < size; ++i) { skipWhitespacesAndTabs(istr); - data_types[i]->deserializeTextCSV(*columns[i], istr, format_settings); + data_types[i]->deserializeAsTextCSV(*columns[i], istr, format_settings); skipWhitespacesAndTabs(istr); skipDelimiter(istr, format_settings.csv.delimiter, i + 1 == size); @@ -215,7 +215,7 @@ bool CSVRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, { skipWhitespacesAndTabs(istr); 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(); skipWhitespacesAndTabs(istr); } diff --git a/dbms/src/Formats/CSVRowOutputStream.cpp b/dbms/src/Formats/CSVRowOutputStream.cpp index ed820a36016..94ed300eaf8 100644 --- a/dbms/src/Formats/CSVRowOutputStream.cpp +++ b/dbms/src/Formats/CSVRowOutputStream.cpp @@ -42,7 +42,7 @@ void CSVRowOutputStream::writePrefix() 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); } diff --git a/dbms/src/Formats/JSONCompactRowOutputStream.cpp b/dbms/src/Formats/JSONCompactRowOutputStream.cpp index 04edadf019b..b9d54604c51 100644 --- a/dbms/src/Formats/JSONCompactRowOutputStream.cpp +++ b/dbms/src/Formats/JSONCompactRowOutputStream.cpp @@ -16,7 +16,7 @@ JSONCompactRowOutputStream::JSONCompactRowOutputStream(WriteBuffer & ostr_, cons 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; } @@ -58,7 +58,7 @@ void JSONCompactRowOutputStream::writeTotals() writeChar(',', *ostr); 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); @@ -79,7 +79,7 @@ static void writeExtremesElement(const char * title, const Block & extremes, siz writeChar(',', ostr); 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); diff --git a/dbms/src/Formats/JSONEachRowRowInputStream.cpp b/dbms/src/Formats/JSONEachRowRowInputStream.cpp index ad67b7a8101..808edbf6976 100644 --- a/dbms/src/Formats/JSONEachRowRowInputStream.cpp +++ b/dbms/src/Formats/JSONEachRowRowInputStream.cpp @@ -134,7 +134,7 @@ void JSONEachRowRowInputStream::readField(size_t index, MutableColumns & columns try { - header.getByPosition(index).type->deserializeTextJSON(*columns[index], istr, format_settings); + header.getByPosition(index).type->deserializeAsTextJSON(*columns[index], istr, format_settings); } catch (Exception & e) { diff --git a/dbms/src/Formats/JSONEachRowRowOutputStream.cpp b/dbms/src/Formats/JSONEachRowRowOutputStream.cpp index 40508b36856..6850060dbfb 100644 --- a/dbms/src/Formats/JSONEachRowRowOutputStream.cpp +++ b/dbms/src/Formats/JSONEachRowRowOutputStream.cpp @@ -27,7 +27,7 @@ void JSONEachRowRowOutputStream::writeField(const IColumn & column, const IDataT { writeString(fields[field_number], ostr); writeChar(':', ostr); - type.serializeTextJSON(column, row_num, ostr, settings); + type.serializeAsTextJSON(column, row_num, ostr, settings); ++field_number; } diff --git a/dbms/src/Formats/JSONRowOutputStream.cpp b/dbms/src/Formats/JSONRowOutputStream.cpp index 89f57669a3e..942b231fc8e 100644 --- a/dbms/src/Formats/JSONRowOutputStream.cpp +++ b/dbms/src/Formats/JSONRowOutputStream.cpp @@ -71,7 +71,7 @@ void JSONRowOutputStream::writeField(const IColumn & column, const IDataType & t writeCString("\t\t\t", *ostr); writeString(fields[field_number].name, *ostr); writeCString(": ", *ostr); - type.serializeTextJSON(column, row_num, *ostr, settings); + type.serializeAsTextJSON(column, row_num, *ostr, settings); ++field_number; } @@ -151,7 +151,7 @@ void JSONRowOutputStream::writeTotals() writeCString("\t\t", *ostr); writeJSONString(column.name, *ostr, settings); writeCString(": ", *ostr); - column.type->serializeTextJSON(*column.column.get(), 0, *ostr, settings); + column.type->serializeAsTextJSON(*column.column.get(), 0, *ostr, settings); } writeChar('\n', *ostr); @@ -178,7 +178,7 @@ static void writeExtremesElement(const char * title, const Block & extremes, siz writeCString("\t\t\t", ostr); writeJSONString(column.name, ostr, settings); 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); diff --git a/dbms/src/Formats/ODBCDriver2BlockOutputStream.cpp b/dbms/src/Formats/ODBCDriver2BlockOutputStream.cpp index 7e1061c3a4a..0d2ba8b8a53 100644 --- a/dbms/src/Formats/ODBCDriver2BlockOutputStream.cpp +++ b/dbms/src/Formats/ODBCDriver2BlockOutputStream.cpp @@ -43,7 +43,7 @@ static void writeRow(const Block & block, size_t row_idx, WriteBuffer & out, con { { 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); } diff --git a/dbms/src/Formats/ODBCDriverBlockOutputStream.cpp b/dbms/src/Formats/ODBCDriverBlockOutputStream.cpp index 96252e67ec7..3cd1ad3c1d6 100644 --- a/dbms/src/Formats/ODBCDriverBlockOutputStream.cpp +++ b/dbms/src/Formats/ODBCDriverBlockOutputStream.cpp @@ -33,7 +33,7 @@ void ODBCDriverBlockOutputStream::write(const Block & block) { 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); diff --git a/dbms/src/Formats/PrettyBlockOutputStream.cpp b/dbms/src/Formats/PrettyBlockOutputStream.cpp index fe102ea5739..9c4c9a20965 100644 --- a/dbms/src/Formats/PrettyBlockOutputStream.cpp +++ b/dbms/src/Formats/PrettyBlockOutputStream.cpp @@ -58,7 +58,7 @@ void PrettyBlockOutputStream::calculateWidths( { { 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(format_settings.pretty.max_column_pad_width, @@ -206,11 +206,11 @@ void PrettyBlockOutputStream::writeValueWithPadding(const ColumnWithTypeAndName if (elem.type->shouldAlignRightInPrettyFormats()) { writePadding(); - elem.type->serializeText(*elem.column.get(), row_num, ostr, format_settings); + elem.type->serializeAsText(*elem.column.get(), row_num, ostr, format_settings); } else { - elem.type->serializeText(*elem.column.get(), row_num, ostr, format_settings); + elem.type->serializeAsText(*elem.column.get(), row_num, ostr, format_settings); writePadding(); } } diff --git a/dbms/src/Formats/TSKVRowInputStream.cpp b/dbms/src/Formats/TSKVRowInputStream.cpp index 4416fcbcbcc..d85e5b9450e 100644 --- a/dbms/src/Formats/TSKVRowInputStream.cpp +++ b/dbms/src/Formats/TSKVRowInputStream.cpp @@ -135,7 +135,7 @@ bool TSKVRowInputStream::read(MutableColumns & columns, RowReadExtension & ext) 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 diff --git a/dbms/src/Formats/TSKVRowOutputStream.cpp b/dbms/src/Formats/TSKVRowOutputStream.cpp index 96be0054363..edf952b45e3 100644 --- a/dbms/src/Formats/TSKVRowOutputStream.cpp +++ b/dbms/src/Formats/TSKVRowOutputStream.cpp @@ -28,7 +28,7 @@ TSKVRowOutputStream::TSKVRowOutputStream(WriteBuffer & ostr_, const Block & samp void TSKVRowOutputStream::writeField(const IColumn & column, const IDataType & type, size_t row_num) { writeString(fields[field_number].name, ostr); - type.serializeTextEscaped(column, row_num, ostr, format_settings); + type.serializeAsTextEscaped(column, row_num, ostr, format_settings); ++field_number; } diff --git a/dbms/src/Formats/TabSeparatedRawRowOutputStream.h b/dbms/src/Formats/TabSeparatedRawRowOutputStream.h index a6a8054e383..4be0c0d2445 100644 --- a/dbms/src/Formats/TabSeparatedRawRowOutputStream.h +++ b/dbms/src/Formats/TabSeparatedRawRowOutputStream.h @@ -18,7 +18,7 @@ public: 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); } }; diff --git a/dbms/src/Formats/TabSeparatedRowInputStream.cpp b/dbms/src/Formats/TabSeparatedRowInputStream.cpp index 181f29113db..f0c232ead0a 100644 --- a/dbms/src/Formats/TabSeparatedRowInputStream.cpp +++ b/dbms/src/Formats/TabSeparatedRowInputStream.cpp @@ -86,7 +86,7 @@ bool TabSeparatedRowInputStream::read(MutableColumns & columns, RowReadExtension 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 if (i + 1 == size) @@ -183,7 +183,7 @@ bool TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns & try { - data_types[i]->deserializeTextEscaped(*columns[i], istr, format_settings); + data_types[i]->deserializeAsTextEscaped(*columns[i], istr, format_settings); } catch (...) { diff --git a/dbms/src/Formats/TabSeparatedRowOutputStream.cpp b/dbms/src/Formats/TabSeparatedRowOutputStream.cpp index b3a498708a3..dca1207dd24 100644 --- a/dbms/src/Formats/TabSeparatedRowOutputStream.cpp +++ b/dbms/src/Formats/TabSeparatedRowOutputStream.cpp @@ -48,7 +48,7 @@ void TabSeparatedRowOutputStream::writePrefix() 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); } diff --git a/dbms/src/Formats/ValuesRowInputStream.cpp b/dbms/src/Formats/ValuesRowInputStream.cpp index eaa8181d185..a48736bad6f 100644 --- a/dbms/src/Formats/ValuesRowInputStream.cpp +++ b/dbms/src/Formats/ValuesRowInputStream.cpp @@ -64,7 +64,7 @@ bool ValuesRowInputStream::read(MutableColumns & columns, RowReadExtension &) bool rollback_on_exception = false; 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; skipWhitespaceIfAny(istr); diff --git a/dbms/src/Formats/ValuesRowOutputStream.cpp b/dbms/src/Formats/ValuesRowOutputStream.cpp index bbd114bfa5a..eacc7e31eba 100644 --- a/dbms/src/Formats/ValuesRowOutputStream.cpp +++ b/dbms/src/Formats/ValuesRowOutputStream.cpp @@ -23,7 +23,7 @@ void ValuesRowOutputStream::flush() 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() diff --git a/dbms/src/Formats/VerticalRowOutputStream.cpp b/dbms/src/Formats/VerticalRowOutputStream.cpp index aafbadb8e59..6a994cf9303 100644 --- a/dbms/src/Formats/VerticalRowOutputStream.cpp +++ b/dbms/src/Formats/VerticalRowOutputStream.cpp @@ -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 { - type.serializeText(column, row_num, ostr, format_settings); + type.serializeAsText(column, row_num, ostr, format_settings); } diff --git a/dbms/src/Formats/XMLRowOutputStream.cpp b/dbms/src/Formats/XMLRowOutputStream.cpp index 7bd5a5d4c84..8b216cdf1de 100644 --- a/dbms/src/Formats/XMLRowOutputStream.cpp +++ b/dbms/src/Formats/XMLRowOutputStream.cpp @@ -87,7 +87,7 @@ void XMLRowOutputStream::writeField(const IColumn & column, const IDataType & ty writeCString("\t\t\t<", *ostr); writeString(field_tag_names[field_number], *ostr); writeCString(">", *ostr); - type.serializeTextXML(column, row_num, *ostr, format_settings); + type.serializeAsTextXML(column, row_num, *ostr, format_settings); writeCString("\n", *ostr); @@ -153,7 +153,7 @@ void XMLRowOutputStream::writeTotals() writeCString("\t\t<", *ostr); writeString(field_tag_names[i], *ostr); writeCString(">", *ostr); - column.type->serializeTextXML(*column.column.get(), 0, *ostr, format_settings); + column.type->serializeAsTextXML(*column.column.get(), 0, *ostr, format_settings); writeCString("\n", *ostr); @@ -179,7 +179,7 @@ static void writeExtremesElement( writeCString("\t\t\t<", ostr); writeString(field_tag_names[i], 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("\n", ostr); diff --git a/dbms/src/Functions/FunctionsCoding.cpp b/dbms/src/Functions/FunctionsCoding.cpp index 7643e77b205..8da8f577efc 100644 --- a/dbms/src/Functions/FunctionsCoding.cpp +++ b/dbms/src/Functions/FunctionsCoding.cpp @@ -26,6 +26,8 @@ void registerFunctionsCoding(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 8742c90cf6c..1483792ffdc 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -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_text_length = 36; @@ -74,10 +73,10 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { const auto ptr = checkAndGetDataType(arguments[0].get()); - if (!ptr || ptr->getN() != ipv6_bytes_length) + if (!ptr || ptr->getN() != IPV6_BINARY_LENGTH) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + - ", expected FixedString(" + toString(ipv6_bytes_length) + ")", + ", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); @@ -92,11 +91,11 @@ public: if (const auto col_in = checkAndGetColumn(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() + " of column " + col_in->getName() + " argument of function " + getName() + - ", expected FixedString(" + toString(ipv6_bytes_length) + ")", + ", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); const auto size = col_in->size(); @@ -112,7 +111,7 @@ public: auto begin = reinterpret_cast(vec_res.data()); 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); offsets_res[i] = pos - begin; @@ -143,10 +142,10 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { const auto ptr = checkAndGetDataType(arguments[0].get()); - if (!ptr || ptr->getN() != ipv6_bytes_length) + if (!ptr || ptr->getN() != IPV6_BINARY_LENGTH) throw Exception("Illegal type " + arguments[0]->getName() + " of argument 1 of function " + getName() + - ", expected FixedString(" + toString(ipv6_bytes_length) + ")", + ", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!WhichDataType(arguments[1]).isUInt8()) @@ -177,11 +176,11 @@ public: if (const auto col_in = checkAndGetColumn(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() + " of column " + col_in->getName() + " argument of function " + getName() + - ", expected FixedString(" + toString(ipv6_bytes_length) + ")", + ", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); const auto ipv6_zeroed_tail_bytes = checkAndGetColumnConst>(col_ipv6_zeroed_tail_bytes.get()); @@ -191,7 +190,7 @@ public: ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); UInt8 ipv6_zeroed_tail_bytes_count = ipv6_zeroed_tail_bytes->getValue(); - 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() + " of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -203,7 +202,7 @@ public: ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); UInt8 ipv4_zeroed_tail_bytes_count = ipv4_zeroed_tail_bytes->getValue(); - 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() + " of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -221,7 +220,7 @@ public: auto begin = reinterpret_cast(vec_res.data()); 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]; 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(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(ipv6_bytes_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((val >> 8) & 0xffu); - *tp++ = static_cast(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((val >> 8) & 0xffu); - *tp++ = static_cast(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 std::make_shared(IPV6_BINARY_LENGTH); } bool useDefaultImplementationForConstants() const override { return true; } @@ -419,10 +279,10 @@ public: if (const auto col_in = checkAndGetColumn(column.get())) { - auto col_res = ColumnFixedString::create(ipv6_bytes_length); + auto col_res = ColumnFixedString::create(IPV6_BINARY_LENGTH); 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::Offsets & offsets_src = col_in->getOffsets(); @@ -430,9 +290,10 @@ public: for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); - out_offset += ipv6_bytes_length, ++i) + out_offset += IPV6_BINARY_LENGTH, ++i) { - ipv6_scan(reinterpret_cast(&vec_src[src_offset]), &vec_res[out_offset]); + //TODO(nemkov): handle failure ? + parseIPv6(reinterpret_cast(&vec_src[src_offset]), &vec_res[out_offset]); src_offset = offsets_src[i]; } @@ -451,59 +312,6 @@ public: template 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(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: static constexpr auto name = Name::name; static FunctionPtr create(const Context &) { return std::make_shared>(); } @@ -547,7 +355,7 @@ public: for (size_t i = 0; i < vec_in.size(); ++i) { - formatIP(vec_in[i], pos); + DB::formatIPv4(reinterpret_cast(&vec_in[i]), pos, mask_tail_octets, "xxx"); offsets_res[i] = pos - begin; } @@ -585,27 +393,12 @@ public: return std::make_shared(); } - static UInt32 parseIPv4(const char * pos) + static inline UInt32 parseIPv4(const char * pos) { - UInt32 res = 0; - for (int offset = 24; offset >= 0; offset -= 8) - { - UInt32 value = 0; - 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; + UInt32 result = 0; + DB::parseIPv4(pos, reinterpret_cast(&result)); + + return result; } bool useDefaultImplementationForConstants() const override { return true; } @@ -670,14 +463,14 @@ public: if (const auto col_in = typeid_cast(column.get())) { - auto col_res = ColumnFixedString::create(ipv6_bytes_length); + auto col_res = ColumnFixedString::create(IPV6_BINARY_LENGTH); 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(); - 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]); 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(); } + + 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(); } + + 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 { diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index e5a2994857f..260939d121f 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -347,7 +347,7 @@ struct ConvertImplGenericToString FormatSettings format_settings; 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); offsets_to[i] = write_buffer.count(); } @@ -631,7 +631,7 @@ struct ConvertImplGenericFromString { 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()) throwExceptionForIncompletelyParsedValue(read_buffer, block, result); diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 3ef24597faa..23d6fc40523 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -204,8 +204,6 @@ inline float roundWithMode(float x, RoundingMode mode) case RoundingMode::Floor: return floorf(x); case RoundingMode::Ceil: return ceilf(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::Ceil: return ceil(x); case RoundingMode::Trunc: return trunc(x); - default: - throw Exception("Logical error: unexpected 'mode' parameter passed to function roundWithMode", ErrorCodes::LOGICAL_ERROR); } } diff --git a/dbms/src/Functions/arrayEnumerateExtended.h b/dbms/src/Functions/arrayEnumerateExtended.h index 9d9c52ab6ea..72958574e83 100644 --- a/dbms/src/Functions/arrayEnumerateExtended.h +++ b/dbms/src/Functions/arrayEnumerateExtended.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -60,11 +61,56 @@ private: /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. static constexpr size_t INITIAL_SIZE_DEGREE = 9; + template + struct MethodOneNumber + { + using Set = ClearableHashMap, HashTableGrower, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; + using Method = ColumnsHashing::HashMethodOneNumber; + }; + + struct MethodString + { + using Set = ClearableHashMap, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; + using Method = ColumnsHashing::HashMethodString; + }; + + struct MethodFixedString + { + using Set = ClearableHashMap, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; + using Method = ColumnsHashing::HashMethodFixedString; + }; + + struct MethodFixed + { + using Set = ClearableHashMap, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; + using Method = ColumnsHashing::HashMethodKeysFixed; + }; + + struct MethodHashed + { + using Set = ClearableHashMap, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; + using Method = ColumnsHashing::HashMethodHashed; + }; + + template + void executeMethod(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, const Sizes & key_sizes, + const NullMap * null_map, ColumnUInt32::Container & res_values); + + template + void executeMethodImpl(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, const Sizes & key_sizes, + const NullMap * null_map, ColumnUInt32::Container & res_values); + template 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 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 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::executeImpl(Block & block, const C if (num_arguments == 1) { - executeNumber(*offsets, *data_columns[0], null_map, res_values) + if (!(executeNumber(*offsets, *data_columns[0], null_map, res_values) || executeNumber(*offsets, *data_columns[0], null_map, res_values) || executeNumber(*offsets, *data_columns[0], null_map, res_values) || executeNumber(*offsets, *data_columns[0], null_map, res_values) @@ -142,47 +188,56 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C || executeNumber(*offsets, *data_columns[0], null_map, res_values) || executeNumber(*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 { - execute128bit(*offsets, data_columns, res_values) - || executeHashed(*offsets, data_columns, res_values); + if (!execute128bit(*offsets, data_columns, res_values)) + executeHashed(*offsets, data_columns, res_values); } block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), offsets_column); } - template -template -bool FunctionArrayEnumerateExtended::executeNumber( - const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) +template +void FunctionArrayEnumerateExtended::executeMethodImpl( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + const Sizes & key_sizes, + [[maybe_unused]] const NullMap * null_map, + ColumnUInt32::Container & res_values) { - const ColumnVector * data_concrete = checkAndGetColumn>(&data); - if (!data_concrete) - return false; - const auto & values = data_concrete->getData(); + typename Method::Set indices; + typename Method::Method method(columns, key_sizes, nullptr); + Arena pool; /// Won't use it; - using ValuesToIndices = ClearableHashMap, HashTableGrower, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; + ColumnArray::Offset prev_off = 0; - ValuesToIndices indices; - size_t prev_off = 0; if constexpr (std::is_same_v) { // Unique - for (size_t i = 0; i < offsets.size(); ++i) + for (size_t off : offsets) { indices.clear(); UInt32 null_count = 0; - size_t off = offsets[i]; 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[j]]; + if constexpr (has_null_map) + { + if ((*null_map)[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; } @@ -190,31 +245,67 @@ bool FunctionArrayEnumerateExtended::executeNumber( else { // Dense - for (size_t i = 0; i < offsets.size(); ++i) + for (size_t off : offsets) { indices.clear(); - size_t rank = 0; - UInt32 null_index = 0; - size_t off = offsets[i]; + UInt32 rank = 0; + [[maybe_unused]] UInt32 null_index = 0; for (size_t j = prev_off; j < off; ++j) { - if (null_map && (*null_map)[j]) + if constexpr (has_null_map) { - if (!null_index) - null_index = ++rank; - res_values[j] = null_index; + if ((*null_map)[j]) + { + 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]]; - if (!idx) - idx = ++rank; - res_values[j] = idx; + idx = ++rank; + emplace_result.setMapped(idx); } + + res_values[j] = idx; } prev_off = off; } } +} + +template +template +void FunctionArrayEnumerateExtended::executeMethod( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + const Sizes & key_sizes, + const NullMap * null_map, + ColumnUInt32::Container & res_values) +{ + if (null_map) + executeMethodImpl(offsets, columns, key_sizes, null_map, res_values); + else + executeMethodImpl(offsets, columns, key_sizes, null_map, res_values); + +} + +template +template +bool FunctionArrayEnumerateExtended::executeNumber( + const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) +{ + const auto * nested = checkAndGetColumn>(&data); + if (!nested) + return false; + + executeMethod>(offsets, {nested}, {}, null_map, res_values); return true; } @@ -222,62 +313,22 @@ template bool FunctionArrayEnumerateExtended::executeString( const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) { - const ColumnString * values = checkAndGetColumn(&data); - if (!values) - return false; + const auto * nested = checkAndGetColumn(&data); + if (nested) + executeMethod(offsets, {nested}, {}, null_map, res_values); - size_t prev_off = 0; - using ValuesToIndices = ClearableHashMap, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; + return nested; +} - ValuesToIndices indices; - if constexpr (std::is_same_v) - { - // Unique - for (size_t i = 0; i < offsets.size(); ++i) - { - indices.clear(); - UInt32 null_count = 0; - size_t off = offsets[i]; - 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 +bool FunctionArrayEnumerateExtended::executeFixedString( + const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) +{ + const auto * nested = checkAndGetColumn(&data); + if (nested) + executeMethod(offsets, {nested}, {}, null_map, res_values); + + return nested; } template @@ -298,95 +349,17 @@ bool FunctionArrayEnumerateExtended::execute128bit( keys_bytes += key_sizes[j]; } - if (keys_bytes > 16) - return false; - - using ValuesToIndices = ClearableHashMap, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; - - ValuesToIndices indices; - size_t prev_off = 0; - if constexpr (std::is_same_v) - { - // 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(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(j, count, columns, key_sizes)]; - if (!idx) - idx = ++rank; - res_values[j] = idx; - } - prev_off = off; - } - } - + executeMethod(offsets, columns, key_sizes, nullptr, res_values); return true; } template -bool FunctionArrayEnumerateExtended::executeHashed( +void FunctionArrayEnumerateExtended::executeHashed( const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values) { - size_t count = columns.size(); - - using ValuesToIndices = ClearableHashMap, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; - - ValuesToIndices indices; - size_t prev_off = 0; - if constexpr (std::is_same_v) - { - // 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; + executeMethod(offsets, columns, {}, nullptr, res_values); } } diff --git a/dbms/src/Functions/arrayUniq.cpp b/dbms/src/Functions/arrayUniq.cpp index 93af86a3441..8e3dc0dbf0e 100644 --- a/dbms/src/Functions/arrayUniq.cpp +++ b/dbms/src/Functions/arrayUniq.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -62,11 +63,56 @@ private: /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. static constexpr size_t INITIAL_SIZE_DEGREE = 9; + template + struct MethodOneNumber + { + using Set = ClearableHashSet, HashTableGrower, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; + using Method = ColumnsHashing::HashMethodOneNumber; + }; + + struct MethodString + { + using Set = ClearableHashSet, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; + using Method = ColumnsHashing::HashMethodString; + }; + + struct MethodFixedString + { + using Set = ClearableHashSet, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; + using Method = ColumnsHashing::HashMethodFixedString; + }; + + struct MethodFixed + { + using Set = ClearableHashSet, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; + using Method = ColumnsHashing::HashMethodKeysFixed; + }; + + struct MethodHashed + { + using Set = ClearableHashSet, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; + using Method = ColumnsHashing::HashMethodHashed; + }; + + template + void executeMethod(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, const Sizes & key_sizes, + const NullMap * null_map, ColumnUInt32::Container & res_values); + + template + void executeMethodImpl(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, const Sizes & key_sizes, + const NullMap * null_map, ColumnUInt32::Container & res_values); + template 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 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 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) { - executeNumber(*offsets, *data_columns[0], null_map, res_values) + if (!(executeNumber(*offsets, *data_columns[0], null_map, res_values) || executeNumber(*offsets, *data_columns[0], null_map, res_values) || executeNumber(*offsets, *data_columns[0], null_map, res_values) || executeNumber(*offsets, *data_columns[0], null_map, res_values) @@ -136,30 +182,31 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen || executeNumber(*offsets, *data_columns[0], null_map, res_values) || executeNumber(*offsets, *data_columns[0], null_map, res_values) || executeNumber(*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) + || executeString(*offsets, *data_columns[0], null_map, res_values))) + executeHashed(*offsets, data_columns, res_values); } else { - execute128bit(*offsets, data_columns, res_values) - || executeHashed(*offsets, data_columns, res_values); + if (!execute128bit(*offsets, data_columns, res_values)) + executeHashed(*offsets, data_columns, res_values); } block.getByPosition(result).column = std::move(res); } -template -bool FunctionArrayUniq::executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) +template +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 * nested = checkAndGetColumn>(&data); - if (!nested) - return false; - const auto & values = nested->getData(); + typename Method::Set set; + typename Method::Method method(columns, key_sizes, nullptr); + Arena pool; /// Won't use it; - using Set = ClearableHashSet, HashTableGrower, - HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; - - Set set; ColumnArray::Offset prev_off = 0; 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]; for (ColumnArray::Offset j = prev_off; j < off; ++j) { - if (null_map && (*null_map)[j]) - found_null = true; - else - set.insert(values[j]); + if constexpr (has_null_map) + { + if ((*null_map)[j]) + { + found_null = true; + continue; + } + } + + method.emplaceKey(set, j, pool); } res_values[i] = set.size() + found_null; prev_off = off; } +} + +template +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(offsets, columns, key_sizes, null_map, res_values); + else + executeMethodImpl(offsets, columns, key_sizes, null_map, res_values); + +} + +template +bool FunctionArrayUniq::executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) +{ + const auto * nested = checkAndGetColumn>(&data); + if (!nested) + return false; + + executeMethod>(offsets, {nested}, {}, null_map, res_values); return true; } bool FunctionArrayUniq::executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) { - const ColumnString * nested = checkAndGetColumn(&data); - if (!nested) - return false; + const auto * nested = checkAndGetColumn(&data); + if (nested) + executeMethod(offsets, {nested}, {}, null_map, res_values); - using Set = ClearableHashSet, - 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; + return nested; } +bool FunctionArrayUniq::executeFixedString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) +{ + const auto * nested = checkAndGetColumn(&data); + if (nested) + executeMethod(offsets, {nested}, {}, null_map, res_values); + + return nested; +} bool FunctionArrayUniq::execute128bit( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - ColumnUInt32::Container & res_values) + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + ColumnUInt32::Container & res_values) { size_t count = columns.size(); size_t keys_bytes = 0; @@ -231,49 +296,16 @@ bool FunctionArrayUniq::execute128bit( if (keys_bytes > 16) return false; - using Set = ClearableHashSet, - 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(j, count, columns, key_sizes)); - - res_values[i] = set.size(); - prev_off = off; - } - + executeMethod(offsets, columns, key_sizes, nullptr, res_values); return true; } -bool FunctionArrayUniq::executeHashed( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - ColumnUInt32::Container & res_values) +void FunctionArrayUniq::executeHashed( + const ColumnArray::Offsets & offsets, + const ColumnRawPtrs & columns, + ColumnUInt32::Container & res_values) { - size_t count = columns.size(); - - using Set = ClearableHashSet, - 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; + executeMethod(offsets, columns, {}, nullptr, res_values); } diff --git a/dbms/src/Functions/visibleWidth.cpp b/dbms/src/Functions/visibleWidth.cpp index 2380bf9a993..c03bdff071d 100644 --- a/dbms/src/Functions/visibleWidth.cpp +++ b/dbms/src/Functions/visibleWidth.cpp @@ -60,7 +60,7 @@ public: { { 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(tmp.data()), tmp.size()); diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 3fb94397956..515b5a4b1eb 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -6,9 +6,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -22,11 +24,9 @@ #include #include #include + #if __has_include() #include -#include -#include - #endif @@ -188,7 +188,7 @@ Aggregator::Aggregator(const Params & params_) } method_chosen = chooseAggregationMethod(); - AggregationStateCache::Settings cache_settings; + HashMethodContext::Settings cache_settings; cache_settings.max_threads = params.max_threads; aggregation_state_cache = AggregatedDataVariants::createCache(method_chosen, cache_settings); } @@ -586,11 +586,7 @@ void NO_INLINE Aggregator::executeImpl( bool no_more_keys, AggregateDataPtr overflow_row) const { - typename Method::State state; - if constexpr (Method::low_cardinality_optimization) - state.init(key_columns, aggregation_state_cache); - else - state.init(key_columns); + typename Method::State state(key_columns, key_sizes, aggregation_state_cache); if (!no_more_keys) executeImplCase(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, Arena * aggregates_pool, size_t rows, - ColumnRawPtrs & key_columns, + ColumnRawPtrs & /*key_columns*/, AggregateFunctionInstruction * aggregate_instructions, - StringRefs & keys, + StringRefs & /*keys*/, AggregateDataPtr overflow_row) const { /// NOTE When editing this code, also pay attention to SpecializedAggregator.h. /// For all rows. - typename Method::Key prev_key{}; - AggregateDataPtr value = nullptr; 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 constexpr (!no_more_keys) /// Insert. { - /// Optimization for consecutive identical keys. - if (!Method::no_consecutive_keys_optimization) + auto emplace_result = state.emplaceKey(method.data, i, *aggregates_pool); + + /// 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) - { - /// 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); + /// exception-safety - if you can not allocate memory or create states, then destructors will not be called. + emplace_result.setMapped(nullptr); - method.onExistingKey(key, keys, *aggregates_pool); - continue; - } - else - prev_key = key; + aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(aggregate_data); + + 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 - { - method.data.emplace(key, it, inserted); - aggregate_data = &Method::getAggregateData(it->second); - } + aggregate_data = emplace_result.getMapped(); } else { /// Add only if the key already exists. - - if constexpr (Method::low_cardinality_optimization) - aggregate_data = state.findFromRow(method.data, i); - else - { - it = method.data.find(key); - if (method.data.end() != it) - aggregate_data = &Method::getAggregateData(it->second); - } + auto find_result = state.findKey(method.data, i, *aggregates_pool); + if (find_result.isFound()) + aggregate_data = find_result.getMapped(); } /// 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 (!aggregate_data && !overflow_row) - { - method.onExistingKey(key, keys, *aggregates_pool); continue; - } - /// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key. - 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; + AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row; /// Add values to the aggregate functions. 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) aggregate_functions[i]->insertResultInto( - Method::getAggregateData(value.second) + offsets_of_aggregate_states[i], + value.second + offsets_of_aggregate_states[i], *final_aggregate_columns[i]); } @@ -1205,9 +1155,9 @@ void NO_INLINE Aggregator::convertToBlockImplNotFinal( /// reserved, so push_back does not throw exceptions 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) aggregate_functions[i]->merge( - Method::getAggregateData(res_it->second) + offsets_of_aggregate_states[i], - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i], + res_it->second + offsets_of_aggregate_states[i], + it->second + offsets_of_aggregate_states[i], arena); for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_functions[i]->destroy( - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + it->second + offsets_of_aggregate_states[i]); } else { res_it->second = it->second; } - Method::getAggregateData(it->second) = nullptr; + it->second = nullptr; } table_src.clearAndShrink(); @@ -1588,19 +1538,18 @@ void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl( AggregateDataPtr res_data = table_dst.end() == res_it ? overflows - : Method::getAggregateData(res_it->second); + : res_it->second; for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_functions[i]->merge( res_data + offsets_of_aggregate_states[i], - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i], + it->second + offsets_of_aggregate_states[i], arena); for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_functions[i]->destroy( - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + aggregate_functions[i]->destroy(it->second + offsets_of_aggregate_states[i]); - Method::getAggregateData(it->second) = nullptr; + it->second = nullptr; } table_src.clearAndShrink(); @@ -1623,19 +1572,18 @@ void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl( if (table_dst.end() == res_it) 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) aggregate_functions[i]->merge( res_data + offsets_of_aggregate_states[i], - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i], + it->second + offsets_of_aggregate_states[i], arena); for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_functions[i]->destroy( - Method::getAggregateData(it->second) + offsets_of_aggregate_states[i]); + aggregate_functions[i]->destroy(it->second + offsets_of_aggregate_states[i]); - Method::getAggregateData(it->second) = nullptr; + it->second = nullptr; } table_src.clearAndShrink(); @@ -1986,7 +1934,7 @@ template void NO_INLINE Aggregator::mergeStreamsImplCase( Block & block, Arena * aggregates_pool, - Method & method, + Method & method [[maybe_unused]], Table & data, AggregateDataPtr overflow_row) const { @@ -2000,77 +1948,43 @@ void NO_INLINE Aggregator::mergeStreamsImplCase( for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_columns[i] = &typeid_cast(*block.safeGetByPosition(params.keys_size + i).column).getData(); - typename Method::State state; - if constexpr (Method::low_cardinality_optimization) - state.init(key_columns, aggregation_state_cache); - else - state.init(key_columns); + typename Method::State state(key_columns, key_sizes, aggregation_state_cache); /// For all rows. - StringRefs keys(params.keys_size); size_t rows = block.rows(); for (size_t i = 0; i < rows; ++i) { - typename Table::iterator it; - 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); + AggregateDataPtr aggregate_data = nullptr; if (!no_more_keys) { - if constexpr (Method::low_cardinality_optimization) - aggregate_data = state.emplaceKeyFromRow(data, i, inserted, params.keys_size, keys, *aggregates_pool); - else + auto emplace_result = state.emplaceKey(data, i, *aggregates_pool); + if (emplace_result.isInserted()) { - data.emplace(key, it, inserted); - aggregate_data = &Method::getAggregateData(it->second); + emplace_result.setMapped(nullptr); + + 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 { - if constexpr (Method::low_cardinality_optimization) - aggregate_data = state.findFromRow(data, i); - else - { - it = data.find(key); - if (data.end() != it) - aggregate_data = &Method::getAggregateData(it->second); - } + auto find_result = state.findKey(data, i, *aggregates_pool); + if (find_result.isFound()) + aggregate_data = find_result.getMapped(); } /// 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 (!aggregate_data && !overflow_row) - { - method.onExistingKey(key, keys, *aggregates_pool); continue; - } - /// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key. - 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; + AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row; /// Merge state of aggregate functions. 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. */ 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) { @@ -2395,15 +2309,11 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl( Method & method, Arena * pool, ColumnRawPtrs & key_columns, - StringRefs & keys, + StringRefs & keys [[maybe_unused]], const Block & source, std::vector & destinations) const { - typename Method::State state; - if constexpr (Method::low_cardinality_optimization) - state.init(key_columns, aggregation_state_cache); - else - state.init(key_columns); + typename Method::State state(key_columns, key_sizes, aggregation_state_cache); size_t rows = source.rows(); size_t columns = source.columns(); @@ -2423,16 +2333,11 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl( } } - /// Obtain a key. Calculate bucket number from it. - typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *pool); - - auto hash = method.data.hash(key); + /// Calculate bucket number from row hash. + auto hash = state.getHash(method.data, i, *pool); auto bucket = method.data.getBucketFromHash(hash); selector[i] = bucket; - - /// We don't need to store this key in pool. - method.onExistingKey(key, keys, *pool); } size_t num_buckets = destinations.size(); @@ -2523,7 +2428,7 @@ void NO_INLINE Aggregator::destroyImpl(Table & table) const { 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 * after inserting the key into a hash table, but before creating all states of aggregate functions, diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index bed147d627d..c4ffb64a15e 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -138,18 +139,6 @@ using AggregatedDataWithNullableStringKeyTwoLevel = AggregationDataWithNullKeyTw TwoLevelHashMapWithSavedHash, 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; /// For the case where there is one numeric key. template /// UInt8/16/32/64 for any type with corresponding bit width. @@ -169,65 +158,16 @@ struct AggregationMethodOneNumber AggregationMethodOneNumber(const Other & other) : data(other.data) {} /// To use one `Method` in different threads, use different `State`. - struct State - { - const char * vec; + using State = ColumnsHashing::HashMethodOneNumber; - /** 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(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. 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_cast(key_columns[0].get())->insertRawData(reinterpret_cast(&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(&value.first), sizeof(value.first)); - } - - static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; } }; @@ -248,58 +188,14 @@ struct AggregationMethodString template AggregationMethodString(const Other & other) : data(other.data) {} - struct State - { - const IColumn::Offset * offsets; - const UInt8 * chars; + using State = ColumnsHashing::HashMethodString; - void init(ColumnRawPtrs & key_columns) - { - const IColumn & column = *key_columns[0]; - const ColumnString & column_string = static_cast(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 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 &) { 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 AggregationMethodFixedString(const Other & other) : data(other.data) {} - struct State - { - size_t n; - const ColumnFixedString::Chars * chars; + using State = ColumnsHashing::HashMethodFixedString; - void init(ColumnRawPtrs & key_columns) - { - const IColumn & column = *key_columns[0]; - const ColumnFixedString & column_string = static_cast(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 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 &) { 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; - - 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; - Cache cache; }; /// Single low cardinality column. @@ -432,342 +241,23 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod using Base::data; - static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & settings) - { - return std::make_shared(settings); - } - AggregationMethodSingleLowCardinalityColumn() = default; template explicit AggregationMethodSingleLowCardinalityColumn(const Other & other) : Base(other) {} - struct State : public BaseState - { - ColumnRawPtrs key_columns; - const IColumn * positions = nullptr; - size_t size_of_index_type = 0; + using State = ColumnsHashing::HashMethodSingleLowCardinalityColumn; - /// 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 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(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(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(); - 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(positions)->getElement(row); - case sizeof(UInt16): return static_cast(positions)->getElement(row); - case sizeof(UInt32): return static_cast(positions)->getElement(row); - case sizeof(UInt64): return static_cast(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 - 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 - 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 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(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 -class BaseStateKeysFixed; - -/// Case where nullable keys are supported. -template -class BaseStateKeysFixed -{ -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(*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 createBitmap(size_t row) const - { - KeysNullMap bitmap{}; - - for (size_t k = 0; k < null_maps.size(); ++k) - { - if (null_maps[k] != nullptr) - { - const auto & null_map = static_cast(*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 -class BaseStateKeysFixed -{ -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 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 -struct LowCardinalityKeys -{ - ColumnRawPtrs nested_columns; - ColumnRawPtrs positions; - Sizes position_sizes; -}; - -template <> -struct LowCardinalityKeys {}; - /// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits. template struct AggregationMethodKeysFixed @@ -787,71 +277,8 @@ struct AggregationMethodKeysFixed template AggregationMethodKeysFixed(const Other & other) : data(other.data) {} - class State final : private aggregator_impl::BaseStateKeysFixed - { - LowCardinalityKeys low_cardinality_keys; + using State = ColumnsHashing::HashMethodKeysFixed; - public: - using Base = aggregator_impl::BaseStateKeysFixed; - - 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(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(i, keys_size, Base::getActualColumns(), key_sizes, bitmap); - } - else - { - if constexpr (has_low_cardinality) - return packFixed(i, keys_size, low_cardinality_keys.nested_columns, key_sizes, - &low_cardinality_keys.positions, &low_cardinality_keys.position_sizes); - - return packFixed(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 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 AggregationMethodSerialized(const Other & other) : data(other.data) {} - struct State - { - void init(ColumnRawPtrs &) - { - } + using State = ColumnsHashing::HashMethodSerialized; - 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 void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &) { auto pos = value.first.data; - for (size_t i = 0; i < key_columns.size(); ++i) - pos = key_columns[i]->deserializeAndInsertFromArena(pos); + for (auto & column : key_columns) + pos = column->deserializeAndInsertFromArena(pos); } - - static AggregationStateCachePtr createCache(const AggregationStateCache::Settings & /*settings*/) { return nullptr; } }; class Aggregator; +using ColumnsHashing::HashMethodContext; +using ColumnsHashing::HashMethodContextPtr; + struct AggregatedDataVariants : private boost::noncopyable { /** 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) { @@ -1309,7 +705,7 @@ struct AggregatedDataVariants : private boost::noncopyable { \ using TPtr ## NAME = decltype(AggregatedDataVariants::NAME); \ using T ## NAME = typename TPtr ## NAME ::element_type; \ - return T ## NAME ::createCache(settings); \ + return T ## NAME ::State::createContext(settings); \ } APPLY_FOR_AGGREGATED_VARIANTS(M) @@ -1496,7 +892,7 @@ protected: AggregatedDataVariants::Type method_chosen; Sizes key_sizes; - AggregationStateCachePtr aggregation_state_cache; + HashMethodContextPtr aggregation_state_cache; AggregateFunctionsPlainPtrs aggregate_functions; diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 8858dd0f2b6..6c5bda21965 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -170,18 +170,54 @@ static size_t getTotalByteCountImpl(const Maps & maps, Join::Type type) } -template -struct KeyGetterForType; +template +struct KeyGetterForTypeImpl; -template <> struct KeyGetterForType { using Type = JoinKeyGetterOneNumber; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterOneNumber; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterOneNumber; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterOneNumber; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterString; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterFixedString; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterFixed; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterFixed; }; -template <> struct KeyGetterForType { using Type = JoinKeyGetterHashed; }; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodOneNumber; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodOneNumber; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodOneNumber; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodOneNumber; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodString; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodFixedString; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodKeysFixed; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodKeysFixed; +}; +template struct KeyGetterForTypeImpl +{ + using Type = ColumnsHashing::HashMethodHashed; +}; + +template +struct KeyGetterForType +{ + using Value = typename Data::value_type; + using Mapped_t = typename Data::mapped_type; + using Mapped = std::conditional_t, const Mapped_t, Mapped_t>; + using Type = typename KeyGetterForTypeImpl::Type; +}; /// 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 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 struct Inserter { - 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; - bool inserted; - map.emplace(key, it, inserted); + auto emplace_result = key_getter.emplaceKey(map, i, pool); - if (inserted) - { - 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); + if (emplace_result.isInserted() || emplace_result.getMapped().overwrite) + new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); } }; template struct Inserter { - 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; - bool inserted; - map.emplace(key, it, inserted); + auto emplace_result = key_getter.emplaceKey(map, i, pool); - if (inserted) - { - KeyGetter::onNewKey(it->first, pool); - new (&it->second) typename Map::mapped_type(stored_block, i); - } + if (emplace_result.isInserted()) + new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); else { /** 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. */ auto elem = pool.alloc(); + auto & mapped = emplace_result.getMapped(); - elem->next = it->second.next; - it->second.next = elem; + elem->next = mapped.next; + mapped.next = elem; elem->block = stored_block; elem->row_num = i; } @@ -372,17 +397,16 @@ namespace template void NO_INLINE insertFromBlockImplTypeCase( 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) { if (has_null_map && (*null_map)[i]) continue; - auto key = key_getter.getKey(key_columns, keys_size, i, key_sizes); - Inserter::insert(map, key, stored_block, i, pool); + Inserter::insert(map, key_getter, stored_block, i, pool); } } @@ -390,19 +414,19 @@ namespace template void insertFromBlockImplType( 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) - insertFromBlockImplTypeCase(map, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); + insertFromBlockImplTypeCase(map, rows, key_columns, key_sizes, stored_block, null_map, pool); else - insertFromBlockImplTypeCase(map, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); + insertFromBlockImplTypeCase(map, rows, key_columns, key_sizes, stored_block, null_map, pool); } template void insertFromBlockImpl( 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) { @@ -411,8 +435,8 @@ namespace #define M(TYPE) \ case Join::Type::TYPE: \ - insertFromBlockImplType::Type>(\ - *maps.TYPE, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); \ + insertFromBlockImplType>::Type>(\ + *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, pool); \ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -499,7 +523,7 @@ bool Join::insertFromBlock(const Block & block) { dispatch([&](auto, auto strictness_, auto & map) { - insertFromBlockImpl(type, map, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); + insertFromBlockImpl(type, map, rows, key_columns, key_sizes, stored_block, null_map, pool); }); } @@ -515,14 +539,14 @@ namespace template struct Adder { - 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*/, const std::vector & right_indexes) { filter[i] = 1; 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, @@ -538,14 +562,14 @@ namespace template struct Adder { - 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*/, const std::vector & right_indexes) { filter[i] = 1; 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*/, @@ -558,14 +582,14 @@ namespace template struct Adder { - 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, const std::vector & right_indexes) { filter[i] = 1; size_t rows_joined = 0; - for (auto current = &static_cast(it->second); current != nullptr; current = current->next) + for (auto current = &static_cast(mapped); current != nullptr; current = current->next) { 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); @@ -605,10 +629,10 @@ namespace const std::vector & right_indexes) { IColumn::Offset current_offset = 0; - size_t keys_size = key_columns.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) { @@ -619,14 +643,14 @@ namespace } else { - auto key = key_getter.getKey(key_columns, keys_size, i, key_sizes); - typename Map::const_iterator it = map.find(key); + auto find_result = key_getter.findKey(map, i, pool); - if (it != map.end()) + if (find_result.isFound()) { - it->second.setUsed(); + auto & mapped = find_result.getMapped(); + mapped.setUsed(); Adder::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 Adder::fill_left, STRICTNESS, Map>::addNotFound( @@ -753,7 +777,7 @@ void Join::joinBlockImpl( #define M(TYPE) \ case Join::Type::TYPE: \ std::tie(filter, offsets_to_replicate) = \ - joinBlockImplType::Type>(\ + joinBlockImplType>::Type>(\ *maps_.TYPE, block.rows(), key_columns, key_sizes, added_columns, null_map, right_indexes); \ break; APPLY_FOR_JOIN_VARIANTS(M) diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 04e9364605b..07b4849fd81 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -8,6 +8,7 @@ #include #include +#include #include #include @@ -21,148 +22,6 @@ 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 -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(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(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(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 -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(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. * 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. diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index f61dd6d4570..a10d520d779 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -75,30 +75,22 @@ void NO_INLINE Set::insertFromBlockImplCase( const ColumnRawPtrs & key_columns, size_t rows, SetVariants & variants, - ConstNullMapPtr null_map, - ColumnUInt8::Container * out_filter) + [[maybe_unused]] ConstNullMapPtr null_map, + [[maybe_unused]] ColumnUInt8::Container * out_filter) { - typename Method::State state; - state.init(key_columns); + typename Method::State state(key_columns, key_sizes, nullptr); /// For all rows for (size_t i = 0; i < rows; ++i) { - if (has_null_map && (*null_map)[i]) - continue; + if constexpr (has_null_map) + if ((*null_map)[i]) + continue; - /// Obtain a key to insert to the set - typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes); + [[maybe_unused]] auto emplace_result = state.emplaceKey(method.data, i, variants.string_pool); - typename Method::Data::iterator it; - bool inserted; - method.data.emplace(key, it, inserted); - - if (inserted) - method.onNewKey(*it, keys_size, variants.string_pool); - - if (build_filter) - (*out_filter)[i] = inserted; + if constexpr (build_filter) + (*out_filter)[i] = emplace_result.isInserted(); } } @@ -392,10 +384,10 @@ void NO_INLINE Set::executeImplCase( size_t rows, ConstNullMapPtr null_map) const { - typename Method::State state; - state.init(key_columns); + Arena pool; + 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 (size_t i = 0; i < rows; ++i) @@ -404,9 +396,8 @@ void NO_INLINE Set::executeImplCase( vec_res[i] = negative; else { - /// Build the key - typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes); - vec_res[i] = negative ^ method.data.has(key); + auto find_result = state.findKey(method.data, i, pool); + vec_res[i] = negative ^ find_result.isFound(); } } } diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index f241ddf4a00..e6c75da91fb 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -27,33 +28,7 @@ struct SetMethodOneNumber Data data; - /// To use one `Method` in different threads, use different `State`. - 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(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*/) {} + using State = ColumnsHashing::HashMethodOneNumber; }; /// For the case where there is one string key. @@ -65,36 +40,7 @@ struct SetMethodString Data data; - struct State - { - 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(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); - } + using State = ColumnsHashing::HashMethodString; }; /// For the case when there is one fixed-length string key. @@ -106,33 +52,7 @@ struct SetMethodFixedString Data data; - struct State - { - 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(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); - } + using State = ColumnsHashing::HashMethodFixedString; }; namespace set_impl @@ -242,34 +162,7 @@ struct SetMethodKeysFixed Data data; - class State : private set_impl::BaseStateKeysFixed - { - public: - using Base = set_impl::BaseStateKeysFixed; - - 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(i, keys_size, Base::getActualColumns(), key_sizes, bitmap); - } - else - return packFixed(i, keys_size, key_columns, key_sizes); - } - }; - - static void onNewKey(typename Data::value_type &, size_t, Arena &) {} + using State = ColumnsHashing::HashMethodKeysFixed; }; /// For other cases. 128 bit hash from the key. @@ -281,23 +174,7 @@ struct SetMethodHashed Data data; - struct State - { - 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 &) {} + using State = ColumnsHashing::HashMethodHashed; }; diff --git a/dbms/src/Interpreters/SpecializedAggregator.h b/dbms/src/Interpreters/SpecializedAggregator.h index de7ea6fd921..8ec6b297111 100644 --- a/dbms/src/Interpreters/SpecializedAggregator.h +++ b/dbms/src/Interpreters/SpecializedAggregator.h @@ -107,11 +107,7 @@ void NO_INLINE Aggregator::executeSpecialized( bool no_more_keys, AggregateDataPtr overflow_row) const { - typename Method::State state; - if constexpr (Method::low_cardinality_optimization) - state.init(key_columns, aggregation_state_cache); - else - state.init(key_columns); + typename Method::State state(key_columns, key_sizes, aggregation_state_cache); if (!no_more_keys) executeSpecializedCase( @@ -130,94 +126,48 @@ void NO_INLINE Aggregator::executeSpecializedCase( typename Method::State & state, Arena * aggregates_pool, size_t rows, - ColumnRawPtrs & key_columns, + ColumnRawPtrs & /*key_columns*/, AggregateColumns & aggregate_columns, - StringRefs & keys, + StringRefs & /*keys*/, AggregateDataPtr overflow_row) const { /// For all rows. - typename Method::Key prev_key{}; - AggregateDataPtr value = nullptr; for (size_t i = 0; i < rows; ++i) { - 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); - - AggregateDataPtr * aggregate_data = nullptr; - typename Method::iterator it; /// Is not used if Method::low_cardinality_optimization + AggregateDataPtr aggregate_data = nullptr; if (!no_more_keys) /// Insert. { - /// Optimization for frequently repeating keys. - if (!Method::no_consecutive_keys_optimization) + auto emplace_result = state.emplaceKey(method.data, i, *aggregates_pool); + + /// 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) - { - /// Add values into aggregate functions. - AggregateFunctionsList::forEach(AggregateFunctionsUpdater( - aggregate_functions, offsets_of_aggregate_states, aggregate_columns, value, i, aggregates_pool)); + /// exception-safety - if you can not allocate memory or create states, then destructors will not be called. + emplace_result.setMapped(nullptr); - method.onExistingKey(key, keys, *aggregates_pool); - continue; - } - else - prev_key = key; + aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + AggregateFunctionsList::forEach(AggregateFunctionsCreator( + aggregate_functions, offsets_of_aggregate_states, aggregate_data)); + + 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 - { - method.data.emplace(key, it, inserted); - aggregate_data = &Method::getAggregateData(it->second); - } + aggregate_data = emplace_result.getMapped(); } else { /// Add only if the key already exists. - if constexpr (Method::low_cardinality_optimization) - aggregate_data = state.findFromRow(method.data, i); - else - { - it = method.data.find(key); - if (method.data.end() != it) - aggregate_data = &Method::getAggregateData(it->second); - } + auto find_result = state.findKey(method.data, i, *aggregates_pool); + if (find_result.isFound()) + aggregate_data = find_result.getMapped(); } /// 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) - { - method.onExistingKey(key, keys, *aggregates_pool); continue; - } - /// If a new key is inserted, initialize the states of the aggregate functions, and possibly some stuff related to the key. - 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; + auto value = aggregate_data ? aggregate_data : overflow_row; /// Add values into the aggregate functions. AggregateFunctionsList::forEach(AggregateFunctionsUpdater( diff --git a/dbms/src/Interpreters/convertFieldToType.cpp b/dbms/src/Interpreters/convertFieldToType.cpp index 0a46078a602..892a5967719 100644 --- a/dbms/src/Interpreters/convertFieldToType.cpp +++ b/dbms/src/Interpreters/convertFieldToType.cpp @@ -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()); + 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: " + Field::Types::toString(src.getType()), ErrorCodes::TYPE_MISMATCH); } diff --git a/dbms/src/Storages/MergeTree/MergeTreePartition.cpp b/dbms/src/Storages/MergeTree/MergeTreePartition.cpp index 57e7acfe986..d26cc7fb627 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartition.cpp @@ -98,7 +98,7 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe const DataTypePtr & type = storage.partition_key_sample.getByPosition(0).type; auto column = type->createColumn(); column->insert(value[0]); - type->serializeText(*column, 0, out, format_settings); + type->serializeAsText(*column, 0, out, format_settings); } else { diff --git a/dbms/tests/performance/columns_hashing/columns_hashing.xml b/dbms/tests/performance/columns_hashing/columns_hashing.xml new file mode 100644 index 00000000000..d8cebcb4447 --- /dev/null +++ b/dbms/tests/performance/columns_hashing/columns_hashing.xml @@ -0,0 +1,48 @@ + + Benchmark + + + columns_hashing + + + + hits_100m_single + hits_1000m_single + + + loop + + + + 5 + 60000 + + + 10 + 150000 + + + + + + + + + + + + + + diff --git a/dbms/tests/performance/ipv4_ipv6/IPv4.xml b/dbms/tests/performance/ipv4_ipv6/IPv4.xml new file mode 100644 index 00000000000..00097c41a2d --- /dev/null +++ b/dbms/tests/performance/ipv4_ipv6/IPv4.xml @@ -0,0 +1,152 @@ + + IPv4 Functions + + once + + + + + + + 2000 + 10000 + + + + + + + + + + + + + ipv4_string + + + 116.253.40.133 + 183.247.232.58 + 116.106.34.242 + 111.56.27.171 + 183.245.137.140 + 183.212.25.70 + 162.144.2.57 + 111.4.229.190 + 59.52.3.168 + 115.11.21.200 + 121.28.97.113 + 111.46.39.248 + 120.192.122.34 + 113.56.44.105 + 116.66.238.92 + 67.22.254.206 + 115.0.24.191 + 182.30.107.86 + 223.73.153.243 + 115.159.103.38 + 36.186.75.121 + 111.56.188.125 + 115.14.93.25 + 211.97.110.141 + 61.58.96.173 + 203.126.212.37 + 192.220.125.142 + 115.22.20.223 + 121.25.160.80 + 117.150.98.199 + 183.211.172.143 + 180.244.18.143 + 209.131.3.252 + 220.200.1.22 + 171.225.130.45 + 115.4.78.200 + 36.183.59.29 + 218.42.159.17 + 115.13.39.164 + 142.254.161.133 + 116.2.211.43 + 36.183.126.25 + 66.150.171.196 + 104.149.148.137 + 120.239.82.212 + 111.14.182.156 + 115.6.63.224 + 153.35.83.233 + 113.142.1.1 + 121.25.82.29 + 62.151.203.189 + 104.27.46.146 + 36.189.46.88 + 116.252.54.207 + 64.77.240.1 + 142.252.102.78 + 36.82.224.170 + 117.33.191.217 + 144.12.164.251 + 122.10.93.66 + 104.25.84.59 + 111.4.242.106 + 222.216.51.186 + 112.33.13.212 + 115.9.240.116 + 171.228.0.153 + 45.3.47.158 + 69.57.193.230 + 115.6.104.199 + 104.24.237.140 + 199.17.84.108 + 120.193.17.57 + 112.40.38.145 + 67.55.90.43 + 180.253.57.249 + 14.204.253.158 + 1.83.241.116 + 202.198.37.147 + 115.6.31.95 + 117.32.14.179 + 23.238.237.26 + 116.97.76.104 + 1.80.2.248 + 59.50.185.152 + 42.117.228.166 + 119.36.22.147 + 210.66.18.184 + 115.19.192.159 + 112.15.128.113 + 1.55.138.211 + 210.183.19.113 + 42.115.43.114 + 58.16.171.31 + 171.234.78.185 + 113.56.43.134 + 111.53.182.225 + 107.160.215.141 + 171.229.231.90 + 58.19.84.138 + 36.79.88.107 + + + + + 1 + 1. + 1.2. + .2. + abc + 127.0.0.1/24 + 127.0.0.1 + 127.0.0.1 + 127.0.0.1? + 999.999.999.999 + + + + + SELECT count() FROM system.numbers WHERE NOT ignore(IPv4StringToNum(materialize('{ipv4_string}'))) + SELECT count() FROM system.numbers WHERE NOT ignore(IPv4NumToString(IPv4StringToNum(materialize('{ipv4_string}')))) + SELECT count() FROM system.numbers WHERE NOT ignore(IPv4NumToStringClassC(IPv4StringToNum(materialize('{ipv4_string}')))) + SELECT count() FROM system.numbers WHERE NOT ignore(IPv4ToIPv6(IPv4StringToNum(materialize('{ipv4_string}')))) + \ No newline at end of file diff --git a/dbms/tests/performance/ipv4_ipv6/IPv6.xml b/dbms/tests/performance/ipv4_ipv6/IPv6.xml new file mode 100644 index 00000000000..747b174fb5c --- /dev/null +++ b/dbms/tests/performance/ipv4_ipv6/IPv6.xml @@ -0,0 +1,257 @@ + + IPv6 Functions + + once + + + + + + + 2000 + 10000 + + + + + + + + + + + + + ipv6_string + + + 2606:2b00::1 + 2001:2000:3080:1351::2 + 2a01:8840:16::1 + 2001:550:0:1000::9a36:2a61 + 2001:578:400:4:2000::19 + 2607:f290::1 + 2a02:23f0:ffff:8::5 + 2400:c700:0:158:: + 2001:67c:24e4:4::250 + 2a02:2a38:37:5::2 + 2001:41a8:400:2::13a + 2405:9800:9800:66::2 + 2a07:a343:f210::1 + 2403:5000:171:46::2 + 2800:c20:1141::8 + 2402:7800:40:2::62 + 2a00:de00::1 + 2001:688:0:2:1::9e + 2001:2000:3080:80::2 + 2001:428::205:171:200:230 + 2001:fb1:fe0:9::8 + 2001:2000:3080:10ca::2 + 2400:dd0b:1003::2 + 2001:1a98:6677::9d9d:140a + 2001:2000:3018:3b::1 + 2607:fa70:3:33::2 + 2001:5b0:23ff:fffa::113 + 2001:450:2001:1000:0:40:6924:23 + 2001:418:0:5000::c2d + 2a01:b740:a09::1 + 2607:f0d0:2:2::243 + 2a01:348::e:1:1 + 2405:4800::3221:3621:2 + 2a02:aa08:e000:3100::2 + 2001:44c8:129:2632:33:0:252:2 + 2a02:e980:1e::1 + 2a0a:6f40:2::1 + 2001:550:2:29::2c9:1 + 2001:c20:4800::175 + 2c0f:feb0:1:2::d1 + 2a0b:7086:fff0::1 + 2a04:2dc0::16d + 2604:7e00::105d + 2001:470:1:946::2 + 2a0c:3240::1 + 2800:630:4010:8::2 + 2001:1af8:4040::12 + 2c0f:fc98:1200::2 + 2001:470:1:248::2 + 2620:44:a000::1 + 2402:800:63ff:40::1 + 2a02:b000:fff::524 + 2001:470:0:327::1 + 2401:7400:8888:2::8 + 2001:500:55::1 + 2001:668:0:3::f000:c2 + 2400:bf40::1 + 2001:67c:754::1 + 2402:28c0:100:ffff:ffff:ffff:ffff:ffff + 2001:470:0:1fa::2 + 2001:550:0:1000::9a18:292a + 2001:470:1:89e::2 + 2001:579:6f05:500:9934:5b3e:b7fe:1447 + 2804:158c::1 + 2600:140e:6::1 + 2a00:18e0:0:bb04::82 + 2a02:2698:5000::1e06 + 2402:800:63ff:10::7:2 + 2a02:e980:19::1 + 2001:4888::342:1:0:0 + 2607:fc68:0:4:0:2:2711:21 + 2606:2800:602a::1 + 2404:c600:1000:2::1d1 + 2001:578:1400:4::9d + 2804:64:0:25::1 + 2605:3e00::1:2:2 + 2c0f:fa18:0:4::b + 2606:2800:602c:b::d004 + 2610:18:181:4000::66 + 2001:48f8:1000:1::16 + 2408:8000:c000::1 + 2a03:4200:441:2::4e + 2400:dd00:1:200a::2 + 2a02:e980:83:5b09:ecb8:c669:b336:650e + 2001:16a0:2:200a::2 + 2001:4888:1f:e891:161:26:: + 2a0c:f743::1 + 2a02:e980:b::1 + 2001:578:201:1::601:9 + 2001:438:ffff::407d:1bc1 + 2001:920:1833::1 + 2001:1b70:a1:610::b102:2 + 2001:13c7:6014::1 + 2003:0:1203:4001::1 + 2804:a8:2:c8::d6 + 2a02:2e00:2080:f000:0:261:1:11 + 2001:578:20::d + 2001:550:2:48::34:1 + 2a03:9d40:fe00:5:: + 2403:e800:200:102::2 + + + ::ffff:116.253.40.133 + ::ffff:183.247.232.58 + ::ffff:116.106.34.242 + ::ffff:111.56.27.171 + ::ffff:183.245.137.140 + ::ffff:183.212.25.70 + ::ffff:162.144.2.57 + ::ffff:111.4.229.190 + ::ffff:59.52.3.168 + ::ffff:115.11.21.200 + ::ffff:121.28.97.113 + ::ffff:111.46.39.248 + ::ffff:120.192.122.34 + ::ffff:113.56.44.105 + ::ffff:116.66.238.92 + ::ffff:67.22.254.206 + ::ffff:115.0.24.191 + ::ffff:182.30.107.86 + ::ffff:223.73.153.243 + ::ffff:115.159.103.38 + ::ffff:36.186.75.121 + ::ffff:111.56.188.125 + ::ffff:115.14.93.25 + ::ffff:211.97.110.141 + ::ffff:61.58.96.173 + ::ffff:203.126.212.37 + ::ffff:192.220.125.142 + ::ffff:115.22.20.223 + ::ffff:121.25.160.80 + ::ffff:117.150.98.199 + ::ffff:183.211.172.143 + ::ffff:180.244.18.143 + ::ffff:209.131.3.252 + ::ffff:220.200.1.22 + ::ffff:171.225.130.45 + ::ffff:115.4.78.200 + ::ffff:36.183.59.29 + ::ffff:218.42.159.17 + ::ffff:115.13.39.164 + ::ffff:142.254.161.133 + ::ffff:116.2.211.43 + ::ffff:36.183.126.25 + ::ffff:66.150.171.196 + ::ffff:104.149.148.137 + ::ffff:120.239.82.212 + ::ffff:111.14.182.156 + ::ffff:115.6.63.224 + ::ffff:153.35.83.233 + ::ffff:113.142.1.1 + ::ffff:121.25.82.29 + ::ffff:62.151.203.189 + ::ffff:104.27.46.146 + ::ffff:36.189.46.88 + ::ffff:116.252.54.207 + ::ffff:64.77.240.1 + ::ffff:142.252.102.78 + ::ffff:36.82.224.170 + ::ffff:117.33.191.217 + ::ffff:144.12.164.251 + ::ffff:122.10.93.66 + ::ffff:104.25.84.59 + ::ffff:111.4.242.106 + ::ffff:222.216.51.186 + ::ffff:112.33.13.212 + ::ffff:115.9.240.116 + ::ffff:171.228.0.153 + ::ffff:45.3.47.158 + ::ffff:69.57.193.230 + ::ffff:115.6.104.199 + ::ffff:104.24.237.140 + ::ffff:199.17.84.108 + ::ffff:120.193.17.57 + ::ffff:112.40.38.145 + ::ffff:67.55.90.43 + ::ffff:180.253.57.249 + ::ffff:14.204.253.158 + ::ffff:1.83.241.116 + ::ffff:202.198.37.147 + ::ffff:115.6.31.95 + ::ffff:117.32.14.179 + ::ffff:23.238.237.26 + ::ffff:116.97.76.104 + ::ffff:1.80.2.248 + ::ffff:59.50.185.152 + ::ffff:42.117.228.166 + ::ffff:119.36.22.147 + ::ffff:210.66.18.184 + ::ffff:115.19.192.159 + ::ffff:112.15.128.113 + ::ffff:1.55.138.211 + ::ffff:210.183.19.113 + ::ffff:42.115.43.114 + ::ffff:58.16.171.31 + ::ffff:171.234.78.185 + ::ffff:113.56.43.134 + ::ffff:111.53.182.225 + ::ffff:107.160.215.141 + ::ffff:171.229.231.90 + ::ffff:58.19.84.138 + ::ffff:36.79.88.107 + + + + + 1 + 1. + 1.2. + .2. + abc + ab:cd:ef:gh:ij:kl:mn + ffffffffffffff + abcdefghijklmn + :::::::::::::: + ::ffff:127.0.0.1 + ::ffff:127.0.0.1 + ::ffff:999.999.999.999 + + + + + SELECT count() FROM system.numbers WHERE NOT ignore(IPv6StringToNum(materialize('{ipv6_string}'))) + SELECT count() FROM system.numbers WHERE NOT ignore(IPv6NumToString(IPv6StringToNum(materialize('{ipv6_string}')))) + \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00076_ip_coding_functions.reference b/dbms/tests/queries/0_stateless/00076_ip_coding_functions.reference index 129d52f3a73..087749d8cf6 100644 --- a/dbms/tests/queries/0_stateless/00076_ip_coding_functions.reference +++ b/dbms/tests/queries/0_stateless/00076_ip_coding_functions.reference @@ -16,6 +16,8 @@ 1 1 1 +1 +1 00000000000000000000FFFF4D583737 00000000000000000000FFFF4D583737 00000000000000000000FFFF7F000001 diff --git a/dbms/tests/queries/0_stateless/00076_ip_coding_functions.sql b/dbms/tests/queries/0_stateless/00076_ip_coding_functions.sql index 56597f53357..eba809afc62 100644 --- a/dbms/tests/queries/0_stateless/00076_ip_coding_functions.sql +++ b/dbms/tests/queries/0_stateless/00076_ip_coding_functions.sql @@ -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 IPv4NumToString(toUInt32(0)) == '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(materialize(''), 16)) == materialize('::'); diff --git a/dbms/tests/queries/0_stateless/00725_ipv4_ipv6_domains.reference b/dbms/tests/queries/0_stateless/00725_ipv4_ipv6_domains.reference new file mode 100644 index 00000000000..9478e02fcaf --- /dev/null +++ b/dbms/tests/queries/0_stateless/00725_ipv4_ipv6_domains.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00725_ipv4_ipv6_domains.sql b/dbms/tests/queries/0_stateless/00725_ipv4_ipv6_domains.sql new file mode 100644 index 00000000000..c0231c6bf16 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00725_ipv4_ipv6_domains.sql @@ -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; diff --git a/docker/packager/packager b/docker/packager/packager index 46e39dd6b26..595891e3c5c 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -52,7 +52,7 @@ def run_image_with_env(image_name, output, env_variables, ch_root): 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 = [] if package_type == "deb": result.append("DEB_CC={}".format(compiler)) @@ -77,6 +77,10 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, di if unbundled: 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 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("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="") 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("--distcc-hosts", nargs="+") 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 pull_image(image_name) or args.force_build_image: 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) logging.info("Output placed into {}".format(args.output_dir)) diff --git a/utils/convert-month-partitioned-parts/CMakeLists.txt b/utils/convert-month-partitioned-parts/CMakeLists.txt index a0308cbe504..abfd60a07a0 100644 --- a/utils/convert-month-partitioned-parts/CMakeLists.txt +++ b/utils/convert-month-partitioned-parts/CMakeLists.txt @@ -1,2 +1,2 @@ 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})