diff --git a/be/src/runtime/datetime_value.h b/be/src/runtime/datetime_value.h index c0ffb9798bf4c3..f7bc0c239923da 100644 --- a/be/src/runtime/datetime_value.h +++ b/be/src/runtime/datetime_value.h @@ -272,13 +272,15 @@ class DateTimeValue { int hour() const { return _hour; } int minute() const { return _minute; } int second() const { return _second; } + int microsecond() const { return _microsecond; } + int neg() const { return _neg; } bool check_loss_accuracy_cast_to_date() { auto loss_accuracy = _hour != 0 || _minute != 0 || _second != 0 || _microsecond != 0; cast_to_date(); return loss_accuracy; } - + void cast_to_date() { _hour = 0; _minute = 0; diff --git a/be/src/runtime/row_batch.cpp b/be/src/runtime/row_batch.cpp index e31c74aab4dd3d..00fc925b4be5bf 100644 --- a/be/src/runtime/row_batch.cpp +++ b/be/src/runtime/row_batch.cpp @@ -26,11 +26,10 @@ #include "runtime/string_value.h" #include "runtime/tuple_row.h" //#include "runtime/mem_tracker.h" -#include "vec/columns/column_vector.h" -#include "vec/core/block.h" - #include "gen_cpp/Data_types.h" #include "gen_cpp/data.pb.h" +#include "vec/columns/column_vector.h" +#include "vec/core/block.h" using std::vector; @@ -553,11 +552,12 @@ vectorized::Block RowBatch::convert_to_vec_block() const { if (slot_desc->is_nullable() && tuple->is_null(slot_desc->null_indicator_offset())) { columns[i]->insertData(nullptr, 0); } else if (slot_desc->type().is_string_type()) { - auto string_value = static_cast(tuple->get_slot(slot_desc->tuple_offset())); + auto string_value = + static_cast(tuple->get_slot(slot_desc->tuple_offset())); columns[i]->insertData(string_value->ptr, string_value->len); } else { columns[i]->insertData( - static_cast(tuple->get_slot(slot_desc->tuple_offset())), + static_cast(tuple->get_slot(slot_desc->tuple_offset())), slot_desc->slot_size()); } } diff --git a/be/src/runtime/types.h b/be/src/runtime/types.h index e0a6bf631f4ea2..47e3d602f93c5e 100644 --- a/be/src/runtime/types.h +++ b/be/src/runtime/types.h @@ -27,6 +27,8 @@ #include "olap/hll.h" #include "runtime/primitive_type.h" #include "thrift/protocol/TDebugProtocol.h" +#include "vec/data_types/data_type_date.h" +#include "vec/data_types/data_type_date_time.h" #include "vec/data_types/data_type_string.h" #include "vec/data_types/data_types_decimal.h" #include "vec/data_types/data_types_number.h" @@ -176,6 +178,8 @@ struct TypeDescriptor { inline bool is_date_type() const { return type == TYPE_DATE || type == TYPE_DATETIME; } + inline bool is_datetime_type() const { return type == TYPE_DATETIME; } + inline bool is_decimal_type() const { return (type == TYPE_DECIMAL || type == TYPE_DECIMALV2); } inline bool is_var_len_string_type() const { @@ -297,10 +301,11 @@ struct TypeDescriptor { return std::make_shared(); case TYPE_LARGEINT: + return std::make_shared(); case TYPE_DATE: + return std::make_shared(); case TYPE_DATETIME: - return std::make_shared(); - + return std::make_shared(); case TYPE_DOUBLE: return std::make_shared(); diff --git a/be/src/vec/CMakeLists.txt b/be/src/vec/CMakeLists.txt index 6529fa68f79bae..752d8b2f314a06 100644 --- a/be/src/vec/CMakeLists.txt +++ b/be/src/vec/CMakeLists.txt @@ -56,6 +56,8 @@ set(VEC_FILES data_types/data_types_number.cpp data_types/get_least_supertype.cpp data_types/nested_utils.cpp + data_types/data_type_date.cpp + data_types/data_type_date_time.cpp exec/vaggregation_node.cpp exec/volap_scan_node.cpp exec/vsort_node.cpp diff --git a/be/src/vec/columns/column_decimal.cpp b/be/src/vec/columns/column_decimal.cpp index 640d0fb9fe30e3..86b36fb53ddc72 100644 --- a/be/src/vec/columns/column_decimal.cpp +++ b/be/src/vec/columns/column_decimal.cpp @@ -15,20 +15,17 @@ // specific language governing permissions and limitations // under the License. +#include "vec/columns/column_decimal.h" + +#include "vec/columns/columns_common.h" #include "vec/common/arena.h" #include "vec/common/assert_cast.h" #include "vec/common/exception.h" #include "vec/common/sip_hash.h" #include "vec/common/unaligned.h" -//#include - -#include "vec/columns/column_decimal.h" -#include "vec/columns/columns_common.h" -//#include - template -bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale); +bool decimalLess(T x, T y, doris::vectorized::UInt32 x_scale, doris::vectorized::UInt32 y_scale); namespace doris::vectorized { @@ -78,17 +75,15 @@ void ColumnDecimal::updateHashWithValue(size_t n, SipHash& hash) const { } template -void ColumnDecimal::getPermutation(bool reverse, size_t limit, int , IColumn::Permutation & res) const -{ +void ColumnDecimal::getPermutation(bool reverse, size_t limit, int, + IColumn::Permutation& res) const { #if 1 /// TODO: perf test - if (data.size() <= std::numeric_limits::max()) - { + if (data.size() <= std::numeric_limits::max()) { PaddedPODArray tmp_res; permutation(reverse, limit, tmp_res); res.resize(tmp_res.size()); - for (size_t i = 0; i < tmp_res.size(); ++i) - res[i] = tmp_res[i]; + for (size_t i = 0; i < tmp_res.size(); ++i) res[i] = tmp_res[i]; return; } #endif diff --git a/be/src/vec/columns/column_string.cpp b/be/src/vec/columns/column_string.cpp index 0e41ea0c2211df..c40ad5246f4e42 100644 --- a/be/src/vec/columns/column_string.cpp +++ b/be/src/vec/columns/column_string.cpp @@ -22,9 +22,6 @@ #include "vec/common/arena.h" #include "vec/common/assert_cast.h" #include "vec/common/memcmp_small.h" -#include "vec/core/defines.h" -//#include - #include "vec/common/unaligned.h" namespace doris::vectorized { @@ -233,25 +230,20 @@ struct ColumnString::less { } }; -void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const -{ +void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, + Permutation& res) const { size_t s = offsets.size(); res.resize(s); - for (size_t i = 0; i < s; ++i) - res[i] = i; + for (size_t i = 0; i < s; ++i) res[i] = i; - if (limit >= s) - limit = 0; + if (limit >= s) limit = 0; - if (limit) - { + if (limit) { if (reverse) std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); else std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); - } - else - { + } else { if (reverse) std::sort(res.begin(), res.end(), less(*this)); else diff --git a/be/src/vec/columns/columns_number.h b/be/src/vec/columns/columns_number.h index 8715c9cdc0614b..d94a2b8e04de3d 100644 --- a/be/src/vec/columns/columns_number.h +++ b/be/src/vec/columns/columns_number.h @@ -34,6 +34,7 @@ using ColumnInt8 = ColumnVector; using ColumnInt16 = ColumnVector; using ColumnInt32 = ColumnVector; using ColumnInt64 = ColumnVector; +using ColumnInt128 = ColumnVector; using ColumnFloat32 = ColumnVector; using ColumnFloat64 = ColumnVector; diff --git a/be/src/vec/common/aggregation_common.h b/be/src/vec/common/aggregation_common.h index b33ed36b208e80..593b54096dbebb 100644 --- a/be/src/vec/common/aggregation_common.h +++ b/be/src/vec/common/aggregation_common.h @@ -1,19 +1,16 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - #include -// #include -// #include + +#include "vec/columns/column.h" +#include "vec/columns/columns_number.h" +#include "vec/common/arena.h" +#include "vec/common/assert_cast.h" +#include "vec/common/hash_table/hash.h" +#include "vec/common/memcpy_small.h" +#include "vec/common/sip_hash.h" +#include "vec/common/string_ref.h" +#include "vec/common/uint128.h" template <> struct DefaultHash : public StringRefHash {}; diff --git a/be/src/vec/common/allocator.h b/be/src/vec/common/allocator.h index be7a55e8ecd157..662c3b0b068930 100644 --- a/be/src/vec/common/allocator.h +++ b/be/src/vec/common/allocator.h @@ -31,9 +31,6 @@ #define ALLOCATOR_ASLR 1 #endif -//#include -// #include - #if !defined(__APPLE__) && !defined(__FreeBSD__) #include #endif @@ -48,13 +45,9 @@ /// Thread sanitizer does not intercept mremap. The usage of mremap will lead to false positives. #define DISABLE_MREMAP 1 #endif -#include "vec/common/mremap.h" - -// #include -#include "vec/common/exception.h" -// #include - #include "vec/common/allocator_fwd.h" +#include "vec/common/exception.h" +#include "vec/common/mremap.h" /// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS #ifndef MAP_ANONYMOUS diff --git a/be/src/vec/common/arena.h b/be/src/vec/common/arena.h index 5ebf798d31eaef..d0a58e15862fc3 100644 --- a/be/src/vec/common/arena.h +++ b/be/src/vec/common/arena.h @@ -26,10 +26,9 @@ #if __has_include() #include #endif +#include "vec/common/allocator.h" #include "vec/common/memcpy_small.h" #include "vec/core/defines.h" -//#include -#include "vec/common/allocator.h" //namespace ProfileEvents //{ diff --git a/be/src/vec/common/args_to_config.h b/be/src/vec/common/args_to_config.h deleted file mode 100644 index c5ba31be75dcda..00000000000000 --- a/be/src/vec/common/args_to_config.h +++ /dev/null @@ -1,27 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once -#include - -namespace Poco::Util { -class LayeredConfiguration; -} - -/// Import extra command line arguments to configuration. These are command line arguments after --. -void argsToConfig(const Poco::Util::Application::ArgVec& argv, - Poco::Util::LayeredConfiguration& config, int priority); diff --git a/be/src/vec/common/columns_hashing.h b/be/src/vec/common/columns_hashing.h index 0fe0f0e1e46a2d..b5be2ba6409d0a 100644 --- a/be/src/vec/common/columns_hashing.h +++ b/be/src/vec/common/columns_hashing.h @@ -1,20 +1,15 @@ #pragma once -#include -#include -#include -#include -// #include -#include -#include -#include -// #include -// #include - -#include - #include +#include "vec/columns/column_string.h" +#include "vec/common/arena.h" +#include "vec/common/assert_cast.h" +#include "vec/common/columns_hashing_impl.h" +#include "vec/common/hash_table/hash_table.h" +#include "vec/common/hash_table/hash_table_key_holder.h" +#include "vec/common/unaligned.h" + namespace doris::vectorized { namespace ColumnsHashing { @@ -81,7 +76,7 @@ struct HashMethodString : public columns_hashing_impl::HashMethodBase< StringRef key(chars + offsets[row - 1], offsets[row] - offsets[row - 1] - 1); if constexpr (place_string_to_arena) { - return ArenaKeyHolder {key, pool}; + return ArenaKeyHolder{key, pool}; } else { return key; } @@ -91,406 +86,6 @@ struct HashMethodString : public columns_hashing_impl::HashMethodBase< friend class columns_hashing_impl::HashMethodBase; }; -/// 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 = assert_cast(column); -// n = column_string.getN(); -// chars = &column_string.getChars(); -// } - -// auto getKeyHolder(size_t row, [[maybe_unused]] Arena & pool) const -// { -// StringRef key(&(*chars)[row * n], n); - -// if constexpr (place_string_to_arena) -// { -// return ArenaKeyHolder{key, pool}; -// } -// else -// { -// return key; -// } -// } - -// protected: -// friend class columns_hashing_impl::HashMethodBase; -// }; - -/// 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 * lcd_cache; -// if constexpr (use_cache) -// { -// lcd_cache = typeid_cast(context.get()); -// if (!lcd_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 = lcd_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; - -// lcd_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 assert_cast(positions)->getElement(row); -// case sizeof(UInt16): return assert_cast(positions)->getElement(row); -// case sizeof(UInt32): return assert_cast(positions)->getElement(row); -// case sizeof(UInt64): return assert_cast(positions)->getElement(row); -// default: throw Exception("Unexpected size of index type for low cardinality column.", ErrorCodes::LOGICAL_ERROR); -// } -// } - -// /// Get the key holder from the key columns for insertion into the hash table. -// ALWAYS_INLINE auto getKeyHolder(size_t row, Arena & pool) const -// { -// return Base::getKeyHolder(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_holder = getKeyHolder(row_, pool); - -// bool inserted = false; -// typename Data::LookupResult it; -// if (saved_hash) -// data.emplace(key_holder, it, inserted, saved_hash[row]); -// else -// data.emplace(key_holder, it, inserted); - -// visit_cache[row] = VisitValue::Found; - -// if constexpr (has_mapped) -// { -// auto & mapped = *lookupResultGetMapped(it); -// if (inserted) -// { -// new (&mapped) Mapped(); -// } -// mapped_cache[row] = mapped; -// return EmplaceResult(mapped, 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_holder = getKeyHolder(row_, pool); - -// typename Data::iterator it; -// if (saved_hash) -// it = data.find(*key_holder, saved_hash[row]); -// else -// it = data.find(*key_holder); - -// 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 getKeyHolder(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. @@ -514,8 +109,8 @@ struct HashMethodSerialized friend class columns_hashing_impl::HashMethodBase; ALWAYS_INLINE SerializedKeyHolder getKeyHolder(size_t row, Arena& pool) const { - return SerializedKeyHolder { - serializeKeysToPoolContiguous(row, keys_size, key_columns, pool), pool}; + return SerializedKeyHolder{serializeKeysToPoolContiguous(row, keys_size, key_columns, pool), + pool}; } }; diff --git a/be/src/vec/common/config_common.h.in b/be/src/vec/common/config_common.h.in deleted file mode 100644 index 810cf0b87f9103..00000000000000 --- a/be/src/vec/common/config_common.h.in +++ /dev/null @@ -1,11 +0,0 @@ -#pragma once - -// .h autogenerated by cmake ! - -#cmakedefine01 USE_TCMALLOC -#cmakedefine01 USE_JEMALLOC -#cmakedefine01 USE_READLINE -#cmakedefine01 USE_LIBEDIT -#cmakedefine01 HAVE_READLINE_HISTORY -#cmakedefine01 UNBUNDLED -#cmakedefine01 WITH_COVERAGE diff --git a/be/src/vec/common/constexpr_helpers.h b/be/src/vec/common/constexpr_helpers.h deleted file mode 100644 index a9b3b8fa67dcdd..00000000000000 --- a/be/src/vec/common/constexpr_helpers.h +++ /dev/null @@ -1,45 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include - -template -inline constexpr bool static_in_v = std::disjunction_v...>; - -template -bool func_wrapper(Func&& func, Arg&& arg) { - if constexpr (std::is_void_v>) { - func(arg); - return false; - } else - return func(arg); -} - -template -constexpr bool static_for_impl(Func&& f, std::integer_sequence) { - return (func_wrapper(std::forward(f), std::integral_constant {}) || ...); -} - -template -constexpr bool static_for(Func&& f) { - using T = decltype(Begin); - return static_for_impl(std::forward(f), - std::make_integer_sequence {}); -} diff --git a/be/src/vec/common/date_lut.h b/be/src/vec/common/date_lut.h deleted file mode 100644 index ee32560ae42edb..00000000000000 --- a/be/src/vec/common/date_lut.h +++ /dev/null @@ -1,75 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include -#include -#include - -#include "DateLUTImpl.h" - -// Also defined in Core/Defines.h -#if !defined(ALWAYS_INLINE) -#if defined(_MSC_VER) -#define ALWAYS_INLINE __forceinline -#else -#define ALWAYS_INLINE __attribute__((__always_inline__)) -#endif -#endif - -/// This class provides lazy initialization and lookup of singleton DateLUTImpl objects for a given timezone. -class DateLUT : private boost::noncopyable { -public: - /// Return singleton DateLUTImpl instance for the default time zone. - static ALWAYS_INLINE const DateLUTImpl& instance() { - const auto& date_lut = getInstance(); - return *date_lut.default_impl.load(std::memory_order_acquire); - } - - /// Return singleton DateLUTImpl instance for a given time zone. - static ALWAYS_INLINE const DateLUTImpl& instance(const std::string& time_zone) { - const auto& date_lut = getInstance(); - if (time_zone.empty()) return *date_lut.default_impl.load(std::memory_order_acquire); - - return date_lut.getImplementation(time_zone); - } - - static void setDefaultTimezone(const std::string& time_zone) { - auto& date_lut = getInstance(); - const auto& impl = date_lut.getImplementation(time_zone); - date_lut.default_impl.store(&impl, std::memory_order_release); - } - -protected: - DateLUT(); - -private: - static DateLUT& getInstance(); - - const DateLUTImpl& getImplementation(const std::string& time_zone) const; - - using DateLUTImplPtr = std::unique_ptr; - - /// Time zone name -> implementation. - mutable std::unordered_map impls; - mutable std::mutex mutex; - - std::atomic default_impl; -}; diff --git a/be/src/vec/common/date_lut_impl.h b/be/src/vec/common/date_lut_impl.h deleted file mode 100644 index 35bd2dc166fc00..00000000000000 --- a/be/src/vec/common/date_lut_impl.h +++ /dev/null @@ -1,790 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include - -#include "DayNum.h" -#include "Types.h" -#include "common/compiler_util.h" - -#define DATE_LUT_MAX (0xFFFFFFFFU - 86400) -#define DATE_LUT_MAX_DAY_NUM (0xFFFFFFFFU / 86400) -/// Table size is bigger than DATE_LUT_MAX_DAY_NUM to fill all indices within UInt16 range: this allows to remove extra check. -#define DATE_LUT_SIZE 0x10000 -#define DATE_LUT_MIN_YEAR 1970 -#define DATE_LUT_MAX_YEAR 2105 /// Last supported year -#define DATE_LUT_YEARS \ - (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table - -#if defined(__PPC__) -#if !__clang__ -#pragma GCC diagnostic ignored "-Wmaybe-uninitialized" -#endif -#endif - -/// Flags for toYearWeek() function. -enum class WeekModeFlag : UInt8 { MONDAY_FIRST = 1, YEAR = 2, FIRST_WEEKDAY = 4, NEWYEAR_DAY = 8 }; -using YearWeek = std::pair; - -/** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on. - * First time was implemented for OLAPServer, that needed to do billions of such transformations. - */ -class DateLUTImpl { -public: - DateLUTImpl(const std::string& time_zone); - -public: - /// The order of fields matters for alignment and sizeof. - struct Values { - /// Least significat 32 bits from time_t at beginning of the day. - /// If the unix timestamp of beginning of the day is negative (example: 1970-01-01 MSK, where time_t == -10800), then value is zero. - /// Change to time_t; change constants above; and recompile the sources if you need to support time after 2105 year. - UInt32 date; - - /// Properties of the day. - UInt16 year; - UInt8 month; - UInt8 day_of_month; - UInt8 day_of_week; - - /// Total number of days in current month. Actually we can use separate table that is independent of time zone. - /// But due to alignment, this field is totally zero cost. - UInt8 days_in_month; - - /// For days, when offset from UTC was changed due to daylight saving time or permanent change, following values could be non zero. - Int16 amount_of_offset_change; /// Usually -3600 or 3600, but look at Lord Howe Island. - UInt32 time_at_offset_change; /// In seconds from beginning of the day. - }; - - static_assert(sizeof(Values) == 16); - -private: - /// Lookup table is indexed by DayNum. - /// Day nums are the same in all time zones. 1970-01-01 is 0 and so on. - /// Table is relatively large, so better not to place the object on stack. - /// In comparison to std::vector, plain array is cheaper by one indirection. - Values lut[DATE_LUT_SIZE]; - - /// Year number after DATE_LUT_MIN_YEAR -> day num for start of year. - DayNum years_lut[DATE_LUT_YEARS]; - - /// Year number after DATE_LUT_MIN_YEAR * month number starting at zero -> day num for first day of month - DayNum years_months_lut[DATE_LUT_YEARS * 12]; - - /// UTC offset at beginning of the Unix epoch. The same as unix timestamp of 1970-01-01 00:00:00 local time. - time_t offset_at_start_of_epoch; - bool offset_is_whole_number_of_hours_everytime; - - /// Time zone name. - std::string time_zone; - - /// We can correctly process only timestamps that less DATE_LUT_MAX (i.e. up to 2105 year inclusively) - /// We don't care about overflow. - inline DayNum findIndex(time_t t) const { - /// First guess. - DayNum guess(t / 86400); - - /// UTC offset is from -12 to +14 in all known time zones. This requires checking only three indices. - - if ((guess == 0 || t >= lut[guess].date) && t < lut[DayNum(guess + 1)].date) return guess; - - /// Time zones that have offset 0 from UTC do daylight saving time change (if any) towards increasing UTC offset (example: British Standard Time). - if (offset_at_start_of_epoch >= 0) return DayNum(guess + 1); - - return DayNum(guess - 1); - } - - inline const Values& find(time_t t) const { return lut[findIndex(t)]; } - -public: - const std::string& getTimeZone() const { return time_zone; } - - /// All functions below are thread-safe; arguments are not checked. - - inline time_t toDate(time_t t) const { return find(t).date; } - inline unsigned toMonth(time_t t) const { return find(t).month; } - inline unsigned toQuarter(time_t t) const { return (find(t).month - 1) / 3 + 1; } - inline unsigned toYear(time_t t) const { return find(t).year; } - inline unsigned toDayOfWeek(time_t t) const { return find(t).day_of_week; } - inline unsigned toDayOfMonth(time_t t) const { return find(t).day_of_month; } - - /// Round down to start of monday. - inline time_t toFirstDayOfWeek(time_t t) const { - DayNum index = findIndex(t); - return lut[DayNum(index - (lut[index].day_of_week - 1))].date; - } - - inline DayNum toFirstDayNumOfWeek(DayNum d) const { - return DayNum(d - (lut[d].day_of_week - 1)); - } - - inline DayNum toFirstDayNumOfWeek(time_t t) const { return toFirstDayNumOfWeek(toDayNum(t)); } - - /// Round down to start of month. - inline time_t toFirstDayOfMonth(time_t t) const { - DayNum index = findIndex(t); - return lut[index - (lut[index].day_of_month - 1)].date; - } - - inline DayNum toFirstDayNumOfMonth(DayNum d) const { - return DayNum(d - (lut[d].day_of_month - 1)); - } - - inline DayNum toFirstDayNumOfMonth(time_t t) const { return toFirstDayNumOfMonth(toDayNum(t)); } - - /// Round down to start of quarter. - inline DayNum toFirstDayNumOfQuarter(DayNum d) const { - DayNum index = d; - size_t month_inside_quarter = (lut[index].month - 1) % 3; - - index -= lut[index].day_of_month; - while (month_inside_quarter) { - index -= lut[index].day_of_month; - --month_inside_quarter; - } - - return DayNum(index + 1); - } - - inline DayNum toFirstDayNumOfQuarter(time_t t) const { - return toFirstDayNumOfQuarter(toDayNum(t)); - } - - inline time_t toFirstDayOfQuarter(time_t t) const { - return fromDayNum(toFirstDayNumOfQuarter(t)); - } - - /// Round down to start of year. - inline time_t toFirstDayOfYear(time_t t) const { - return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date; - } - - inline DayNum toFirstDayNumOfYear(DayNum d) const { - return years_lut[lut[d].year - DATE_LUT_MIN_YEAR]; - } - - inline DayNum toFirstDayNumOfYear(time_t t) const { return toFirstDayNumOfYear(toDayNum(t)); } - - inline time_t toFirstDayOfNextMonth(time_t t) const { - DayNum index = findIndex(t); - index += 32 - lut[index].day_of_month; - return lut[index - (lut[index].day_of_month - 1)].date; - } - - inline time_t toFirstDayOfPrevMonth(time_t t) const { - DayNum index = findIndex(t); - index -= lut[index].day_of_month; - return lut[index - (lut[index].day_of_month - 1)].date; - } - - inline UInt8 daysInMonth(DayNum d) const { return lut[d].days_in_month; } - - inline UInt8 daysInMonth(time_t t) const { return find(t).days_in_month; } - - inline UInt8 daysInMonth(UInt16 year, UInt8 month) const { - /// 32 makes arithmetic more simple. - DayNum any_day_of_month = DayNum(years_lut[year - DATE_LUT_MIN_YEAR] + 32 * (month - 1)); - return lut[any_day_of_month].days_in_month; - } - - /** Round to start of day, then shift for specified amount of days. - */ - inline time_t toDateAndShift(time_t t, Int32 days) const { - return lut[DayNum(findIndex(t) + days)].date; - } - - inline time_t toTime(time_t t) const { - DayNum index = findIndex(t); - - if (UNLIKELY(index == 0)) return t + offset_at_start_of_epoch; - - time_t res = t - lut[index].date; - - if (res >= lut[index].time_at_offset_change) res += lut[index].amount_of_offset_change; - - return res - offset_at_start_of_epoch; /// Starting at 1970-01-01 00:00:00 local time. - } - - inline unsigned toHour(time_t t) const { - DayNum index = findIndex(t); - - /// If it is not 1970 year (findIndex found nothing appropriate), - /// than limit number of hours to avoid insane results like 1970-01-01 89:28:15 - if (UNLIKELY(index == 0)) - return static_cast((t + offset_at_start_of_epoch) / 3600) % 24; - - time_t res = t - lut[index].date; - - /// Data is cleaned to avoid possibility of underflow. - if (res >= lut[index].time_at_offset_change) res += lut[index].amount_of_offset_change; - - return res / 3600; - } - - /** Only for time zones with/when offset from UTC is multiple of five minutes. - * This is true for all time zones: right now, all time zones have an offset that is multiple of 15 minutes. - * - * "By 1929, most major countries had adopted hourly time zones. Nepal was the last - * country to adopt a standard offset, shifting slightly to UTC+5:45 in 1986." - * - https://en.wikipedia.org/wiki/Time_zone#Offsets_from_UTC - * - * Also please note, that unix timestamp doesn't count "leap seconds": - * each minute, with added or subtracted leap second, spans exactly 60 unix timestamps. - */ - - inline unsigned toSecond(time_t t) const { return t % 60; } - - inline unsigned toMinute(time_t t) const { - if (offset_is_whole_number_of_hours_everytime) return (t / 60) % 60; - - time_t date = find(t).date; - return (t - date) / 60 % 60; - } - - inline time_t toStartOfMinute(time_t t) const { return t / 60 * 60; } - inline time_t toStartOfFiveMinute(time_t t) const { return t / 300 * 300; } - inline time_t toStartOfFifteenMinutes(time_t t) const { return t / 900 * 900; } - inline time_t toStartOfTenMinutes(time_t t) const { return t / 600 * 600; } - - inline time_t toStartOfHour(time_t t) const { - if (offset_is_whole_number_of_hours_everytime) return t / 3600 * 3600; - - time_t date = find(t).date; - /// Still can return wrong values for time at 1970-01-01 if the UTC offset was non-whole number of hours. - return date + (t - date) / 3600 * 3600; - } - - /** Number of calendar day since the beginning of UNIX epoch (1970-01-01 is zero) - * We use just two bytes for it. It covers the range up to 2105 and slightly more. - * - * This is "calendar" day, it itself is independent of time zone - * (conversion from/to unix timestamp will depend on time zone, - * because the same calendar day starts/ends at different timestamps in different time zones) - */ - - inline DayNum toDayNum(time_t t) const { return findIndex(t); } - inline time_t fromDayNum(DayNum d) const { return lut[d].date; } - - inline time_t toDate(DayNum d) const { return lut[d].date; } - inline unsigned toMonth(DayNum d) const { return lut[d].month; } - inline unsigned toQuarter(DayNum d) const { return (lut[d].month - 1) / 3 + 1; } - inline unsigned toYear(DayNum d) const { return lut[d].year; } - inline unsigned toDayOfWeek(DayNum d) const { return lut[d].day_of_week; } - inline unsigned toDayOfMonth(DayNum d) const { return lut[d].day_of_month; } - inline unsigned toDayOfYear(DayNum d) const { return d + 1 - toFirstDayNumOfYear(d); } - - inline unsigned toDayOfYear(time_t t) const { return toDayOfYear(toDayNum(t)); } - - /// Number of week from some fixed moment in the past. Week begins at monday. - /// (round down to monday and divide DayNum by 7; we made an assumption, - /// that in domain of the function there was no weeks with any other number of days than 7) - inline unsigned toRelativeWeekNum(DayNum d) const { - /// We add 8 to avoid underflow at beginning of unix epoch. - return (d + 8 - toDayOfWeek(d)) / 7; - } - - inline unsigned toRelativeWeekNum(time_t t) const { return toRelativeWeekNum(toDayNum(t)); } - - /// Get year that contains most of the current week. Week begins at monday. - inline unsigned toISOYear(DayNum d) const { - /// That's effectively the year of thursday of current week. - return toYear(DayNum(d + 4 - toDayOfWeek(d))); - } - - inline unsigned toISOYear(time_t t) const { return toISOYear(toDayNum(t)); } - - /// ISO year begins with a monday of the week that is contained more than by half in the corresponding calendar year. - /// Example: ISO year 2019 begins at 2018-12-31. And ISO year 2017 begins at 2017-01-02. - /// https://en.wikipedia.org/wiki/ISO_week_date - inline DayNum toFirstDayNumOfISOYear(DayNum d) const { - auto iso_year = toISOYear(d); - - DayNum first_day_of_year = years_lut[iso_year - DATE_LUT_MIN_YEAR]; - auto first_day_of_week_of_year = lut[first_day_of_year].day_of_week; - - return DayNum(first_day_of_week_of_year <= 4 - ? first_day_of_year + 1 - first_day_of_week_of_year - : first_day_of_year + 8 - first_day_of_week_of_year); - } - - inline DayNum toFirstDayNumOfISOYear(time_t t) const { - return toFirstDayNumOfISOYear(toDayNum(t)); - } - - inline time_t toFirstDayOfISOYear(time_t t) const { - return fromDayNum(toFirstDayNumOfISOYear(t)); - } - - /// ISO 8601 week number. Week begins at monday. - /// The week number 1 is the first week in year that contains 4 or more days (that's more than half). - inline unsigned toISOWeek(DayNum d) const { - return 1 + DayNum(toFirstDayNumOfWeek(d) - toFirstDayNumOfISOYear(d)) / 7; - } - - inline unsigned toISOWeek(time_t t) const { return toISOWeek(toDayNum(t)); } - - /* - The bits in week_mode has the following meaning: - WeekModeFlag::MONDAY_FIRST (0) If not set Sunday is first day of week - If set Monday is first day of week - WeekModeFlag::YEAR (1) If not set Week is in range 0-53 - - Week 0 is returned for the the last week of the previous year (for - a date at start of january) In this case one can get 53 for the - first week of next year. This flag ensures that the week is - relevant for the given year. Note that this flag is only - releveant if WeekModeFlag::JANUARY is not set. - - If set Week is in range 1-53. - - In this case one may get week 53 for a date in January (when - the week is that last week of previous year) and week 1 for a - date in December. - - WeekModeFlag::FIRST_WEEKDAY (2) If not set Weeks are numbered according - to ISO 8601:1988 - If set The week that contains the first - 'first-day-of-week' is week 1. - - WeekModeFlag::NEWYEAR_DAY (3) If not set no meaning - If set The week that contains the January 1 is week 1. - Week is in range 1-53. - And ignore WeekModeFlag::YEAR, WeekModeFlag::FIRST_WEEKDAY - - ISO 8601:1988 means that if the week containing January 1 has - four or more days in the new year, then it is week 1; - Otherwise it is the last week of the previous year, and the - next week is week 1. - */ - inline YearWeek toYearWeek(DayNum d, UInt8 week_mode) const { - bool newyear_day_mode = week_mode & static_cast(WeekModeFlag::NEWYEAR_DAY); - week_mode = check_week_mode(week_mode); - bool monday_first_mode = week_mode & static_cast(WeekModeFlag::MONDAY_FIRST); - bool week_year_mode = week_mode & static_cast(WeekModeFlag::YEAR); - bool first_weekday_mode = week_mode & static_cast(WeekModeFlag::FIRST_WEEKDAY); - - // Calculate week number of WeekModeFlag::NEWYEAR_DAY mode - if (newyear_day_mode) { - return toYearWeekOfNewyearMode(d, monday_first_mode); - } - - YearWeek yw(toYear(d), 0); - UInt16 days = 0; - UInt16 daynr = makeDayNum(yw.first, toMonth(d), toDayOfMonth(d)); - UInt16 first_daynr = makeDayNum(yw.first, 1, 1); - - // 0 for monday, 1 for tuesday ... - // get weekday from first day in year. - UInt16 weekday = calc_weekday(DayNum(first_daynr), !monday_first_mode); - - if (toMonth(d) == 1 && toDayOfMonth(d) <= static_cast(7 - weekday)) { - if (!week_year_mode && - ((first_weekday_mode && weekday != 0) || (!first_weekday_mode && weekday >= 4))) - return yw; - week_year_mode = 1; - (yw.first)--; - first_daynr -= (days = calc_days_in_year(yw.first)); - weekday = (weekday + 53 * 7 - days) % 7; - } - - if ((first_weekday_mode && weekday != 0) || (!first_weekday_mode && weekday >= 4)) - days = daynr - (first_daynr + (7 - weekday)); - else - days = daynr - (first_daynr - weekday); - - if (week_year_mode && days >= 52 * 7) { - weekday = (weekday + calc_days_in_year(yw.first)) % 7; - if ((!first_weekday_mode && weekday < 4) || (first_weekday_mode && weekday == 0)) { - (yw.first)++; - yw.second = 1; - return yw; - } - } - yw.second = days / 7 + 1; - return yw; - } - - /// Calculate week number of WeekModeFlag::NEWYEAR_DAY mode - /// The week number 1 is the first week in year that contains January 1, - inline YearWeek toYearWeekOfNewyearMode(DayNum d, bool monday_first_mode) const { - YearWeek yw(0, 0); - UInt16 offset_day = monday_first_mode ? 0U : 1U; - - // Checking the week across the year - yw.first = toYear(DayNum(d + 7 - toDayOfWeek(DayNum(d + offset_day)))); - - DayNum first_day = makeDayNum(yw.first, 1, 1); - DayNum this_day = d; - - if (monday_first_mode) { - // Rounds down a date to the nearest Monday. - first_day = toFirstDayNumOfWeek(first_day); - this_day = toFirstDayNumOfWeek(d); - } else { - // Rounds down a date to the nearest Sunday. - if (toDayOfWeek(first_day) != 7) first_day = DayNum(first_day - toDayOfWeek(first_day)); - if (toDayOfWeek(d) != 7) this_day = DayNum(d - toDayOfWeek(d)); - } - yw.second = (this_day - first_day) / 7 + 1; - return yw; - } - - /** - * get first day of week with week_mode, return Sunday or Monday - */ - inline DayNum toFirstDayNumOfWeek(DayNum d, UInt8 week_mode) const { - bool monday_first_mode = week_mode & static_cast(WeekModeFlag::MONDAY_FIRST); - if (monday_first_mode) { - return toFirstDayNumOfWeek(d); - } else { - return (toDayOfWeek(d) != 7) ? DayNum(d - toDayOfWeek(d)) : d; - } - } - - /* - * check and change mode to effective - */ - inline UInt8 check_week_mode(UInt8 mode) const { - UInt8 week_format = (mode & 7); - if (!(week_format & static_cast(WeekModeFlag::MONDAY_FIRST))) - week_format ^= static_cast(WeekModeFlag::FIRST_WEEKDAY); - return week_format; - } - - /* - * Calc weekday from d - * Returns 0 for monday, 1 for tuesday ... - */ - inline unsigned calc_weekday(DayNum d, bool sunday_first_day_of_week) const { - if (!sunday_first_day_of_week) - return toDayOfWeek(d) - 1; - else - return toDayOfWeek(DayNum(d + 1)) - 1; - } - - /* Calc days in one year. */ - inline unsigned calc_days_in_year(UInt16 year) const { - return ((year & 3) == 0 && (year % 100 || (year % 400 == 0 && year)) ? 366 : 365); - } - - /// Number of month from some fixed moment in the past (year * 12 + month) - inline unsigned toRelativeMonthNum(DayNum d) const { return lut[d].year * 12 + lut[d].month; } - - inline unsigned toRelativeMonthNum(time_t t) const { return toRelativeMonthNum(toDayNum(t)); } - - inline unsigned toRelativeQuarterNum(DayNum d) const { - return lut[d].year * 4 + (lut[d].month - 1) / 3; - } - - inline unsigned toRelativeQuarterNum(time_t t) const { - return toRelativeQuarterNum(toDayNum(t)); - } - - /// We count all hour-length intervals, unrelated to offset changes. - inline time_t toRelativeHourNum(time_t t) const { - if (offset_is_whole_number_of_hours_everytime) return t / 3600; - - /// Assume that if offset was fractional, then the fraction is the same as at the beginning of epoch. - /// NOTE This assumption is false for "Pacific/Pitcairn" time zone. - return (t + 86400 - offset_at_start_of_epoch) / 3600; - } - - inline time_t toRelativeHourNum(DayNum d) const { return toRelativeHourNum(lut[d].date); } - - inline time_t toRelativeMinuteNum(time_t t) const { return t / 60; } - - inline time_t toRelativeMinuteNum(DayNum d) const { return toRelativeMinuteNum(lut[d].date); } - - inline DayNum toStartOfYearInterval(DayNum d, UInt64 years) const { - if (years == 1) return toFirstDayNumOfYear(d); - return years_lut[(lut[d].year - DATE_LUT_MIN_YEAR) / years * years]; - } - - inline DayNum toStartOfQuarterInterval(DayNum d, UInt64 quarters) const { - if (quarters == 1) return toFirstDayNumOfQuarter(d); - return toStartOfMonthInterval(d, quarters * 3); - } - - inline DayNum toStartOfMonthInterval(DayNum d, UInt64 months) const { - if (months == 1) return toFirstDayNumOfMonth(d); - const auto& date = lut[d]; - UInt32 month_total_index = (date.year - DATE_LUT_MIN_YEAR) * 12 + date.month - 1; - return years_months_lut[month_total_index / months * months]; - } - - inline DayNum toStartOfWeekInterval(DayNum d, UInt64 weeks) const { - if (weeks == 1) return toFirstDayNumOfWeek(d); - UInt64 days = weeks * 7; - // January 1st 1970 was Thursday so we need this 4-days offset to make weeks start on Monday. - return DayNum(4 + (d - 4) / days * days); - } - - inline time_t toStartOfDayInterval(DayNum d, UInt64 days) const { - if (days == 1) return toDate(d); - return lut[d / days * days].date; - } - - inline time_t toStartOfHourInterval(time_t t, UInt64 hours) const { - if (hours == 1) return toStartOfHour(t); - UInt64 seconds = hours * 3600; - t = t / seconds * seconds; - if (offset_is_whole_number_of_hours_everytime) return t; - return toStartOfHour(t); - } - - inline time_t toStartOfMinuteInterval(time_t t, UInt64 minutes) const { - if (minutes == 1) return toStartOfMinute(t); - UInt64 seconds = 60 * minutes; - return t / seconds * seconds; - } - - inline time_t toStartOfSecondInterval(time_t t, UInt64 seconds) const { - if (seconds == 1) return t; - return t / seconds * seconds; - } - - /// Create DayNum from year, month, day of month. - inline DayNum makeDayNum(UInt16 year, UInt8 month, UInt8 day_of_month) const { - if (UNLIKELY(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || - month > 12 || day_of_month < 1 || day_of_month > 31)) - return DayNum(0); - - return DayNum(years_months_lut[(year - DATE_LUT_MIN_YEAR) * 12 + month - 1] + day_of_month - - 1); - } - - inline time_t makeDate(UInt16 year, UInt8 month, UInt8 day_of_month) const { - return lut[makeDayNum(year, month, day_of_month)].date; - } - - /** Does not accept daylight saving time as argument: in case of ambiguity, it choose greater timestamp. - */ - inline time_t makeDateTime(UInt16 year, UInt8 month, UInt8 day_of_month, UInt8 hour, - UInt8 minute, UInt8 second) const { - size_t index = makeDayNum(year, month, day_of_month); - time_t time_offset = hour * 3600 + minute * 60 + second; - - if (time_offset >= lut[index].time_at_offset_change) - time_offset -= lut[index].amount_of_offset_change; - - return lut[index].date + time_offset; - } - - inline const Values& getValues(DayNum d) const { return lut[d]; } - inline const Values& getValues(time_t t) const { return lut[findIndex(t)]; } - - inline UInt32 toNumYYYYMM(time_t t) const { - const Values& values = find(t); - return values.year * 100 + values.month; - } - - inline UInt32 toNumYYYYMM(DayNum d) const { - const Values& values = lut[d]; - return values.year * 100 + values.month; - } - - inline UInt32 toNumYYYYMMDD(time_t t) const { - const Values& values = find(t); - return values.year * 10000 + values.month * 100 + values.day_of_month; - } - - inline UInt32 toNumYYYYMMDD(DayNum d) const { - const Values& values = lut[d]; - return values.year * 10000 + values.month * 100 + values.day_of_month; - } - - inline time_t YYYYMMDDToDate(UInt32 num) const { - return makeDate(num / 10000, num / 100 % 100, num % 100); - } - - inline DayNum YYYYMMDDToDayNum(UInt32 num) const { - return makeDayNum(num / 10000, num / 100 % 100, num % 100); - } - - inline UInt64 toNumYYYYMMDDhhmmss(time_t t) const { - const Values& values = find(t); - return toSecond(t) + toMinute(t) * 100 + toHour(t) * 10000 + - UInt64(values.day_of_month) * 1000000 + UInt64(values.month) * 100000000 + - UInt64(values.year) * 10000000000; - } - - inline time_t YYYYMMDDhhmmssToTime(UInt64 num) const { - return makeDateTime(num / 10000000000, num / 100000000 % 100, num / 1000000 % 100, - num / 10000 % 100, num / 100 % 100, num % 100); - } - - /// Adding calendar intervals. - /// Implementation specific behaviour when delta is too big. - - inline time_t addDays(time_t t, Int64 delta) const { - DayNum index = findIndex(t); - time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t); - - index += delta; - - if (time_offset >= lut[index].time_at_offset_change) - time_offset -= lut[index].amount_of_offset_change; - - return lut[index].date + time_offset; - } - - inline time_t addWeeks(time_t t, Int64 delta) const { return addDays(t, delta * 7); } - - inline UInt8 saturateDayOfMonth(UInt16 year, UInt8 month, UInt8 day_of_month) const { - if (LIKELY(day_of_month <= 28)) return day_of_month; - - UInt8 days_in_month = daysInMonth(year, month); - - if (day_of_month > days_in_month) day_of_month = days_in_month; - - return day_of_month; - } - - /// If resulting month has less deys than source month, then saturation can happen. - /// Example: 31 Aug + 1 month = 30 Sep. - inline time_t addMonths(time_t t, Int64 delta) const { - DayNum result_day = addMonths(toDayNum(t), delta); - - time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t); - - if (time_offset >= lut[result_day].time_at_offset_change) - time_offset -= lut[result_day].amount_of_offset_change; - - return lut[result_day].date + time_offset; - } - - inline DayNum addMonths(DayNum d, Int64 delta) const { - const Values& values = lut[d]; - - Int64 month = static_cast(values.month) + delta; - - if (month > 0) { - auto year = values.year + (month - 1) / 12; - month = ((month - 1) % 12) + 1; - auto day_of_month = saturateDayOfMonth(year, month, values.day_of_month); - - return makeDayNum(year, month, day_of_month); - } else { - auto year = values.year - (12 - month) / 12; - month = 12 - (-month % 12); - auto day_of_month = saturateDayOfMonth(year, month, values.day_of_month); - - return makeDayNum(year, month, day_of_month); - } - } - - inline time_t addQuarters(time_t t, Int64 delta) const { return addMonths(t, delta * 3); } - - inline DayNum addQuarters(DayNum d, Int64 delta) const { return addMonths(d, delta * 3); } - - /// Saturation can occur if 29 Feb is mapped to non-leap year. - inline time_t addYears(time_t t, Int64 delta) const { - DayNum result_day = addYears(toDayNum(t), delta); - - time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t); - - if (time_offset >= lut[result_day].time_at_offset_change) - time_offset -= lut[result_day].amount_of_offset_change; - - return lut[result_day].date + time_offset; - } - - inline DayNum addYears(DayNum d, Int64 delta) const { - const Values& values = lut[d]; - - auto year = values.year + delta; - auto month = values.month; - auto day_of_month = values.day_of_month; - - /// Saturation to 28 Feb can happen. - if (UNLIKELY(day_of_month == 29 && month == 2)) - day_of_month = saturateDayOfMonth(year, month, day_of_month); - - return makeDayNum(year, month, day_of_month); - } - - inline std::string timeToString(time_t t) const { - const Values& values = find(t); - - std::string s {"0000-00-00 00:00:00"}; - - s[0] += values.year / 1000; - s[1] += (values.year / 100) % 10; - s[2] += (values.year / 10) % 10; - s[3] += values.year % 10; - s[5] += values.month / 10; - s[6] += values.month % 10; - s[8] += values.day_of_month / 10; - s[9] += values.day_of_month % 10; - - auto hour = toHour(t); - auto minute = toMinute(t); - auto second = toSecond(t); - - s[11] += hour / 10; - s[12] += hour % 10; - s[14] += minute / 10; - s[15] += minute % 10; - s[17] += second / 10; - s[18] += second % 10; - - return s; - } - - inline std::string dateToString(time_t t) const { - const Values& values = find(t); - - std::string s {"0000-00-00"}; - - s[0] += values.year / 1000; - s[1] += (values.year / 100) % 10; - s[2] += (values.year / 10) % 10; - s[3] += values.year % 10; - s[5] += values.month / 10; - s[6] += values.month % 10; - s[8] += values.day_of_month / 10; - s[9] += values.day_of_month % 10; - - return s; - } - - inline std::string dateToString(DayNum d) const { - const Values& values = lut[d]; - - std::string s {"0000-00-00"}; - - s[0] += values.year / 1000; - s[1] += (values.year / 100) % 10; - s[2] += (values.year / 10) % 10; - s[3] += values.year % 10; - s[5] += values.month / 10; - s[6] += values.month % 10; - s[8] += values.day_of_month / 10; - s[9] += values.day_of_month % 10; - - return s; - } -}; - -#if defined(__PPC__) -#if !__clang__ -#pragma GCC diagnostic pop -#endif -#endif diff --git a/be/src/vec/common/day_num.h b/be/src/vec/common/day_num.h deleted file mode 100644 index 7b9d8324122485..00000000000000 --- a/be/src/vec/common/day_num.h +++ /dev/null @@ -1,26 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include "vec/common/strong_typedef.h" -#include "vec/common/types.h" - -/** Represents number of days since 1970-01-01. - * See DateLUTImpl for usage examples. - */ -STRONG_TYPEDEF(UInt16, DayNum) diff --git a/be/src/vec/common/error_handlers.h b/be/src/vec/common/error_handlers.h deleted file mode 100644 index 40dc311d2b5688..00000000000000 --- a/be/src/vec/common/error_handlers.h +++ /dev/null @@ -1,48 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include - -#include "vec/common/exception.h" -#include "vec/common/logger_useful.h" - -/** ErrorHandler for Poco::Thread, - * that in case of unhandled exception, - * logs exception message and terminates the process. - */ -class KillingErrorHandler : public Poco::ErrorHandler { -public: - void exception(const Poco::Exception&) { std::terminate(); } - void exception(const std::exception&) { std::terminate(); } - void exception() { std::terminate(); } -}; - -/** Log exception message. - */ -class ServerErrorHandler : public Poco::ErrorHandler { -public: - void exception(const Poco::Exception&) { logException(); } - void exception(const std::exception&) { logException(); } - void exception() { logException(); } - -private: - Logger* log = &Logger::get("ServerErrorHandler"); - - void logException() { doris::vectorized::tryLogCurrentException(log); } -}; diff --git a/be/src/vec/common/exception.cpp b/be/src/vec/common/exception.cpp index bf8c88e54f3e26..03827f94d76315 100644 --- a/be/src/vec/common/exception.cpp +++ b/be/src/vec/common/exception.cpp @@ -20,17 +20,8 @@ #include #include -#include -// #include -// #include -// #include -// #include -// #include -// #include -// #include -// #include -// #include #include +#include #include #include diff --git a/be/src/vec/common/find_symbols.h b/be/src/vec/common/find_symbols.h deleted file mode 100644 index d1b382d6713235..00000000000000 --- a/be/src/vec/common/find_symbols.h +++ /dev/null @@ -1,295 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include - -#if defined(__SSE2__) -#include -#endif -#if defined(__SSE4_2__) -#include -#endif - -/** find_first_symbols(begin, end): - * - * Allow to search for next character from the set of 'symbols...' in a string. - * It is similar to 'strpbrk', 'strcspn' (and 'strchr', 'memchr' in the case of one symbol and '\0'), - * but with the following differencies: - * - works with any memory ranges, including containing zero bytes; - * - doesn't require terminating zero byte: end of memory range is passed explicitly; - * - if not found, returns pointer to end instead of nullptr; - * - maximum number of symbols to search is 16. - * - * Uses SSE 2 in case of small number of symbols for search and SSE 4.2 in the case of large number of symbols, - * that have more than 2x performance advantage over trivial loop - * in the case of parsing tab-separated dump with (probably escaped) string fields. - * In the case of parsing tab separated dump with short strings, there is no performance degradation over trivial loop. - * - * Note: the optimal threshold to choose between SSE 2 and SSE 4.2 may depend on CPU model. - * - * find_last_symbols_or_null(begin, end): - * - * Allow to search for the last matching character in a string. - * If no such characters, returns nullptr. - */ - -namespace detail { - -template -inline bool is_in(char x) { - return x == s0; -} - -template -inline bool is_in(char x) { - return x == s0 || is_in(x); -} - -#if defined(__SSE2__) -template -inline __m128i mm_is_in(__m128i bytes) { - __m128i eq0 = _mm_cmpeq_epi8(bytes, _mm_set1_epi8(s0)); - return eq0; -} - -template -inline __m128i mm_is_in(__m128i bytes) { - __m128i eq0 = _mm_cmpeq_epi8(bytes, _mm_set1_epi8(s0)); - __m128i eq = mm_is_in(bytes); - return _mm_or_si128(eq0, eq); -} -#endif - -template -bool maybe_negate(bool x) { - if constexpr (positive) - return x; - else - return !x; -} - -template -uint16_t maybe_negate(uint16_t x) { - if constexpr (positive) - return x; - else - return ~x; -} - -enum class ReturnMode { - End, - Nullptr, -}; - -template -inline const char* find_first_symbols_sse2(const char* const begin, const char* const end) { - const char* pos = begin; - -#if defined(__SSE2__) - for (; pos + 15 < end; pos += 16) { - __m128i bytes = _mm_loadu_si128(reinterpret_cast(pos)); - - __m128i eq = mm_is_in(bytes); - - uint16_t bit_mask = maybe_negate(uint16_t(_mm_movemask_epi8(eq))); - if (bit_mask) return pos + __builtin_ctz(bit_mask); - } -#endif - - for (; pos < end; ++pos) - if (maybe_negate(is_in(*pos))) return pos; - - return return_mode == ReturnMode::End ? end : nullptr; -} - -template -inline const char* find_last_symbols_sse2(const char* const begin, const char* const end) { - const char* pos = end; - -#if defined(__SSE2__) - for (; pos - 16 >= begin; - pos -= - 16) /// Assuming the pointer cannot overflow. Assuming we can compare these pointers. - { - __m128i bytes = _mm_loadu_si128(reinterpret_cast(pos - 16)); - - __m128i eq = mm_is_in(bytes); - - uint16_t bit_mask = maybe_negate(uint16_t(_mm_movemask_epi8(eq))); - if (bit_mask) - return pos - 1 - - (__builtin_clz(bit_mask) - - 16); /// because __builtin_clz works with mask as uint32. - } -#endif - - --pos; - for (; pos >= begin; --pos) - if (maybe_negate(is_in(*pos))) return pos; - - return return_mode == ReturnMode::End ? end : nullptr; -} - -template -inline const char* find_first_symbols_sse42_impl(const char* const begin, const char* const end) { - const char* pos = begin; - -#if defined(__SSE4_2__) -#define MODE (_SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_ANY | _SIDD_LEAST_SIGNIFICANT) - __m128i set = _mm_setr_epi8(c01, c02, c03, c04, c05, c06, c07, c08, c09, c10, c11, c12, c13, - c14, c15, c16); - - for (; pos + 15 < end; pos += 16) { - __m128i bytes = _mm_loadu_si128(reinterpret_cast(pos)); - - if constexpr (positive) { - if (_mm_cmpestrc(set, num_chars, bytes, 16, MODE)) - return pos + _mm_cmpestri(set, num_chars, bytes, 16, MODE); - } else { - if (_mm_cmpestrc(set, num_chars, bytes, 16, MODE | _SIDD_NEGATIVE_POLARITY)) - return pos + - _mm_cmpestri(set, num_chars, bytes, 16, MODE | _SIDD_NEGATIVE_POLARITY); - } - } -#undef MODE -#endif - - for (; pos < end; ++pos) - if ((num_chars >= 1 && maybe_negate(*pos == c01)) || - (num_chars >= 2 && maybe_negate(*pos == c02)) || - (num_chars >= 3 && maybe_negate(*pos == c03)) || - (num_chars >= 4 && maybe_negate(*pos == c04)) || - (num_chars >= 5 && maybe_negate(*pos == c05)) || - (num_chars >= 6 && maybe_negate(*pos == c06)) || - (num_chars >= 7 && maybe_negate(*pos == c07)) || - (num_chars >= 8 && maybe_negate(*pos == c08)) || - (num_chars >= 9 && maybe_negate(*pos == c09)) || - (num_chars >= 10 && maybe_negate(*pos == c10)) || - (num_chars >= 11 && maybe_negate(*pos == c11)) || - (num_chars >= 12 && maybe_negate(*pos == c12)) || - (num_chars >= 13 && maybe_negate(*pos == c13)) || - (num_chars >= 14 && maybe_negate(*pos == c14)) || - (num_chars >= 15 && maybe_negate(*pos == c15)) || - (num_chars >= 16 && maybe_negate(*pos == c16))) - return pos; - return return_mode == ReturnMode::End ? end : nullptr; -} - -template -inline const char* find_first_symbols_sse42(const char* begin, const char* end) { - return find_first_symbols_sse42_impl( - begin, end); -} - -/// NOTE No SSE 4.2 implementation for find_last_symbols_or_null. Not worth to do. - -template -inline const char* find_first_symbols_dispatch(const char* begin, const char* end) { -#if defined(__SSE4_2__) - if (sizeof...(symbols) >= 5) - return find_first_symbols_sse42(begin, end); - else -#endif - return find_first_symbols_sse2(begin, end); -} - -} // namespace detail - -template -inline const char* find_first_symbols(const char* begin, const char* end) { - return detail::find_first_symbols_dispatch(begin, - end); -} - -/// Returning non const result for non const arguments. -/// It is convenient when you are using this function to iterate through non-const buffer. -template -inline char* find_first_symbols(char* begin, char* end) { - return const_cast( - detail::find_first_symbols_dispatch(begin, - end)); -} - -template -inline const char* find_first_not_symbols(const char* begin, const char* end) { - return detail::find_first_symbols_dispatch(begin, - end); -} - -template -inline char* find_first_not_symbols(char* begin, char* end) { - return const_cast( - detail::find_first_symbols_dispatch(begin, - end)); -} - -template -inline const char* find_first_symbols_or_null(const char* begin, const char* end) { - return detail::find_first_symbols_dispatch(begin, - end); -} - -template -inline char* find_first_symbols_or_null(char* begin, char* end) { - return const_cast( - detail::find_first_symbols_dispatch( - begin, end)); -} - -template -inline const char* find_first_not_symbols_or_null(const char* begin, const char* end) { - return detail::find_first_symbols_dispatch( - begin, end); -} - -template -inline char* find_first_not_symbols_or_null(char* begin, char* end) { - return const_cast( - detail::find_first_symbols_dispatch( - begin, end)); -} - -template -inline const char* find_last_symbols_or_null(const char* begin, const char* end) { - return detail::find_last_symbols_sse2(begin, - end); -} - -template -inline char* find_last_symbols_or_null(char* begin, char* end) { - return const_cast( - detail::find_last_symbols_sse2(begin, - end)); -} - -template -inline const char* find_last_not_symbols_or_null(const char* begin, const char* end) { - return detail::find_last_symbols_sse2(begin, - end); -} - -template -inline char* find_last_not_symbols_or_null(char* begin, char* end) { - return const_cast( - detail::find_last_symbols_sse2(begin, - end)); -} diff --git a/be/src/vec/common/get_memory_amount.h b/be/src/vec/common/get_memory_amount.h deleted file mode 100644 index 81e07d8654c1ab..00000000000000 --- a/be/src/vec/common/get_memory_amount.h +++ /dev/null @@ -1,29 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include - -/** Returns the size of physical memory (RAM) in bytes. - * Returns 0 on unsupported platform or if it cannot determine the size of physical memory. - */ -uint64_t getMemoryAmountOrZero(); - -/** Throws exception if it cannot determine the size of physical memory. - */ -uint64_t getMemoryAmount(); diff --git a/be/src/vec/common/get_thread_number.h b/be/src/vec/common/get_thread_number.h deleted file mode 100644 index 3a112f48bb5580..00000000000000 --- a/be/src/vec/common/get_thread_number.h +++ /dev/null @@ -1,24 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -/** Последовательный номер потока, начиная с 1, среди тех потоков, для которых был получен этот номер. - * Используется при логгировании. - */ - -unsigned getThreadNumber(); diff --git a/be/src/vec/common/hash_table/hash_table.h b/be/src/vec/common/hash_table/hash_table.h index 549b4c604da52b..30f7218fb5ea8b 100644 --- a/be/src/vec/common/hash_table/hash_table.h +++ b/be/src/vec/common/hash_table/hash_table.h @@ -2,23 +2,14 @@ #include #include - -#include -#include - -// #include - #include +#include +#include #include #include -// #include -// #include -// #include -// #include -// #include -#include -#include +#include +#include #include "vec/io/io_helper.h" @@ -234,7 +225,7 @@ template struct HashTableGrower { /// The state of this structure is enough to get the buffer size of the hash table. - UInt8 size_degree = initial_size_degree; + doris::vectorized::UInt8 size_degree = initial_size_degree; /// The size of the hash table in the cells. size_t bufSize() const { return 1ULL << size_degree; } @@ -512,7 +503,7 @@ class HashTable : private boost::noncopyable, } auto& operator*() const { return *ptr; } - auto* operator-> () const { return ptr; } + auto* operator->() const { return ptr; } auto getPtr() const { return ptr; } size_t getHash() const { return ptr->getHash(*container); } diff --git a/be/src/vec/common/iostream_debug_helpers.h b/be/src/vec/common/iostream_debug_helpers.h deleted file mode 100644 index 41e20cc6b678b9..00000000000000 --- a/be/src/vec/common/iostream_debug_helpers.h +++ /dev/null @@ -1,206 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include -#include - -#include "demangle.h" -#include "getThreadNumber.h" - -/** Usage: - * - * DUMP(variable...) - */ - -template -Out& dumpValue(Out&, T&&); - -/// Catch-all case. -template -std::enable_if_t& dumpImpl(Out& out, T&&) { - return out << "{...}"; -} - -/// An object, that could be output with operator <<. -template -std::enable_if_t& dumpImpl( - Out& out, T&& x, - std::decay_t() << std::declval())>* = nullptr) { - return out << x; -} - -/// A pointer-like object. -template -std::enable_if_t< - priority == 1 - /// Protect from the case when operator * do effectively nothing (function pointer). - && !std::is_same_v, std::decay_t())>>, - Out>& -dumpImpl(Out& out, T&& x, std::decay_t())>* = nullptr) { - if (!x) return out << "nullptr"; - return dumpValue(out, *x); -} - -/// Container. -template -std::enable_if_t& dumpImpl( - Out& out, T&& x, std::decay_t()))>* = nullptr) { - bool first = true; - out << "{"; - for (const auto& elem : x) { - if (first) - first = false; - else - out << ", "; - dumpValue(out, elem); - } - return out << "}"; -} - -/// string and const char * - output not as container or pointer. - -template -std::enable_if_t, std::string> || - std::is_same_v, const char*>), - Out>& -dumpImpl(Out& out, T&& x) { - return out << std::quoted(x); -} - -/// UInt8 - output as number, not char. - -template -std::enable_if_t, unsigned char>, Out>& dumpImpl( - Out& out, T&& x) { - return out << int(x); -} - -/// Tuple, pair -template -Out& dumpTupleImpl(Out& out, T&& x) { - if constexpr (N == 0) - out << "{"; - else - out << ", "; - - dumpValue(out, std::get(x)); - - if constexpr (N + 1 == std::tuple_size_v>) - out << "}"; - else - dumpTupleImpl(out, x); - - return out; -} - -template -std::enable_if_t& dumpImpl( - Out& out, T&& x, std::decay_t(std::declval()))>* = nullptr) { - return dumpTupleImpl<0>(out, x); -} - -template -Out& dumpDispatchPriorities( - Out& out, T&& x, - std::decay_t(std::declval(), std::declval()))>*) { - return dumpImpl(out, x); -} - -struct LowPriority { - LowPriority(void*) {} -}; - -template -Out& dumpDispatchPriorities(Out& out, T&& x, LowPriority) { - return dumpDispatchPriorities(out, x, nullptr); -} - -template -Out& dumpValue(Out& out, T&& x) { - return dumpDispatchPriorities<5>(out, x, nullptr); -} - -template -Out& dump(Out& out, const char* name, T&& x) { - out << demangle(typeid(x).name()) << " " << name << " = "; - return dumpValue(out, x); -} - -#ifdef __clang__ -#pragma clang diagnostic ignored "-Wgnu-zero-variadic-macro-arguments" -#endif - -#define DUMPVAR(VAR) \ - dump(std::cerr, #VAR, (VAR)); \ - std::cerr << "; "; -#define DUMPHEAD std::cerr << __FILE__ << ':' << __LINE__ << " [ " << getThreadNumber() << " ] "; -#define DUMPTAIL std::cerr << '\n'; - -#define DUMP1(V1) \ - do { \ - DUMPHEAD DUMPVAR(V1) DUMPTAIL \ - } while (0) -#define DUMP2(V1, V2) \ - do { \ - DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPTAIL \ - } while (0) -#define DUMP3(V1, V2, V3) \ - do { \ - DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPTAIL \ - } while (0) -#define DUMP4(V1, V2, V3, V4) \ - do { \ - DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPTAIL \ - } while (0) -#define DUMP5(V1, V2, V3, V4, V5) \ - do { \ - DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPVAR(V5) DUMPTAIL \ - } while (0) -#define DUMP6(V1, V2, V3, V4, V5, V6) \ - do { \ - DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPVAR(V5) DUMPVAR(V6) DUMPTAIL \ - } while (0) -#define DUMP7(V1, V2, V3, V4, V5, V6, V7) \ - do { \ - DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPVAR(V5) DUMPVAR(V6) \ - DUMPVAR(V7) DUMPTAIL \ - } while (0) -#define DUMP8(V1, V2, V3, V4, V5, V6, V7, V8) \ - do { \ - DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPVAR(V5) DUMPVAR(V6) \ - DUMPVAR(V7) DUMPVAR(V8) DUMPTAIL \ - } while (0) -#define DUMP9(V1, V2, V3, V4, V5, V6, V7, V8, V9) \ - do { \ - DUMPHEAD DUMPVAR(V1) DUMPVAR(V2) DUMPVAR(V3) DUMPVAR(V4) DUMPVAR(V5) DUMPVAR(V6) \ - DUMPVAR(V7) DUMPVAR(V8) DUMPVAR(V9) DUMPTAIL \ - } while (0) - -/// https://groups.google.com/forum/#!searchin/kona-dev/variadic$20macro%7Csort:date/kona-dev/XMA-lDOqtlI/GCzdfZsD41sJ - -#define VA_NUM_ARGS_IMPL(x1, x2, x3, x4, x5, x6, x7, x8, x9, N, ...) N -#define VA_NUM_ARGS(...) VA_NUM_ARGS_IMPL(__VA_ARGS__, 9, 8, 7, 6, 5, 4, 3, 2, 1) - -#define MAKE_VAR_MACRO_IMPL_CONCAT(PREFIX, NUM_ARGS) PREFIX##NUM_ARGS -#define MAKE_VAR_MACRO_IMPL(PREFIX, NUM_ARGS) MAKE_VAR_MACRO_IMPL_CONCAT(PREFIX, NUM_ARGS) -#define MAKE_VAR_MACRO(PREFIX, ...) MAKE_VAR_MACRO_IMPL(PREFIX, VA_NUM_ARGS(__VA_ARGS__)) - -#define DUMP(...) MAKE_VAR_MACRO(DUMP, __VA_ARGS__)(__VA_ARGS__) diff --git a/be/src/vec/common/itoa.h b/be/src/vec/common/itoa.h deleted file mode 100644 index 685fcd6534b0b0..00000000000000 --- a/be/src/vec/common/itoa.h +++ /dev/null @@ -1,377 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -// Based on https://github.com/amdn/itoa and combined with our optimizations -// -//=== itoa.h - Fast integer to ascii conversion --*- C++ -*-// -// -// The MIT License (MIT) -// Copyright (c) 2016 Arturo Martin-de-Nicolas -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included -// in all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -// SOFTWARE. -//===----------------------------------------------------------------------===// - -#include -#include -#include -#include - -#include "common/compiler_util.h" - -using int128_t = __int128; -using uint128_t = unsigned __int128; - -namespace impl { - -template -static constexpr T pow10(size_t x) { - return x ? 10 * pow10(x - 1) : 1; -} - -// Division by a power of 10 is implemented using a multiplicative inverse. -// This strength reduction is also done by optimizing compilers, but -// presently the fastest results are produced by using the values -// for the multiplication and the shift as given by the algorithm -// described by Agner Fog in "Optimizing Subroutines in Assembly Language" -// -// http://www.agner.org/optimize/optimizing_assembly.pdf -// -// "Integer division by a constant (all processors) -// A floating point number can be divided by a constant by multiplying -// with the reciprocal. If we want to do the same with integers, we have -// to scale the reciprocal by 2n and then shift the product to the right -// by n. There are various algorithms for finding a suitable value of n -// and compensating for rounding errors. The algorithm described below -// was invented by Terje Mathisen, Norway, and not published elsewhere." - -/// Division by constant is performed by: -/// 1. Adding 1 if needed; -/// 2. Multiplying by another constant; -/// 3. Shifting right by another constant. -template -struct Division { - static constexpr bool add {add_}; - static constexpr UInt multiplier {multiplier_}; - static constexpr unsigned shift {shift_}; -}; - -/// Select a type with appropriate number of bytes from the list of types. -/// First parameter is the number of bytes requested. Then goes a list of types with 1, 2, 4, ... number of bytes. -/// Example: SelectType<4, uint8_t, uint16_t, uint32_t, uint64_t> will select uint32_t. -template -struct SelectType { - using Result = typename SelectType::Result; -}; - -template -struct SelectType<1, T, Ts...> { - using Result = T; -}; - -/// Division by 10^N where N is the size of the type. -template -using DivisionBy10PowN = typename SelectType< - N, Division, /// divide by 10 - Division, /// divide by 100 - Division, /// divide by 10000 - Division /// divide by 100000000 - >::Result; - -template -using UnsignedOfSize = - typename SelectType::Result; - -/// Holds the result of dividing an unsigned N-byte variable by 10^N resulting in -template -struct QuotientAndRemainder { - UnsignedOfSize quotient; // quotient with fewer than 2*N decimal digits - UnsignedOfSize remainder; // remainder with at most N decimal digits -}; - -template -QuotientAndRemainder static inline split(UnsignedOfSize value) { - constexpr DivisionBy10PowN division; - - UnsignedOfSize quotient = - (division.multiplier * (UnsignedOfSize<2 * N>(value) + division.add)) >> division.shift; - UnsignedOfSize remainder = value - quotient * pow10>(N); - - return {quotient, remainder}; -} - -static inline char* outDigit(char* p, uint8_t value) { - *p = '0' + value; - ++p; - return p; -} - -// Using a lookup table to convert binary numbers from 0 to 99 -// into ascii characters as described by Andrei Alexandrescu in -// https://www.facebook.com/notes/facebook-engineering/three-optimization-tips-for-c/10151361643253920/ - -static const char digits[201] = - "00010203040506070809" - "10111213141516171819" - "20212223242526272829" - "30313233343536373839" - "40414243444546474849" - "50515253545556575859" - "60616263646566676869" - "70717273747576777879" - "80818283848586878889" - "90919293949596979899"; - -static inline char* outTwoDigits(char* p, uint8_t value) { - memcpy(p, &digits[value * 2], 2); - p += 2; - return p; -} - -namespace convert { -template -static char* head(char* p, UInt u); -template -static char* tail(char* p, UInt u); - -//===----------------------------------------------------------===// -// head: find most significant digit, skip leading zeros -//===----------------------------------------------------------===// - -// "x" contains quotient and remainder after division by 10^N -// quotient is less than 10^N -template -static inline char* head(char* p, QuotientAndRemainder x) { - p = head(p, UnsignedOfSize(x.quotient)); - p = tail(p, x.remainder); - return p; -} - -// "u" is less than 10^2*N -template -static inline char* head(char* p, UInt u) { - return u < pow10>(N) ? head(p, UnsignedOfSize(u)) - : head(p, split(u)); -} - -// recursion base case, selected when "u" is one byte -template <> -inline char* head, 1>(char* p, UnsignedOfSize<1> u) { - return u < 10 ? outDigit(p, u) : outTwoDigits(p, u); -} - -//===----------------------------------------------------------===// -// tail: produce all digits including leading zeros -//===----------------------------------------------------------===// - -// recursive step, "u" is less than 10^2*N -template -static inline char* tail(char* p, UInt u) { - QuotientAndRemainder x = split(u); - p = tail(p, UnsignedOfSize(x.quotient)); - p = tail(p, x.remainder); - return p; -} - -// recursion base case, selected when "u" is one byte -template <> -inline char* tail, 1>(char* p, UnsignedOfSize<1> u) { - return outTwoDigits(p, u); -} - -//===----------------------------------------------------------===// -// large values are >= 10^2*N -// where x contains quotient and remainder after division by 10^N -//===----------------------------------------------------------===// - -template -static inline char* large(char* p, QuotientAndRemainder x) { - QuotientAndRemainder y = split(x.quotient); - p = head(p, UnsignedOfSize(y.quotient)); - p = tail(p, y.remainder); - p = tail(p, x.remainder); - return p; -} - -//===----------------------------------------------------------===// -// handle values of "u" that might be >= 10^2*N -// where N is the size of "u" in bytes -//===----------------------------------------------------------===// - -template -static inline char* uitoa(char* p, UInt u) { - if (u < pow10>(N)) return head(p, UnsignedOfSize(u)); - QuotientAndRemainder x = split(u); - - return u < pow10>(2 * N) ? head(p, x) : large(p, x); -} - -// selected when "u" is one byte -template <> -inline char* uitoa, 1>(char* p, UnsignedOfSize<1> u) { - if (u < 10) - return outDigit(p, u); - else if (u < 100) - return outTwoDigits(p, u); - else { - p = outDigit(p, u / 100); - p = outTwoDigits(p, u % 100); - return p; - } -} - -//===----------------------------------------------------------===// -// handle unsigned and signed integral operands -//===----------------------------------------------------------===// - -// itoa: handle unsigned integral operands (selected by SFINAE) -template && std::is_integral_v>* = nullptr> -static inline char* itoa(U u, char* p) { - return convert::uitoa(p, u); -} - -// itoa: handle signed integral operands (selected by SFINAE) -template && std::is_integral_v>* = nullptr> -static inline char* itoa(I i, char* p) { - // Need "mask" to be filled with a copy of the sign bit. - // If "i" is a negative value, then the result of "operator >>" - // is implementation-defined, though usually it is an arithmetic - // right shift that replicates the sign bit. - // Use a conditional expression to be portable, - // a good optimizing compiler generates an arithmetic right shift - // and avoids the conditional branch. - UnsignedOfSize mask = i < 0 ? ~UnsignedOfSize(0) : 0; - // Now get the absolute value of "i" and cast to unsigned type UnsignedOfSize. - // Cannot use std::abs() because the result is undefined - // in 2's complement systems for the most-negative value. - // Want to avoid conditional branch for performance reasons since - // CPU branch prediction will be ineffective when negative values - // occur randomly. - // Let "u" be "i" cast to unsigned type UnsignedOfSize. - // Subtract "u" from 2*u if "i" is positive or 0 if "i" is negative. - // This yields the absolute value with the desired type without - // using a conditional branch and without invoking undefined or - // implementation defined behavior: - UnsignedOfSize u = ((2 * UnsignedOfSize(i)) & ~mask) - UnsignedOfSize(i); - // Unconditionally store a minus sign when producing digits - // in a forward direction and increment the pointer only if - // the value is in fact negative. - // This avoids a conditional branch and is safe because we will - // always produce at least one digit and it will overwrite the - // minus sign when the value is not negative. - *p = '-'; - p += (mask & 1); - p = convert::uitoa(p, u); - return p; -} -} // namespace convert - -static inline int digits10(uint128_t x) { - if (x < 10ULL) return 1; - if (x < 100ULL) return 2; - if (x < 1000ULL) return 3; - - if (x < 1000000000000ULL) { - if (x < 100000000ULL) { - if (x < 1000000ULL) { - if (x < 10000ULL) - return 4; - else - return 5 + (x >= 100000ULL); - } - - return 7 + (x >= 10000000ULL); - } - - if (x < 10000000000ULL) return 9 + (x >= 1000000000ULL); - - return 11 + (x >= 100000000000ULL); - } - - return 12 + digits10(x / 1000000000000ULL); -} - -static inline char* writeUIntText(uint128_t x, char* p) { - int len = digits10(x); - auto pp = p + len; - while (x >= 100) { - const auto i = x % 100; - x /= 100; - pp -= 2; - outTwoDigits(pp, i); - } - if (x < 10) - *p = '0' + x; - else - outTwoDigits(p, x); - return p + len; -} - -static inline char* writeLeadingMinus(char* pos) { - *pos = '-'; - return pos + 1; -} - -static inline char* writeSIntText(int128_t x, char* pos) { - static const int128_t min_int128 = int128_t(0x8000000000000000ll) << 64; - - if (UNLIKELY(x == min_int128)) { - memcpy(pos, "-170141183460469231731687303715884105728", 40); - return pos + 40; - } - - if (x < 0) { - x = -x; - pos = writeLeadingMinus(pos); - } - return writeUIntText(static_cast(x), pos); -} - -} // namespace impl - -template -char* itoa(I i, char* p) { - return impl::convert::itoa(i, p); -} - -template <> -inline char* itoa(uint128_t i, char* p) { - return impl::writeUIntText(i, p); -} - -template <> -inline char* itoa(int128_t i, char* p) { - return impl::writeSIntText(i, p); -} diff --git a/be/src/vec/common/json.h b/be/src/vec/common/json.h deleted file mode 100644 index 4741e7c331cd80..00000000000000 --- a/be/src/vec/common/json.h +++ /dev/null @@ -1,219 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include - -#include - -#include "vec/common/string_ref.h" -#include "vec/common/types.h" - -/** Очень простой класс для чтения JSON (или его кусочков). - * Представляет собой ссылку на кусок памяти, в котором содержится JSON (или его кусочек). - * Не создаёт никаких структур данных в оперативке. Не выделяет память (кроме std::string). - * Не парсит JSON до конца (парсит только часть, необходимую для выполнения вызванного метода). - * Парсинг необходимой части запускается каждый раз при вызове методов. - * Может работать с обрезанным JSON-ом. - * При этом, (в отличие от SAX-подобных парсеров), предоставляет удобные методы для работы. - * - * Эта структура данных более оптимальна, если нужно доставать несколько элементов из большого количества маленьких JSON-ов. - * То есть, подходит для обработки "параметров визитов" и "параметров интернет магазинов" в Яндекс.Метрике. - * Если нужно много работать с одним большим JSON-ом, то этот класс может быть менее оптимальным. - * - * Имеются следующие соглашения: - * 1. Предполагается, что в JSON-е нет пробельных символов. - * 2. Предполагается, что строки в JSON в кодировке UTF-8; также могут использоваться \u-последовательности. - * Строки возвращаются в кодировке UTF-8, \u-последовательности переводятся в UTF-8. - * 3. Но суррогатная пара из двух \uXXXX\uYYYY переводится не в UTF-8, а в CESU-8. - * 4. Корректный JSON парсится корректно. - * При работе с некорректным JSON-ом, кидается исключение или возвращаются неверные результаты. - * (пример: считается, что если встретился символ 'n', то после него идёт 'ull' (null); - * если после него идёт ',1,', то исключение не кидается, и, таким образом, возвращается неверный результат) - * 5. Глубина вложенности JSON ограничена (см. MAX_JSON_DEPTH в cpp файле). - * При необходимости спуститься на большую глубину, кидается исключение. - * 6. В отличие от JSON, пользоволяет парсить значения вида 64-битное число, со знаком, или без. - * При этом, если число дробное - то дробная часть тихо отбрасывается. - * 7. Числа с плавающей запятой парсятся не с максимальной точностью. - * - * Подходит только для чтения JSON, модификация не предусмотрена. - * Все методы immutable, кроме operator++. - */ - -POCO_DECLARE_EXCEPTION(Foundation_API, JSONException, Poco::Exception) - -class JSON { -private: - using Pos = const char*; - Pos ptr_begin; - Pos ptr_end; - unsigned level; - -public: - JSON(Pos ptr_begin_, Pos ptr_end_, unsigned level_ = 0) - : ptr_begin(ptr_begin_), ptr_end(ptr_end_), level(level_) { - checkInit(); - } - - JSON(const std::string& s) : ptr_begin(s.data()), ptr_end(s.data() + s.size()), level(0) { - checkInit(); - } - - JSON(const JSON& rhs) { *this = rhs; } - - JSON& operator=(const JSON& rhs) { - ptr_begin = rhs.ptr_begin; - ptr_end = rhs.ptr_end; - level = rhs.level; - return *this; - } - - const char* data() const { return ptr_begin; } - const char* dataEnd() const { return ptr_end; } - - enum ElementType { - TYPE_OBJECT, - TYPE_ARRAY, - TYPE_NUMBER, - TYPE_STRING, - TYPE_BOOL, - TYPE_NULL, - TYPE_NAME_VALUE_PAIR, - TYPE_NOTYPE, - }; - - ElementType getType() const; - - bool isObject() const { return getType() == TYPE_OBJECT; } - bool isArray() const { return getType() == TYPE_ARRAY; } - bool isNumber() const { return getType() == TYPE_NUMBER; } - bool isString() const { return getType() == TYPE_STRING; } - bool isBool() const { return getType() == TYPE_BOOL; } - bool isNull() const { return getType() == TYPE_NULL; } - bool isNameValuePair() const { return getType() == TYPE_NAME_VALUE_PAIR; } - - /// Количество элементов в массиве или объекте; если элемент - не массив или объект, то исключение. - size_t size() const; - - /// Является ли массив или объект пустыми; если элемент - не массив или объект, то исключение. - bool empty() const; - - /// Получить элемент массива по индексу; если элемент - не массив, то исключение. - JSON operator[](size_t n) const; - - /// Получить элемент объекта по имени; если элемент - не объект, то исключение. - JSON operator[](const std::string& name) const; - - /// Есть ли в объекте элемент с заданным именем; если элемент - не объект, то исключение. - bool has(const std::string& name) const { return has(name.data(), name.size()); } - bool has(const char* data, size_t size) const; - - /// Получить значение элемента; исключение, если элемент имеет неправильный тип. - template - T get() const; - - /// если значения нет, или тип неверный, то возвращает дефолтное значение - template - T getWithDefault(const std::string& key, const T& default_ = T()) const; - - double getDouble() const; - Int64 getInt() const; /// Отбросить дробную часть. - UInt64 getUInt() const; /// Отбросить дробную часть. Если число отрицательное - исключение. - std::string getString() const; - bool getBool() const; - std::string getName() const; /// Получить имя name-value пары. - JSON getValue() const; /// Получить значение name-value пары. - - StringRef getRawString() const; - StringRef getRawName() const; - - /// Получить значение элемента; если элемент - строка, то распарсить значение из строки; если не строка или число - то исключение. - double toDouble() const; - Int64 toInt() const; - UInt64 toUInt() const; - - /** Преобразовать любой элемент в строку. - * Для строки возвращается её значение, для всех остальных элементов - сериализованное представление. - */ - std::string toString() const; - - /// Класс JSON одновременно является итератором по самому себе. - using iterator = JSON; - using const_iterator = JSON; - - iterator operator*() const { return *this; } - const JSON* operator->() const { return this; } - bool operator==(const JSON& rhs) const { return ptr_begin == rhs.ptr_begin; } - bool operator!=(const JSON& rhs) const { return ptr_begin != rhs.ptr_begin; } - - /** Если элемент - массив или объект, то begin() возвращает iterator, - * который указывает на первый элемент массива или первую name-value пару объекта. - */ - iterator begin() const; - - /** end() - значение, которое нельзя использовать; сигнализирует о том, что элементы закончились. - */ - iterator end() const; - - /// Перейти к следующему элементу массива или следующей name-value паре объекта. - iterator& operator++(); - iterator operator++(int); - - /// Есть ли в строке escape-последовательности - bool hasEscapes() const; - - /// Есть ли в строке спец-символы из набора \, ', \0, \b, \f, \r, \n, \t, возможно, заэскейпленные. - bool hasSpecialChars() const; - -private: - /// Проверить глубину рекурсии, а также корректность диапазона памяти. - void checkInit() const; - /// Проверить, что pos лежит внутри диапазона памяти. - void checkPos(Pos pos) const; - - /// Вернуть позицию после заданного элемента. - Pos skipString() const; - Pos skipNumber() const; - Pos skipBool() const; - Pos skipNull() const; - Pos skipNameValuePair() const; - Pos skipObject() const; - Pos skipArray() const; - - Pos skipElement() const; - - /// Найти name-value пару с заданным именем в объекте. - Pos searchField(const std::string& name) const { return searchField(name.data(), name.size()); } - Pos searchField(const char* data, size_t size) const; - - template - bool isType() const; -}; - -template -T JSON::getWithDefault(const std::string& key, const T& default_) const { - if (has(key)) { - JSON key_json = (*this)[key]; - - if (key_json.isType()) - return key_json.get(); - else - return default_; - } else - return default_; -} diff --git a/be/src/vec/common/local_date.h b/be/src/vec/common/local_date.h deleted file mode 100644 index 2bd0e6f13fe7b3..00000000000000 --- a/be/src/vec/common/local_date.h +++ /dev/null @@ -1,153 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include - -#include -#include -#include - -#include "vec/common/date_lut.h" - -/** Stores a calendar date in broken-down form (year, month, day-in-month). - * Could be initialized from date in text form, like '2011-01-01' or from time_t with rounding to date. - * Also could be initialized from date in text form like '20110101... (only first 8 symbols are used). - * Could be implicitly casted to time_t. - * NOTE: Transforming between time_t and LocalDate is done in local time zone! - * - * When local time was shifted backwards (due to daylight saving time or whatever reason) - * - then to resolve the ambiguity of transforming to time_t, lowest of two possible values is selected. - * - * packed - for memcmp to work naturally (but because m_year is 2 bytes, on little endian, comparison is correct only before year 2047) - */ -class LocalDate { -private: - unsigned short m_year; - unsigned char m_month; - unsigned char m_day; - - void init(time_t time) { - const auto& date_lut = DateLUT::instance(); - const auto& values = date_lut.getValues(time); - - m_year = values.year; - m_month = values.month; - m_day = values.day_of_month; - } - - void init(const char* s, size_t length) { - if (length < 8) - throw std::runtime_error("Cannot parse LocalDate: " + std::string(s, length)); - - m_year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); - - if (s[4] == '-') { - if (length < 10) - throw std::runtime_error("Cannot parse LocalDate: " + std::string(s, length)); - m_month = (s[5] - '0') * 10 + (s[6] - '0'); - m_day = (s[8] - '0') * 10 + (s[9] - '0'); - } else { - m_month = (s[4] - '0') * 10 + (s[5] - '0'); - m_day = (s[6] - '0') * 10 + (s[7] - '0'); - } - } - -public: - explicit LocalDate(time_t time) { init(time); } - - LocalDate(DayNum day_num) { - const auto& values = DateLUT::instance().getValues(day_num); - m_year = values.year; - m_month = values.month; - m_day = values.day_of_month; - } - - LocalDate(unsigned short year_, unsigned char month_, unsigned char day_) - : m_year(year_), m_month(month_), m_day(day_) {} - - explicit LocalDate(const std::string& s) { init(s.data(), s.size()); } - - LocalDate(const char* data, size_t length) { init(data, length); } - - LocalDate() : m_year(0), m_month(0), m_day(0) {} - - LocalDate(const LocalDate&) noexcept = default; - LocalDate& operator=(const LocalDate&) noexcept = default; - - LocalDate& operator=(time_t time) { - init(time); - return *this; - } - - operator time_t() const { return DateLUT::instance().makeDate(m_year, m_month, m_day); } - - DayNum getDayNum() const { return DateLUT::instance().makeDayNum(m_year, m_month, m_day); } - - operator DayNum() const { return getDayNum(); } - - unsigned short year() const { return m_year; } - unsigned char month() const { return m_month; } - unsigned char day() const { return m_day; } - - void year(unsigned short x) { m_year = x; } - void month(unsigned char x) { m_month = x; } - void day(unsigned char x) { m_day = x; } - - bool operator<(const LocalDate& other) const { return 0 > memcmp(this, &other, sizeof(*this)); } - - bool operator>(const LocalDate& other) const { return 0 < memcmp(this, &other, sizeof(*this)); } - - bool operator<=(const LocalDate& other) const { - return 0 >= memcmp(this, &other, sizeof(*this)); - } - - bool operator>=(const LocalDate& other) const { - return 0 <= memcmp(this, &other, sizeof(*this)); - } - - bool operator==(const LocalDate& other) const { - return 0 == memcmp(this, &other, sizeof(*this)); - } - - bool operator!=(const LocalDate& other) const { return !(*this == other); } - - /// NOTE Inefficient. - std::string toString(char separator = '-') const { - std::stringstream ss; - if (separator) - ss << year() << separator << (month() / 10) << (month() % 10) << separator - << (day() / 10) << (day() % 10); - else - ss << year() << (month() / 10) << (month() % 10) << (day() / 10) << (day() % 10); - return ss.str(); - } -}; - -static_assert(sizeof(LocalDate) == 4); - -inline std::ostream& operator<<(std::ostream& ostr, const LocalDate& date) { - return ostr << date.year() << '-' << (date.month() / 10) << (date.month() % 10) << '-' - << (date.day() / 10) << (date.day() % 10); -} - -namespace std { -inline string to_string(const LocalDate& date) { - return date.toString(); -} -} // namespace std diff --git a/be/src/vec/common/local_date_time.h b/be/src/vec/common/local_date_time.h deleted file mode 100644 index f618149458bc45..00000000000000 --- a/be/src/vec/common/local_date_time.h +++ /dev/null @@ -1,184 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include - -#include "vec/common/date_lut.h" -#include "vec/common/local_date.h" - -/** Stores calendar date and time in broken-down form. - * Could be initialized from date and time in text form like '2011-01-01 00:00:00' or from time_t. - * Could be implicitly casted to time_t. - * NOTE: Transforming between time_t and LocalDate is done in local time zone! - * - * When local time was shifted backwards (due to daylight saving time or whatever reason) - * - then to resolve the ambiguity of transforming to time_t, lowest of two possible values is selected. - */ -class LocalDateTime { -private: - unsigned short m_year; - unsigned char m_month; - unsigned char m_day; - unsigned char m_hour; - unsigned char m_minute; - unsigned char m_second; - - /// For struct to fill 8 bytes and for safe invocation of memcmp. - /// NOTE We may use attribute packed instead, but it is less portable. - unsigned char pad = 0; - - void init(time_t time) { - if (UNLIKELY(time > DATE_LUT_MAX || time == 0)) { - m_year = 0; - m_month = 0; - m_day = 0; - m_hour = 0; - m_minute = 0; - m_second = 0; - - return; - } - - const auto& date_lut = DateLUT::instance(); - const auto& values = date_lut.getValues(time); - - m_year = values.year; - m_month = values.month; - m_day = values.day_of_month; - m_hour = date_lut.toHour(time); - m_minute = date_lut.toMinute(time); - m_second = date_lut.toSecond(time); - - (void)pad; /// Suppress unused private field warning. - } - - void init(const char* s, size_t length) { - if (length < 19) - throw std::runtime_error("Cannot parse LocalDateTime: " + std::string(s, length)); - - m_year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); - m_month = (s[5] - '0') * 10 + (s[6] - '0'); - m_day = (s[8] - '0') * 10 + (s[9] - '0'); - - m_hour = (s[11] - '0') * 10 + (s[12] - '0'); - m_minute = (s[14] - '0') * 10 + (s[15] - '0'); - m_second = (s[17] - '0') * 10 + (s[18] - '0'); - - (void)pad; - } - -public: - explicit LocalDateTime(time_t time) { init(time); } - - LocalDateTime(unsigned short year_, unsigned char month_, unsigned char day_, - unsigned char hour_, unsigned char minute_, unsigned char second_) - : m_year(year_), - m_month(month_), - m_day(day_), - m_hour(hour_), - m_minute(minute_), - m_second(second_) {} - - explicit LocalDateTime(const std::string& s) { - if (s.size() < 19) throw std::runtime_error("Cannot parse LocalDateTime: " + s); - - init(s.data(), s.size()); - } - - LocalDateTime() : m_year(0), m_month(0), m_day(0), m_hour(0), m_minute(0), m_second(0) {} - - LocalDateTime(const char* data, size_t length) { init(data, length); } - - LocalDateTime(const LocalDateTime&) noexcept = default; - LocalDateTime& operator=(const LocalDateTime&) noexcept = default; - - LocalDateTime& operator=(time_t time) { - init(time); - return *this; - } - - operator time_t() const { - return m_year == 0 ? 0 - : DateLUT::instance().makeDateTime(m_year, m_month, m_day, m_hour, - m_minute, m_second); - } - - unsigned short year() const { return m_year; } - unsigned char month() const { return m_month; } - unsigned char day() const { return m_day; } - unsigned char hour() const { return m_hour; } - unsigned char minute() const { return m_minute; } - unsigned char second() const { return m_second; } - - void year(unsigned short x) { m_year = x; } - void month(unsigned char x) { m_month = x; } - void day(unsigned char x) { m_day = x; } - void hour(unsigned char x) { m_hour = x; } - void minute(unsigned char x) { m_minute = x; } - void second(unsigned char x) { m_second = x; } - - LocalDate toDate() const { return LocalDate(m_year, m_month, m_day); } - - LocalDateTime toStartOfDate() { return LocalDateTime(m_year, m_month, m_day, 0, 0, 0); } - - bool operator<(const LocalDateTime& other) const { - return 0 > memcmp(this, &other, sizeof(*this)); - } - - bool operator>(const LocalDateTime& other) const { - return 0 < memcmp(this, &other, sizeof(*this)); - } - - bool operator<=(const LocalDateTime& other) const { - return 0 >= memcmp(this, &other, sizeof(*this)); - } - - bool operator>=(const LocalDateTime& other) const { - return 0 <= memcmp(this, &other, sizeof(*this)); - } - - bool operator==(const LocalDateTime& other) const { - return 0 == memcmp(this, &other, sizeof(*this)); - } - - bool operator!=(const LocalDateTime& other) const { return !(*this == other); } -}; - -static_assert(sizeof(LocalDateTime) == 8); - -inline std::ostream& operator<<(std::ostream& ostr, const LocalDateTime& datetime) { - ostr << std::setfill('0') << std::setw(4) << datetime.year(); - - ostr << '-' << (datetime.month() / 10) << (datetime.month() % 10) << '-' - << (datetime.day() / 10) << (datetime.day() % 10) << ' ' << (datetime.hour() / 10) - << (datetime.hour() % 10) << ':' << (datetime.minute() / 10) << (datetime.minute() % 10) - << ':' << (datetime.second() / 10) << (datetime.second() % 10); - - return ostr; -} - -namespace std { -inline string to_string(const LocalDateTime& datetime) { - stringstream str; - str << datetime; - return str.str(); -} -} // namespace std diff --git a/be/src/vec/common/memory.h b/be/src/vec/common/memory.h deleted file mode 100644 index 93ad8e73559c54..00000000000000 --- a/be/src/vec/common/memory.h +++ /dev/null @@ -1,91 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include - -#include "common/compiler_util.h" - -#if __has_include() -#include "vec/common/config_common.h" -#endif - -#if USE_JEMALLOC -#include - -#if JEMALLOC_VERSION_MAJOR < 4 -#undef USE_JEMALLOC -#define USE_JEMALLOC 0 -#include -#endif -#else -#include -#endif - -// Also defined in Core/Defines.h -#if !defined(ALWAYS_INLINE) -#if defined(_MSC_VER) -#define ALWAYS_INLINE inline __forceinline -#else -#define ALWAYS_INLINE inline __attribute__((__always_inline__)) -#endif -#endif - -#if !defined(NO_INLINE) -#if defined(_MSC_VER) -#define NO_INLINE static __declspec(noinline) -#else -#define NO_INLINE __attribute__((__noinline__)) -#endif -#endif - -namespace Memory { - -ALWAYS_INLINE void* newImpl(std::size_t size) { - auto* ptr = malloc(size); - if (LIKELY(ptr != nullptr)) return ptr; - - /// @note no std::get_new_handler logic implemented - throw std::bad_alloc {}; -} - -ALWAYS_INLINE void* newNoExept(std::size_t size) noexcept { - return malloc(size); -} - -ALWAYS_INLINE void deleteImpl(void* ptr) noexcept { - free(ptr); -} - -#if USE_JEMALLOC - -ALWAYS_INLINE void deleteSized(void* ptr, std::size_t size) noexcept { - if (UNLIKELY(ptr == nullptr)) return; - - sdallocx(ptr, size, 0); -} - -#else - -ALWAYS_INLINE void deleteSized(void* ptr, std::size_t size [[maybe_unused]]) noexcept { - free(ptr); -} - -#endif - -} // namespace Memory diff --git a/be/src/vec/common/phdr_cache.h b/be/src/vec/common/phdr_cache.h deleted file mode 100644 index 1be39423e16746..00000000000000 --- a/be/src/vec/common/phdr_cache.h +++ /dev/null @@ -1,36 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -/// This code was based on the code by Fedor Korotkiy (prime@yandex-team.ru) for YT product in Yandex. - -/** Collects all dl_phdr_info items and caches them in a static array. - * Also rewrites dl_iterate_phdr with a lock-free version which consults the above cache - * thus eliminating scalability bottleneck in C++ exception unwinding. - * As a drawback, this only works if no dynamic object unloading happens after this point. - * This function is thread-safe. You should call it to update cache after loading new shared libraries. - * Otherwise exception handling from dlopened libraries won't work (will call std::terminate immediately). - * - * NOTE: It is disabled with Thread Sanitizer because TSan can only use original "dl_iterate_phdr" function. - */ -void updatePHDRCache(); - -/** Check if "dl_iterate_phdr" will be lock-free - * to determine if some features like Query Profiler can be used. - */ -bool hasPHDRCache(); diff --git a/be/src/vec/common/pipe.h b/be/src/vec/common/pipe.h deleted file mode 100644 index 99aae1f3091967..00000000000000 --- a/be/src/vec/common/pipe.h +++ /dev/null @@ -1,50 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include - -#include - -/** - * Struct containing a pipe with lazy initialization. - * Use `open` and `close` methods to manipulate pipe and `fds_rw` field to access - * pipe's file descriptors. - */ -struct LazyPipe { - int fds_rw[2] = {-1, -1}; - - LazyPipe() = default; - - void open(); - - void close(); - - virtual ~LazyPipe() = default; -}; - -/** - * Struct which opens new pipe on creation and closes it on destruction. - * Use `fds_rw` field to access pipe's file descriptors. - */ -struct Pipe : public LazyPipe { - Pipe(); - - ~Pipe(); -}; diff --git a/be/src/vec/common/precise_exp10.h b/be/src/vec/common/precise_exp10.h deleted file mode 100644 index 8e2e9585ed4621..00000000000000 --- a/be/src/vec/common/precise_exp10.h +++ /dev/null @@ -1,31 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -/** exp10 from GNU libm fails to give precise result for integer arguments. - * For example, exp10(3) gives 1000.0000000000001 - * despite the fact that 1000 is exactly representable in double and float. - * Better to always use implementation from MUSL. - * - * Note: the function names are different to avoid confusion with symbols from the system libm. - */ - -extern "C" { - -double preciseExp10(double x); -} diff --git a/be/src/vec/common/radix_sort.h b/be/src/vec/common/radix_sort.h index d32510054d1cf7..34654215d69399 100644 --- a/be/src/vec/common/radix_sort.h +++ b/be/src/vec/common/radix_sort.h @@ -1,20 +1,16 @@ #pragma once - #include "string.h" #if !defined(__APPLE__) && !defined(__FreeBSD__) #include "malloc.h" #endif #include "algorithm" #include "cmath" -#include "cstdlib" #include "cstdint" +#include "cstdlib" #include "type_traits" - #include "vec/common/bit_cast.h" #include "vec/core/types.h" -#include "vec/core/defines.h" - /** Radix sort, has the following functionality: * Can sort unsigned, signed numbers, and floats. @@ -26,23 +22,14 @@ * as well as radix-select, radix-partial-sort, radix-get-permutation algorithms based on it. */ - /** Used as a template parameter. See below. */ -struct RadixSortMallocAllocator -{ - void * allocate(size_t size) - { - return malloc(size); - } +struct RadixSortMallocAllocator { + void* allocate(size_t size) { return malloc(size); } - void deallocate(void * ptr, size_t /*size*/) - { - return free(ptr); - } + void deallocate(void* ptr, size_t /*size*/) { return free(ptr); } }; - /** A transformation that transforms the bit representation of a key into an unsigned integer number, * that the order relation over the keys will match the order relation over the obtained unsigned numbers. * For floats this conversion does the following: @@ -50,34 +37,34 @@ struct RadixSortMallocAllocator * In this case, NaN-s are bigger than all normal numbers. */ template -struct RadixSortFloatTransform -{ +struct RadixSortFloatTransform { /// Is it worth writing the result in memory, or is it better to do calculation every time again? static constexpr bool transform_is_simple = false; - static KeyBits forward(KeyBits x) - { - return x ^ ((-(x >> (sizeof(KeyBits) * 8 - 1))) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1))); + static KeyBits forward(KeyBits x) { + return x ^ + ((-(x >> (sizeof(KeyBits) * 8 - 1))) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1))); } - static KeyBits backward(KeyBits x) - { - return x ^ (((x >> (sizeof(KeyBits) * 8 - 1)) - 1) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1))); + static KeyBits backward(KeyBits x) { + return x ^ + (((x >> (sizeof(KeyBits) * 8 - 1)) - 1) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1))); } }; - template -struct RadixSortFloatTraits -{ - using Element = TElement; /// The type of the element. It can be a structure with a key and some other payload. Or just a key. - using Key = Element; /// The key to sort by. - using CountType = uint32_t; /// Type for calculating histograms. In the case of a known small number of elements, it can be less than size_t. +struct RadixSortFloatTraits { + using Element = + TElement; /// The type of the element. It can be a structure with a key and some other payload. Or just a key. + using Key = Element; /// The key to sort by. + using CountType = + uint32_t; /// Type for calculating histograms. In the case of a known small number of elements, it can be less than size_t. /// The type to which the key is transformed to do bit operations. This UInt is the same size as the key. using KeyBits = std::conditional_t; - static constexpr size_t PART_SIZE_BITS = 8; /// With what pieces of the key, in bits, to do one pass - reshuffle of the array. + static constexpr size_t PART_SIZE_BITS = + 8; /// With what pieces of the key, in bits, to do one pass - reshuffle of the array. /// Converting a key into KeyBits is such that the order relation over the key corresponds to the order relation over KeyBits. using Transform = RadixSortFloatTransform; @@ -88,31 +75,23 @@ struct RadixSortFloatTraits using Allocator = RadixSortMallocAllocator; /// The function to get the key from an array element. - static Key & extractKey(Element & elem) { return elem; } + static Key& extractKey(Element& elem) { return elem; } /// Used when fallback to comparison based sorting is needed. /// TODO: Correct handling of NaNs, NULLs, etc - static bool less(Key x, Key y) - { - return x < y; - } + static bool less(Key x, Key y) { return x < y; } }; - template -struct RadixSortIdentityTransform -{ +struct RadixSortIdentityTransform { static constexpr bool transform_is_simple = true; - static KeyBits forward(KeyBits x) { return x; } - static KeyBits backward(KeyBits x) { return x; } + static KeyBits forward(KeyBits x) { return x; } + static KeyBits backward(KeyBits x) { return x; } }; - - template -struct RadixSortUIntTraits -{ +struct RadixSortUIntTraits { using Element = TElement; using Key = Element; using CountType = uint32_t; @@ -123,28 +102,21 @@ struct RadixSortUIntTraits using Transform = RadixSortIdentityTransform; using Allocator = RadixSortMallocAllocator; - static Key & extractKey(Element & elem) { return elem; } + static Key& extractKey(Element& elem) { return elem; } - static bool less(Key x, Key y) - { - return x < y; - } + static bool less(Key x, Key y) { return x < y; } }; - template -struct RadixSortSignedTransform -{ +struct RadixSortSignedTransform { static constexpr bool transform_is_simple = true; - static KeyBits forward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); } - static KeyBits backward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); } + static KeyBits forward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); } + static KeyBits backward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); } }; - template -struct RadixSortIntTraits -{ +struct RadixSortIntTraits { using Element = TElement; using Key = Element; using CountType = uint32_t; @@ -155,32 +127,24 @@ struct RadixSortIntTraits using Transform = RadixSortSignedTransform; using Allocator = RadixSortMallocAllocator; - static Key & extractKey(Element & elem) { return elem; } + static Key& extractKey(Element& elem) { return elem; } - static bool less(Key x, Key y) - { - return x < y; - } + static bool less(Key x, Key y) { return x < y; } }; - template -using RadixSortNumTraits = - std::conditional_t, - std::conditional_t, - RadixSortUIntTraits, - RadixSortIntTraits>, +using RadixSortNumTraits = std::conditional_t< + std::is_integral_v, + std::conditional_t, RadixSortUIntTraits, RadixSortIntTraits>, RadixSortFloatTraits>; - template -struct RadixSort -{ +struct RadixSort { private: - using Element = typename Traits::Element; - using Key = typename Traits::Key; - using CountType = typename Traits::CountType; - using KeyBits = typename Traits::KeyBits; + using Element = typename Traits::Element; + using Key = typename Traits::Key; + using CountType = typename Traits::CountType; + using KeyBits = typename Traits::KeyBits; // Use insertion sort if the size of the array is less than equal to this threshold static constexpr size_t INSERTION_SORT_THRESHOLD = 64; @@ -188,12 +152,11 @@ struct RadixSort static constexpr size_t HISTOGRAM_SIZE = 1 << Traits::PART_SIZE_BITS; static constexpr size_t PART_BITMASK = HISTOGRAM_SIZE - 1; static constexpr size_t KEY_BITS = sizeof(Key) * 8; - static constexpr size_t NUM_PASSES = (KEY_BITS + (Traits::PART_SIZE_BITS - 1)) / Traits::PART_SIZE_BITS; + static constexpr size_t NUM_PASSES = + (KEY_BITS + (Traits::PART_SIZE_BITS - 1)) / Traits::PART_SIZE_BITS; - static ALWAYS_INLINE KeyBits getPart(size_t N, KeyBits x) - { - if (Traits::Transform::transform_is_simple) - x = Traits::Transform::forward(x); + static ALWAYS_INLINE KeyBits getPart(size_t N, KeyBits x) { + if (Traits::Transform::transform_is_simple) x = Traits::Transform::forward(x); return (x >> (N * Traits::PART_SIZE_BITS)) & PART_BITMASK; } @@ -201,17 +164,16 @@ struct RadixSort static KeyBits keyToBits(Key x) { return ext::bit_cast(x); } static Key bitsToKey(KeyBits x) { return ext::bit_cast(x); } - static void insertionSortInternal(Element *arr, size_t size) - { - Element * end = arr + size; - for (Element * i = arr + 1; i < end; ++i) - { - if (Traits::less(Traits::extractKey(*i), Traits::extractKey(*(i - 1)))) - { - Element * j; + static void insertionSortInternal(Element* arr, size_t size) { + Element* end = arr + size; + for (Element* i = arr + 1; i < end; ++i) { + if (Traits::less(Traits::extractKey(*i), Traits::extractKey(*(i - 1)))) { + Element* j; Element tmp = *i; *i = *(i - 1); - for (j = i - 1; j > arr && Traits::less(Traits::extractKey(tmp), Traits::extractKey(*(j - 1))); --j) + for (j = i - 1; + j > arr && Traits::less(Traits::extractKey(tmp), Traits::extractKey(*(j - 1))); + --j) *j = *(j - 1); *j = tmp; } @@ -222,24 +184,20 @@ struct RadixSort * Puts elements to buckets based on PASS-th digit, then recursively calls insertion sort or itself on the buckets */ template - static inline void radixSortMSDInternal(Element * arr, size_t size, size_t limit) - { - Element * last_list[HISTOGRAM_SIZE + 1]; - Element ** last = last_list + 1; + static inline void radixSortMSDInternal(Element* arr, size_t size, size_t limit) { + Element* last_list[HISTOGRAM_SIZE + 1]; + Element** last = last_list + 1; size_t count[HISTOGRAM_SIZE] = {0}; - for (Element * i = arr; i < arr + size; ++i) - ++count[getPart(PASS, *i)]; + for (Element* i = arr; i < arr + size; ++i) ++count[getPart(PASS, *i)]; last_list[0] = last_list[1] = arr; size_t buckets_for_recursion = HISTOGRAM_SIZE; - Element * finish = arr + size; - for (size_t i = 1; i < HISTOGRAM_SIZE; ++i) - { + Element* finish = arr + size; + for (size_t i = 1; i < HISTOGRAM_SIZE; ++i) { last[i] = last[i - 1] + count[i - 1]; - if (last[i] >= arr + limit) - { + if (last[i] >= arr + limit) { buckets_for_recursion = i; finish = last[i]; } @@ -253,22 +211,17 @@ struct RadixSort */ // Scatter array elements to buckets until the first buckets_for_recursion buckets are full - for (size_t i = 0; i < buckets_for_recursion; ++i) - { - Element * end = last[i - 1] + count[i]; - if (end == finish) - { + for (size_t i = 0; i < buckets_for_recursion; ++i) { + Element* end = last[i - 1] + count[i]; + if (end == finish) { last[i] = end; break; } - while (last[i] != end) - { + while (last[i] != end) { Element swapper = *last[i]; KeyBits tag = getPart(PASS, swapper); - if (tag != i) - { - do - { + if (tag != i) { + do { std::swap(swapper, *last[tag]++); } while ((tag = getPart(PASS, swapper)) != i); *last[i] = swapper; @@ -277,18 +230,16 @@ struct RadixSort } } - if constexpr (PASS > 0) - { + if constexpr (PASS > 0) { // Recursively sort buckets, except the last one - for (size_t i = 0; i < buckets_for_recursion - 1; ++i) - { - Element * start = last[i - 1]; + for (size_t i = 0; i < buckets_for_recursion - 1; ++i) { + Element* start = last[i - 1]; size_t subsize = last[i] - last[i - 1]; radixSortMSDInternalHelper(start, subsize, subsize); } // Sort last necessary bucket with limit - Element * start = last[buckets_for_recursion - 2]; + Element* start = last[buckets_for_recursion - 2]; size_t subsize = last[buckets_for_recursion - 1] - last[buckets_for_recursion - 2]; size_t sublimit = limit - (last[buckets_for_recursion - 1] - arr); radixSortMSDInternalHelper(start, subsize, sublimit); @@ -297,8 +248,7 @@ struct RadixSort // A helper to choose sorting algorithm based on array length template - static inline void radixSortMSDInternalHelper(Element * arr, size_t size, size_t limit) - { + static inline void radixSortMSDInternalHelper(Element* arr, size_t size, size_t limit) { if (size <= INSERTION_SORT_THRESHOLD) insertionSortInternal(arr, size); else @@ -307,8 +257,7 @@ struct RadixSort public: /// Least significant digit radix sort (stable) - static void executeLSD(Element * arr, size_t size) - { + static void executeLSD(Element* arr, size_t size) { /// If the array is smaller than 256, then it is better to use another algorithm. /// There are loops of NUM_PASSES. It is very important that they are unfolded at compile-time. @@ -319,27 +268,27 @@ struct RadixSort typename Traits::Allocator allocator; /// We will do several passes through the array. On each pass, the data is transferred to another array. Let's allocate this temporary array. - Element * swap_buffer = reinterpret_cast(allocator.allocate(size * sizeof(Element))); + Element* swap_buffer = + reinterpret_cast(allocator.allocate(size * sizeof(Element))); /// Transform the array and calculate the histogram. /// NOTE This is slightly suboptimal. Look at https://github.com/powturbo/TurboHist - for (size_t i = 0; i < size; ++i) - { + for (size_t i = 0; i < size; ++i) { if (!Traits::Transform::transform_is_simple) - Traits::extractKey(arr[i]) = bitsToKey(Traits::Transform::forward(keyToBits(Traits::extractKey(arr[i])))); + Traits::extractKey(arr[i]) = bitsToKey( + Traits::Transform::forward(keyToBits(Traits::extractKey(arr[i])))); for (size_t pass = 0; pass < NUM_PASSES; ++pass) - ++histograms[pass * HISTOGRAM_SIZE + getPart(pass, keyToBits(Traits::extractKey(arr[i])))]; + ++histograms[pass * HISTOGRAM_SIZE + + getPart(pass, keyToBits(Traits::extractKey(arr[i])))]; } { /// Replace the histograms with the accumulated sums: the value in position i is the sum of the previous positions minus one. size_t sums[NUM_PASSES] = {0}; - for (size_t i = 0; i < HISTOGRAM_SIZE; ++i) - { - for (size_t pass = 0; pass < NUM_PASSES; ++pass) - { + for (size_t i = 0; i < HISTOGRAM_SIZE; ++i) { + for (size_t pass = 0; pass < NUM_PASSES; ++pass) { size_t tmp = histograms[pass * HISTOGRAM_SIZE + i] + sums[pass]; histograms[pass * HISTOGRAM_SIZE + i] = sums[pass] - 1; sums[pass] = tmp; @@ -348,29 +297,27 @@ struct RadixSort } /// Move the elements in the order starting from the least bit piece, and then do a few passes on the number of pieces. - for (size_t pass = 0; pass < NUM_PASSES; ++pass) - { - Element * writer = pass % 2 ? arr : swap_buffer; - Element * reader = pass % 2 ? swap_buffer : arr; + for (size_t pass = 0; pass < NUM_PASSES; ++pass) { + Element* writer = pass % 2 ? arr : swap_buffer; + Element* reader = pass % 2 ? swap_buffer : arr; - for (size_t i = 0; i < size; ++i) - { + for (size_t i = 0; i < size; ++i) { size_t pos = getPart(pass, keyToBits(Traits::extractKey(reader[i]))); /// Place the element on the next free position. - auto & dest = writer[++histograms[pass * HISTOGRAM_SIZE + pos]]; + auto& dest = writer[++histograms[pass * HISTOGRAM_SIZE + pos]]; dest = reader[i]; /// On the last pass, we do the reverse transformation. if (!Traits::Transform::transform_is_simple && pass == NUM_PASSES - 1) - Traits::extractKey(dest) = bitsToKey(Traits::Transform::backward(keyToBits(Traits::extractKey(reader[i])))); + Traits::extractKey(dest) = bitsToKey( + Traits::Transform::backward(keyToBits(Traits::extractKey(reader[i])))); } } /// If the number of passes is odd, the result array is in a temporary buffer. Copy it to the place of the original array. /// NOTE Sometimes it will be more optimal to provide non-destructive interface, that will not modify original array. - if (NUM_PASSES % 2) - memcpy(arr, swap_buffer, size * sizeof(Element)); + if (NUM_PASSES % 2) memcpy(arr, swap_buffer, size * sizeof(Element)); allocator.deallocate(swap_buffer, size * sizeof(Element)); } @@ -402,25 +349,21 @@ struct RadixSort * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE * SOFTWARE. */ - static void executeMSD(Element * arr, size_t size, size_t limit) - { + static void executeMSD(Element* arr, size_t size, size_t limit) { limit = std::min(limit, size); radixSortMSDInternalHelper(arr, size, limit); } }; - /// Helper functions for numeric types. /// Use RadixSort with custom traits for complex types instead. template -void radixSortLSD(T *arr, size_t size) -{ +void radixSortLSD(T* arr, size_t size) { RadixSort>::executeLSD(arr, size); } template -void radixSortMSD(T *arr, size_t size, size_t limit) -{ +void radixSortMSD(T* arr, size_t size, size_t limit) { RadixSort>::executeMSD(arr, size, limit); } diff --git a/be/src/vec/common/readline_use.h b/be/src/vec/common/readline_use.h deleted file mode 100644 index 4fc9759a7ebbc7..00000000000000 --- a/be/src/vec/common/readline_use.h +++ /dev/null @@ -1,48 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#if __has_include() -#include "vec/common/config_common.h" -#endif - -/// Different line editing libraries can be used depending on the environment. -#if USE_READLINE -#include -#include -#elif USE_LIBEDIT -#include -#else -#include -#include -#include -inline char* readline(const char* prompt) { - std::string s; - std::cout << prompt; - std::getline(std::cin, s); - - if (!std::cin.good()) return nullptr; - return strdup(s.data()); -} -#define add_history(...) \ - do { \ - } while (0) -#define rl_bind_key(...) \ - do { \ - } while (0) -#endif diff --git a/be/src/vec/common/set_terminal_echo.h b/be/src/vec/common/set_terminal_echo.h deleted file mode 100644 index 076d1cee4978f1..00000000000000 --- a/be/src/vec/common/set_terminal_echo.h +++ /dev/null @@ -1,21 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -/// Enable or disable echoing of typed characters. Throws std::runtime_error on error. -void setTerminalEcho(bool enable); diff --git a/be/src/vec/common/shift10.h b/be/src/vec/common/shift10.h deleted file mode 100644 index de1122bb9f197d..00000000000000 --- a/be/src/vec/common/shift10.h +++ /dev/null @@ -1,33 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include "vec/common/types.h" - -/** Almost the same as x = x * exp10(exponent), but gives more accurate result. - * Example: - * 5 * 1e-11 = 4.9999999999999995e-11 - * != - * 5e-11 = shift10(5.0, -11) - */ - -double shift10(double x, int exponent); -float shift10(float x, int exponent); - -double shift10(UInt64 x, int exponent); -double shift10(Int64 x, int exponent); diff --git a/be/src/vec/common/simple_cache.h b/be/src/vec/common/simple_cache.h deleted file mode 100644 index ac6ab9592cd472..00000000000000 --- a/be/src/vec/common/simple_cache.h +++ /dev/null @@ -1,90 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include -#include - -/** The simplest cache for a free function. - * You can also pass a static class method or lambda without captures. - * The size is unlimited. Values are stored permanently and never evicted. - * But single record or all cache can be manually dropped. - * Mutex is used for synchronization. - * Suitable only for the simplest cases. - * - * Usage - * - * SimpleCache func_cached; - * std::cerr << func_cached(args...); - */ -template -class SimpleCache { -private: - using Key = typename function_traits::arguments_decay; - using Result = typename function_traits::result; - - std::map cache; - mutable std::mutex mutex; - -public: - template - Result operator()(Args&&... args) { - { - std::lock_guard lock(mutex); - - Key key {std::forward(args)...}; - auto it = cache.find(key); - - if (cache.end() != it) return it->second; - } - - /// The calculations themselves are not done under mutex. - Result res = f(std::forward(args)...); - - { - std::lock_guard lock(mutex); - - cache.emplace(std::forward_as_tuple(args...), res); - } - - return res; - } - - template - void update(Args&&... args) { - Result res = f(std::forward(args)...); - { - std::lock_guard lock(mutex); - - Key key {std::forward(args)...}; - cache[key] = std::move(res); - } - } - - size_t size() const { - std::lock_guard lock(mutex); - return cache.size(); - } - - void drop() { - std::lock_guard lock(mutex); - cache.clear(); - } -}; diff --git a/be/src/vec/common/sip_hash.h b/be/src/vec/common/sip_hash.h index 129fcac7625efb..7acc80d489d1d1 100644 --- a/be/src/vec/common/sip_hash.h +++ b/be/src/vec/common/sip_hash.h @@ -33,11 +33,11 @@ #include #include -#include "vec/common/types.h" #include "vec/common/unaligned.h" #include "vec/core/defines.h" +#include "vec/core/types.h" -#define ROTL(x, b) static_cast(((x) << (b)) | ((x) >> (64 - (b)))) +#define ROTL(x, b) static_cast(((x) << (b)) | ((x) >> (64 - (b)))) #define SIPROUND \ do { \ @@ -60,18 +60,18 @@ class SipHash { private: /// State. - UInt64 v0; - UInt64 v1; - UInt64 v2; - UInt64 v3; + doris::vectorized::UInt64 v0; + doris::vectorized::UInt64 v1; + doris::vectorized::UInt64 v2; + doris::vectorized::UInt64 v3; /// How many bytes have been processed. - UInt64 cnt; + doris::vectorized::UInt64 cnt; /// The current 8 bytes of input data. union { - UInt64 current_word; - UInt8 current_bytes[8]; + doris::vectorized::UInt64 current_word; + doris::vectorized::UInt8 current_bytes[8]; }; ALWAYS_INLINE void finalize() { @@ -92,7 +92,7 @@ class SipHash { public: /// Arguments - seed. - SipHash(UInt64 k0 = 0, UInt64 k1 = 0) { + SipHash(doris::vectorized::UInt64 k0 = 0, doris::vectorized::UInt64 k1 = 0) { /// Initialize the state with some random bytes and seed. v0 = 0x736f6d6570736575ULL ^ k0; v1 = 0x646f72616e646f6dULL ^ k1; @@ -103,7 +103,7 @@ class SipHash { current_word = 0; } - void update(const char* data, UInt64 size) { + void update(const char* data, doris::vectorized::UInt64 size) { const char* end = data + size; /// We'll finish to process the remainder of the previous update, if any. @@ -126,7 +126,7 @@ class SipHash { cnt += end - data; while (data + 8 <= end) { - current_word = unalignedLoad(data); + current_word = unalignedLoad(data); v3 ^= current_word; SIPROUND; @@ -178,8 +178,8 @@ class SipHash { void get128(char* out) { finalize(); - reinterpret_cast(out)[0] = v0 ^ v1; - reinterpret_cast(out)[1] = v2 ^ v3; + reinterpret_cast(out)[0] = v0 ^ v1; + reinterpret_cast(out)[1] = v2 ^ v3; } /// template for avoiding 'unsigned long long' vs 'unsigned long' problem on old poco in macos @@ -191,7 +191,7 @@ class SipHash { hi = v2 ^ v3; } - UInt64 get64() { + doris::vectorized::UInt64 get64() { finalize(); return v0 ^ v1 ^ v2 ^ v3; } @@ -208,20 +208,21 @@ inline void sipHash128(const char* data, const size_t size, char* out) { hash.get128(out); } -inline UInt64 sipHash64(const char* data, const size_t size) { +inline doris::vectorized::UInt64 sipHash64(const char* data, const size_t size) { SipHash hash; hash.update(data, size); return hash.get64(); } template -std::enable_if_t, UInt64> +std::enable_if_t, + doris::vectorized::UInt64> sipHash64(const T& x) { SipHash hash; hash.update(x); return hash.get64(); } -inline UInt64 sipHash64(const std::string& s) { +inline doris::vectorized::UInt64 sipHash64(const std::string& s) { return sipHash64(s.data(), s.size()); } diff --git a/be/src/vec/common/string_ref.h b/be/src/vec/common/string_ref.h index 404f1bc724d32f..5f7662732e3efc 100644 --- a/be/src/vec/common/string_ref.h +++ b/be/src/vec/common/string_ref.h @@ -24,8 +24,8 @@ #include #include -#include "vec/common/types.h" #include "vec/common/unaligned.h" +#include "vec/core/types.h" #if defined(__SSE2__) #include @@ -209,24 +209,25 @@ struct StringRefHash64 { /// Parts are taken from CityHash. -inline UInt64 hashLen16(UInt64 u, UInt64 v) { +inline doris::vectorized::UInt64 hashLen16(doris::vectorized::UInt64 u, + doris::vectorized::UInt64 v) { return CityHash_v1_0_2::Hash128to64(CityHash_v1_0_2::uint128(u, v)); } -inline UInt64 shiftMix(UInt64 val) { +inline doris::vectorized::UInt64 shiftMix(doris::vectorized::UInt64 val) { return val ^ (val >> 47); } -inline UInt64 rotateByAtLeast1(UInt64 val, int shift) { +inline doris::vectorized::UInt64 rotateByAtLeast1(doris::vectorized::UInt64 val, int shift) { return (val >> shift) | (val << (64 - shift)); } inline size_t hashLessThan8(const char* data, size_t size) { - static constexpr UInt64 k2 = 0x9ae16a3b2f90404fULL; - static constexpr UInt64 k3 = 0xc949d7c7509e6557ULL; + static constexpr doris::vectorized::UInt64 k2 = 0x9ae16a3b2f90404fULL; + static constexpr doris::vectorized::UInt64 k3 = 0xc949d7c7509e6557ULL; if (size >= 4) { - UInt64 a = unalignedLoad(data); + doris::vectorized::UInt64 a = unalignedLoad(data); return hashLen16(size + (a << 3), unalignedLoad(data + size - 4)); } @@ -244,8 +245,8 @@ inline size_t hashLessThan8(const char* data, size_t size) { inline size_t hashLessThan16(const char* data, size_t size) { if (size > 8) { - UInt64 a = unalignedLoad(data); - UInt64 b = unalignedLoad(data + size - 8); + doris::vectorized::UInt64 a = unalignedLoad(data); + doris::vectorized::UInt64 b = unalignedLoad(data + size - 8); return hashLen16(a, rotateByAtLeast1(b + size, size)) ^ b; } @@ -267,13 +268,14 @@ struct CRC32Hash { size_t res = -1ULL; do { - UInt64 word = unalignedLoad(pos); + doris::vectorized::UInt64 word = unalignedLoad(pos); res = _mm_crc32_u64(res, word); pos += 8; } while (pos + 8 < end); - UInt64 word = unalignedLoad(end - 8); /// I'm not sure if this is normal. + doris::vectorized::UInt64 word = unalignedLoad( + end - 8); /// I'm not sure if this is normal. res = _mm_crc32_u64(res, word); return res; @@ -286,7 +288,7 @@ struct StringRefHash : CRC32Hash {}; struct CRC32Hash { size_t operator()(StringRef /* x */) const { - throw std::logic_error {"Not implemented CRC32Hash without SSE"}; + throw std::logic_error{"Not implemented CRC32Hash without SSE"}; } }; diff --git a/be/src/vec/common/types.h b/be/src/vec/common/types.h deleted file mode 100644 index 7b43b4587c518b..00000000000000 --- a/be/src/vec/common/types.h +++ /dev/null @@ -1,32 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once -#include -#include -#include -#include - -using Int8 = int8_t; -using Int16 = int16_t; -using Int32 = int32_t; -using Int64 = int64_t; - -using UInt8 = uint8_t; -using UInt16 = uint16_t; -using UInt32 = uint32_t; -using UInt64 = uint64_t; \ No newline at end of file diff --git a/be/src/vec/core/accurate_comparison.h b/be/src/vec/core/accurate_comparison.h index 3db29091b2fb29..390ce5858eec60 100644 --- a/be/src/vec/core/accurate_comparison.h +++ b/be/src/vec/core/accurate_comparison.h @@ -21,9 +21,9 @@ #include #include "vec/common/nan_utils.h" -#include "vec/common/types.h" #include "vec/common/uint128.h" #include "vec/core/defines.h" +#include "vec/core/types.h" /** Preceptually-correct number comparisons. * Example: Int8(-1) != UInt8(255) @@ -31,8 +31,6 @@ namespace accurate { -using doris::vectorized::UInt64; - /** Cases: 1) Safe conversion (in case of default C++ operators) a) int vs any int @@ -220,7 +218,7 @@ inline bool greaterOp( return (f < 0) || (f < static_cast( std::numeric_limits::max()) && - u > static_cast(f)); + u > static_cast(f)); } // Case 3b for float32 @@ -342,7 +340,7 @@ inline bool NO_SANITIZE_UNDEFINED equalsOp inline bool NO_SANITIZE_UNDEFINED equalsOp( doris::vectorized::UInt128 u, doris::vectorized::Float64 f) { - return u.low == 0 && equalsOp(static_cast(u.high), f); + return u.low == 0 && equalsOp(static_cast(u.high), f); } template <> diff --git a/be/src/vec/core/defines.h b/be/src/vec/core/defines.h index 95815c18478cd7..14ac9881d15cbe 100644 --- a/be/src/vec/core/defines.h +++ b/be/src/vec/core/defines.h @@ -17,103 +17,13 @@ #pragma once -#define DBMS_DEFAULT_HOST "localhost" -#define DBMS_DEFAULT_PORT 9000 -#define DBMS_DEFAULT_SECURE_PORT 9440 -#define DBMS_DEFAULT_HTTP_PORT 8123 -#define DBMS_DEFAULT_CONNECT_TIMEOUT_SEC 10 -#define DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS 50 -#define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300 -#define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300 -/// Timeout for synchronous request-result protocol call (like Ping or TablesStatus). -#define DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC 5 -#define DBMS_DEFAULT_POLL_INTERVAL 10 - -/// The size of the I/O buffer by default. -#define DBMS_DEFAULT_BUFFER_SIZE 1048576ULL - -/** Which blocks by default read the data (by number of rows). - * Smaller values give better cache locality, less consumption of RAM, but more overhead to process the query. - */ -//#define DEFAULT_BLOCK_SIZE 65536 - -/** Which blocks should be formed for insertion into the table, if we control the formation of blocks. - * (Sometimes the blocks are inserted exactly such blocks that have been read / transmitted from the outside, and this parameter does not affect their size.) - * More than DEFAULT_BLOCK_SIZE, because in some tables a block of data on the disk is created for each block (quite a big thing), - * and if the parts were small, then it would be costly then to combine them. - */ -#define DEFAULT_INSERT_BLOCK_SIZE 1048576 - -/** The same, but for merge operations. Less DEFAULT_BLOCK_SIZE for saving RAM (since all the columns are read). - * Significantly less, since there are 10-way mergers. - */ -#define DEFAULT_MERGE_BLOCK_SIZE 8192 - -#define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5 -#define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160) -#define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15 -#define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024 -#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3 -/// each period reduces the error counter by 2 times -/// too short a period can cause errors to disappear immediately after creation. -#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD 60 -/// replica error max cap, this is to prevent replica from accumulating too many errors and taking to long to recover. -#define DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT 1000 - -#define DBMS_MIN_REVISION_WITH_CLIENT_INFO 54032 -#define DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE 54058 -#define DBMS_MIN_REVISION_WITH_QUOTA_KEY_IN_CLIENT_INFO 54060 -#define DBMS_MIN_REVISION_WITH_TABLES_STATUS 54226 -#define DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE 54337 -#define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372 -#define DBMS_MIN_REVISION_WITH_VERSION_PATCH 54401 -#define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54406 -#define DBMS_MIN_REVISION_WITH_CLIENT_SUPPORT_EMBEDDED_DATA 54415 -/// Minimum revision with exactly the same set of aggregation methods and rules to select them. -/// Two-level (bucketed) aggregation is incompatible if servers are inconsistent in these rules -/// (keys will be placed in different buckets and result will not be fully aggregated). -#define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54408 -#define DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA 54410 - -#define DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE 54405 - -#define DBMS_MIN_REVISION_WITH_CLIENT_WRITE_INFO 54420 - -/// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change. -#define DBMS_TCP_PROTOCOL_VERSION 54226 - -/// The boundary on which the blocks for asynchronous file operations should be aligned. -#define DEFAULT_AIO_FILE_BLOCK_SIZE 4096 - -#define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 1800 -#define DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT 1 -/// Maximum namber of http-connections between two endpoints -/// the number is unmotivated -#define DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT 15 - -#define DBMS_DEFAULT_PATH "/var/lib/clickhouse/" - // more aliases: https://mailman.videolan.org/pipermail/x264-devel/2014-May/010660.html #ifdef ALWAYS_INLINE #undef ALWAYS_INLINE #endif - -#if defined(_MSC_VER) -#define ALWAYS_INLINE __forceinline -#define NO_INLINE static __declspec(noinline) -#define MAY_ALIAS -#else #define ALWAYS_INLINE __attribute__((__always_inline__)) #define NO_INLINE __attribute__((__noinline__)) #define MAY_ALIAS __attribute__((__may_alias__)) -#endif - -#define PLATFORM_NOT_SUPPORTED \ - "The only supported platforms are x86_64 and AArch64, PowerPC (work in progress)" - -#if !defined(__x86_64__) && !defined(__aarch64__) && !defined(__PPC__) -#error PLATFORM_NOT_SUPPORTED -#endif /// Check for presence of address sanitizer #if !defined(ADDRESS_SANITIZER) @@ -161,22 +71,3 @@ #define NO_SANITIZE_ADDRESS #define NO_SANITIZE_THREAD #endif - -#if defined __GNUC__ && !defined __clang__ -#define OPTIMIZE(x) __attribute__((__optimize__(x))) -#else -#define OPTIMIZE(x) -#endif - -/// This number is only used for distributed version compatible. -/// It could be any magic number. -#define DBMS_DISTRIBUTED_SENDS_MAGIC_NUMBER 0xCAFECABE - -#if !__has_include() -#define ASAN_UNPOISON_MEMORY_REGION(a, b) -#define ASAN_POISON_MEMORY_REGION(a, b) -#endif - -/// A macro for suppressing warnings about unused variables or function results. -/// Useful for structured bindings which have no standard way to declare this. -#define UNUSED(...) (void)(__VA_ARGS__) diff --git a/be/src/vec/core/field.h b/be/src/vec/core/field.h index 079984dcffbac5..83830ebf220beb 100644 --- a/be/src/vec/core/field.h +++ b/be/src/vec/core/field.h @@ -24,13 +24,10 @@ #include #include "vec/common/exception.h" -#include "vec/common/uint128.h" -#include "vec/core/defines.h" -#include "vec/core/types.h" -//#include -#include "vec/common/day_num.h" #include "vec/common/int_exp.h" #include "vec/common/strong_typedef.h" +#include "vec/common/uint128.h" +#include "vec/core/types.h" namespace doris::vectorized { @@ -780,10 +777,6 @@ struct NearestFieldTypeImpl { using Type = UInt64; }; -template <> -struct NearestFieldTypeImpl { - using Type = UInt64; -}; template <> struct NearestFieldTypeImpl { using Type = UInt128; diff --git a/be/src/vec/core/types.h b/be/src/vec/core/types.h index bd3876b0b53d29..63939464a9794c 100644 --- a/be/src/vec/core/types.h +++ b/be/src/vec/core/types.h @@ -21,7 +21,6 @@ #include #include #include -#include "vec/common/types.h" namespace doris::vectorized { @@ -211,6 +210,9 @@ struct TypeId { static constexpr const TypeIndex value = TypeIndex::Int128; }; +using Date = Int128; +using DateTime = Int128; + /// Own FieldType for Decimal. /// It is only a "storage" for decimal. To perform operations, you also have to provide a scale (number of digits after point). template diff --git a/be/src/vec/data_types/data_type.cpp b/be/src/vec/data_types/data_type.cpp index 6ba83d20789309..0d2b780eb26b0d 100644 --- a/be/src/vec/data_types/data_type.cpp +++ b/be/src/vec/data_types/data_type.cpp @@ -20,7 +20,6 @@ #include "vec/columns/column.h" #include "vec/columns/column_const.h" #include "vec/common/exception.h" -#include "vec/core/defines.h" #include "vec/data_types/nested_utils.h" namespace doris::vectorized { @@ -72,56 +71,11 @@ DataTypePtr IDataType::promoteNumericType() const { ErrorCodes::DATA_TYPE_CANNOT_BE_PROMOTED); } -// void IDataType::serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const -// { -// throw Exception("Data type " + getName() + " must be serialized with multiple streams", ErrorCodes::MULTIPLE_STREAMS_REQUIRED); -// } - -// void IDataType::deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) const -// { -// throw Exception("Data type " + getName() + " must be deserialized with multiple streams", ErrorCodes::MULTIPLE_STREAMS_REQUIRED); -// } - size_t IDataType::getSizeOfValueInMemory() const { throw Exception("Value of type " + getName() + " in memory is not of fixed size.", ErrorCodes::LOGICAL_ERROR); } -// String IDataType::getFileNameForStream(const String & column_name, const IDataType::SubstreamPath & path) -// { -// /// Sizes of arrays (elements of Nested type) are shared (all reside in single file). -// String nested_table_name = Nested::extractTableName(column_name); - -// bool is_sizes_of_nested_type = -// path.size() == 1 /// Nested structure may have arrays as nested elements (so effectively we have multidimensional arrays). -// /// Sizes of arrays are shared only at first level. -// && path[0].type == IDataType::Substream::ArraySizes -// && nested_table_name != column_name; - -// size_t array_level = 0; -// String stream_name = escapeForFileName(is_sizes_of_nested_type ? nested_table_name : column_name); -// for (const Substream & elem : path) -// { -// if (elem.type == Substream::NullMap) -// stream_name += ".null"; -// else if (elem.type == Substream::ArraySizes) -// stream_name += ".size" + toString(array_level); -// else if (elem.type == Substream::ArrayElements) -// ++array_level; -// else if (elem.type == Substream::TupleElement) -// { -// /// For compatibility reasons, we use %2E instead of dot. -// /// Because nested data may be represented not by Array of Tuple, -// /// but by separate Array columns with names in a form of a.b, -// /// and name is encoded as a whole. -// stream_name += "%2E" + escapeForFileName(elem.tuple_element_name); -// } -// else if (elem.type == Substream::DictionaryKeys) -// stream_name += ".dict"; -// } -// return stream_name; -// } - void IDataType::to_string(const IColumn& column, size_t row_num, BufferWritable& ostr) const { throw Exception("Data type " + getName() + "to_string not implement.", ErrorCodes::NOT_IMPLEMENTED); @@ -135,103 +89,5 @@ std::string IDataType::to_string(const IColumn& column, size_t row_num) const { 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 (custom_text_serialization) - custom_text_serialization->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 (custom_text_serialization) - custom_text_serialization->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 (custom_text_serialization) - custom_text_serialization->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 (custom_text_serialization) - custom_text_serialization->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 (custom_text_serialization) - custom_text_serialization->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 (custom_text_serialization) - custom_text_serialization->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 (custom_text_serialization) - custom_text_serialization->serializeText(column, row_num, ostr, settings); - else - serializeText(column, row_num, ostr, settings); -} - -void IDataType::deserializeAsWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const -{ - if (custom_text_serialization) - custom_text_serialization->deserializeWholeText(column, istr, settings); - else - deserializeWholeText(column, istr, settings); -} - -void IDataType::serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - if (custom_text_serialization) - custom_text_serialization->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 (custom_text_serialization) - custom_text_serialization->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 (custom_text_serialization) - custom_text_serialization->serializeTextXML(column, row_num, ostr, settings); - else - serializeTextXML(column, row_num, ostr, settings); -} - -void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const -{ - /// replace only if not null - if (custom_desc_->name) - custom_name = std::move(custom_desc_->name); - - if (custom_desc_->text_serialization) - custom_text_serialization = std::move(custom_desc_->text_serialization); -}*/ } // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type.h b/be/src/vec/data_types/data_type.h index 145765663a008c..7484049a12b2ec 100644 --- a/be/src/vec/data_types/data_type.h +++ b/be/src/vec/data_types/data_type.h @@ -59,12 +59,6 @@ class IDataType : private boost::noncopyable { 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. - /// Place it in descendants: - /// static constexpr bool is_parametric = false; - /// Name of data type (examples: UInt64, Array(String)). String getName() const; @@ -77,232 +71,9 @@ class IDataType : private boost::noncopyable { virtual void to_string(const IColumn& column, size_t row_num, BufferWritable& ostr) const; virtual std::string to_string(const IColumn& column, size_t row_num) const; - /** Binary serialization for range of values in column - for writing to disk/network, etc. - * - * Some data types are represented in multiple streams while being serialized. - * Example: - * - Arrays are represented as stream of all elements and stream of array sizes. - * - Nullable types are represented as stream of values (with unspecified values in place of NULLs) and stream of NULL flags. - * - * Different streams are identified by "path". - * If the data type require single stream (it's true for most of data types), the stream will have empty path. - * Otherwise, the path can have components like "array elements", "array sizes", etc. - * - * For multidimensional arrays, path can have arbiraty length. - * As an example, for 2-dimensional arrays of numbers we have at least three streams: - * - array sizes; (sizes of top level arrays) - * - array elements / array sizes; (sizes of second level (nested) arrays) - * - array elements / array elements; (the most deep elements, placed contiguously) - * - * Descendants must override either serializeBinaryBulk, deserializeBinaryBulk methods (for simple cases with single stream) - * or serializeBinaryBulkWithMultipleStreams, deserializeBinaryBulkWithMultipleStreams, enumerateStreams methods (for cases with multiple streams). - * - * Default implementations of ...WithMultipleStreams methods will call serializeBinaryBulk, deserializeBinaryBulk for single stream. - */ - - // struct Substream - // { - // enum Type - // { - // ArrayElements, - // ArraySizes, - - // NullableElements, - // NullMap, - - // TupleElement, - - // DictionaryKeys, - // DictionaryIndexes, - // }; - // Type type; - - // /// Index of tuple element, starting at 1. - // String tuple_element_name; - - // Substream(Type type_) : type(type_) {} - // }; - - // using SubstreamPath = std::vector; - - // using StreamCallback = std::function; - // virtual void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const - // { - // callback(path); - // } - // void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); } - // void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); } - - // // using OutputStreamGetter = std::function; - // // using InputStreamGetter = std::function; - - // struct SerializeBinaryBulkState - // { - // virtual ~SerializeBinaryBulkState() = default; - // }; - // struct DeserializeBinaryBulkState - // { - // virtual ~DeserializeBinaryBulkState() = default; - // }; - - // using SerializeBinaryBulkStatePtr = std::shared_ptr; - // using DeserializeBinaryBulkStatePtr = std::shared_ptr; - - // struct SerializeBinaryBulkSettings - // { - // OutputStreamGetter getter; - // SubstreamPath path; - - // size_t low_cardinality_max_dictionary_size = 0; - // bool low_cardinality_use_single_dictionary_for_part = true; - - // bool position_independent_encoding = true; - // }; - - // struct DeserializeBinaryBulkSettings - // { - // InputStreamGetter getter; - // SubstreamPath path; - - // /// True if continue reading from previous positions in file. False if made fseek to the start of new granule. - // bool continuous_reading = true; - - // bool position_independent_encoding = true; - // /// If not zero, may be used to avoid reallocations while reading column of String type. - // double avg_value_size_hint = 0; - // }; - - /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. - // virtual void serializeBinaryBulkStatePrefix( - // SerializeBinaryBulkSettings & /*settings*/, - // SerializeBinaryBulkStatePtr & /*state*/) const {} - - // /// Call after serializeBinaryBulkWithMultipleStreams chain to finish serialization. - // virtual void serializeBinaryBulkStateSuffix( - // SerializeBinaryBulkSettings & /*settings*/, - // SerializeBinaryBulkStatePtr & /*state*/) const {} - - // /// Call before before deserializeBinaryBulkWithMultipleStreams chain to get DeserializeBinaryBulkStatePtr. - // virtual void deserializeBinaryBulkStatePrefix( - // DeserializeBinaryBulkSettings & /*settings*/, - // DeserializeBinaryBulkStatePtr & /*state*/) const {} - - /** 'offset' and 'limit' are used to specify range. - * limit = 0 - means no limit. - * offset must be not greater than size of column. - * offset + limit could be greater than size of column - * - in that case, column is serialized till the end. - */ - // virtual void serializeBinaryBulkWithMultipleStreams( - // const IColumn & column, - // size_t offset, - // size_t limit, - // SerializeBinaryBulkSettings & settings, - // SerializeBinaryBulkStatePtr & /*state*/) const - // { - // if (WriteBuffer * stream = settings.getter(settings.path)) - // serializeBinaryBulk(column, *stream, offset, limit); - // } - - // /// Read no more than limit values and append them into column. - // virtual void deserializeBinaryBulkWithMultipleStreams( - // IColumn & column, - // size_t limit, - // DeserializeBinaryBulkSettings & settings, - // DeserializeBinaryBulkStatePtr & /*state*/) const - // { - // if (ReadBuffer * stream = settings.getter(settings.path)) - // deserializeBinaryBulk(column, *stream, limit, settings.avg_value_size_hint); - // } - - /** Override these methods for data types that require just single stream (most of data types). - */ - // virtual void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const; - // virtual void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const; - - /** Serialization/deserialization of individual values. - * - * These are helper methods for implementation of various formats to input/output for user (like CSV, JSON, etc.). - * There is no one-to-one correspondence between formats and these methods. - * For example, TabSeparated and Pretty formats could use same helper method serializeTextEscaped. - * - * For complex data types (like arrays) binary serde for individual values may differ from bulk serde. - * For example, if you serialize single array, it will be represented as its size and elements in single contiguous stream, - * but if you bulk serialize column with arrays, then sizes and elements will be written to separate streams. - */ - - // /// There is two variants for binary serde. First variant work with Field. - // virtual void serializeBinary(const Field & field, WriteBuffer & ostr) const = 0; - // virtual void deserializeBinary(Field & field, ReadBuffer & istr) const = 0; - - // /// Other variants takes a column, to avoid creating temporary Field object. - // /// Column must be non-constant. - - // /// Serialize one value of a column at specified row number. - // virtual void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const = 0; - // /// Deserialize one value and insert into a column. - // /// 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; - - // /** Serialize to a protobuf. */ - // virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const = 0; - // virtual void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const = 0; - - // /** Text serialization with escaping but without quoting. - // */ - // void serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const; - - // void deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const; - - // /** Text serialization as a literal that may be inserted into a query. - // */ - // void serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const; - - // void deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const; - - // /** Text serialization for the CSV format. - // */ - // void serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const; - // 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. - // */ - // void serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const; - - // /** Text deserialization in case when buffer contains only one value, without any escaping and delimiters. - // */ - // void deserializeAsWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const; - - // /** Text serialization intended for using in JSON format. - // */ - // void serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const; - // void deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const; - - // /** Text serialization for putting into the XML format. - // */ - // void serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; - protected: virtual String doGetName() const; - /// Default implementations of text serialization in case of 'custom_text_serialization' is not set. - - // 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; - // 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; - // 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; - // virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0; - // virtual void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0; - // 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; - // virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const - // { - // serializeText(column, row_num, ostr, settings); - // } - public: /** Create empty column for corresponding type. */ diff --git a/be/src/vec/data_types/data_type_date.cpp b/be/src/vec/data_types/data_type_date.cpp new file mode 100644 index 00000000000000..5aaebcf9d9ea21 --- /dev/null +++ b/be/src/vec/data_types/data_type_date.cpp @@ -0,0 +1,45 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "vec/data_types/data_type_date.h" + +#include "runtime/datetime_value.h" +#include "vec/columns/columns_number.h" + +namespace doris::vectorized { + +bool DataTypeDate::equals(const IDataType& rhs) const { + return typeid(rhs) == typeid(*this); +} + +std::string DataTypeDate::to_string(const IColumn& column, size_t row_num) const { + Int128 int_val = assert_cast(*column.convertToFullColumnIfConst().get()) + .getData()[row_num]; + doris::DateTimeValue value = *reinterpret_cast(&int_val); + std::stringstream ss; + // Year + uint32_t temp = value.year() / 100; + ss << (char)('0' + (temp / 10)) << (char)('0' + (temp % 10)); + temp = value.year() % 100; + ss << (char)('0' + (temp / 10)) << (char)('0' + (temp % 10)) << '-'; + // Month + ss << (char)('0' + (value.month() / 10)) << (char)('0' + (value.month() % 10)) << '-'; + // Day + ss << (char)('0' + (value.day() / 10)) << (char)('0' + (value.day() % 10)); + return ss.str(); +} +} // namespace doris::vectorized diff --git a/be/src/vec/common/sleep.h b/be/src/vec/data_types/data_type_date.h similarity index 58% rename from be/src/vec/common/sleep.h rename to be/src/vec/data_types/data_type_date.h index ff759e656b0088..cd3778e0cfa3e8 100644 --- a/be/src/vec/common/sleep.h +++ b/be/src/vec/data_types/data_type_date.h @@ -17,17 +17,20 @@ #pragma once -#include +#include "vec/data_types/data_type_number_base.h" -/** - * Sleep functions tolerant to signal interruptions (which can happen - * when query profiler is turned on for example) - */ +namespace doris::vectorized { -void sleepForNanoseconds(uint64_t nanoseconds); +class DataTypeDate final : public DataTypeNumberBase { +public: + TypeIndex getTypeId() const override { return TypeIndex::Date; } + const char* getFamilyName() const override { return "Date"; } -void sleepForMicroseconds(uint64_t microseconds); + bool canBeUsedAsVersion() const override { return true; } + bool canBeInsideNullable() const override { return true; } -void sleepForMilliseconds(uint64_t milliseconds); + bool equals(const IDataType& rhs) const override; + std::string to_string(const IColumn& column, size_t row_num) const; +}; -void sleepForSeconds(uint64_t seconds); +} // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_date_time.cpp b/be/src/vec/data_types/data_type_date_time.cpp new file mode 100644 index 00000000000000..cfd2ea10c264ea --- /dev/null +++ b/be/src/vec/data_types/data_type_date_time.cpp @@ -0,0 +1,74 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "vec/data_types/data_type_date_time.h" + +#include "runtime/datetime_value.h" +#include "vec/columns/columns_number.h" + +namespace doris::vectorized { + +DataTypeDateTime::DataTypeDateTime() {} + +std::string DataTypeDateTime::doGetName() const { + return "DateTime"; +} + +bool DataTypeDateTime::equals(const IDataType& rhs) const { + return typeid(rhs) == typeid(*this); +} +std::string DataTypeDateTime::to_string(const IColumn& column, size_t row_num) const { + Int128 int_val = assert_cast(*column.convertToFullColumnIfConst().get()) + .getData()[row_num]; + doris::DateTimeValue value = *reinterpret_cast(&int_val); + std::stringstream ss; + // Year + uint32_t temp = value.year() / 100; + ss << (char)('0' + (temp / 10)) << (char)('0' + (temp % 10)); + temp = value.year() % 100; + ss << (char)('0' + (temp / 10)) << (char)('0' + (temp % 10)) << '-'; + // Month + ss << (char)('0' + (value.month() / 10)) << (char)('0' + (value.month() % 10)) << '-'; + // Day + ss << (char)('0' + (value.day() / 10)) << (char)('0' + (value.day() % 10)); + if (value.neg()) { + ss << '-'; + } + ss << ' '; + // Hour + temp = value.hour(); + if (temp >= 100) { + ss << (char)('0' + (temp / 100)); + temp %= 100; + } + ss << (char)('0' + (temp / 10)) << (char)('0' + (temp % 10)) << ':'; + // Minute + ss << (char)('0' + (value.minute() / 10)) << (char)('0' + (value.minute() % 10)) << ':'; + /* Second */ + ss << (char)('0' + (value.second() / 10)) << (char)('0' + (value.second() % 10)); + if (value.microsecond() > 0) { + ss << '.'; + uint32_t first = value.microsecond() / 10000; + uint32_t second = (value.microsecond() % 10000) / 100; + uint32_t third = value.microsecond() % 100; + ss << (char)('0' + first / 10) << (char)('0' + first % 10) << (char)('0' + second / 10); + ss << (char)('0' + second % 10) << (char)('0' + third / 10) << (char)('0' + third % 10); + } + return ss.str(); +} + +} // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_date_time.h b/be/src/vec/data_types/data_type_date_time.h new file mode 100644 index 00000000000000..c1bd1c9b5636b3 --- /dev/null +++ b/be/src/vec/data_types/data_type_date_time.h @@ -0,0 +1,63 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "vec/data_types/data_type_number_base.h" + +class DateLUTImpl; + +namespace doris::vectorized { + +/** DateTime stores time as unix timestamp. + * The value itself is independent of time zone. + * + * In binary format it is represented as unix timestamp. + * In text format it is serialized to and parsed from YYYY-MM-DD hh:mm:ss format. + * The text format is dependent of time zone. + * + * To convert from/to text format, time zone may be specified explicitly or implicit time zone may be used. + * + * Time zone may be specified explicitly as type parameter, example: DateTime('Europe/Moscow'). + * As it does not affect the internal representation of values, + * all types with different time zones are equivalent and may be used interchangingly. + * Time zone only affects parsing and displaying in text formats. + * + * If time zone is not specified (example: DateTime without parameter), then default time zone is used. + * Default time zone is server time zone, if server is doing transformations + * and if client is doing transformations, unless 'use_client_time_zone' setting is passed to client; + * Server time zone is the time zone specified in 'timezone' parameter in configuration file, + * or system time zone at the moment of server startup. + */ +class DataTypeDateTime final : public DataTypeNumberBase { +public: + DataTypeDateTime(const std::string& time_zone_name = ""); + + const char* getFamilyName() const override { return "DateTime"; } + std::string doGetName() const override; + TypeIndex getTypeId() const override { return TypeIndex::DateTime; } + + bool canBeUsedAsVersion() const override { return true; } + bool canBeInsideNullable() const override { return true; } + + bool equals(const IDataType& rhs) const override; + + const std::string& getTimeZone() const { return time_zone; } + std::string to_string(const IColumn& column, size_t row_num) const; +}; + +} // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_factory.hpp b/be/src/vec/data_types/data_type_factory.hpp index 29a36a50423df0..285458a4ad25b4 100644 --- a/be/src/vec/data_types/data_type_factory.hpp +++ b/be/src/vec/data_types/data_type_factory.hpp @@ -20,6 +20,8 @@ #include #include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_date.h" +#include "vec/data_types/data_type_date_time.h" #include "vec/data_types/data_type_nothing.h" #include "vec/data_types/data_type_string.h" #include "vec/data_types/data_types_decimal.h" @@ -46,6 +48,9 @@ class DataTypeFactory { instance.regist_data_type("Int64", DataTypePtr(std::make_shared())); instance.regist_data_type("Float32", DataTypePtr(std::make_shared())); instance.regist_data_type("Float64", DataTypePtr(std::make_shared())); + instance.regist_data_type("Date", DataTypePtr(std::make_shared())); + instance.regist_data_type("DateTime", + DataTypePtr(std::make_shared())); instance.regist_data_type("String", DataTypePtr(std::make_shared())); }); return instance; diff --git a/be/src/vec/data_types/data_type_nothing.cpp b/be/src/vec/data_types/data_type_nothing.cpp index 8bcde2b7c06c58..09f23bd194e55f 100644 --- a/be/src/vec/data_types/data_type_nothing.cpp +++ b/be/src/vec/data_types/data_type_nothing.cpp @@ -27,22 +27,6 @@ MutableColumnPtr DataTypeNothing::createColumn() const { return ColumnNothing::create(0); } -// void DataTypeNothing::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const -// { -// size_t size = column.size(); - -// if (limit == 0 || offset + limit > size) -// limit = size - offset; - -// for (size_t i = 0; i < limit; ++i) -// ostr.write('0'); -// } - -// void DataTypeNothing::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const -// { -// typeid_cast(column).addSize(istr.tryIgnore(limit)); -// } - void DataTypeNothing::serialize(const IColumn& column, PColumn* pcolumn) const {} void DataTypeNothing::deserialize(const PColumn& pcolumn, IColumn* column) const {} @@ -50,9 +34,4 @@ bool DataTypeNothing::equals(const IDataType& rhs) const { return typeid(rhs) == typeid(*this); } -// void registerDataTypeNothing(DataTypeFactory & factory) -// { -// factory.registerSimpleDataType("Nothing", [] { return DataTypePtr(std::make_shared()); }); -// } - } // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_nothing.h b/be/src/vec/data_types/data_type_nothing.h index d08ab65e7bfffa..7ba395c8beac9c 100644 --- a/be/src/vec/data_types/data_type_nothing.h +++ b/be/src/vec/data_types/data_type_nothing.h @@ -37,10 +37,6 @@ class DataTypeNothing final : public IDataType { MutableColumnPtr createColumn() const override; - /// These methods read and write zero bytes just to allow to figure out size of column. - // void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; - // void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; - bool equals(const IDataType& rhs) const override; bool isParametric() const override { return false; } diff --git a/be/src/vec/data_types/data_type_nullable.cpp b/be/src/vec/data_types/data_type_nullable.cpp index 0e8199b6eeda75..740d1e6f596c77 100644 --- a/be/src/vec/data_types/data_type_nullable.cpp +++ b/be/src/vec/data_types/data_type_nullable.cpp @@ -55,438 +55,6 @@ std::string DataTypeNullable::to_string(const IColumn& column, size_t row_num) c } } -// void DataTypeNullable::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const -// { -// path.push_back(Substream::NullMap); -// callback(path); -// path.back() = Substream::NullableElements; -// nested_data_type->enumerateStreams(callback, path); -// path.pop_back(); -// } - -// void DataTypeNullable::serializeBinaryBulkStatePrefix( -// SerializeBinaryBulkSettings & settings, -// SerializeBinaryBulkStatePtr & state) const -// { -// settings.path.push_back(Substream::NullableElements); -// nested_data_type->serializeBinaryBulkStatePrefix(settings, state); -// settings.path.pop_back(); -// } - -// void DataTypeNullable::serializeBinaryBulkStateSuffix( -// SerializeBinaryBulkSettings & settings, -// SerializeBinaryBulkStatePtr & state) const -// { -// settings.path.push_back(Substream::NullableElements); -// nested_data_type->serializeBinaryBulkStateSuffix(settings, state); -// settings.path.pop_back(); -// } - -// void DataTypeNullable::deserializeBinaryBulkStatePrefix( -// DeserializeBinaryBulkSettings & settings, -// DeserializeBinaryBulkStatePtr & state) const -// { -// settings.path.push_back(Substream::NullableElements); -// nested_data_type->deserializeBinaryBulkStatePrefix(settings, state); -// settings.path.pop_back(); -// } - -// void DataTypeNullable::serializeBinaryBulkWithMultipleStreams( -// const IColumn & column, -// size_t offset, -// size_t limit, -// SerializeBinaryBulkSettings & settings, -// SerializeBinaryBulkStatePtr & state) const -// { -// const ColumnNullable & col = assert_cast(column); -// col.checkConsistency(); - -// /// First serialize null map. -// settings.path.push_back(Substream::NullMap); -// if (auto stream = settings.getter(settings.path)) -// DataTypeUInt8().serializeBinaryBulk(col.getNullMapColumn(), *stream, offset, limit); - -// /// Then serialize contents of arrays. -// settings.path.back() = Substream::NullableElements; -// nested_data_type->serializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), offset, limit, settings, state); -// settings.path.pop_back(); -// } - -// void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( -// IColumn & column, -// size_t limit, -// DeserializeBinaryBulkSettings & settings, -// DeserializeBinaryBulkStatePtr & state) const -// { -// ColumnNullable & col = assert_cast(column); - -// settings.path.push_back(Substream::NullMap); -// if (auto stream = settings.getter(settings.path)) -// DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0); - -// settings.path.back() = Substream::NullableElements; -// nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), limit, settings, state); -// settings.path.pop_back(); -// } - -// void DataTypeNullable::serializeBinary(const Field & field, WriteBuffer & ostr) const -// { -// if (field.isNull()) -// { -// writeBinary(true, ostr); -// } -// else -// { -// writeBinary(false, ostr); -// nested_data_type->serializeBinary(field, ostr); -// } -// } - -// void DataTypeNullable::deserializeBinary(Field & field, ReadBuffer & istr) const -// { -// bool is_null = false; -// readBinary(is_null, istr); -// if (!is_null) -// { -// nested_data_type->deserializeBinary(field, istr); -// } -// else -// { -// field = Null(); -// } -// } - -// void DataTypeNullable::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const -// { -// const ColumnNullable & col = assert_cast(column); - -// bool is_null = col.isNullAt(row_num); -// writeBinary(is_null, ostr); -// if (!is_null) -// nested_data_type->serializeBinary(col.getNestedColumn(), row_num, ostr); -// } - -// /// Deserialize value into ColumnNullable. -// /// We need to insert both to nested column and to null byte map, or, in case of exception, to not insert at all. -// template , ReturnType>* = nullptr> -// static ReturnType safeDeserialize( -// IColumn & column, const IDataType & /*nested_data_type*/, -// CheckForNull && check_for_null, DeserializeNested && deserialize_nested) -// { -// ColumnNullable & col = assert_cast(column); - -// if (check_for_null()) -// { -// col.insertDefault(); -// } -// else -// { -// deserialize_nested(col.getNestedColumn()); - -// try -// { -// col.getNullMapData().push_back(0); -// } -// catch (...) -// { -// col.getNestedColumn().popBack(1); -// throw; -// } -// } -// } - -// /// Deserialize value into non-nullable column. In case of NULL, insert default value and return false. -// template , ReturnType>* = nullptr> -// static ReturnType safeDeserialize( -// IColumn & column, const IDataType & nested_data_type, -// CheckForNull && check_for_null, DeserializeNested && deserialize_nested) -// { -// assert(!dynamic_cast(&column)); -// assert(!dynamic_cast(&nested_data_type)); -// bool insert_default = check_for_null(); -// if (insert_default) -// nested_data_type.insertDefaultInto(column); -// else -// deserialize_nested(column); -// return !insert_default; -// } - -// void DataTypeNullable::deserializeBinary(IColumn & column, ReadBuffer & istr) const -// { -// safeDeserialize(column, *nested_data_type, -// [&istr] { bool is_null = 0; readBinary(is_null, istr); return is_null; }, -// [this, &istr] (IColumn & nested) { nested_data_type->deserializeBinary(nested, istr); }); -// } - -// void DataTypeNullable::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -// { -// const ColumnNullable & col = assert_cast(column); - -// if (col.isNullAt(row_num)) -// writeCString("\\N", ostr); -// else -// nested_data_type->serializeAsTextEscaped(col.getNestedColumn(), row_num, ostr, settings); -// } - -// void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const -// { -// deserializeTextEscaped(column, istr, settings, nested_data_type); -// } - -// template -// ReturnType DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, -// const DataTypePtr & nested_data_type) -// { -// /// Little tricky, because we cannot discriminate null from first character. - -// if (istr.eof()) -// throw Exception("Unexpected end of stream, while parsing value of Nullable type", ErrorCodes::CANNOT_READ_ALL_DATA); - -// /// This is not null, surely. -// if (*istr.position() != '\\') -// { -// return safeDeserialize(column, *nested_data_type, -// [] { return false; }, -// [&nested_data_type, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsTextEscaped(nested, istr, settings); }); -// } -// else -// { -// /// Now we know, that data in buffer starts with backslash. -// ++istr.position(); - -// if (istr.eof()) -// throw Exception("Unexpected end of stream, while parsing value of Nullable type, after backslash", ErrorCodes::CANNOT_READ_ALL_DATA); - -// return safeDeserialize(column, *nested_data_type, -// [&istr] -// { -// if (*istr.position() == 'N') -// { -// ++istr.position(); -// return true; -// } -// return false; -// }, -// [&nested_data_type, &istr, &settings] (IColumn & nested) -// { -// if (istr.position() != istr.buffer().begin()) -// { -// /// We could step back to consume backslash again. -// --istr.position(); -// nested_data_type->deserializeAsTextEscaped(nested, istr, settings); -// } -// else -// { -// /// Otherwise, we need to place backslash back in front of istr. -// ReadBufferFromMemory prefix("\\", 1); -// ConcatReadBuffer prepended_istr(prefix, istr); - -// nested_data_type->deserializeAsTextEscaped(nested, prepended_istr, settings); - -// /// Synchronise cursor position in original buffer. - -// if (prepended_istr.count() > 1) -// istr.position() = prepended_istr.position(); -// } -// }); -// } -// } - -// void DataTypeNullable::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -// { -// const ColumnNullable & col = assert_cast(column); - -// if (col.isNullAt(row_num)) -// writeCString("NULL", ostr); -// else -// nested_data_type->serializeAsTextQuoted(col.getNestedColumn(), row_num, ostr, settings); -// } - -// void DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const -// { -// deserializeTextQuoted(column, istr, settings, nested_data_type); -// } - -// template -// ReturnType DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, -// const DataTypePtr & nested_data_type) -// { -// return safeDeserialize(column, *nested_data_type, -// [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); }, -// [&nested_data_type, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsTextQuoted(nested, istr, settings); }); -// } - -// void DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const -// { -// safeDeserialize(column, *nested_data_type, -// [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); }, -// [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsWholeText(nested, istr, settings); }); -// } - -// void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -// { -// const ColumnNullable & col = assert_cast(column); - -// if (col.isNullAt(row_num)) -// writeCString("\\N", ostr); -// else -// nested_data_type->serializeAsTextCSV(col.getNestedColumn(), row_num, ostr, settings); -// } - -// void DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const -// { -// deserializeTextCSV(column, istr, settings, nested_data_type); -// } - -// template -// ReturnType DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, -// const DataTypePtr & nested_data_type) -// { -// constexpr char const * null_literal = "NULL"; -// constexpr size_t len = 4; -// size_t null_prefix_len = 0; - -// auto check_for_null = [&istr, &settings, &null_prefix_len] -// { -// if (checkStringByFirstCharacterAndAssertTheRest("\\N", istr)) -// return true; -// if (!settings.csv.unquoted_null_literal_as_null) -// return false; - -// /// Check for unquoted NULL -// while (!istr.eof() && null_prefix_len < len && null_literal[null_prefix_len] == *istr.position()) -// { -// ++null_prefix_len; -// ++istr.position(); -// } -// if (null_prefix_len == len) -// return true; - -// /// Value and "NULL" have common prefix, but value is not "NULL". -// /// Restore previous buffer position if possible. -// if (null_prefix_len <= istr.offset()) -// { -// istr.position() -= null_prefix_len; -// null_prefix_len = 0; -// } -// return false; -// }; - -// auto deserialize_nested = [&nested_data_type, &settings, &istr, &null_prefix_len] (IColumn & nested) -// { -// if (likely(!null_prefix_len)) -// nested_data_type->deserializeAsTextCSV(nested, istr, settings); -// else -// { -// /// Previous buffer position was not restored, -// /// so we need to prepend extracted characters (rare case) -// ReadBufferFromMemory prepend(null_literal, null_prefix_len); -// ConcatReadBuffer buf(prepend, istr); -// nested_data_type->deserializeAsTextCSV(nested, buf, settings); - -// /// Check if all extracted characters were read by nested parser and update buffer position -// if (null_prefix_len < buf.count()) -// istr.position() = buf.position(); -// else if (null_prefix_len > buf.count()) -// { -// /// It can happen only if there is an unquoted string instead of a number -// /// or if someone uses 'U' or 'L' as delimiter in CSV. -// /// In the first case we cannot continue reading anyway. The second case seems to be unlikely. -// if (settings.csv.delimiter == 'U' || settings.csv.delimiter == 'L') -// throw doris::vectorized::Exception("Enabled setting input_format_csv_unquoted_null_literal_as_null may not work correctly " -// "with format_csv_delimiter = 'U' or 'L' for large input.", ErrorCodes::CANNOT_READ_ALL_DATA); -// WriteBufferFromOwnString parsed_value; -// nested_data_type->serializeAsTextCSV(nested, nested.size() - 1, parsed_value, settings); -// throw doris::vectorized::Exception("Error while parsing \"" + std::string(null_literal, null_prefix_len) -// + std::string(istr.position(), std::min(size_t{10}, istr.available())) + "\" as Nullable(" + nested_data_type->getName() -// + ") at position " + std::to_string(istr.count()) + ": expected \"NULL\" or " + nested_data_type->getName() -// + ", got \"" + std::string(null_literal, buf.count()) + "\", which was deserialized as \"" -// + parsed_value.str() + "\". It seems that input data is ill-formatted.", -// ErrorCodes::CANNOT_READ_ALL_DATA); -// } -// } -// }; - -// return safeDeserialize(column, *nested_data_type, check_for_null, deserialize_nested); -// } - -// void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -// { -// const ColumnNullable & col = assert_cast(column); - -// /// In simple text format (like 'Pretty' format) (these formats are suitable only for output and cannot be parsed back), -// /// data is printed without escaping. -// /// It makes theoretically impossible to distinguish between NULL and some string value, regardless on how do we print NULL. -// /// For this reason, we output NULL in a bit strange way. -// /// This assumes UTF-8 and proper font support. This is Ok, because Pretty formats are "presentational", not for data exchange. - -// if (col.isNullAt(row_num)) -// writeCString("ᴺᵁᴸᴸ", ostr); -// else -// 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 -// { -// const ColumnNullable & col = assert_cast(column); - -// if (col.isNullAt(row_num)) -// writeCString("null", ostr); -// else -// nested_data_type->serializeAsTextJSON(col.getNestedColumn(), row_num, ostr, settings); -// } - -// void DataTypeNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const -// { -// deserializeTextJSON(column, istr, settings, nested_data_type); -// } - -// template -// ReturnType DataTypeNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, -// const DataTypePtr & nested_data_type) -// { -// return safeDeserialize(column, *nested_data_type, -// [&istr] { return checkStringByFirstCharacterAndAssertTheRest("null", istr); }, -// [&nested_data_type, &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 -// { -// const ColumnNullable & col = assert_cast(column); - -// if (col.isNullAt(row_num)) -// writeCString("\\N", ostr); -// else -// nested_data_type->serializeAsTextXML(col.getNestedColumn(), row_num, ostr, settings); -// } - -// void DataTypeNullable::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const -// { -// const ColumnNullable & col = assert_cast(column); -// if (!col.isNullAt(row_num)) -// nested_data_type->serializeProtobuf(col.getNestedColumn(), row_num, protobuf, value_index); -// } - -// void DataTypeNullable::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const -// { -// ColumnNullable & col = assert_cast(column); -// IColumn & nested_column = col.getNestedColumn(); -// size_t old_size = nested_column.size(); -// try -// { -// nested_data_type->deserializeProtobuf(nested_column, protobuf, allow_add_row, row_added); -// if (row_added) -// col.getNullMapData().push_back(0); -// } -// catch (...) -// { -// nested_column.popBack(nested_column.size() - old_size); -// col.getNullMapData().resize_assume_reserved(old_size); -// row_added = false; -// throw; -// } -// } - void DataTypeNullable::serialize(const IColumn& column, PColumn* pcolumn) const { const ColumnNullable& col = assert_cast(*column.convertToFullColumnIfConst().get()); @@ -528,21 +96,6 @@ bool DataTypeNullable::equals(const IDataType& rhs) const { nested_data_type->equals(*static_cast(rhs).nested_data_type); } -// static DataTypePtr create(const ASTPtr & arguments) -// { -// if (!arguments || arguments->children.size() != 1) -// throw Exception("Nullable data type family must have exactly one argument - nested type", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - -// DataTypePtr nested_type = DataTypeFactory::instance().get(arguments->children[0]); - -// return std::make_shared(nested_type); -// } - -// void registerDataTypeNullable(DataTypeFactory & factory) -// { -// factory.registerDataType("Nullable", create); -// } - DataTypePtr makeNullable(const DataTypePtr& type) { if (type->isNullable()) return type; return std::make_shared(type); @@ -553,9 +106,4 @@ DataTypePtr removeNullable(const DataTypePtr& type) { return type; } -// template bool DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); -// template bool DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &, const DataTypePtr & nested); -// template bool DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); -// template bool DataTypeNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &, const DataTypePtr & nested); - } // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_nullable.h b/be/src/vec/data_types/data_type_nullable.h index 887633538e52f7..a661bf4a11c55b 100644 --- a/be/src/vec/data_types/data_type_nullable.h +++ b/be/src/vec/data_types/data_type_nullable.h @@ -35,63 +35,6 @@ class DataTypeNullable final : public IDataType { const char* getFamilyName() const override { return "Nullable"; } TypeIndex getTypeId() const override { return TypeIndex::Nullable; } - // void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; - - // void serializeBinaryBulkStatePrefix( - // SerializeBinaryBulkSettings & settings, - // SerializeBinaryBulkStatePtr & state) const override; - - // void serializeBinaryBulkStateSuffix( - // SerializeBinaryBulkSettings & settings, - // SerializeBinaryBulkStatePtr & state) const override; - - // void deserializeBinaryBulkStatePrefix( - // DeserializeBinaryBulkSettings & settings, - // DeserializeBinaryBulkStatePtr & state) const override; - - // void serializeBinaryBulkWithMultipleStreams( - // const IColumn & column, - // size_t offset, - // size_t limit, - // SerializeBinaryBulkSettings & settings, - // SerializeBinaryBulkStatePtr & state) const override; - - // void deserializeBinaryBulkWithMultipleStreams( - // IColumn & column, - // size_t limit, - // DeserializeBinaryBulkSettings & settings, - // DeserializeBinaryBulkStatePtr & state) const override; - - // void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - // void deserializeBinary(Field & field, ReadBuffer & istr) const override; - // void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - // void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; - // 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; - // 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; - // void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; - - // void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - - /** It is questionable, how NULL values could be represented in CSV. There are three variants: - * 1. \N - * 2. empty string (without quotes) - * 3. NULL - * We support all of them (however, second variant is supported by CSVRowInputStream, not by deserializeTextCSV). - * (see also input_format_defaults_for_omitted_fields and input_format_csv_unquoted_null_literal_as_null settings) - * In CSV, non-NULL string value, starting with \N characters, must be placed in quotes, to avoid ambiguity. - */ - // void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; - - // 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; - // void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - // void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - - // void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override; - // void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override; - void serialize(const IColumn& column, PColumn* pcolumn) const override; void deserialize(const PColumn& pcolumn, IColumn* column) const override; MutableColumnPtr createColumn() const override; @@ -136,17 +79,6 @@ class DataTypeNullable final : public IDataType { const DataTypePtr& getNestedType() const { return nested_data_type; } - /// If ReturnType is bool, check for NULL and deserialize value into non-nullable column (and return true) or insert default value of nested type (and return false) - /// If ReturnType is void, deserialize Nullable(T) - // template - // static ReturnType deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); - // template - // static ReturnType deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &, const DataTypePtr & nested); - // template - // static ReturnType deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); - // template - // static ReturnType deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &, const DataTypePtr & nested); - private: DataTypePtr nested_data_type; }; diff --git a/be/src/vec/data_types/data_type_number_base.cpp b/be/src/vec/data_types/data_type_number_base.cpp index db62e419e0c36e..1bb8e6751aa2a0 100644 --- a/be/src/vec/data_types/data_type_number_base.cpp +++ b/be/src/vec/data_types/data_type_number_base.cpp @@ -33,237 +33,35 @@ template void DataTypeNumberBase::to_string(const IColumn& column, size_t row_num, BufferWritable& ostr) const { if constexpr (std::is_same::value || std::is_same::value) { - // write int 128 + std::string hex = int128_to_string( + assert_cast&>(*column.convertToFullColumnIfConst().get()) + .getData()[row_num]); + ostr.write(hex.data(), hex.size()); } else if constexpr (std::is_integral::value || std::numeric_limits::is_iec559) { - ostr.write_number(assert_cast&>(column).getData()[row_num]); + ostr.write_number( + assert_cast&>(*column.convertToFullColumnIfConst().get()) + .getData()[row_num]); } } +template +Field DataTypeNumberBase::getDefault() const { + return NearestFieldType(); +} + template std::string DataTypeNumberBase::to_string(const IColumn& column, size_t row_num) const { if constexpr (std::is_same::value || std::is_same::value) { - // write int 128 + return int128_to_string( + assert_cast&>(*column.convertToFullColumnIfConst().get()) + .getData()[row_num]); } else if constexpr (std::is_integral::value || std::numeric_limits::is_iec559) { return std::to_string( assert_cast&>(*column.convertToFullColumnIfConst().get()) .getData()[row_num]); } - throw Exception("to_string not support", -1); -} - -// template -// void DataTypeNumberBase::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -// { -// writeText(assert_cast &>(column).getData()[row_num], ostr); -// } - -// template -// void DataTypeNumberBase::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -// { -// T x; - -// if constexpr (std::is_integral_v && std::is_arithmetic_v) -// readIntTextUnsafe(x, istr); -// else -// readText(x, istr); - -// assert_cast &>(column).getData().push_back(x); -// } - -// template -// static inline void writeDenormalNumber(T x, WriteBuffer & ostr) -// { -// if constexpr (std::is_floating_point_v) -// { -// if (std::signbit(x)) -// { -// if (isNaN(x)) -// writeCString("-nan", ostr); -// else -// writeCString("-inf", ostr); -// } -// else -// { -// if (isNaN(x)) -// writeCString("nan", ostr); -// else -// writeCString("inf", ostr); -// } -// } -// else -// { -// /// This function is not called for non floating point numbers. -// (void)x; -// } -// } - -// template -// void DataTypeNumberBase::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -// { -// auto x = assert_cast &>(column).getData()[row_num]; -// bool is_finite = isFinite(x); - -// const bool need_quote = (std::is_integral_v && (sizeof(T) == 8) && settings.json.quote_64bit_integers) -// || (settings.json.quote_denormals && !is_finite); - -// if (need_quote) -// writeChar('"', ostr); - -// if (is_finite) -// writeText(x, ostr); -// else if (!settings.json.quote_denormals) -// writeCString("null", ostr); -// else -// writeDenormalNumber(x, ostr); - -// if (need_quote) -// writeChar('"', ostr); -// } - -// template -// void DataTypeNumberBase::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -// { -// bool has_quote = false; -// if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without. -// { -// has_quote = true; -// ++istr.position(); -// } - -// FieldType x; - -// /// null -// if (!has_quote && !istr.eof() && *istr.position() == 'n') -// { -// ++istr.position(); -// assertString("ull", istr); - -// x = NaNOrZero(); -// } -// else -// { -// static constexpr bool is_uint8 = std::is_same_v; -// static constexpr bool is_int8 = std::is_same_v; - -// if (is_uint8 || is_int8) -// { -// // extra conditions to parse true/false strings into 1/0 -// if (istr.eof()) -// throwReadAfterEOF(); -// if (*istr.position() == 't' || *istr.position() == 'f') -// { -// bool tmp = false; -// readBoolTextWord(tmp, istr); -// x = tmp; -// } -// else -// readText(x, istr); -// } -// else -// { -// readText(x, istr); -// } - -// if (has_quote) -// assertChar('"', istr); -// } - -// assert_cast &>(column).getData().push_back(x); -// } - -// template -// void DataTypeNumberBase::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -// { -// FieldType x; -// readCSV(x, istr); -// assert_cast &>(column).getData().push_back(x); -// } - -template -Field DataTypeNumberBase::getDefault() const { - return NearestFieldType(); } -// template -// void DataTypeNumberBase::serializeBinary(const Field & field, WriteBuffer & ostr) const -// { -// /// ColumnVector::value_type is a narrower type. For example, UInt8, when the Field type is UInt64 -// typename ColumnVector::value_type x = get>(field); -// writeBinary(x, ostr); -// } - -// template -// void DataTypeNumberBase::deserializeBinary(Field & field, ReadBuffer & istr) const -// { -// typename ColumnVector::value_type x; -// readBinary(x, istr); -// field = NearestFieldType(x); -// } - -// template -// void DataTypeNumberBase::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const -// { -// writeBinary(assert_cast &>(column).getData()[row_num], ostr); -// } - -// template -// void DataTypeNumberBase::deserializeBinary(IColumn & column, ReadBuffer & istr) const -// { -// typename ColumnVector::value_type x; -// readBinary(x, istr); -// assert_cast &>(column).getData().push_back(x); -// } - -// template -// void DataTypeNumberBase::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const -// { -// const typename ColumnVector::Container & x = typeid_cast &>(column).getData(); - -// size_t size = x.size(); - -// if (limit == 0 || offset + limit > size) -// limit = size - offset; - -// if (limit) -// ostr.write(reinterpret_cast(&x[offset]), sizeof(typename ColumnVector::value_type) * limit); -// } - -// template -// void DataTypeNumberBase::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const -// { -// typename ColumnVector::Container & x = typeid_cast &>(column).getData(); -// size_t initial_size = x.size(); -// x.resize(initial_size + limit); -// size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(typename ColumnVector::value_type) * limit); -// x.resize(initial_size + size / sizeof(typename ColumnVector::value_type)); -// } - -// template -// void DataTypeNumberBase::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const -// { -// if (value_index) -// return; -// value_index = static_cast(protobuf.writeNumber(assert_cast &>(column).getData()[row_num])); -// } - -// template -// void DataTypeNumberBase::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const -// { -// row_added = false; -// T value; -// if (!protobuf.readNumber(value)) -// return; - -// auto & container = typeid_cast &>(column).getData(); -// if (allow_add_row) -// { -// container.emplace_back(value); -// row_added = true; -// } -// else -// container.back() = value; -// } - template void DataTypeNumberBase::serialize(const IColumn& column, PColumn* pcolumn) const { std::ostringstream buf; diff --git a/be/src/vec/data_types/data_type_number_base.h b/be/src/vec/data_types/data_type_number_base.h index 51c84b7dc4f843..e5f626d7a4e3cb 100644 --- a/be/src/vec/data_types/data_type_number_base.h +++ b/be/src/vec/data_types/data_type_number_base.h @@ -36,25 +36,8 @@ class DataTypeNumberBase : public IDataType { const char* getFamilyName() const override { return TypeName::get(); } TypeIndex getTypeId() const override { return TypeId::value; } - - // void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - // void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; - // void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; - // void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; - // void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; Field getDefault() const override; - /** Format is platform-dependent. */ - // void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - // void deserializeBinary(Field & field, ReadBuffer & istr) const override; - // void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - // void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; - // void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; - // void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; - - // void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override; - // void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override; - void serialize(const IColumn& column, PColumn* pcolumn) const override; void deserialize(const PColumn& pcolumn, IColumn* column) const override; MutableColumnPtr createColumn() const override; diff --git a/be/src/vec/data_types/data_type_string.cpp b/be/src/vec/data_types/data_type_string.cpp index d39ea9535e6a33..6fafc23854acf6 100644 --- a/be/src/vec/data_types/data_type_string.cpp +++ b/be/src/vec/data_types/data_type_string.cpp @@ -23,7 +23,6 @@ #include "vec/columns/columns_number.h" #include "vec/common/assert_cast.h" #include "vec/common/typeid_cast.h" -#include "vec/core/defines.h" #include "vec/core/field.h" #include "vec/io/io_helper.h" @@ -33,202 +32,6 @@ namespace doris::vectorized { -//void DataTypeString::serializeBinary(const Field & field, WriteBuffer & ostr) const -//{ -// const String & s = get(field); -// writeVarUInt(s.size(), ostr); -// writeString(s, ostr); -//} -// -// -//void DataTypeString::deserializeBinary(Field & field, ReadBuffer & istr) const -//{ -// UInt64 size; -// readVarUInt(size, istr); -// field = String(); -// String & s = get(field); -// s.resize(size); -// istr.readStrict(s.data(), size); -//} -// -// -//void DataTypeString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const -//{ -// const StringRef & s = assert_cast(column).getDataAt(row_num); -// writeVarUInt(s.size, ostr); -// writeString(s, ostr); -//} -// -// -//void DataTypeString::deserializeBinary(IColumn & column, ReadBuffer & istr) const -//{ -// ColumnString & column_string = assert_cast(column); -// ColumnString::Chars & data = column_string.getChars(); -// ColumnString::Offsets & offsets = column_string.getOffsets(); -// -// UInt64 size; -// readVarUInt(size, istr); -// -// size_t old_chars_size = data.size(); -// size_t offset = old_chars_size + size + 1; -// offsets.push_back(offset); -// -// try -// { -// data.resize(offset); -// istr.readStrict(reinterpret_cast(&data[offset - size - 1]), size); -// data.back() = 0; -// } -// catch (...) -// { -// offsets.pop_back(); -// data.resize_assume_reserved(old_chars_size); -// throw; -// } -//} -// -// -//void DataTypeString::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const -//{ -// const ColumnString & column_string = typeid_cast(column); -// const ColumnString::Chars & data = column_string.getChars(); -// const ColumnString::Offsets & offsets = column_string.getOffsets(); -// -// size_t size = column.size(); -// if (!size) -// return; -// -// size_t end = limit && offset + limit < size -// ? offset + limit -// : size; -// -// if (offset == 0) -// { -// UInt64 str_size = offsets[0] - 1; -// writeVarUInt(str_size, ostr); -// ostr.write(reinterpret_cast(data.data()), str_size); -// -// ++offset; -// } -// -// for (size_t i = offset; i < end; ++i) -// { -// UInt64 str_size = offsets[i] - offsets[i - 1] - 1; -// writeVarUInt(str_size, ostr); -// ostr.write(reinterpret_cast(&data[offsets[i - 1]]), str_size); -// } -//} -// -// -//template -//static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnString::Offsets & offsets, ReadBuffer & istr, size_t limit) -//{ -// size_t offset = data.size(); -// for (size_t i = 0; i < limit; ++i) -// { -// if (istr.eof()) -// break; -// -// UInt64 size; -// readVarUInt(size, istr); -// -// offset += size + 1; -// offsets.push_back(offset); -// -// data.resize(offset); -// -// if (size) -// { -//#ifdef __SSE2__ -// /// An optimistic branch in which more efficient copying is possible. -// if (offset + 16 * UNROLL_TIMES <= data.capacity() && istr.position() + size + 16 * UNROLL_TIMES <= istr.buffer().end()) -// { -// const __m128i * sse_src_pos = reinterpret_cast(istr.position()); -// const __m128i * sse_src_end = sse_src_pos + (size + (16 * UNROLL_TIMES - 1)) / 16 / UNROLL_TIMES * UNROLL_TIMES; -// __m128i * sse_dst_pos = reinterpret_cast<__m128i *>(&data[offset - size - 1]); -// -// while (sse_src_pos < sse_src_end) -// { -// for (size_t j = 0; j < UNROLL_TIMES; ++j) -// _mm_storeu_si128(sse_dst_pos + j, _mm_loadu_si128(sse_src_pos + j)); -// -// sse_src_pos += UNROLL_TIMES; -// sse_dst_pos += UNROLL_TIMES; -// } -// -// istr.position() += size; -// } -// else -//#endif -// { -// istr.readStrict(reinterpret_cast(&data[offset - size - 1]), size); -// } -// } -// -// data[offset - 1] = 0; -// } -//} -// -// -//void DataTypeString::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const -//{ -// ColumnString & column_string = typeid_cast(column); -// ColumnString::Chars & data = column_string.getChars(); -// ColumnString::Offsets & offsets = column_string.getOffsets(); -// -// double avg_chars_size = 1; /// By default reserve only for empty strings. -// -// if (avg_value_size_hint && avg_value_size_hint > sizeof(offsets[0])) -// { -// /// Randomly selected. -// constexpr auto avg_value_size_hint_reserve_multiplier = 1.2; -// -// avg_chars_size = (avg_value_size_hint - sizeof(offsets[0])) * avg_value_size_hint_reserve_multiplier; -// } -// -// size_t size_to_reserve = data.size() + std::ceil(limit * avg_chars_size); -// -// /// Never reserve for too big size. -// if (size_to_reserve < 256 * 1024 * 1024) -// { -// try -// { -// data.reserve(size_to_reserve); -// } -// catch (Exception & e) -// { -// e.addMessage( -// "(avg_value_size_hint = " + toString(avg_value_size_hint) -// + ", avg_chars_size = " + toString(avg_chars_size) -// + ", limit = " + toString(limit) + ")"); -// throw; -// } -// } -// -// offsets.reserve(offsets.size() + limit); -// -// if (avg_chars_size >= 64) -// deserializeBinarySSE2<4>(data, offsets, istr, limit); -// else if (avg_chars_size >= 48) -// deserializeBinarySSE2<3>(data, offsets, istr, limit); -// else if (avg_chars_size >= 32) -// deserializeBinarySSE2<2>(data, offsets, istr, limit); -// else -// deserializeBinarySSE2<1>(data, offsets, istr, limit); -//} -// -// -//void DataTypeString::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -//{ -// writeString(assert_cast(column).getDataAt(row_num), ostr); -//} -// -// -//void DataTypeString::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -//{ -// writeEscapedString(assert_cast(column).getDataAt(row_num), ostr); -//} - template static inline void read(IColumn& column, Reader&& reader) { ColumnString& column_string = assert_cast(column); @@ -254,6 +57,18 @@ std::string DataTypeString::to_string(const IColumn& column, size_t row_num) con return s.toString(); } +Field DataTypeString::getDefault() const { + return String(); +} + +MutableColumnPtr DataTypeString::createColumn() const { + return ColumnString::create(); +} + +bool DataTypeString::equals(const IDataType& rhs) const { + return typeid(rhs) == typeid(*this); +} + void DataTypeString::serialize(const IColumn& column, PColumn* pcolumn) const { std::ostringstream buf; for (size_t i = 0; i < column.size(); ++i) { @@ -283,140 +98,4 @@ void DataTypeString::deserialize(const PColumn& pcolumn, IColumn* column) const data.back() = 0; } } - -//void DataTypeString::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -//{ -// read(column, [&](ColumnString::Chars & data) { readStringInto(data, istr); }); -//} -// -// -//void DataTypeString::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -//{ -// read(column, [&](ColumnString::Chars & data) { readEscapedStringInto(data, istr); }); -//} -// -// -//void DataTypeString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -//{ -// writeQuotedString(assert_cast(column).getDataAt(row_num), ostr); -//} -// -// -//void DataTypeString::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -//{ -// read(column, [&](ColumnString::Chars & data) { readQuotedStringInto(data, istr); }); -//} -// -// -//void DataTypeString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -//{ -// writeJSONString(assert_cast(column).getDataAt(row_num), ostr, settings); -//} -// -// -//void DataTypeString::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -//{ -// read(column, [&](ColumnString::Chars & data) { readJSONStringInto(data, istr); }); -//} -// -// -//void DataTypeString::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -//{ -// writeXMLString(assert_cast(column).getDataAt(row_num), ostr); -//} -// -// -//void DataTypeString::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -//{ -// writeCSVString<>(assert_cast(column).getDataAt(row_num), ostr); -//} -// -// -//void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const -//{ -// read(column, [&](ColumnString::Chars & data) { readCSVStringInto(data, istr, settings.csv); }); -//} -// -// -//void DataTypeString::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const -//{ -// if (value_index) -// return; -// value_index = static_cast(protobuf.writeString(assert_cast(column).getDataAt(row_num))); -//} -// -// -//void DataTypeString::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const -//{ -// row_added = false; -// auto & column_string = assert_cast(column); -// ColumnString::Chars & data = column_string.getChars(); -// ColumnString::Offsets & offsets = column_string.getOffsets(); -// size_t old_size = offsets.size(); -// try -// { -// if (allow_add_row) -// { -// if (protobuf.readStringInto(data)) -// { -// data.emplace_back(0); -// offsets.emplace_back(data.size()); -// row_added = true; -// } -// else -// data.resize_assume_reserved(offsets.back()); -// } -// else -// { -// ColumnString::Chars temp_data; -// if (protobuf.readStringInto(temp_data)) -// { -// temp_data.emplace_back(0); -// column_string.popBack(1); -// old_size = offsets.size(); -// data.insertSmallAllowReadWriteOverflow15(temp_data.begin(), temp_data.end()); -// offsets.emplace_back(data.size()); -// } -// } -// } -// catch (...) -// { -// offsets.resize_assume_reserved(old_size); -// data.resize_assume_reserved(offsets.back()); -// throw; -// } -//} - -Field DataTypeString::getDefault() const { - return String(); -} - -MutableColumnPtr DataTypeString::createColumn() const { - return ColumnString::create(); -} - -bool DataTypeString::equals(const IDataType& rhs) const { - return typeid(rhs) == typeid(*this); -} - -//void registerDataTypeString(DataTypeFactory & factory) -//{ -// auto creator = static_cast([] { return DataTypePtr(std::make_shared()); }); -// -// factory.registerSimpleDataType("String", creator); -// -// /// These synonyms are added for compatibility. -// -// factory.registerAlias("CHAR", "String", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("VARCHAR", "String", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("TEXT", "String", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("TINYTEXT", "String", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("LONGTEXT", "String", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("BLOB", "String", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("TINYBLOB", "String", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("LONGBLOB", "String", DataTypeFactory::CaseInsensitive); -//} - } // namespace doris::vectorized diff --git a/be/src/vec/data_types/data_type_string.h b/be/src/vec/data_types/data_type_string.h index fd0bef02edb7f8..6b4d3b0a3b05a3 100644 --- a/be/src/vec/data_types/data_type_string.h +++ b/be/src/vec/data_types/data_type_string.h @@ -31,34 +31,6 @@ class DataTypeString final : public IDataType { const char* getFamilyName() const override { return "String"; } TypeIndex getTypeId() const override { return TypeIndex::String; } - - // void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - // void deserializeBinary(Field & field, ReadBuffer & istr) const override; - // void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - // void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; - // - // void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; - // void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; - // - // void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - // void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; - // - // 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; - // - // 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; - // - // 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; - // - // void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - // - // void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - // void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; - // - // void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override; - // void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override; void serialize(const IColumn& column, PColumn* pcolumn) const override; void deserialize(const PColumn& pcolumn, IColumn* column) const override; diff --git a/be/src/vec/data_types/data_types_decimal.cpp b/be/src/vec/data_types/data_types_decimal.cpp index 9cbd858fec61c8..3da3f6f39cf858 100644 --- a/be/src/vec/data_types/data_types_decimal.cpp +++ b/be/src/vec/data_types/data_types_decimal.cpp @@ -33,11 +33,6 @@ extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ARGUMENT_OUT_OF_BOUND; } // namespace ErrorCodes -//bool decimalCheckComparisonOverflow(const Context & context) { return context.getSettingsRef().decimal_check_overflow; } -//bool decimalCheckArithmeticOverflow(const Context & context) { return context.getSettingsRef().decimal_check_overflow; } - -// - template std::string DataTypeDecimal::doGetName() const { std::stringstream ss; @@ -61,140 +56,6 @@ std::string DataTypeDecimal::to_string(const IColumn& column, size_t row_num) return buf.str(); } -//template -//void DataTypeDecimal::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -//{ -// T value = assert_cast(column).getData()[row_num]; -// writeText(value, scale, ostr); -//} - -//template -//bool DataTypeDecimal::tryReadText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale) -//{ -// UInt32 unread_scale = scale; -// bool done = tryReadDecimalText(istr, x, precision, unread_scale); -// x *= getScaleMultiplier(unread_scale); -// return done; -//} - -//template -//void DataTypeDecimal::readText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale, bool csv) -//{ -// UInt32 unread_scale = scale; -// if (csv) -// readCSVDecimalText(istr, x, precision, unread_scale); -// else -// readDecimalText(istr, x, precision, unread_scale); -// x *= getScaleMultiplier(unread_scale); -//} -// -//template -//void DataTypeDecimal::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -//{ -// T x; -// readText(x, istr); -// assert_cast(column).getData().push_back(x); -//} - -//template -//void DataTypeDecimal::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -//{ -// T x; -// readText(x, istr, true); -// assert_cast(column).getData().push_back(x); -//} - -//template -//T DataTypeDecimal::parseFromString(const String & str) const -//{ -// ReadBufferFromMemory buf(str.data(), str.size()); -// T x; -// UInt32 unread_scale = scale; -// readDecimalText(buf, x, precision, unread_scale, true); -// x *= getScaleMultiplier(unread_scale); -// return x; -//} - -//template -//void DataTypeDecimal::serializeBinary(const Field & field, WriteBuffer & ostr) const -//{ -// FieldType x = get>(field); -// writeBinary(x, ostr); -//} -// -//template -//void DataTypeDecimal::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const -//{ -// const FieldType & x = assert_cast(column).getElement(row_num); -// writeBinary(x, ostr); -//} -// -//template -//void DataTypeDecimal::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const -//{ -// const typename ColumnType::Container & x = typeid_cast(column).getData(); -// -// size_t size = x.size(); -// -// if (limit == 0 || offset + limit > size) -// limit = size - offset; -// -// ostr.write(reinterpret_cast(&x[offset]), sizeof(FieldType) * limit); -//} - -//template -//void DataTypeDecimal::deserializeBinary(Field & field, ReadBuffer & istr) const -//{ -// typename FieldType::NativeType x; -// readBinary(x, istr); -// field = DecimalField(T(x), scale); -//} -// -//template -//void DataTypeDecimal::deserializeBinary(IColumn & column, ReadBuffer & istr) const -//{ -// typename FieldType::NativeType x; -// readBinary(x, istr); -// assert_cast(column).getData().push_back(FieldType(x)); -//} -// -//template -//void DataTypeDecimal::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double) const -//{ -// typename ColumnType::Container & x = typeid_cast(column).getData(); -// size_t initial_size = x.size(); -// x.resize(initial_size + limit); -// size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(FieldType) * limit); -// x.resize(initial_size + size / sizeof(FieldType)); -//} - -//template -//void DataTypeDecimal::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const -//{ -// if (value_index) -// return; -// value_index = static_cast(protobuf.writeDecimal(assert_cast(column).getData()[row_num], scale)); -//} -// - -//template -//void DataTypeDecimal::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const -//{ -// row_added = false; -// T decimal; -// if (!protobuf.readDecimal(decimal, precision, scale)) -// return; -// -// auto & container = assert_cast(column).getData(); -// if (allow_add_row) -// { -// container.emplace_back(decimal); -// row_added = true; -// } -// else -// container.back() = decimal; -//} - template void DataTypeDecimal::serialize(const IColumn& column, PColumn* pcolumn) const { std::ostringstream buf; @@ -256,53 +117,6 @@ DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value) { return std::make_shared>(precision_value, scale_value); } -//static DataTypePtr create(const ASTPtr & arguments) -//{ -// if (!arguments || arguments->children.size() != 2) -// throw Exception("Decimal data type family must have exactly two arguments: precision and scale", -// ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); -// -// const auto * precision = arguments->children[0]->as(); -// const auto * scale = arguments->children[1]->as(); -// -// if (!precision || precision->value.getType() != Field::Types::UInt64 || -// !scale || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64)) -// throw Exception("Decimal data type family must have a two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); -// -// UInt64 precision_value = precision->value.get(); -// UInt64 scale_value = scale->value.get(); -// -// return createDecimal(precision_value, scale_value); -//} -// -//template -//static DataTypePtr createExact(const ASTPtr & arguments) -//{ -// if (!arguments || arguments->children.size() != 1) -// throw Exception("Decimal data type family must have exactly two arguments: precision and scale", -// ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); -// -// const auto * scale_arg = arguments->children[0]->as(); -// -// if (!scale_arg || !(scale_arg->value.getType() == Field::Types::Int64 || scale_arg->value.getType() == Field::Types::UInt64)) -// throw Exception("Decimal data type family must have a two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); -// -// UInt64 precision = maxDecimalPrecision(); -// UInt64 scale = scale_arg->value.get(); -// -// return createDecimal(precision, scale); -//} -// -//void registerDataTypeDecimal(DataTypeFactory & factory) -//{ -// factory.registerDataType("Decimal32", createExact, DataTypeFactory::CaseInsensitive); -// factory.registerDataType("Decimal64", createExact, DataTypeFactory::CaseInsensitive); -// factory.registerDataType("Decimal128", createExact, DataTypeFactory::CaseInsensitive); -// -// factory.registerDataType("Decimal", create, DataTypeFactory::CaseInsensitive); -// factory.registerAlias("DEC", "Decimal", DataTypeFactory::CaseInsensitive); -//} - template <> Decimal32 DataTypeDecimal::getScaleMultiplier(UInt32 scale_) { return common::exp10_i32(scale_); diff --git a/be/src/vec/data_types/data_types_decimal.h b/be/src/vec/data_types/data_types_decimal.h index a6cf5e08e44915..e3e04fc58299c2 100644 --- a/be/src/vec/data_types/data_types_decimal.h +++ b/be/src/vec/data_types/data_types_decimal.h @@ -115,21 +115,6 @@ class DataTypeDecimal final : public IDataType { 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; - // void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; - // void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; - // - // void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - // void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - // void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; - // - // void deserializeBinary(Field & field, ReadBuffer & istr) const override; - // void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; - // void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; - // - // void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override; - // void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override; - void serialize(const IColumn& column, PColumn* pcolumn) const override; void deserialize(const PColumn& pcolumn, IColumn* column) const override; Field getDefault() const override; @@ -194,11 +179,6 @@ class DataTypeDecimal final : public IDataType { return getScaleMultiplier(); } - // T parseFromString(const String & str) const; - // - // void readText(T & x, ReadBuffer & istr, bool csv = false) const { readText(x, istr, precision, scale, csv); } - // static void readText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale, bool csv = false); - // static bool tryReadText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale); static T getScaleMultiplier(UInt32 scale); private: diff --git a/be/src/vec/data_types/data_types_number.cpp b/be/src/vec/data_types/data_types_number.cpp index 4855ec94645332..8ec44aebcc7d0f 100644 --- a/be/src/vec/data_types/data_types_number.cpp +++ b/be/src/vec/data_types/data_types_number.cpp @@ -16,34 +16,5 @@ // under the License. #include "vec/data_types/data_types_number.h" -// #include -namespace doris::vectorized { - -// void registerDataTypeNumbers(DataTypeFactory & factory) -// { -// factory.registerSimpleDataType("UInt8", [] { return DataTypePtr(std::make_shared()); }); -// factory.registerSimpleDataType("UInt16", [] { return DataTypePtr(std::make_shared()); }); -// factory.registerSimpleDataType("UInt32", [] { return DataTypePtr(std::make_shared()); }); -// factory.registerSimpleDataType("UInt64", [] { return DataTypePtr(std::make_shared()); }); - -// factory.registerSimpleDataType("Int8", [] { return DataTypePtr(std::make_shared()); }); -// factory.registerSimpleDataType("Int16", [] { return DataTypePtr(std::make_shared()); }); -// factory.registerSimpleDataType("Int32", [] { return DataTypePtr(std::make_shared()); }); -// factory.registerSimpleDataType("Int64", [] { return DataTypePtr(std::make_shared()); }); - -// factory.registerSimpleDataType("Float32", [] { return DataTypePtr(std::make_shared()); }); -// factory.registerSimpleDataType("Float64", [] { return DataTypePtr(std::make_shared()); }); - -// /// These synonyms are added for compatibility. - -// factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("INT", "Int32", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("INTEGER", "Int32", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("BIGINT", "Int64", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("FLOAT", "Float32", DataTypeFactory::CaseInsensitive); -// factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive); -// } - -} // namespace doris::vectorized +namespace doris::vectorized {} // namespace doris::vectorized diff --git a/be/src/vec/data_types/get_least_supertype.cpp b/be/src/vec/data_types/get_least_supertype.cpp index f8d165cb2791f4..2cb48ae6b34d0e 100644 --- a/be/src/vec/data_types/get_least_supertype.cpp +++ b/be/src/vec/data_types/get_least_supertype.cpp @@ -15,21 +15,18 @@ // specific language governing permissions and limitations // under the License. -#include +#include "vec/data_types/get_least_supertype.h" -//#include -//#include -#include -#include +#include -//#include -//#include -#include -#include -#include -//#include -#include -#include +#include "vec/common/typeid_cast.h" +#include "vec/data_types/data_type_date.h" +#include "vec/data_types/data_type_date_time.h" +#include "vec/data_types/data_type_nothing.h" +#include "vec/data_types/data_type_nullable.h" +#include "vec/data_types/data_type_string.h" +#include "vec/data_types/data_types_decimal.h" +#include "vec/data_types/data_types_number.h" namespace doris::vectorized { @@ -89,78 +86,6 @@ DataTypePtr getLeastSupertype(const DataTypes& types) { if (non_nothing_types.size() < types.size()) return getLeastSupertype(non_nothing_types); } - // /// For Arrays - // { - // bool have_array = false; - // bool all_arrays = true; - // - // DataTypes nested_types; - // nested_types.reserve(types.size()); - // - // for (const auto & type : types) - // { - // if (const DataTypeArray * type_array = typeid_cast(type.get())) - // { - // have_array = true; - // nested_types.emplace_back(type_array->getNestedType()); - // } - // else - // all_arrays = false; - // } - // - // if (have_array) - // { - // if (!all_arrays) - // throw Exception(getExceptionMessagePrefix(types) + " because some of them are Array and some of them are not", ErrorCodes::NO_COMMON_TYPE); - // - // return std::make_shared(getLeastSupertype(nested_types)); - // } - // } - - // /// For tuples - // { - // bool have_tuple = false; - // bool all_tuples = true; - // size_t tuple_size = 0; - // - // std::vector nested_types; - // - // for (const auto & type : types) - // { - // if (const DataTypeTuple * type_tuple = typeid_cast(type.get())) - // { - // if (!have_tuple) - // { - // tuple_size = type_tuple->getElements().size(); - // nested_types.resize(tuple_size); - // for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx) - // nested_types[elem_idx].reserve(types.size()); - // } - // else if (tuple_size != type_tuple->getElements().size()) - // throw Exception(getExceptionMessagePrefix(types) + " because Tuples have different sizes", ErrorCodes::NO_COMMON_TYPE); - // - // have_tuple = true; - // - // for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx) - // nested_types[elem_idx].emplace_back(type_tuple->getElements()[elem_idx]); - // } - // else - // all_tuples = false; - // } - // - // if (have_tuple) - // { - // if (!all_tuples) - // throw Exception(getExceptionMessagePrefix(types) + " because some of them are Tuple and some of them are not", ErrorCodes::NO_COMMON_TYPE); - // - // DataTypes common_tuple_types(tuple_size); - // for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx) - // common_tuple_types[elem_idx] = getLeastSupertype(nested_types[elem_idx]); - // - // return std::make_shared(common_tuple_types); - // } - // } - /// For Nullable { bool have_nullable = false; @@ -207,20 +132,22 @@ DataTypePtr getLeastSupertype(const DataTypes& types) { } } - // /// For Date and DateTime, the common type is DateTime. No other types are compatible. - // { - // UInt32 have_date = type_ids.count(TypeIndex::Date); - // UInt32 have_datetime = type_ids.count(TypeIndex::DateTime); - // - // if (have_date || have_datetime) - // { - // bool all_date_or_datetime = type_ids.size() == (have_date + have_datetime); - // if (!all_date_or_datetime) - // throw Exception(getExceptionMessagePrefix(types) + " because some of them are Date/DateTime and some of them are not", ErrorCodes::NO_COMMON_TYPE); - // - // return std::make_shared(); - // } - // } + /// For Date and DateTime, the common type is DateTime. No other types are compatible. + { + UInt32 have_date = type_ids.count(TypeIndex::Date); + UInt32 have_datetime = type_ids.count(TypeIndex::DateTime); + + if (have_date || have_datetime) { + bool all_date_or_datetime = type_ids.size() == (have_date + have_datetime); + if (!all_date_or_datetime) + throw Exception( + getExceptionMessagePrefix(types) + + " because some of them are Date/DateTime and some of them are not", + ErrorCodes::NO_COMMON_TYPE); + + return std::make_shared(); + } + } /// Decimals { diff --git a/be/src/vec/data_types/nested_utils.cpp b/be/src/vec/data_types/nested_utils.cpp index a358d7c5adabfd..62dd31e4345618 100644 --- a/be/src/vec/data_types/nested_utils.cpp +++ b/be/src/vec/data_types/nested_utils.cpp @@ -15,22 +15,15 @@ // specific language governing permissions and limitations // under the License. +#include "vec/data_types/nested_utils.h" + #include +#include "vec/columns/column_const.h" #include "vec/common/assert_cast.h" #include "vec/common/string_utils/string_utils.h" #include "vec/common/typeid_cast.h" -// #include -// #include -#include "vec/data_types/nested_utils.h" - -// #include -// #include -#include "vec/columns/column_const.h" - -// #include - namespace doris::vectorized { namespace ErrorCodes { @@ -80,122 +73,6 @@ std::string extractTableName(const std::string& nested_name) { return splitted.first; } -// Block flatten(const Block & block) -// { -// Block res; - -// for (const auto & elem : block) -// { -// if (const DataTypeArray * type_arr = typeid_cast(elem.type.get())) -// { -// if (const DataTypeTuple * type_tuple = typeid_cast(type_arr->getNestedType().get())) -// { -// const DataTypes & element_types = type_tuple->getElements(); -// const Strings & names = type_tuple->getElementNames(); -// size_t tuple_size = element_types.size(); - -// bool is_const = isColumnConst(*elem.column); -// const ColumnArray * column_array; -// if (is_const) -// column_array = typeid_cast(&assert_cast(*elem.column).getDataColumn()); -// else -// column_array = typeid_cast(elem.column.get()); - -// const ColumnPtr & column_offsets = column_array->getOffsetsPtr(); - -// const ColumnTuple & column_tuple = typeid_cast(column_array->getData()); -// const auto & element_columns = column_tuple.getColumns(); - -// for (size_t i = 0; i < tuple_size; ++i) -// { -// String nested_name = concatenateName(elem.name, names[i]); -// ColumnPtr column_array_of_element = ColumnArray::create(element_columns[i], column_offsets); - -// res.insert(ColumnWithTypeAndName( -// is_const -// ? ColumnConst::create(std::move(column_array_of_element), block.rows()) -// : std::move(column_array_of_element), -// std::make_shared(element_types[i]), -// nested_name)); -// } -// } -// else -// res.insert(elem); -// } -// else -// res.insert(elem); -// } - -// return res; -// } - -// NamesAndTypesList collect(const NamesAndTypesList & names_and_types) -// { -// NamesAndTypesList res; - -// std::map nested; -// for (const auto & name_type : names_and_types) -// { -// bool collected = false; -// if (const DataTypeArray * type_arr = typeid_cast(name_type.type.get())) -// { -// auto splitted = splitName(name_type.name); -// if (!splitted.second.empty()) -// { -// nested[splitted.first].emplace_back(splitted.second, type_arr->getNestedType()); -// collected = true; -// } -// } - -// if (!collected) -// res.push_back(name_type); -// } - -// for (const auto & name_elems : nested) -// res.emplace_back(name_elems.first, std::make_shared( -// std::make_shared(name_elems.second.getTypes(), name_elems.second.getNames()))); - -// return res; -// } - -// void validateArraySizes(const Block & block) -// { -// /// Nested prefix -> position of first column in block. -// std::map nested; - -// for (size_t i = 0, size = block.columns(); i < size; ++i) -// { -// const auto & elem = block.getByPosition(i); - -// if (isArray(elem.type)) -// { -// if (!typeid_cast(elem.column.get())) -// throw Exception("Column with Array type is not represented by ColumnArray column: " + elem.column->dumpStructure(), ErrorCodes::ILLEGAL_COLUMN); - -// auto splitted = splitName(elem.name); - -// /// Is it really a column of Nested data structure. -// if (!splitted.second.empty()) -// { -// auto [it, inserted] = nested.emplace(splitted.first, i); - -// /// It's not the first column of Nested data structure. -// if (!inserted) -// { -// const ColumnArray & first_array_column = assert_cast(*block.getByPosition(it->second).column); -// const ColumnArray & another_array_column = assert_cast(*elem.column); - -// if (!first_array_column.hasEqualOffsets(another_array_column)) -// throw Exception("Elements '" + block.getByPosition(it->second).name -// + "' and '" + elem.name -// + "' of Nested data structure '" + splitted.first -// + "' (Array columns) have different array sizes.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); -// } -// } -// } -// } -// } - } // namespace Nested } // namespace doris::vectorized diff --git a/be/src/vec/functions/functions_comparison.h b/be/src/vec/functions/functions_comparison.h index bded075fba5ecb..29a3082ee74c2a 100644 --- a/be/src/vec/functions/functions_comparison.h +++ b/be/src/vec/functions/functions_comparison.h @@ -17,41 +17,25 @@ #pragma once +#include +#include + #include "vec/columns/column_const.h" #include "vec/columns/column_decimal.h" +#include "vec/columns/column_nullable.h" #include "vec/columns/column_string.h" #include "vec/columns/columns_number.h" -#include "vec/columns/column_nullable.h" #include "vec/common/assert_cast.h" #include "vec/common/memcmp_small.h" -//#include -//#include -//#include - -#include "vec/data_types/data_types_number.h" -//#include -//#include -#include "vec/data_types/data_type_string.h" -//#include -//#include -//#include -//#include -#include "vec/data_types/get_least_supertype.h" - -//#include - #include "vec/core/accurate_comparison.h" #include "vec/core/decimal_comparison.h" +#include "vec/data_types/data_type_string.h" +#include "vec/data_types/data_types_number.h" +#include "vec/data_types/get_least_supertype.h" #include "vec/functions/function.h" #include "vec/functions/function_helpers.h" #include "vec/functions/functions_logical.h" -//#include -//#include - -#include -#include - namespace doris::vectorized { namespace ErrorCodes { @@ -545,6 +529,7 @@ class FunctionComparison : public IFunction { FunctionComparison() {} bool useDefaultImplementationForNulls() const override { return false; } + private: // const Context & context; // bool check_decimal_overflow = true; @@ -809,241 +794,19 @@ class FunctionComparison : public IFunction { } } - // bool executeDateOrDateTimeOrEnumOrUUIDWithConstString( - // Block & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped, - // const DataTypePtr & left_type, const DataTypePtr & right_type, bool left_is_num, size_t input_rows_count) - // { - // /// This is no longer very special case - comparing dates, datetimes, and enumerations with a string constant. - // const IColumn * column_string_untyped = !left_is_num ? col_left_untyped : col_right_untyped; - // const IColumn * column_number = left_is_num ? col_left_untyped : col_right_untyped; - // const IDataType * number_type = left_is_num ? left_type.get() : right_type.get(); - // - // WhichDataType which(number_type); - // - // const bool legal_types = which.isDateOrDateTime() || which.isEnum() || which.isUUID(); - // - // const auto column_string = checkAndGetColumnConst(column_string_untyped); - // if (!column_string || !legal_types) - // return false; - // - // StringRef string_value = column_string->getDataAt(0); - // - // if (which.isDate()) - // { - // DayNum date; - // ReadBufferFromMemory in(string_value.data, string_value.size); - // readDateText(date, in); - // if (!in.eof()) - // throw Exception("String is too long for Date: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); - // - // ColumnPtr parsed_const_date_holder = DataTypeDate().createColumnConst(input_rows_count, date); - // const ColumnConst * parsed_const_date = assert_cast(parsed_const_date_holder.get()); - // executeNumLeftType(block, result, - // left_is_num ? col_left_untyped : parsed_const_date, - // left_is_num ? parsed_const_date : col_right_untyped); - // } - // else if (which.isDateTime()) - // { - // time_t date_time; - // ReadBufferFromMemory in(string_value.data, string_value.size); - // readDateTimeText(date_time, in); - // if (!in.eof()) - // throw Exception("String is too long for DateTime: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); - // - // ColumnPtr parsed_const_date_time_holder = DataTypeDateTime().createColumnConst(input_rows_count, UInt64(date_time)); - // const ColumnConst * parsed_const_date_time = assert_cast(parsed_const_date_time_holder.get()); - // executeNumLeftType(block, result, - // left_is_num ? col_left_untyped : parsed_const_date_time, - // left_is_num ? parsed_const_date_time : col_right_untyped); - // } - // else if (which.isUUID()) - // { - // UUID uuid; - // ReadBufferFromMemory in(string_value.data, string_value.size); - // readText(uuid, in); - // if (!in.eof()) - // throw Exception("String is too long for UUID: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); - // - // ColumnPtr parsed_const_uuid_holder = DataTypeUUID().createColumnConst(input_rows_count, uuid); - // const ColumnConst * parsed_const_uuid = assert_cast(parsed_const_uuid_holder.get()); - // executeNumLeftType(block, result, - // left_is_num ? col_left_untyped : parsed_const_uuid, - // left_is_num ? parsed_const_uuid : col_right_untyped); - // } - // - // else if (which.isEnum8()) - // executeEnumWithConstString(block, result, column_number, column_string, - // number_type, left_is_num, input_rows_count); - // else if (which.isEnum16()) - // executeEnumWithConstString(block, result, column_number, column_string, - // number_type, left_is_num, input_rows_count); - // - // return true; - // } - - // /// Comparison between DataTypeEnum and string constant containing the name of an enum element - // template - // void executeEnumWithConstString( - // Block & block, const size_t result, const IColumn * column_number, const ColumnConst * column_string, - // const IDataType * type_untyped, const bool left_is_num, size_t input_rows_count) - // { - // const auto type = static_cast(type_untyped); - // - // const Field x = castToNearestFieldType(type->getValue(column_string->getValue())); - // const auto enum_col = type->createColumnConst(input_rows_count, x); - // - // executeNumLeftType(block, result, - // left_is_num ? column_number : enum_col.get(), - // left_is_num ? enum_col.get() : column_number); - // } - - // void executeTuple(Block & block, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1, - // size_t input_rows_count) - // { - // /** We will lexicographically compare the tuples. This is done as follows: - // * x == y : x1 == y1 && x2 == y2 ... - // * x != y : x1 != y1 || x2 != y2 ... - // * - // * x < y: x1 < y1 || (x1 == y1 && (x2 < y2 || (x2 == y2 ... && xn < yn)) - // * x > y: x1 > y1 || (x1 == y1 && (x2 > y2 || (x2 == y2 ... && xn > yn)) - // * x <= y: x1 < y1 || (x1 == y1 && (x2 < y2 || (x2 == y2 ... && xn <= yn)) - // * - // * Recursive form: - // * x <= y: x1 < y1 || (x1 == y1 && x_tail <= y_tail) - // * - // * x >= y: x1 > y1 || (x1 == y1 && (x2 > y2 || (x2 == y2 ... && xn >= yn)) - // */ - // - // const size_t tuple_size = typeid_cast(*c0.type).getElements().size(); - // - // if (0 == tuple_size) - // throw Exception("Comparison of zero-sized tuples is not implemented.", ErrorCodes::NOT_IMPLEMENTED); - // - // ColumnsWithTypeAndName x(tuple_size); - // ColumnsWithTypeAndName y(tuple_size); - // - // auto x_const = checkAndGetColumnConst(c0.column.get()); - // auto y_const = checkAndGetColumnConst(c1.column.get()); - // - // Columns x_columns; - // Columns y_columns; - // - // if (x_const) - // x_columns = convertConstTupleToConstantElements(*x_const); - // else - // x_columns = assert_cast(*c0.column).getColumnsCopy(); - // - // if (y_const) - // y_columns = convertConstTupleToConstantElements(*y_const); - // else - // y_columns = assert_cast(*c1.column).getColumnsCopy(); - // - // for (size_t i = 0; i < tuple_size; ++i) - // { - // x[i].type = static_cast(*c0.type).getElements()[i]; - // y[i].type = static_cast(*c1.type).getElements()[i]; - // - // x[i].column = x_columns[i]; - // y[i].column = y_columns[i]; - // } - // - // executeTupleImpl(block, result, x, y, tuple_size, input_rows_count); - // } - - // void executeTupleImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x, - // const ColumnsWithTypeAndName & y, size_t tuple_size, - // size_t input_rows_count); - - // template - // void executeTupleEqualityImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, - // size_t tuple_size, size_t input_rows_count) - // { - // ComparisonFunction func_compare(context); - // ConvolutionFunction func_convolution; - // - // Block tmp_block; - // for (size_t i = 0; i < tuple_size; ++i) - // { - // tmp_block.insert(x[i]); - // tmp_block.insert(y[i]); - // - // /// Comparison of the elements. - // tmp_block.insert({ nullptr, std::make_shared(), "" }); - // func_compare.execute(tmp_block, {i * 3, i * 3 + 1}, i * 3 + 2, input_rows_count); - // } - // - // /// Logical convolution. - // tmp_block.insert({ nullptr, std::make_shared(), "" }); - // - // ColumnNumbers convolution_args(tuple_size); - // for (size_t i = 0; i < tuple_size; ++i) - // convolution_args[i] = i * 3 + 2; - // - // func_convolution.execute(tmp_block, convolution_args, tuple_size * 3, input_rows_count); - // block.getByPosition(result).column = tmp_block.getByPosition(tuple_size * 3).column; - // } - - // template - // void executeTupleLessGreaterImpl(Block & block, size_t result, const ColumnsWithTypeAndName & x, - // const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) - // { - // HeadComparisonFunction func_compare_head(context); - // TailComparisonFunction func_compare_tail(context); - // FunctionAnd func_and; - // FunctionOr func_or; - // FunctionComparison func_equals(context); - // - // Block tmp_block; - // - // /// Pairwise comparison of the inequality of all elements; on the equality of all elements except the last. - // for (size_t i = 0; i < tuple_size; ++i) - // { - // tmp_block.insert(x[i]); - // tmp_block.insert(y[i]); - // - // tmp_block.insert({ nullptr, std::make_shared(), "" }); - // - // if (i + 1 != tuple_size) - // { - // func_compare_head.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2, input_rows_count); - // - // tmp_block.insert({ nullptr, std::make_shared(), "" }); - // func_equals.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 3, input_rows_count); - // - // } - // else - // func_compare_tail.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2, input_rows_count); - // } - // - // /// Combination. Complex code - make a drawing. It can be replaced by a recursive comparison of tuples. - // size_t i = tuple_size - 1; - // while (i > 0) - // { - // tmp_block.insert({ nullptr, std::make_shared(), "" }); - // func_and.execute(tmp_block, {tmp_block.columns() - 2, (i - 1) * 4 + 3}, tmp_block.columns() - 1, input_rows_count); - // tmp_block.insert({ nullptr, std::make_shared(), "" }); - // func_or.execute(tmp_block, {tmp_block.columns() - 2, (i - 1) * 4 + 2}, tmp_block.columns() - 1, input_rows_count); - // --i; - // } - // - // block.getByPosition(result).column = tmp_block.getByPosition(tmp_block.columns() - 1).column; - // } - - void executeGenericIdenticalTypes(Block & block, size_t result, const IColumn * c0, const IColumn * c1) - { + void executeGenericIdenticalTypes(Block& block, size_t result, const IColumn* c0, + const IColumn* c1) { bool c0_const = isColumnConst(*c0); bool c1_const = isColumnConst(*c1); - if (c0_const && c1_const) - { + if (c0_const && c1_const) { UInt8 res = 0; GenericComparisonImpl>::constant_constant(*c0, *c1, res); - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(c0->size(), toField(res)); - } - else - { + block.getByPosition(result).column = + DataTypeUInt8().createColumnConst(c0->size(), toField(res)); + } else { auto c_res = ColumnUInt8::create(); - ColumnUInt8::Container & vec_res = c_res->getData(); + ColumnUInt8::Container& vec_res = c_res->getData(); vec_res.resize(c0->size()); if (c0_const) @@ -1057,12 +820,12 @@ class FunctionComparison : public IFunction { } } - void executeGeneric(Block & block, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1) - { + void executeGeneric(Block& block, size_t result, const ColumnWithTypeAndName& c0, + const ColumnWithTypeAndName& c1) { DataTypePtr common_type = getLeastSupertype({c0.type, c1.type}); // TODO: Support full castColumn -// ColumnPtr c0_converted = castColumn(c0, common_type, context); -// ColumnPtr c1_converted = castColumn(c1, common_type, context); + // ColumnPtr c0_converted = castColumn(c0, common_type, context); + // ColumnPtr c1_converted = castColumn(c1, common_type, context); ColumnPtr c0_converted = castColumnNullable(c0, common_type); ColumnPtr c1_converted = castColumnNullable(c1, common_type); @@ -1071,9 +834,8 @@ class FunctionComparison : public IFunction { } private: - ColumnPtr castColumnNullable(const ColumnWithTypeAndName & arg, const DataTypePtr & type) { - if (arg.type->equals(*type)) - return arg.column; + ColumnPtr castColumnNullable(const ColumnWithTypeAndName& arg, const DataTypePtr& type) { + if (arg.type->equals(*type)) return arg.column; auto bool_column = ColumnUInt8::create(); bool_column->insertManyDefaults(arg.column->size()); @@ -1167,8 +929,8 @@ class FunctionComparison : public IFunction { } } - WhichDataType which_left {left_type}; - WhichDataType which_right {right_type}; + WhichDataType which_left{left_type}; + WhichDataType which_right{right_type}; const bool left_is_num = col_left_untyped->isNumeric(); const bool right_is_num = col_right_untyped->isNumeric(); @@ -1220,7 +982,8 @@ class FunctionComparison : public IFunction { // { // } else { - executeGeneric(block, result, col_with_type_and_name_left, col_with_type_and_name_right); + executeGeneric(block, result, col_with_type_and_name_left, + col_with_type_and_name_right); } } diff --git a/be/src/vec/functions/functions_logical.h b/be/src/vec/functions/functions_logical.h index af8d6be5a3b9cc..84b71e652fb5e5 100644 --- a/be/src/vec/functions/functions_logical.h +++ b/be/src/vec/functions/functions_logical.h @@ -17,21 +17,11 @@ #pragma once -#include "vec/core/defines.h" +#include + #include "vec/core/types.h" #include "vec/data_types/data_type.h" #include "vec/functions/function.h" -//#include -#include - -#if USE_EMBEDDED_COMPILER -#include - -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" -#include -#pragma GCC diagnostic pop -#endif /** Logical functions AND, OR, XOR and NOT support three-valued (or ternary) logic * https://en.wikibooks.org/wiki/Structured_Query_Language/NULLs_and_the_Three_Valued_Logic @@ -163,7 +153,7 @@ class FunctionAnyArityLogical : public IFunction { b.SetInsertPoint(next); auto* value = values[i](); auto* truth = nativeBoolCast(b, types[i], value); - if (!types[i]->equals(DataTypeUInt8 {})) + if (!types[i]->equals(DataTypeUInt8{})) value = b.CreateSelect(truth, b.getInt8(1), b.getInt8(0)); phi->addIncoming(value, b.GetInsertBlock()); if (i + 1 < types.size()) { diff --git a/be/src/vec/functions/minus.cpp b/be/src/vec/functions/minus.cpp index b7c9bdf23c53b2..b1d6186807aa71 100644 --- a/be/src/vec/functions/minus.cpp +++ b/be/src/vec/functions/minus.cpp @@ -1,41 +1,39 @@ -#include "vec/functions/simple_function_factory.h" -//#include "vec/functions/function_factory.h" -#include "vec/functions/function_binary_arithmetic.h" #include "vec/common/arithmetic_overflow.h" +#include "vec/functions/function_binary_arithmetic.h" +#include "vec/functions/simple_function_factory.h" -namespace doris::vectorized -{ +namespace doris::vectorized { template -struct MinusImpl -{ +struct MinusImpl { using ResultType = typename NumberTraits::ResultOfSubtraction::Type; static const constexpr bool allow_decimal = true; template - static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b) - { + static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b) { return static_cast(a) - b; } /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false otherwise. template - static inline bool apply(A a, B b, Result & c) - { + static inline bool apply(A a, B b, Result& c) { return common::subOverflow(static_cast(a), b, c); } #if USE_EMBEDDED_COMPILER static constexpr bool compilable = true; - static inline llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool) - { - return left->getType()->isIntegerTy() ? b.CreateSub(left, right) : b.CreateFSub(left, right); + static inline llvm::Value* compile(llvm::IRBuilder<>& b, llvm::Value* left, llvm::Value* right, + bool) { + return left->getType()->isIntegerTy() ? b.CreateSub(left, right) + : b.CreateFSub(left, right); } #endif }; -struct NameMinus { static constexpr auto name = "subtract"; }; +struct NameMinus { + static constexpr auto name = "subtract"; +}; using FunctionMinus = FunctionBinaryArithmetic; //void registerFunctionMinus(FunctionFactory & factory) @@ -45,4 +43,4 @@ using FunctionMinus = FunctionBinaryArithmetic; void registerFunctionMinus(SimpleFunctionFactory& factory) { factory.registerFunction(); } -} +} // namespace doris::vectorized diff --git a/be/src/vec/io/io_helper.h b/be/src/vec/io/io_helper.h index 0caabbdf87dc87..353f8ed82e40ac 100644 --- a/be/src/vec/io/io_helper.h +++ b/be/src/vec/io/io_helper.h @@ -24,7 +24,6 @@ #include "gen_cpp/data.pb.h" #include "vec/common/exception.h" #include "vec/common/string_ref.h" -#include "vec/common/types.h" #include "vec/common/uint128.h" #include "vec/core/types.h" #include "vec/io/var_int.h" @@ -49,6 +48,26 @@ inline Int128 decimalScaleMultiplier(UInt32 scale) { return common::exp10_i128(scale); } +inline std::string int128_to_string(__int128_t value) { + char buffer[128]; + char* d = std::end(buffer); + do { + --d; + *d = "0123456789"[value % 10]; + value /= 10; + } while (value != 0); + if (value < 0) { + --d; + *d = '-'; + } + int len = std::end(buffer) - d; + return std::string(d, len); +} + +inline std::string int128_to_string(UInt128 value) { + return value.toHexString(); +} + template void writeText(Decimal value, UInt32 scale, std::ostream& ostr) { if (value < Decimal(0)) { @@ -57,9 +76,11 @@ void writeText(Decimal value, UInt32 scale, std::ostream& ostr) { } T whole_part = value; - if (scale) whole_part = value / decimalScaleMultiplier(scale); + if (scale) { + whole_part = value / decimalScaleMultiplier(scale); + } if constexpr (std::is_same::value || std::is_same::value) { - // int128 + ostr << int128_to_string(whole_part); } else { ostr << whole_part; } diff --git a/be/src/vec/io/var_int.h b/be/src/vec/io/var_int.h index 609b3f03979138..ec1f16af6a7998 100644 --- a/be/src/vec/io/var_int.h +++ b/be/src/vec/io/var_int.h @@ -19,7 +19,7 @@ #include -#include "vec/common/types.h" +#include "vec/core/types.h" namespace doris::vectorized { namespace ErrorCodes { @@ -28,12 +28,10 @@ extern const int ATTEMPT_TO_READ_AFTER_EOF; /** Write UInt64 in variable length format (base128) NOTE Only up to 2^63 - 1 are supported. */ void writeVarUInt(UInt64 x, std::ostream& ostr); -// void writeVarUInt(UInt64 x, WriteBuffer & ostr); char* writeVarUInt(UInt64 x, char* ostr); /** Read UInt64, written in variable length format (base128) */ void readVarUInt(UInt64& x, std::istream& istr); -// void readVarUInt(UInt64 & x, ReadBuffer & istr); const char* readVarUInt(UInt64& x, const char* istr, size_t size); /** Get the length of UInt64 in VarUInt format */ @@ -71,8 +69,7 @@ inline void writeVarT(UInt64 x, std::ostream& ostr) { inline void writeVarT(Int64 x, std::ostream& ostr) { writeVarInt(x, ostr); } -// inline void writeVarT(UInt64 x, WriteBuffer & ostr) { writeVarUInt(x, ostr); } -// inline void writeVarT(Int64 x, WriteBuffer & ostr) { writeVarInt(x, ostr); } + inline char* writeVarT(UInt64 x, char*& ostr) { return writeVarUInt(x, ostr); } @@ -86,8 +83,7 @@ inline void readVarT(UInt64& x, std::istream& istr) { inline void readVarT(Int64& x, std::istream& istr) { readVarInt(x, istr); } -// inline void readVarT(UInt64 & x, ReadBuffer & istr) { readVarUInt(x, istr); } -// inline void readVarT(Int64 & x, ReadBuffer & istr) { readVarInt(x, istr); } + inline const char* readVarT(UInt64& x, const char* istr, size_t size) { return readVarUInt(x, istr, size); } @@ -95,75 +91,6 @@ inline const char* readVarT(Int64& x, const char* istr, size_t size) { return readVarInt(x, istr, size); } -/// For [U]Int32, [U]Int16, size_t. - -// inline void readVarUInt(UInt32 & x, ReadBuffer & istr) -// { -// UInt64 tmp; -// readVarUInt(tmp, istr); -// x = tmp; -// } - -// inline void readVarInt(Int32 & x, ReadBuffer & istr) -// { -// Int64 tmp; -// readVarInt(tmp, istr); -// x = tmp; -// } - -// inline void readVarUInt(UInt16 & x, ReadBuffer & istr) -// { -// UInt64 tmp; -// readVarUInt(tmp, istr); -// x = tmp; -// } - -// inline void readVarInt(Int16 & x, ReadBuffer & istr) -// { -// Int64 tmp; -// readVarInt(tmp, istr); -// x = tmp; -// } - -// template -// inline std::enable_if_t, void> -// readVarUInt(T & x, ReadBuffer & istr) -// { -// UInt64 tmp; -// readVarUInt(tmp, istr); -// x = tmp; -// } - -// [[noreturn]] inline void throwReadAfterEOF() { -// throw Exception("Attempt to read after eof", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF); -// } - -// template -// inline void readVarUIntImpl(UInt64 & x, ReadBuffer & istr) -// { -// x = 0; -// for (size_t i = 0; i < 9; ++i) -// { -// if constexpr (!fast) -// if (istr.eof()) -// throwReadAfterEOF(); - -// UInt64 byte = *istr.position(); -// ++istr.position(); -// x |= (byte & 0x7F) << (7 * i); - -// if (!(byte & 0x80)) -// return; -// } -// } - -// inline void readVarUInt(UInt64 & x, ReadBuffer & istr) -// { -// if (istr.buffer().end() - istr.position() >= 9) -// return readVarUIntImpl(x, istr); -// return readVarUIntImpl(x, istr); -// } - inline void readVarUInt(UInt64& x, std::istream& istr) { x = 0; for (size_t i = 0; i < 9; ++i) { @@ -174,41 +101,6 @@ inline void readVarUInt(UInt64& x, std::istream& istr) { } } -// inline const char* readVarUInt(UInt64& x, const char* istr, size_t size) { -// const char* end = istr + size; - -// x = 0; -// for (size_t i = 0; i < 9; ++i) { -// if (istr == end) throwReadAfterEOF(); - -// UInt64 byte = *istr; -// ++istr; -// x |= (byte & 0x7F) << (7 * i); - -// if (!(byte & 0x80)) return istr; -// } - -// return istr; -// } - -// inline void writeVarUInt(UInt64 x, WriteBuffer & ostr) -// { -// for (size_t i = 0; i < 9; ++i) -// { -// uint8_t byte = x & 0x7F; -// if (x > 0x7F) -// byte |= 0x80; - -// ostr.nextIfAtEnd(); -// *ostr.position() = byte; -// ++ostr.position(); - -// x >>= 7; -// if (!x) -// return; -// } -// } - inline void writeVarUInt(UInt64 x, std::ostream& ostr) { for (size_t i = 0; i < 9; ++i) { uint8_t byte = x & 0x7F; diff --git a/be/test/vec/core/block_test.cpp b/be/test/vec/core/block_test.cpp index 65dd3a8f9ab451..559ab47658aad3 100644 --- a/be/test/vec/core/block_test.cpp +++ b/be/test/vec/core/block_test.cpp @@ -19,6 +19,7 @@ #include +#include #include #include @@ -195,10 +196,7 @@ TEST(BlockTest, SerializeAndDeserializeBlock) { decimal_column.get()) ->getData(); for (int i = 0; i < 1024; ++i) { - __int128_t value = i; - for (int j = 0; j < 9; ++j) { - value *= 10; - } + __int128_t value = i * pow(10, 9) + i * pow(10, 8); data.push_back(value); } vectorized::ColumnWithTypeAndName type_and_name(decimal_column->getPtr(), decimal_data_type, @@ -259,10 +257,7 @@ TEST(BlockTest, DumpData) { decimal_column.get()) ->getData(); for (int i = 0; i < 1024; ++i) { - __int128_t value = i; - for (int j = 0; j < 9; ++j) { - value *= 10; - } + __int128_t value = i * pow(10, 9) + i * pow(10, 8); decimal_data.push_back(value); } vectorized::ColumnWithTypeAndName test_decimal(decimal_column->getPtr(), decimal_data_type, @@ -277,7 +272,31 @@ TEST(BlockTest, DumpData) { auto nint32_type = makeNullable(std::make_shared()); vectorized::ColumnWithTypeAndName test_nullable_int32(mutable_nullable_vector->getPtr(), nint32_type, "test_nullable_int32"); - vectorized::Block block({test_int, test_string, test_decimal, test_nullable_int32}); + + auto column_vector_date = vectorized::ColumnVector::create(); + auto& date_data = column_vector_date->getData(); + for (int i = 0; i < 1024; ++i) { + DateTimeValue value; + value.from_date_int64(20210501); + date_data.push_back(*reinterpret_cast(&value)); + } + vectorized::DataTypePtr date_type(std::make_shared()); + vectorized::ColumnWithTypeAndName test_date(column_vector_date->getPtr(), date_type, + "test_date"); + + auto column_vector_datetime = vectorized::ColumnVector::create(); + auto& datetime_data = column_vector_datetime->getData(); + for (int i = 0; i < 1024; ++i) { + DateTimeValue value; + value.from_date_int64(20210501080910); + datetime_data.push_back(*reinterpret_cast(&value)); + } + vectorized::DataTypePtr datetime_type(std::make_shared()); + vectorized::ColumnWithTypeAndName test_datetime(column_vector_datetime->getPtr(), datetime_type, + "test_datetime"); + + vectorized::Block block( + {test_int, test_string, test_decimal, test_nullable_int32, test_date, test_datetime}); EXPECT_GT(block.dumpData().size(), 1); } } // namespace doris