diff --git a/CHANGELOG.md b/CHANGELOG.md index 258936d6b52..305021728a9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -128,6 +128,8 @@ Yu](https://github.com/yuzhichang)) * Introduce CustomSeparated data format that supports custom escaping and delimiter rules. [#7118](https://github.com/ClickHouse/ClickHouse/pull/7118) ([tavplubix](https://github.com/tavplubix)) +* Support Redis as source of external dictionary. [#4361](https://github.com/ClickHouse/ClickHouse/pull/4361) [#6962](https://github.com/ClickHouse/ClickHouse/pull/6962) ([comunodi](https://github.com/comunodi), [Anton +Popov](https://github.com/CurtizJ)) ### Bug Fix * Fix wrong query result if it has `WHERE IN (SELECT ...)` section and `optimize_read_in_order` is diff --git a/README.md b/README.md index 5d9faa11fbe..a545c91886f 100644 --- a/README.md +++ b/README.md @@ -11,7 +11,3 @@ ClickHouse is an open-source column-oriented database management system that all * [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events. * [Contacts](https://clickhouse.yandex/#contacts) can help to get your questions answered if there are any. * You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person. - -## Upcoming Events - -* [ClickHouse Meetup in Moscow](https://yandex.ru/promo/clickhouse/moscow-december-2019) on December 11. diff --git a/dbms/src/Columns/ColumnDecimal.h b/dbms/src/Columns/ColumnDecimal.h index 73fd5a18f0b..5664417e8d5 100644 --- a/dbms/src/Columns/ColumnDecimal.h +++ b/dbms/src/Columns/ColumnDecimal.h @@ -66,6 +66,7 @@ private: friend class COWHelper; public: + using ValueType = T; using Container = DecimalPaddedPODArray; private: diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 9d56281ea1c..a359d5b2f1b 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -204,7 +204,7 @@ MutableColumnPtr ColumnVector::cloneResized(size_t size) const memcpy(new_col.data.data(), data.data(), count * sizeof(data[0])); if (size > count) - memset(static_cast(&new_col.data[count]), static_cast(value_type()), (size - count) * sizeof(value_type)); + memset(static_cast(&new_col.data[count]), static_cast(ValueType()), (size - count) * sizeof(ValueType)); } return res; diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 85ca1839853..a90f1bdb6e8 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -104,13 +104,13 @@ private: struct greater; public: - using value_type = T; - using Container = PaddedPODArray; + using ValueType = T; + using Container = PaddedPODArray; private: ColumnVector() {} ColumnVector(const size_t n) : data(n) {} - ColumnVector(const size_t n, const value_type x) : data(n, x) {} + ColumnVector(const size_t n, const ValueType x) : data(n, x) {} ColumnVector(const ColumnVector & src) : data(src.data.begin(), src.data.end()) {} /// Sugar constructor. diff --git a/dbms/src/Columns/ReverseIndex.h b/dbms/src/Columns/ReverseIndex.h index 1e80164ca05..f95f5e6fcb0 100644 --- a/dbms/src/Columns/ReverseIndex.h +++ b/dbms/src/Columns/ReverseIndex.h @@ -116,7 +116,7 @@ namespace return (*state.saved_hash_column)[index]; else { - using ValueType = typename ColumnType::value_type; + using ValueType = typename ColumnType::ValueType; ValueType value = unalignedLoad(state.index_column->getDataAt(index).data); return DefaultHash()(value); } @@ -367,7 +367,7 @@ private: { if constexpr (is_numeric_column) { - using ValueType = typename ColumnType::value_type; + using ValueType = typename ColumnType::ValueType; ValueType value = unalignedLoad(ref.data); return DefaultHash()(value); } diff --git a/dbms/src/Common/quoteString.cpp b/dbms/src/Common/quoteString.cpp index bcc6906ddfa..6fc928ff022 100644 --- a/dbms/src/Common/quoteString.cpp +++ b/dbms/src/Common/quoteString.cpp @@ -14,6 +14,15 @@ String quoteString(const StringRef & x) } +String doubleQuoteString(const StringRef & x) +{ + String res(x.size, '\0'); + WriteBufferFromString wb(res); + writeDoubleQuotedString(x, wb); + return res; +} + + String backQuote(const StringRef & x) { String res(x.size, '\0'); diff --git a/dbms/src/Common/quoteString.h b/dbms/src/Common/quoteString.h index f17f6c7015d..426034e4803 100644 --- a/dbms/src/Common/quoteString.h +++ b/dbms/src/Common/quoteString.h @@ -9,6 +9,9 @@ namespace DB /// Quote the string. String quoteString(const StringRef & x); +/// Double quote the string. +String doubleQuoteString(const StringRef & x); + /// Quote the identifier with backquotes. String backQuote(const StringRef & x); diff --git a/dbms/src/Core/DecimalComparison.h b/dbms/src/Core/DecimalComparison.h index cb332ad3779..b9f47952d10 100644 --- a/dbms/src/Core/DecimalComparison.h +++ b/dbms/src/Core/DecimalComparison.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -22,12 +23,12 @@ namespace ErrorCodes /// inline bool allowDecimalComparison(const DataTypePtr & left_type, const DataTypePtr & right_type) { - if (isDecimal(left_type)) + if (isColumnedAsDecimal(left_type)) { - if (isDecimal(right_type) || isNotDecimalButComparableToDecimal(right_type)) + if (isColumnedAsDecimal(right_type) || isNotDecimalButComparableToDecimal(right_type)) return true; } - else if (isNotDecimalButComparableToDecimal(left_type) && isDecimal(right_type)) + else if (isNotDecimalButComparableToDecimal(left_type) && isColumnedAsDecimal(right_type)) return true; return false; } @@ -82,7 +83,7 @@ public: static bool compare(A a, B b, UInt32 scale_a, UInt32 scale_b) { - static const UInt32 max_scale = maxDecimalPrecision(); + static const UInt32 max_scale = DecimalUtils::maxPrecision(); if (scale_a > max_scale || scale_b > max_scale) throw Exception("Bad scale of decimal field", ErrorCodes::DECIMAL_OVERFLOW); diff --git a/dbms/src/Core/DecimalFunctions.h b/dbms/src/Core/DecimalFunctions.h new file mode 100644 index 00000000000..f098f2427eb --- /dev/null +++ b/dbms/src/Core/DecimalFunctions.h @@ -0,0 +1,142 @@ +#pragma once +// Moved Decimal-related functions out from Core/Types.h to reduce compilation time. + +#include +#include + +#include + +class DateLUTImpl; + +namespace DB +{ + +namespace DecimalUtils +{ + +static constexpr size_t minPrecision() { return 1; } +template static constexpr size_t maxPrecision() { return 0; } +template <> constexpr size_t maxPrecision() { return 9; } +template <> constexpr size_t maxPrecision() { return 18; } +template <> constexpr size_t maxPrecision() { return 38; } + +template T scaleMultiplier(UInt32 scale); +template <> inline Int32 scaleMultiplier(UInt32 scale) { return common::exp10_i32(scale); } +template <> inline Int64 scaleMultiplier(UInt32 scale) { return common::exp10_i64(scale); } +template <> inline Int128 scaleMultiplier(UInt32 scale) { return common::exp10_i128(scale); } + +/** Components of DecimalX value: + * whole - represents whole part of decimal, can be negatve or positive. + * fractional - for fractional part of decimal, always positive. + */ +template +struct DecimalComponents +{ + T whole; + T fractional; +}; + +/** Make a decimal value from whole and fractional components with given scale multiplier. + * where scale_multiplier = scaleMultiplier(scale) + * this is to reduce number of calls to scaleMultiplier when scale is known. + * + * Sign of `whole` controls sign of result: negative whole => negative result, positive whole => positive result. + * Sign of `fractional` is expected to be positive, otherwise result is undefined. + * If `scale` is to big (scale > maxPrecision), result is undefined. + */ +template +DecimalType decimalFromComponentsWithMultiplier(const typename DecimalType::NativeType & whole, + const typename DecimalType::NativeType & fractional, + typename DecimalType::NativeType scale_multiplier) +{ + using T = typename DecimalType::NativeType; + const auto fractional_sign = whole < 0 ? -1 : 1; + + const T value = whole * scale_multiplier + fractional_sign * (fractional % scale_multiplier); + return DecimalType(value); +} + +/** Make a decimal value from whole and fractional components with given scale. + * + * @see `decimalFromComponentsWithMultiplier` for details. + */ +template +DecimalType decimalFromComponents(const typename DecimalType::NativeType & whole, const typename DecimalType::NativeType & fractional, UInt32 scale) +{ + using T = typename DecimalType::NativeType; + + return decimalFromComponentsWithMultiplier(whole, fractional, scaleMultiplier(scale)); +} + +/** Make a decimal value from whole and fractional components with given scale. + * @see `decimalFromComponentsWithMultiplier` for details. + */ +template +DecimalType decimalFromComponents(const DecimalComponents & components, UInt32 scale) +{ + return decimalFromComponents(components.whole, components.fractional, scale); +} + +/** Split decimal into whole and fractional parts with given scale_multiplier. + * This is an optimization to reduce number of calls to scaleMultiplier on known scale. + */ +template +DecimalComponents splitWithScaleMultiplier(const DecimalType & decimal, typename DecimalType::NativeType scale_multiplier) +{ + using T = typename DecimalType::NativeType; + const auto whole = decimal.value / scale_multiplier; + auto fractional = decimal.value % scale_multiplier; + if (fractional < T(0)) + fractional *= T(-1); + + return {whole, fractional}; +} + +/// Split decimal into components: whole and fractional part, @see `DecimalComponents` for details. +template +DecimalComponents split(const DecimalType & decimal, UInt32 scale) +{ + if (scale == 0) + { + return {decimal.value, 0}; + } + return splitWithScaleMultiplier(decimal, scaleMultiplier(scale)); +} + +/** Get whole part from decimal. + * + * Sign of result follows sign of `decimal` value. + * If scale is to big, result is undefined. + */ +template +typename DecimalType::NativeType getWholePart(const DecimalType & decimal, size_t scale) +{ + if (scale == 0) + return decimal.value; + + return decimal.value / scaleMultiplier(scale); +} + +/** Get fractional part from decimal + * + * Result is always positive. + * If scale is to big, result is undefined. + */ +template +typename DecimalType::NativeType getFractionalPart(const DecimalType & decimal, size_t scale) +{ + using T = typename DecimalType::NativeType; + + if (scale == 0) + return 0; + + T result = decimal.value; + if (result < T(0)) + result *= T(-1); + + return result % scaleMultiplier(scale); +} + +} + +} diff --git a/dbms/src/Core/Types.h b/dbms/src/Core/Types.h index 511446b442f..ea80ab7d427 100644 --- a/dbms/src/Core/Types.h +++ b/dbms/src/Core/Types.h @@ -31,6 +31,8 @@ enum class TypeIndex Float64, Date, DateTime, + DateTime32 = DateTime, + DateTime64, String, FixedString, Enum8, @@ -151,10 +153,15 @@ struct Decimal T value; }; + using Decimal32 = Decimal; using Decimal64 = Decimal; using Decimal128 = Decimal; +// TODO (nemkov): consider making a strong typedef +//using DateTime32 = time_t; +using DateTime64 = Decimal64; + template <> struct TypeName { static const char * get() { return "Decimal32"; } }; template <> struct TypeName { static const char * get() { return "Decimal64"; } }; template <> struct TypeName { static const char * get() { return "Decimal128"; } }; @@ -196,6 +203,7 @@ inline const char * getTypeName(TypeIndex idx) case TypeIndex::Float64: return TypeName::get(); case TypeIndex::Date: return "Date"; case TypeIndex::DateTime: return "DateTime"; + case TypeIndex::DateTime64: return "DateTime64"; case TypeIndex::String: return TypeName::get(); case TypeIndex::FixedString: return "FixedString"; case TypeIndex::Enum8: return "Enum8"; diff --git a/dbms/src/Core/callOnTypeIndex.h b/dbms/src/Core/callOnTypeIndex.h index ad2a98d8112..bc09e7fd1ac 100644 --- a/dbms/src/Core/callOnTypeIndex.h +++ b/dbms/src/Core/callOnTypeIndex.h @@ -71,6 +71,7 @@ bool callOnBasicType(TypeIndex number, F && f) { case TypeIndex::Date: return f(TypePair()); case TypeIndex::DateTime: return f(TypePair()); + case TypeIndex::DateTime64: return f(TypePair()); default: break; } @@ -135,6 +136,7 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f) { case TypeIndex::Date: return callOnBasicType(type_num2, std::forward(f)); case TypeIndex::DateTime: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::DateTime64: return callOnBasicType(type_num2, std::forward(f)); default: break; } @@ -145,10 +147,11 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f) class DataTypeDate; -class DataTypeDateTime; class DataTypeString; class DataTypeFixedString; class DataTypeUUID; +class DataTypeDateTime; +class DataTypeDateTime64; template class DataTypeEnum; template class DataTypeNumber; template class DataTypeDecimal; @@ -178,6 +181,7 @@ bool callOnIndexAndDataType(TypeIndex number, F && f) case TypeIndex::Date: return f(TypePair()); case TypeIndex::DateTime: return f(TypePair()); + case TypeIndex::DateTime64: return f(TypePair()); case TypeIndex::String: return f(TypePair()); case TypeIndex::FixedString: return f(TypePair()); diff --git a/dbms/src/Core/tests/gtest_DecimalFunctions.cpp b/dbms/src/Core/tests/gtest_DecimalFunctions.cpp new file mode 100644 index 00000000000..8bbd3b8d836 --- /dev/null +++ b/dbms/src/Core/tests/gtest_DecimalFunctions.cpp @@ -0,0 +1,171 @@ +#include + +#include + +namespace +{ +using namespace DB; + +struct DecimalUtilsSplitAndCombineTestParam +{ + const char * description; + + Decimal64 decimal_value; + UInt8 scale; + + DecimalUtils::DecimalComponents components; +}; + +std::ostream & operator << (std::ostream & ostr, const DecimalUtilsSplitAndCombineTestParam & param) +{ + return ostr << param.description; +} + +class DecimalUtilsSplitAndCombineTest : public ::testing::TestWithParam +{}; + +template +void testSplit(const DecimalUtilsSplitAndCombineTestParam & param) +{ + const DecimalType decimal_value = param.decimal_value; + const auto & actual_components = DecimalUtils::split(decimal_value, param.scale); + + EXPECT_EQ(param.components.whole, actual_components.whole); + EXPECT_EQ(param.components.fractional, actual_components.fractional); +} + +template +void testDecimalFromComponents(const DecimalUtilsSplitAndCombineTestParam & param) +{ + EXPECT_EQ(param.decimal_value, + DecimalUtils::decimalFromComponents(param.components.whole, param.components.fractional, param.scale)); +} + +template +void testGetWhole(const DecimalUtilsSplitAndCombineTestParam & param) +{ + EXPECT_EQ(param.components.whole, + DecimalUtils::getWholePart(DecimalType{param.decimal_value}, param.scale)); +} + +template +void testGetFractional(const DecimalUtilsSplitAndCombineTestParam & param) +{ + EXPECT_EQ(param.components.fractional, + DecimalUtils::getFractionalPart(DecimalType{param.decimal_value}, param.scale)); +} + +// unfortunatelly typed parametrized tests () are not supported in this version of gtest, so I have to emulate by hand. +TEST_P(DecimalUtilsSplitAndCombineTest, split_Decimal32) +{ + testSplit(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, split_Decimal64) +{ + testSplit(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, split_Decimal128) +{ + testSplit(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, combine_Decimal32) +{ + testDecimalFromComponents(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, combine_Decimal64) +{ + testDecimalFromComponents(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, combine_Decimal128) +{ + testDecimalFromComponents(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, getWholePart_Decimal32) +{ + testGetWhole(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, getWholePart_Decimal64) +{ + testGetWhole(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, getWholePart_Decimal128) +{ + testGetWhole(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, getFractionalPart_Decimal32) +{ + testGetFractional(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, getFractionalPart_Decimal64) +{ + testGetFractional(GetParam()); +} + +TEST_P(DecimalUtilsSplitAndCombineTest, getFractionalPart_Decimal128) +{ + testGetFractional(GetParam()); +} + +} + +// Intentionally small values that fit into 32-bit in order to cover Decimal32, Decimal64 and Decimal128 with single set of data. +INSTANTIATE_TEST_CASE_P(Basic, + DecimalUtilsSplitAndCombineTest, + ::testing::ValuesIn(std::initializer_list{ + { + "Positive value with non-zero scale, whole, and fractional parts.", + 1234567'89, + 2, + { + 1234567, + 89 + } + }, + { + "When scale is 0, fractional part is 0.", + 1234567'89, + 0, + { + 123456789, + 0 + } + }, + { + "When scale is not 0 and fractional part is 0.", + 1234567'00, + 2, + { + 1234567, + 0 + } + }, + { + "When scale is not 0 and whole part is 0.", + 123, + 3, + { + 0, + 123 + } + }, + { + "For negative Decimal value whole part is negative, fractional is non-negative.", + -1234567'89, + 2, + { + -1234567, + 89 + } + } + } +),); diff --git a/dbms/src/DataStreams/AddingDefaultsBlockInputStream.cpp b/dbms/src/DataStreams/AddingDefaultsBlockInputStream.cpp index 168dea36b36..f6b6b290428 100644 --- a/dbms/src/DataStreams/AddingDefaultsBlockInputStream.cpp +++ b/dbms/src/DataStreams/AddingDefaultsBlockInputStream.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/DataStreams/TTLBlockInputStream.cpp b/dbms/src/DataStreams/TTLBlockInputStream.cpp index 02191aec52d..339f81321e4 100644 --- a/dbms/src/DataStreams/TTLBlockInputStream.cpp +++ b/dbms/src/DataStreams/TTLBlockInputStream.cpp @@ -85,6 +85,8 @@ Block TTLBlockInputStream::readImpl() removeValuesWithExpiredColumnTTL(block); + updateMovesTTL(block); + return block; } @@ -145,7 +147,8 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) defaults_expression->execute(block_with_defaults); } - for (const auto & [name, ttl_entry] : storage.ttl_entries_by_name) + std::vector columns_to_remove; + for (const auto & [name, ttl_entry] : storage.column_ttl_entries_by_name) { const auto & old_ttl_info = old_ttl_infos.columns_ttl[name]; auto & new_ttl_info = new_ttl_infos.columns_ttl[name]; @@ -159,7 +162,10 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) continue; if (!block.has(ttl_entry.result_column)) + { + columns_to_remove.push_back(ttl_entry.result_column); ttl_entry.expression->execute(block); + } ColumnPtr default_column = nullptr; if (block_with_defaults.has(name)) @@ -192,9 +198,34 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) column_with_type.column = std::move(result_column); } - for (const auto & elem : storage.ttl_entries_by_name) - if (block.has(elem.second.result_column)) - block.erase(elem.second.result_column); + for (const String & column : columns_to_remove) + block.erase(column); +} + +void TTLBlockInputStream::updateMovesTTL(Block & block) +{ + std::vector columns_to_remove; + for (const auto & ttl_entry : storage.move_ttl_entries) + { + auto & new_ttl_info = new_ttl_infos.moves_ttl[ttl_entry.result_column]; + + if (!block.has(ttl_entry.result_column)) + { + columns_to_remove.push_back(ttl_entry.result_column); + ttl_entry.expression->execute(block); + } + + const IColumn * ttl_column = block.getByName(ttl_entry.result_column).column.get(); + + for (size_t i = 0; i < block.rows(); ++i) + { + UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); + new_ttl_info.update(cur_ttl); + } + } + + for (const String & column : columns_to_remove) + block.erase(column); } UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t ind) diff --git a/dbms/src/DataStreams/TTLBlockInputStream.h b/dbms/src/DataStreams/TTLBlockInputStream.h index 5ed6aa9e520..05893600fe6 100644 --- a/dbms/src/DataStreams/TTLBlockInputStream.h +++ b/dbms/src/DataStreams/TTLBlockInputStream.h @@ -58,6 +58,9 @@ private: /// Removes rows with expired table ttl and computes new ttl_infos for part void removeRowsWithExpiredTableTTL(Block & block); + /// Updates TTL for moves + void updateMovesTTL(Block & block); + UInt32 getTimestampByIndex(const IColumn * column, size_t ind); bool isTTLExpired(time_t ttl); }; diff --git a/dbms/src/DataTypes/DataTypeDate.h b/dbms/src/DataTypes/DataTypeDate.h index 7bd4c0d6b02..00afba424e4 100644 --- a/dbms/src/DataTypes/DataTypeDate.h +++ b/dbms/src/DataTypes/DataTypeDate.h @@ -9,8 +9,10 @@ namespace DB class DataTypeDate final : public DataTypeNumberBase { public: + static constexpr auto family_name = "Date"; + TypeIndex getTypeId() const override { return TypeIndex::Date; } - const char * getFamilyName() const override { return "Date"; } + const char * getFamilyName() const override { return family_name; } 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; diff --git a/dbms/src/DataTypes/DataTypeDateTime.cpp b/dbms/src/DataTypes/DataTypeDateTime.cpp index 867833b4b2d..24215f3e590 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.cpp +++ b/dbms/src/DataTypes/DataTypeDateTime.cpp @@ -1,54 +1,25 @@ -#include -#include -#include +#include -#include -#include -#include +#include +#include #include +#include +#include +#include +#include #include #include #include -#include -#include - -#include #include - +#include +#include +#include +#include #include - -namespace DB +namespace { - -DataTypeDateTime::DataTypeDateTime(const std::string & time_zone_name) - : has_explicit_time_zone(!time_zone_name.empty()), - time_zone(DateLUT::instance(time_zone_name)), - utc_time_zone(DateLUT::instance("UTC")) -{ -} - -std::string DataTypeDateTime::doGetName() const -{ - if (!has_explicit_time_zone) - return "DateTime"; - - WriteBufferFromOwnString out; - out << "DateTime(" << quote << time_zone.getTimeZone() << ")"; - return out.str(); -} - -void DataTypeDateTime::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -{ - writeDateTimeText(assert_cast(column).getData()[row_num], ostr, time_zone); -} - -void DataTypeDateTime::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -{ - serializeText(column, row_num, ostr, settings); -} - - +using namespace DB; static inline void readText(time_t & x, ReadBuffer & istr, const FormatSettings & settings, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone) { switch (settings.date_time_input_format) @@ -61,7 +32,45 @@ static inline void readText(time_t & x, ReadBuffer & istr, const FormatSettings return; } } +} +namespace DB +{ + +TimezoneMixin::TimezoneMixin(const String & time_zone_name) + : has_explicit_time_zone(!time_zone_name.empty()), + time_zone(DateLUT::instance(time_zone_name)), + utc_time_zone(DateLUT::instance("UTC")) +{} + +DataTypeDateTime::DataTypeDateTime(const String & time_zone_name) + : TimezoneMixin(time_zone_name) +{ +} + +DataTypeDateTime::DataTypeDateTime(const TimezoneMixin & time_zone_) + : TimezoneMixin(time_zone_) +{} + +String DataTypeDateTime::doGetName() const +{ + if (!has_explicit_time_zone) + return "DateTime"; + + WriteBufferFromOwnString out; + out << "DateTime(" << quote << time_zone.getTimeZone() << ")"; + return out.str(); +} + +void DataTypeDateTime::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const +{ + writeDateTimeText(assert_cast(column).getData()[row_num], ostr, time_zone); +} + +void DataTypeDateTime::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeText(column, row_num, ostr, settings); +} void DataTypeDateTime::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { @@ -71,8 +80,8 @@ void DataTypeDateTime::deserializeWholeText(IColumn & column, ReadBuffer & istr, void DataTypeDateTime::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { time_t x; - readText(x, istr, settings, time_zone, utc_time_zone); - assert_cast(column).getData().push_back(x); + ::readText(x, istr, settings, time_zone, utc_time_zone); + assert_cast(column).getData().push_back(x); } void DataTypeDateTime::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -87,14 +96,14 @@ void DataTypeDateTime::deserializeTextQuoted(IColumn & column, ReadBuffer & istr time_t x; if (checkChar('\'', istr)) /// Cases: '2017-08-31 18:36:48' or '1504193808' { - readText(x, istr, settings, time_zone, utc_time_zone); + ::readText(x, istr, settings, time_zone, utc_time_zone); assertChar('\'', istr); } else /// Just 1504193808 or 01504193808 { readIntText(x, istr); } - assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. + assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } void DataTypeDateTime::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -109,14 +118,14 @@ void DataTypeDateTime::deserializeTextJSON(IColumn & column, ReadBuffer & istr, time_t x; if (checkChar('"', istr)) { - readText(x, istr, settings, time_zone, utc_time_zone); + ::readText(x, istr, settings, time_zone, utc_time_zone); assertChar('"', istr); } else { readIntText(x, istr); } - assert_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } void DataTypeDateTime::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -138,19 +147,21 @@ void DataTypeDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c if (maybe_quote == '\'' || maybe_quote == '\"') ++istr.position(); - readText(x, istr, settings, time_zone, utc_time_zone); + ::readText(x, istr, settings, time_zone, utc_time_zone); if (maybe_quote == '\'' || maybe_quote == '\"') assertChar(maybe_quote, istr); - assert_cast(column).getData().push_back(x); + assert_cast(column).getData().push_back(x); } void DataTypeDateTime::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const { if (value_index) return; - value_index = static_cast(protobuf.writeDateTime(assert_cast(column).getData()[row_num])); + + // On some platforms `time_t` is `long` but not `unsigned int` (UInt32 that we store in column), hence static_cast. + value_index = static_cast(protobuf.writeDateTime(static_cast(assert_cast(column).getData()[row_num]))); } void DataTypeDateTime::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const @@ -160,7 +171,7 @@ void DataTypeDateTime::deserializeProtobuf(IColumn & column, ProtobufReader & pr if (!protobuf.readDateTime(t)) return; - auto & container = assert_cast(column).getData(); + auto & container = assert_cast(column).getData(); if (allow_add_row) { container.emplace_back(t); @@ -177,7 +188,6 @@ bool DataTypeDateTime::equals(const IDataType & rhs) const return typeid(rhs) == typeid(*this); } - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -205,5 +215,4 @@ void registerDataTypeDateTime(DataTypeFactory & factory) factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive); } - } diff --git a/dbms/src/DataTypes/DataTypeDateTime.h b/dbms/src/DataTypes/DataTypeDateTime.h index 6a951e0e288..47c7f361091 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.h +++ b/dbms/src/DataTypes/DataTypeDateTime.h @@ -1,13 +1,31 @@ #pragma once +#include #include - class DateLUTImpl; namespace DB { +/** Mixin-class that manages timezone info for timezone-aware DateTime implementations + * + * Must be used as a (second) base for class implementing IDateType-interface. + */ +class TimezoneMixin +{ +public: + explicit TimezoneMixin(const String & time_zone_name = ""); + TimezoneMixin(const TimezoneMixin &) = default; + + const DateLUTImpl & getTimeZone() const { return time_zone; } + +protected: + bool has_explicit_time_zone; + const DateLUTImpl & time_zone; + const DateLUTImpl & utc_time_zone; +}; + /** DateTime stores time as unix timestamp. * The value itself is independent of time zone. * @@ -15,7 +33,7 @@ namespace DB * 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. + * To cast 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, @@ -28,13 +46,16 @@ namespace DB * 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 +class DataTypeDateTime final : public DataTypeNumberBase, public TimezoneMixin { public: - DataTypeDateTime(const std::string & time_zone_name = ""); + explicit DataTypeDateTime(const String & time_zone_name = ""); + explicit DataTypeDateTime(const TimezoneMixin & time_zone); - const char * getFamilyName() const override { return "DateTime"; } - std::string doGetName() const override; + static constexpr auto family_name = "DateTime"; + + const char * getFamilyName() const override { return family_name; } + String doGetName() const override; TypeIndex getTypeId() const override { return TypeIndex::DateTime; } void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; @@ -54,13 +75,7 @@ public: bool canBeInsideNullable() const override { return true; } bool equals(const IDataType & rhs) const override; - - const DateLUTImpl & getTimeZone() const { return time_zone; } - -private: - bool has_explicit_time_zone; - const DateLUTImpl & time_zone; - const DateLUTImpl & utc_time_zone; }; } + diff --git a/dbms/src/DataTypes/DataTypeDateTime64.cpp b/dbms/src/DataTypes/DataTypeDateTime64.cpp new file mode 100644 index 00000000000..d790dd60ce4 --- /dev/null +++ b/dbms/src/DataTypes/DataTypeDateTime64.cpp @@ -0,0 +1,252 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name) + : DataTypeDecimalBase(DecimalUtils::maxPrecision(), scale_), + TimezoneMixin(time_zone_name) +{ +} + +DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info) + : DataTypeDecimalBase(DecimalUtils::maxPrecision() - scale_, scale_), + TimezoneMixin(time_zone_info) +{} + +std::string DataTypeDateTime64::doGetName() const +{ + if (!has_explicit_time_zone) + return std::string(getFamilyName()) + "(" + std::to_string(this->scale) + ")"; + + WriteBufferFromOwnString out; + out << "DateTime64(" << this->scale << ", " << quote << time_zone.getTimeZone() << ")"; + return out.str(); +} + +void DataTypeDateTime64::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & /*settings*/) const +{ + writeDateTimeText(assert_cast(column).getData()[row_num], scale, ostr, time_zone); +} + +void DataTypeDateTime64::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +{ + DateTime64 result = 0; + readDateTime64Text(result, this->getScale(), istr, time_zone); + assert_cast(column).getData().push_back(result); +} + +void DataTypeDateTime64::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeTextEscaped(column, istr, settings); +} + +void DataTypeDateTime64::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeText(column, row_num, ostr, settings); +} + +static inline void readText(DateTime64 & x, UInt32 scale, ReadBuffer & istr, const FormatSettings & settings, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone) +{ + switch (settings.date_time_input_format) + { + case FormatSettings::DateTimeInputFormat::Basic: + readDateTime64Text(x, scale, istr, time_zone); + return; + case FormatSettings::DateTimeInputFormat::BestEffort: + parseDateTime64BestEffort(x, scale, istr, time_zone, utc_time_zone); + return; + } +} + +void DataTypeDateTime64::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + DateTime64 x = 0; + readText(x, scale, istr, settings, time_zone, utc_time_zone); + assert_cast(column).getData().push_back(x); +} + +void DataTypeDateTime64::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + writeChar('\'', ostr); + serializeText(column, row_num, ostr, settings); + writeChar('\'', ostr); +} + +void DataTypeDateTime64::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + DateTime64 x = 0; + if (checkChar('\'', istr)) /// Cases: '2017-08-31 18:36:48' or '1504193808' + { + readText(x, scale, istr, settings, time_zone, utc_time_zone); + assertChar('\'', istr); + } + else /// Just 1504193808 or 01504193808 + { + readIntText(x, istr); + } + assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. +} + +void DataTypeDateTime64::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + writeChar('"', ostr); + serializeText(column, row_num, ostr, settings); + writeChar('"', ostr); +} + +void DataTypeDateTime64::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + DateTime64 x = 0; + if (checkChar('"', istr)) + { + readText(x, scale, istr, settings, time_zone, utc_time_zone); + assertChar('"', istr); + } + else + { + readIntText(x, istr); + } + assert_cast(column).getData().push_back(x); +} + +void DataTypeDateTime64::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + writeChar('"', ostr); + serializeText(column, row_num, ostr, settings); + writeChar('"', ostr); +} + +void DataTypeDateTime64::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + DateTime64 x = 0; + + if (istr.eof()) + throwReadAfterEOF(); + + char maybe_quote = *istr.position(); + + if (maybe_quote == '\'' || maybe_quote == '\"') + ++istr.position(); + + readText(x, scale, istr, settings, time_zone, utc_time_zone); + + if (maybe_quote == '\'' || maybe_quote == '\"') + assertChar(maybe_quote, istr); + + assert_cast(column).getData().push_back(x); +} + +void DataTypeDateTime64::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const +{ + if (value_index) + return; + value_index = static_cast(protobuf.writeDateTime64(assert_cast(column).getData()[row_num], scale)); +} + +void DataTypeDateTime64::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const +{ + row_added = false; + DateTime64 t = 0; + if (!protobuf.readDateTime64(t, scale)) + return; + + auto & container = assert_cast(column).getData(); + if (allow_add_row) + { + container.emplace_back(t); + row_added = true; + } + else + container.back() = t; +} + +bool DataTypeDateTime64::equals(const IDataType & rhs) const +{ + if (auto * ptype = typeid_cast(&rhs)) + return this->scale == ptype->getScale(); + return false; +} + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +enum class ArgumentKind +{ + Optional, + Mandatory +}; + +template +std::conditional_t, T> +getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name) +{ + using NearestResultType = NearestFieldType; + const auto fieldType = Field::TypeToEnum::value; + const ASTLiteral * argument = nullptr; + + auto exceptionMessage = [=](const String & message) + { + return std::string("Parameter #") + std::to_string(argument_index) + " '" + + argument_name + "' for " + context_data_type_name + + message + + ", expected: " + Field::Types::toString(fieldType) + " literal."; + }; + + if (!arguments || arguments->children.size() <= argument_index + || !(argument = arguments->children[argument_index]->as())) + { + if constexpr (Kind == ArgumentKind::Optional) + return {}; + else + throw Exception(exceptionMessage(" is missing"), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + if (argument->value.getType() != fieldType) + throw Exception(exceptionMessage(String(" has wrong type: ") + argument->value.getTypeName()), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return argument->value.get(); +} + +static DataTypePtr create64(const ASTPtr & arguments) +{ + if (!arguments || arguments->size() == 0) + return std::make_shared(DataTypeDateTime64::default_scale); + + const auto scale = getArgument(arguments, 0, "scale", "DateType64"); + const auto timezone = getArgument(arguments, !!scale, "timezone", "DateType64"); + + return std::make_shared(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{})); +} + +void registerDataTypeDateTime64(DataTypeFactory & factory) +{ + factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/DataTypes/DataTypeDateTime64.h b/dbms/src/DataTypes/DataTypeDateTime64.h new file mode 100644 index 00000000000..44efe4f41b6 --- /dev/null +++ b/dbms/src/DataTypes/DataTypeDateTime64.h @@ -0,0 +1,84 @@ +#pragma once + +#include +#include +#include + +class DateLUTImpl; + +namespace DB +{ + +/** DateTime64 is same as DateTime, but it stores values as Int64 and has configurable sub-second part. + * + * `scale` determines number of decimal places for sub-second part of the DateTime64. + */ +class DataTypeDateTime64 final : public DataTypeDecimalBase, public TimezoneMixin +{ +public: + static constexpr UInt8 default_scale = 3; + static constexpr auto family_name = "DateTime64"; + + explicit DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name = ""); + + // reuse timezone from other DateTime/DateTime64 + DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info); + + const char * getFamilyName() const override { return family_name; } + std::string doGetName() const override; + TypeIndex getTypeId() const override { return TypeIndex::DateTime64; } + + 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 deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) 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 serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) 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; + + bool equals(const IDataType & rhs) const override; +}; + + +/** Basic wrapper for Tansform-types for DateTime64. + * + * Allows reusing existing Transform (that takes DateTime-values as UInt32) with DateTime64-values, + * by discarding fractional part and producing SAME return type as original Transform. + * + * Such Transfotm-types are commonly used in Date/DateTime manipulation functions, + * and implement static execute fucntion with following signature: + * R execute(UInt32, T, const DateLUTImpl &) + * + * Wehere R and T could be arbitrary types. +*/ +template +class DateTime64BasicTransformWrapper : public Transform +{ +public: + using Transform::execute; + + explicit DateTime64BasicTransformWrapper(UInt32 scale_) + : scale_multiplier(DecimalUtils::scaleMultiplier(scale_)) + {} + + template + auto execute(DateTime64 t, T v, const DateLUTImpl & time_zone) const + { + const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); + return static_cast(this)->execute( + static_cast(components.whole), v, time_zone); + } + +private: + UInt32 scale_multiplier = 1; +}; + + +} + diff --git a/dbms/src/DataTypes/DataTypeDecimalBase.cpp b/dbms/src/DataTypes/DataTypeDecimalBase.cpp new file mode 100644 index 00000000000..7b9a391427c --- /dev/null +++ b/dbms/src/DataTypes/DataTypeDecimalBase.cpp @@ -0,0 +1,109 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; +} + + +bool decimalCheckComparisonOverflow(const Context & context) { return context.getSettingsRef().decimal_check_overflow; } +bool decimalCheckArithmeticOverflow(const Context & context) { return context.getSettingsRef().decimal_check_overflow; } + +template +Field DataTypeDecimalBase::getDefault() const +{ + return DecimalField(T(0), scale); +} + +template +MutableColumnPtr DataTypeDecimalBase::createColumn() const +{ + return ColumnType::create(0, scale); +} + +template +void DataTypeDecimalBase::serializeBinary(const Field & field, WriteBuffer & ostr) const +{ + FieldType x = get>(field); + writeBinary(x, ostr); +} + +template +void DataTypeDecimalBase::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 DataTypeDecimalBase::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 DataTypeDecimalBase::deserializeBinary(Field & field, ReadBuffer & istr) const +{ + typename FieldType::NativeType x; + readBinary(x, istr); + field = DecimalField(T(x), this->scale); +} + +template +void DataTypeDecimalBase::deserializeBinary(IColumn & column, ReadBuffer & istr) const +{ + typename FieldType::NativeType x; + readBinary(x, istr); + assert_cast(column).getData().push_back(FieldType(x)); +} + +template +void DataTypeDecimalBase::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 +T DataTypeDecimalBase::getScaleMultiplier(UInt32 scale_) +{ + return DecimalUtils::scaleMultiplier(scale_); +} + + +/// Explicit template instantiations. +template class DataTypeDecimalBase; +template class DataTypeDecimalBase; +template class DataTypeDecimalBase; + +} diff --git a/dbms/src/DataTypes/DataTypeDecimalBase.h b/dbms/src/DataTypes/DataTypeDecimalBase.h new file mode 100644 index 00000000000..897e94bb8e1 --- /dev/null +++ b/dbms/src/DataTypes/DataTypeDecimalBase.h @@ -0,0 +1,213 @@ +#pragma once +#include + +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int CANNOT_CONVERT_TYPE; + extern const int DECIMAL_OVERFLOW; +} + +class Context; +bool decimalCheckComparisonOverflow(const Context & context); +bool decimalCheckArithmeticOverflow(const Context & context); + +inline UInt32 leastDecimalPrecisionFor(TypeIndex int_type) +{ + switch (int_type) + { + case TypeIndex::Int8: [[fallthrough]]; + case TypeIndex::UInt8: + return 3; + case TypeIndex::Int16: [[fallthrough]]; + case TypeIndex::UInt16: + return 5; + case TypeIndex::Int32: [[fallthrough]]; + case TypeIndex::UInt32: + return 10; + case TypeIndex::Int64: + return 19; + case TypeIndex::UInt64: + return 20; + default: + break; + } + return 0; +} + +/// Base class for decimals, like Decimal(P, S), where P is precision, S is scale. +/// Maximum precisions for underlying types are: +/// Int32 9 +/// Int64 18 +/// Int128 38 +/// Operation between two decimals leads to Decimal(P, S), where +/// P is one of (9, 18, 38); equals to the maximum precision for the biggest underlying type of operands. +/// S is maximum scale of operands. The allowed valuas are [0, precision] +template +class DataTypeDecimalBase : public DataTypeWithSimpleSerialization +{ + static_assert(IsDecimalNumber); + +public: + using FieldType = T; + using ColumnType = ColumnDecimal; + + static constexpr bool is_parametric = true; + + static constexpr size_t maxPrecision() { return DecimalUtils::maxPrecision(); } + + DataTypeDecimalBase(UInt32 precision_, UInt32 scale_) + : precision(precision_), + scale(scale_) + { + if (unlikely(precision < 1 || precision > maxPrecision())) + throw Exception("Precision " + std::to_string(precision) + " is out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + if (unlikely(scale < 0 || static_cast(scale) > maxPrecision())) + throw Exception("Scale " + std::to_string(scale) + " is out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + + TypeIndex getTypeId() const override { return TypeId::value; } + + Field getDefault() const override; + MutableColumnPtr createColumn() const override; + + bool isParametric() const override { return true; } + bool haveSubtypes() const override { return false; } + bool shouldAlignRightInPrettyFormats() const override { return true; } + bool textCanContainOnlyValidUTF8() const override { return true; } + bool isComparable() const override { return true; } + bool isValueRepresentedByNumber() const override { return true; } + bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } + bool haveMaximumSizeOfValue() const override { return true; } + size_t getSizeOfValueInMemory() const override { return sizeof(T); } + + bool isSummable() const override { return true; } + bool canBeUsedInBooleanContext() const override { return true; } + bool canBeInsideNullable() const override { return true; } + + 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; + + /// Decimal specific + + UInt32 getPrecision() const { return precision; } + UInt32 getScale() const { return scale; } + T getScaleMultiplier() const { return getScaleMultiplier(scale); } + + T wholePart(T x) const + { + return DecimalUtils::getWholePart(x, scale); + } + + T fractionalPart(T x) const + { + return DecimalUtils::getFractionalPart(x, scale); + } + + T maxWholeValue() const { return getScaleMultiplier(maxPrecision() - scale) - T(1); } + + bool canStoreWhole(T x) const + { + T max = maxWholeValue(); + if (x > max || x < -max) + return false; + return true; + } + + /// @returns multiplier for U to become T with correct scale + template + T scaleFactorFor(const DataTypeDecimalBase & x, bool) const + { + if (getScale() < x.getScale()) + throw Exception("Decimal result's scale is less than argument's one", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + UInt32 scale_delta = getScale() - x.getScale(); /// scale_delta >= 0 + return getScaleMultiplier(scale_delta); + } + + template + T scaleFactorFor(const DataTypeNumber & , bool is_multiply_or_divisor) const + { + if (is_multiply_or_divisor) + return 1; + return getScaleMultiplier(); + } + + static T getScaleMultiplier(UInt32 scale); + +protected: + const UInt32 precision; + const UInt32 scale; +}; + + +template typename DecimalType> +typename std::enable_if_t<(sizeof(T) >= sizeof(U)), DecimalType> +decimalResultType(const DecimalType & tx, const DecimalType & ty, bool is_multiply, bool is_divide) +{ + UInt32 scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale()); + if (is_multiply) + scale = tx.getScale() + ty.getScale(); + else if (is_divide) + scale = tx.getScale(); + return DecimalType(DecimalUtils::maxPrecision(), scale); +} + +template typename DecimalType> +typename std::enable_if_t<(sizeof(T) < sizeof(U)), const DecimalType> +decimalResultType(const DecimalType & tx, const DecimalType & ty, bool is_multiply, bool is_divide) +{ + UInt32 scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale()); + if (is_multiply) + scale = tx.getScale() * ty.getScale(); + else if (is_divide) + scale = tx.getScale(); + return DecimalType(DecimalUtils::maxPrecision(), scale); +} + +template typename DecimalType> +const DecimalType decimalResultType(const DecimalType & tx, const DataTypeNumber &, bool, bool) +{ + return DecimalType(DecimalUtils::maxPrecision(), tx.getScale()); +} + +template typename DecimalType> +const DecimalType decimalResultType(const DataTypeNumber &, const DecimalType & ty, bool, bool) +{ + return DecimalType(DecimalUtils::maxPrecision(), ty.getScale()); +} + +template