Merge branch 'master' into decimal

This commit is contained in:
chertus 2019-12-12 15:35:30 +03:00
commit 16844012e4
122 changed files with 5848 additions and 957 deletions

View File

@ -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

View File

@ -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.

View File

@ -66,6 +66,7 @@ private:
friend class COWHelper<ColumnVectorHelper, Self>;
public:
using ValueType = T;
using Container = DecimalPaddedPODArray<T>;
private:

View File

@ -204,7 +204,7 @@ MutableColumnPtr ColumnVector<T>::cloneResized(size_t size) const
memcpy(new_col.data.data(), data.data(), count * sizeof(data[0]));
if (size > count)
memset(static_cast<void *>(&new_col.data[count]), static_cast<int>(value_type()), (size - count) * sizeof(value_type));
memset(static_cast<void *>(&new_col.data[count]), static_cast<int>(ValueType()), (size - count) * sizeof(ValueType));
}
return res;

View File

@ -104,13 +104,13 @@ private:
struct greater;
public:
using value_type = T;
using Container = PaddedPODArray<value_type>;
using ValueType = T;
using Container = PaddedPODArray<ValueType>;
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.

View File

@ -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<ValueType>(state.index_column->getDataAt(index).data);
return DefaultHash<ValueType>()(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<ValueType>(ref.data);
return DefaultHash<ValueType>()(value);
}

View File

@ -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');

View File

@ -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);

View File

@ -4,6 +4,7 @@
#include <Core/Block.h>
#include <Core/AccurateComparison.h>
#include <Core/callOnTypeIndex.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
@ -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<Decimal128>();
static const UInt32 max_scale = DecimalUtils::maxPrecision<Decimal128>();
if (scale_a > max_scale || scale_b > max_scale)
throw Exception("Bad scale of decimal field", ErrorCodes::DECIMAL_OVERFLOW);

View File

@ -0,0 +1,142 @@
#pragma once
// Moved Decimal-related functions out from Core/Types.h to reduce compilation time.
#include <Core/Types.h>
#include <Common/intExp.h>
#include <limits>
class DateLUTImpl;
namespace DB
{
namespace DecimalUtils
{
static constexpr size_t minPrecision() { return 1; }
template <typename T> static constexpr size_t maxPrecision() { return 0; }
template <> constexpr size_t maxPrecision<Decimal32>() { return 9; }
template <> constexpr size_t maxPrecision<Decimal64>() { return 18; }
template <> constexpr size_t maxPrecision<Decimal128>() { return 38; }
template <typename T> T scaleMultiplier(UInt32 scale);
template <> inline Int32 scaleMultiplier<Int32>(UInt32 scale) { return common::exp10_i32(scale); }
template <> inline Int64 scaleMultiplier<Int64>(UInt32 scale) { return common::exp10_i64(scale); }
template <> inline Int128 scaleMultiplier<Int128>(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 <typename T>
struct DecimalComponents
{
T whole;
T fractional;
};
/** Make a decimal value from whole and fractional components with given scale multiplier.
* where scale_multiplier = scaleMultiplier<T>(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<DecimalType::NativeType>), result is undefined.
*/
template <typename DecimalType>
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 <typename DecimalType>
DecimalType decimalFromComponents(const typename DecimalType::NativeType & whole, const typename DecimalType::NativeType & fractional, UInt32 scale)
{
using T = typename DecimalType::NativeType;
return decimalFromComponentsWithMultiplier<DecimalType>(whole, fractional, scaleMultiplier<T>(scale));
}
/** Make a decimal value from whole and fractional components with given scale.
* @see `decimalFromComponentsWithMultiplier` for details.
*/
template <typename DecimalType>
DecimalType decimalFromComponents(const DecimalComponents<typename DecimalType::NativeType> & components, UInt32 scale)
{
return decimalFromComponents<DecimalType>(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 <typename DecimalType>
DecimalComponents<typename DecimalType::NativeType> 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 <typename DecimalType>
DecimalComponents<typename DecimalType::NativeType> split(const DecimalType & decimal, UInt32 scale)
{
if (scale == 0)
{
return {decimal.value, 0};
}
return splitWithScaleMultiplier(decimal, scaleMultiplier<typename DecimalType::NativeType>(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>
typename DecimalType::NativeType getWholePart(const DecimalType & decimal, size_t scale)
{
if (scale == 0)
return decimal.value;
return decimal.value / scaleMultiplier<typename DecimalType::NativeType>(scale);
}
/** Get fractional part from decimal
*
* Result is always positive.
* If scale is to big, result is undefined.
*/
template <typename DecimalType>
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<T>(scale);
}
}
}

View File

@ -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<Int32>;
using Decimal64 = Decimal<Int64>;
using Decimal128 = Decimal<Int128>;
// TODO (nemkov): consider making a strong typedef
//using DateTime32 = time_t;
using DateTime64 = Decimal64;
template <> struct TypeName<Decimal32> { static const char * get() { return "Decimal32"; } };
template <> struct TypeName<Decimal64> { static const char * get() { return "Decimal64"; } };
template <> struct TypeName<Decimal128> { static const char * get() { return "Decimal128"; } };
@ -196,6 +203,7 @@ inline const char * getTypeName(TypeIndex idx)
case TypeIndex::Float64: return TypeName<Float64>::get();
case TypeIndex::Date: return "Date";
case TypeIndex::DateTime: return "DateTime";
case TypeIndex::DateTime64: return "DateTime64";
case TypeIndex::String: return TypeName<String>::get();
case TypeIndex::FixedString: return "FixedString";
case TypeIndex::Enum8: return "Enum8";

View File

@ -71,6 +71,7 @@ bool callOnBasicType(TypeIndex number, F && f)
{
case TypeIndex::Date: return f(TypePair<T, UInt16>());
case TypeIndex::DateTime: return f(TypePair<T, UInt32>());
case TypeIndex::DateTime64: return f(TypePair<T, DateTime64>());
default:
break;
}
@ -135,6 +136,7 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f)
{
case TypeIndex::Date: return callOnBasicType<UInt16, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::DateTime: return callOnBasicType<UInt32, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::DateTime64: return callOnBasicType<Decimal64, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(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 <typename T> class DataTypeEnum;
template <typename T> class DataTypeNumber;
template <typename T> class DataTypeDecimal;
@ -178,6 +181,7 @@ bool callOnIndexAndDataType(TypeIndex number, F && f)
case TypeIndex::Date: return f(TypePair<DataTypeDate, T>());
case TypeIndex::DateTime: return f(TypePair<DataTypeDateTime, T>());
case TypeIndex::DateTime64: return f(TypePair<DataTypeDateTime64, T>());
case TypeIndex::String: return f(TypePair<DataTypeString, T>());
case TypeIndex::FixedString: return f(TypePair<DataTypeFixedString, T>());

View File

@ -0,0 +1,171 @@
#include <gtest/gtest.h>
#include <Core/DecimalFunctions.h>
namespace
{
using namespace DB;
struct DecimalUtilsSplitAndCombineTestParam
{
const char * description;
Decimal64 decimal_value;
UInt8 scale;
DecimalUtils::DecimalComponents<typename Decimal64::NativeType> components;
};
std::ostream & operator << (std::ostream & ostr, const DecimalUtilsSplitAndCombineTestParam & param)
{
return ostr << param.description;
}
class DecimalUtilsSplitAndCombineTest : public ::testing::TestWithParam<DecimalUtilsSplitAndCombineTestParam>
{};
template <typename DecimalType>
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 <typename DecimalType>
void testDecimalFromComponents(const DecimalUtilsSplitAndCombineTestParam & param)
{
EXPECT_EQ(param.decimal_value,
DecimalUtils::decimalFromComponents<DecimalType>(param.components.whole, param.components.fractional, param.scale));
}
template <typename DecimalType>
void testGetWhole(const DecimalUtilsSplitAndCombineTestParam & param)
{
EXPECT_EQ(param.components.whole,
DecimalUtils::getWholePart(DecimalType{param.decimal_value}, param.scale));
}
template <typename DecimalType>
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<Decimal32>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, split_Decimal64)
{
testSplit<Decimal64>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, split_Decimal128)
{
testSplit<Decimal128>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, combine_Decimal32)
{
testDecimalFromComponents<Decimal32>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, combine_Decimal64)
{
testDecimalFromComponents<Decimal64>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, combine_Decimal128)
{
testDecimalFromComponents<Decimal64>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, getWholePart_Decimal32)
{
testGetWhole<Decimal32>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, getWholePart_Decimal64)
{
testGetWhole<Decimal64>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, getWholePart_Decimal128)
{
testGetWhole<Decimal128>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, getFractionalPart_Decimal32)
{
testGetFractional<Decimal32>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, getFractionalPart_Decimal64)
{
testGetFractional<Decimal64>(GetParam());
}
TEST_P(DecimalUtilsSplitAndCombineTest, getFractionalPart_Decimal128)
{
testGetFractional<Decimal128>(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<DecimalUtilsSplitAndCombineTestParam>{
{
"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
}
}
}
),);

View File

@ -14,6 +14,7 @@
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeString.h>

View File

@ -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<String> 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<String> 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)

View File

@ -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);
};

View File

@ -9,8 +9,10 @@ namespace DB
class DataTypeDate final : public DataTypeNumberBase<UInt16>
{
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;

View File

@ -1,54 +1,25 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/parseDateTimeBestEffort.h>
#include <DataTypes/DataTypeDateTime.h>
#include <common/DateLUT.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include <common/DateLUT.h>
#include <DataTypes/DataTypeFactory.h>
#include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeFactory.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <IO/parseDateTimeBestEffort.h>
#include <Parsers/ASTLiteral.h>
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<const ColumnUInt32 &>(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<const ColumnType &>(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<ColumnUInt32 &>(column).getData().push_back(x);
::readText(x, istr, settings, time_zone, utc_time_zone);
assert_cast<ColumnType &>(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<ColumnUInt32 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
assert_cast<ColumnType &>(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<ColumnUInt32 &>(column).getData().push_back(x);
assert_cast<ColumnType &>(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<ColumnUInt32 &>(column).getData().push_back(x);
assert_cast<ColumnType &>(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<bool>(protobuf.writeDateTime(assert_cast<const ColumnUInt32 &>(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<bool>(protobuf.writeDateTime(static_cast<time_t>(assert_cast<const ColumnType &>(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<ColumnUInt32 &>(column).getData();
auto & container = assert_cast<ColumnType &>(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);
}
}

View File

@ -1,13 +1,31 @@
#pragma once
#include <Core/Types.h>
#include <DataTypes/DataTypeNumberBase.h>
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<UInt32>
class DataTypeDateTime final : public DataTypeNumberBase<UInt32>, 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;
};
}

View File

@ -0,0 +1,252 @@
#include <DataTypes/DataTypeDateTime64.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include <common/DateLUT.h>
#include <DataTypes/DataTypeFactory.h>
#include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <IO/Operators.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <IO/parseDateTimeBestEffort.h>
#include <Parsers/ASTLiteral.h>
#include <optional>
#include <string>
namespace DB
{
DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name)
: DataTypeDecimalBase<DateTime64>(DecimalUtils::maxPrecision<DateTime64>(), scale_),
TimezoneMixin(time_zone_name)
{
}
DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info)
: DataTypeDecimalBase<DateTime64>(DecimalUtils::maxPrecision<DateTime64>() - 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<const ColumnType &>(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<ColumnType &>(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<ColumnType &>(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<ColumnType &>(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<ColumnType &>(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<ColumnType &>(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<bool>(protobuf.writeDateTime64(assert_cast<const ColumnType &>(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<ColumnType &>(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<const DataTypeDateTime64 *>(&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 <typename T, ArgumentKind Kind>
std::conditional_t<Kind == ArgumentKind::Optional, std::optional<T>, T>
getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name)
{
using NearestResultType = NearestFieldType<T>;
const auto fieldType = Field::TypeToEnum<NearestResultType>::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<ASTLiteral>()))
{
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<NearestResultType>();
}
static DataTypePtr create64(const ASTPtr & arguments)
{
if (!arguments || arguments->size() == 0)
return std::make_shared<DataTypeDateTime64>(DataTypeDateTime64::default_scale);
const auto scale = getArgument<UInt64, ArgumentKind::Optional>(arguments, 0, "scale", "DateType64");
const auto timezone = getArgument<String, ArgumentKind::Optional>(arguments, !!scale, "timezone", "DateType64");
return std::make_shared<DataTypeDateTime64>(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{}));
}
void registerDataTypeDateTime64(DataTypeFactory & factory)
{
factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,84 @@
#pragma once
#include <Core/Types.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDecimalBase.h>
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<DateTime64>, 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 <typename Transform>
class DateTime64BasicTransformWrapper : public Transform
{
public:
using Transform::execute;
explicit DateTime64BasicTransformWrapper(UInt32 scale_)
: scale_multiplier(DecimalUtils::scaleMultiplier<DateTime64::NativeType>(scale_))
{}
template <typename T>
auto execute(DateTime64 t, T v, const DateLUTImpl & time_zone) const
{
const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier);
return static_cast<const Transform *>(this)->execute(
static_cast<UInt32>(components.whole), v, time_zone);
}
private:
UInt32 scale_multiplier = 1;
};
}

View File

@ -0,0 +1,109 @@
#include <DataTypes/DataTypeDecimalBase.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include <Core/DecimalFunctions.h>
#include <DataTypes/DataTypeFactory.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/readDecimalText.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/IAST.h>
#include <type_traits>
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 <typename T>
Field DataTypeDecimalBase<T>::getDefault() const
{
return DecimalField(T(0), scale);
}
template <typename T>
MutableColumnPtr DataTypeDecimalBase<T>::createColumn() const
{
return ColumnType::create(0, scale);
}
template <typename T>
void DataTypeDecimalBase<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
FieldType x = get<DecimalField<T>>(field);
writeBinary(x, ostr);
}
template <typename T>
void DataTypeDecimalBase<T>::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const FieldType & x = assert_cast<const ColumnType &>(column).getElement(row_num);
writeBinary(x, ostr);
}
template <typename T>
void DataTypeDecimalBase<T>::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
{
const typename ColumnType::Container & x = typeid_cast<const ColumnType &>(column).getData();
size_t size = x.size();
if (limit == 0 || offset + limit > size)
limit = size - offset;
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(FieldType) * limit);
}
template <typename T>
void DataTypeDecimalBase<T>::deserializeBinary(Field & field, ReadBuffer & istr) const
{
typename FieldType::NativeType x;
readBinary(x, istr);
field = DecimalField(T(x), this->scale);
}
template <typename T>
void DataTypeDecimalBase<T>::deserializeBinary(IColumn & column, ReadBuffer & istr) const
{
typename FieldType::NativeType x;
readBinary(x, istr);
assert_cast<ColumnType &>(column).getData().push_back(FieldType(x));
}
template <typename T>
void DataTypeDecimalBase<T>::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double) const
{
typename ColumnType::Container & x = typeid_cast<ColumnType &>(column).getData();
size_t initial_size = x.size();
x.resize(initial_size + limit);
size_t size = istr.readBig(reinterpret_cast<char*>(&x[initial_size]), sizeof(FieldType) * limit);
x.resize(initial_size + size / sizeof(FieldType));
}
template <typename T>
T DataTypeDecimalBase<T>::getScaleMultiplier(UInt32 scale_)
{
return DecimalUtils::scaleMultiplier<typename T::NativeType>(scale_);
}
/// Explicit template instantiations.
template class DataTypeDecimalBase<Decimal32>;
template class DataTypeDecimalBase<Decimal64>;
template class DataTypeDecimalBase<Decimal128>;
}

View File

@ -0,0 +1,213 @@
#pragma once
#include <cmath>
#include <common/likely.h>
#include <Columns/ColumnDecimal.h>
#include <Core/DecimalFunctions.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeWithSimpleSerialization.h>
#include <type_traits>
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 <typename T>
class DataTypeDecimalBase : public DataTypeWithSimpleSerialization
{
static_assert(IsDecimalNumber<T>);
public:
using FieldType = T;
using ColumnType = ColumnDecimal<T>;
static constexpr bool is_parametric = true;
static constexpr size_t maxPrecision() { return DecimalUtils::maxPrecision<T>(); }
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<UInt32>(scale) > maxPrecision()))
throw Exception("Scale " + std::to_string(scale) + " is out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
TypeIndex getTypeId() const override { return TypeId<T>::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 <typename U>
T scaleFactorFor(const DataTypeDecimalBase<U> & 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 <typename U>
T scaleFactorFor(const DataTypeNumber<U> & , 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 T, typename U, template <typename> typename DecimalType>
typename std::enable_if_t<(sizeof(T) >= sizeof(U)), DecimalType<T>>
decimalResultType(const DecimalType<T> & tx, const DecimalType<U> & 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<T>(DecimalUtils::maxPrecision<T>(), scale);
}
template <typename T, typename U, template <typename> typename DecimalType>
typename std::enable_if_t<(sizeof(T) < sizeof(U)), const DecimalType<U>>
decimalResultType(const DecimalType<T> & tx, const DecimalType<U> & 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<U>(DecimalUtils::maxPrecision<U>(), scale);
}
template <typename T, typename U, template <typename> typename DecimalType>
const DecimalType<T> decimalResultType(const DecimalType<T> & tx, const DataTypeNumber<U> &, bool, bool)
{
return DecimalType<T>(DecimalUtils::maxPrecision<T>(), tx.getScale());
}
template <typename T, typename U, template <typename> typename DecimalType>
const DecimalType<U> decimalResultType(const DataTypeNumber<T> &, const DecimalType<U> & ty, bool, bool)
{
return DecimalType<U>(DecimalUtils::maxPrecision<U>(), ty.getScale());
}
template <template <typename> typename DecimalType>
DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value)
{
if (precision_value < DecimalUtils::minPrecision() || precision_value > DecimalUtils::maxPrecision<Decimal128>())
throw Exception("Wrong precision", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (static_cast<UInt64>(scale_value) > precision_value)
throw Exception("Negative scales and scales larger than precision are not supported", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (precision_value <= DecimalUtils::maxPrecision<Decimal32>())
return std::make_shared<DecimalType<Decimal32>>(precision_value, scale_value);
else if (precision_value <= DecimalUtils::maxPrecision<Decimal64>())
return std::make_shared<DecimalType<Decimal64>>(precision_value, scale_value);
return std::make_shared<DecimalType<Decimal128>>(precision_value, scale_value);
}
}

View File

@ -127,7 +127,7 @@ void DataTypeEnum<Type>::serializeBinary(const IColumn & column, size_t row_num,
template <typename Type>
void DataTypeEnum<Type>::deserializeBinary(IColumn & column, ReadBuffer & istr) const
{
typename ColumnType::value_type x;
typename ColumnType::ValueType x;
readBinary(x, istr);
assert_cast<ColumnType &>(column).getData().push_back(x);
}

View File

@ -161,6 +161,7 @@ void registerDataTypeNumbers(DataTypeFactory & factory);
void registerDataTypeDecimal(DataTypeFactory & factory);
void registerDataTypeDate(DataTypeFactory & factory);
void registerDataTypeDateTime(DataTypeFactory & factory);
void registerDataTypeDateTime64(DataTypeFactory & factory);
void registerDataTypeString(DataTypeFactory & factory);
void registerDataTypeFixedString(DataTypeFactory & factory);
void registerDataTypeEnum(DataTypeFactory & factory);
@ -183,6 +184,7 @@ DataTypeFactory::DataTypeFactory()
registerDataTypeDecimal(*this);
registerDataTypeDate(*this);
registerDataTypeDateTime(*this);
registerDataTypeDateTime64(*this);
registerDataTypeString(*this);
registerDataTypeFixedString(*this);
registerDataTypeEnum(*this);

View File

@ -153,15 +153,15 @@ Field DataTypeNumberBase<T>::getDefault() const
template <typename T>
void DataTypeNumberBase<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
/// ColumnVector<T>::value_type is a narrower type. For example, UInt8, when the Field type is UInt64
typename ColumnVector<T>::value_type x = get<NearestFieldType<FieldType>>(field);
/// ColumnVector<T>::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64
typename ColumnVector<T>::ValueType x = get<NearestFieldType<FieldType>>(field);
writeBinary(x, ostr);
}
template <typename T>
void DataTypeNumberBase<T>::deserializeBinary(Field & field, ReadBuffer & istr) const
{
typename ColumnVector<T>::value_type x;
typename ColumnVector<T>::ValueType x;
readBinary(x, istr);
field = NearestFieldType<FieldType>(x);
}
@ -175,7 +175,7 @@ void DataTypeNumberBase<T>::serializeBinary(const IColumn & column, size_t row_n
template <typename T>
void DataTypeNumberBase<T>::deserializeBinary(IColumn & column, ReadBuffer & istr) const
{
typename ColumnVector<T>::value_type x;
typename ColumnVector<T>::ValueType x;
readBinary(x, istr);
assert_cast<ColumnVector<T> &>(column).getData().push_back(x);
}
@ -191,7 +191,7 @@ void DataTypeNumberBase<T>::serializeBinaryBulk(const IColumn & column, WriteBuf
limit = size - offset;
if (limit)
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(typename ColumnVector<T>::value_type) * limit);
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(typename ColumnVector<T>::ValueType) * limit);
}
template <typename T>
@ -200,8 +200,8 @@ void DataTypeNumberBase<T>::deserializeBinaryBulk(IColumn & column, ReadBuffer &
typename ColumnVector<T>::Container & x = typeid_cast<ColumnVector<T> &>(column).getData();
size_t initial_size = x.size();
x.resize(initial_size + limit);
size_t size = istr.readBig(reinterpret_cast<char*>(&x[initial_size]), sizeof(typename ColumnVector<T>::value_type) * limit);
x.resize(initial_size + size / sizeof(typename ColumnVector<T>::value_type));
size_t size = istr.readBig(reinterpret_cast<char*>(&x[initial_size]), sizeof(typename ColumnVector<T>::ValueType) * limit);
x.resize(initial_size + size / sizeof(typename ColumnVector<T>::ValueType));
}

View File

@ -7,6 +7,9 @@
namespace DB
{
template <typename T>
class ColumnVector;
/** Implements part of the IDataType interface, common to all numbers and for Date and DateTime.
*/
template <typename T>
@ -18,6 +21,8 @@ public:
static constexpr bool is_parametric = false;
using FieldType = T;
using ColumnType = ColumnVector<T>;
const char * getFamilyName() const override { return TypeName<T>::get(); }
TypeIndex getTypeId() const override { return TypeId<T>::value; }

View File

@ -1,17 +1,19 @@
#include <type_traits>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
#include <Core/DecimalFunctions.h>
#include <DataTypes/DataTypeFactory.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/readDecimalText.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/IAST.h>
#include <type_traits>
namespace DB
{
@ -23,34 +25,38 @@ namespace ErrorCodes
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 <typename T>
std::string DataTypeDecimal<T>::doGetName() const
{
std::stringstream ss;
ss << "Decimal(" << precision << ", " << scale << ")";
ss << "Decimal(" << this->precision << ", " << this->scale << ")";
return ss.str();
}
template <typename T>
bool DataTypeDecimal<T>::equals(const IDataType & rhs) const
{
if (auto * ptype = typeid_cast<const DataTypeDecimal<T> *>(&rhs))
return scale == ptype->getScale();
return this->scale == ptype->getScale();
return false;
}
template <typename T>
DataTypePtr DataTypeDecimal<T>::promoteNumericType() const
{
using PromotedType = DataTypeDecimal<Decimal128>;
return std::make_shared<PromotedType>(PromotedType::maxPrecision(), this->scale);
}
template <typename T>
void DataTypeDecimal<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
T value = assert_cast<const ColumnType &>(column).getData()[row_num];
writeText(value, scale, ostr);
writeText(value, this->scale, ostr);
}
template <typename T>
@ -58,6 +64,7 @@ bool DataTypeDecimal<T>::tryReadText(T & x, ReadBuffer & istr, UInt32 precision,
{
UInt32 unread_scale = scale;
bool done = tryReadDecimalText(istr, x, precision, unread_scale);
x *= T::getScaleMultiplier(unread_scale);
return done;
}
@ -94,74 +101,19 @@ T DataTypeDecimal<T>::parseFromString(const String & str) const
{
ReadBufferFromMemory buf(str.data(), str.size());
T x;
UInt32 unread_scale = scale;
readDecimalText(buf, x, precision, unread_scale, true);
UInt32 unread_scale = this->scale;
readDecimalText(buf, x, this->precision, unread_scale, true);
x *= T::getScaleMultiplier(unread_scale);
return x;
}
template <typename T>
void DataTypeDecimal<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
FieldType x = get<DecimalField<T>>(field);
writeBinary(x, ostr);
}
template <typename T>
void DataTypeDecimal<T>::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const FieldType & x = assert_cast<const ColumnType &>(column).getElement(row_num);
writeBinary(x, ostr);
}
template <typename T>
void DataTypeDecimal<T>::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
{
const typename ColumnType::Container & x = typeid_cast<const ColumnType &>(column).getData();
size_t size = x.size();
if (limit == 0 || offset + limit > size)
limit = size - offset;
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(FieldType) * limit);
}
template <typename T>
void DataTypeDecimal<T>::deserializeBinary(Field & field, ReadBuffer & istr) const
{
typename FieldType::NativeType x;
readBinary(x, istr);
field = DecimalField(T(x), scale);
}
template <typename T>
void DataTypeDecimal<T>::deserializeBinary(IColumn & column, ReadBuffer & istr) const
{
typename FieldType::NativeType x;
readBinary(x, istr);
assert_cast<ColumnType &>(column).getData().push_back(FieldType(x));
}
template <typename T>
void DataTypeDecimal<T>::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double) const
{
typename ColumnType::Container & x = typeid_cast<ColumnType &>(column).getData();
size_t initial_size = x.size();
x.resize(initial_size + limit);
size_t size = istr.readBig(reinterpret_cast<char*>(&x[initial_size]), sizeof(FieldType) * limit);
x.resize(initial_size + size / sizeof(FieldType));
}
template <typename T>
void DataTypeDecimal<T>::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
if (value_index)
return;
value_index = static_cast<bool>(protobuf.writeDecimal(assert_cast<const ColumnType &>(column).getData()[row_num], scale));
value_index = static_cast<bool>(protobuf.writeDecimal(assert_cast<const ColumnType &>(column).getData()[row_num], this->scale));
}
@ -170,7 +122,7 @@ void DataTypeDecimal<T>::deserializeProtobuf(IColumn & column, ProtobufReader &
{
row_added = false;
T decimal;
if (!protobuf.readDecimal(decimal, precision, scale))
if (!protobuf.readDecimal(decimal, this->precision, this->scale))
return;
auto & container = assert_cast<ColumnType &>(column).getData();
@ -184,45 +136,6 @@ void DataTypeDecimal<T>::deserializeProtobuf(IColumn & column, ProtobufReader &
}
template <typename T>
Field DataTypeDecimal<T>::getDefault() const
{
return DecimalField(T(0), scale);
}
template <typename T>
DataTypePtr DataTypeDecimal<T>::promoteNumericType() const
{
using PromotedType = DataTypeDecimal<Decimal128>;
return std::make_shared<PromotedType>(PromotedType::maxPrecision(), scale);
}
template <typename T>
MutableColumnPtr DataTypeDecimal<T>::createColumn() const
{
return ColumnType::create(0, scale);
}
//
DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value)
{
if (precision_value < minDecimalPrecision() || precision_value > maxDecimalPrecision<Decimal128>())
throw Exception("Wrong precision", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (static_cast<UInt64>(scale_value) > precision_value)
throw Exception("Negative scales and scales larger than precision are not supported", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
if (precision_value <= maxDecimalPrecision<Decimal32>())
return std::make_shared<DataTypeDecimal<Decimal32>>(precision_value, scale_value);
else if (precision_value <= maxDecimalPrecision<Decimal64>())
return std::make_shared<DataTypeDecimal<Decimal64>>(precision_value, scale_value);
return std::make_shared<DataTypeDecimal<Decimal128>>(precision_value, scale_value);
}
static DataTypePtr create(const ASTPtr & arguments)
{
if (!arguments || arguments->children.size() != 2)
@ -234,12 +147,12 @@ static DataTypePtr create(const ASTPtr & arguments)
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);
throw Exception("Decimal data type family must have two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
UInt64 precision_value = precision->value.get<UInt64>();
UInt64 scale_value = scale->value.get<UInt64>();
return createDecimal(precision_value, scale_value);
return createDecimal<DataTypeDecimal>(precision_value, scale_value);
}
template <typename T>
@ -254,10 +167,10 @@ static DataTypePtr createExact(const ASTPtr & arguments)
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<T>();
UInt64 precision = DecimalUtils::maxPrecision<T>();
UInt64 scale = scale_arg->value.get<UInt64>();
return createDecimal(precision, scale);
return createDecimal<DataTypeDecimal>(precision, scale);
}
void registerDataTypeDecimal(DataTypeFactory & factory)
@ -270,7 +183,6 @@ void registerDataTypeDecimal(DataTypeFactory & factory)
factory.registerAlias("DEC", "Decimal", DataTypeFactory::CaseInsensitive);
}
/// Explicit template instantiations.
template class DataTypeDecimal<Decimal32>;
template class DataTypeDecimal<Decimal64>;

View File

@ -1,12 +1,9 @@
#pragma once
#include <cmath>
#include <common/arithmeticOverflow.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnDecimal.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeWithSimpleSerialization.h>
#include <DataTypes/DataTypeDecimalBase.h>
namespace DB
@ -19,43 +16,6 @@ namespace ErrorCodes
extern const int DECIMAL_OVERFLOW;
}
class Context;
bool decimalCheckComparisonOverflow(const Context & context);
bool decimalCheckArithmeticOverflow(const Context & context);
static constexpr size_t minDecimalPrecision() { return 1; }
template <typename T> static constexpr size_t maxDecimalPrecision() { return 0; }
template <> constexpr size_t maxDecimalPrecision<Decimal32>() { return 9; }
template <> constexpr size_t maxDecimalPrecision<Decimal64>() { return 18; }
template <> constexpr size_t maxDecimalPrecision<Decimal128>() { return 38; }
DataTypePtr createDecimal(UInt64 precision, UInt64 scale);
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;
}
/// Implements Decimal(P, S), where P is precision, S is scale.
/// Maximum precisions for underlying types are:
/// Int32 9
@ -65,166 +25,40 @@ inline UInt32 leastDecimalPrecisionFor(TypeIndex int_type)
/// 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 <typename T>
class DataTypeDecimal final : public DataTypeWithSimpleSerialization
class DataTypeDecimal final : public DataTypeDecimalBase<T>
{
using Base = DataTypeDecimalBase<T>;
static_assert(IsDecimalNumber<T>);
public:
using FieldType = T;
using ColumnType = ColumnDecimal<T>;
using typename Base::FieldType;
using typename Base::ColumnType;
using Base::Base;
static constexpr bool is_parametric = true;
static constexpr auto family_name = "Decimal";
static constexpr size_t maxPrecision() { return maxDecimalPrecision<T>(); }
DataTypeDecimal(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<UInt32>(scale) > maxPrecision()))
throw Exception("Scale " + std::to_string(scale) + " is out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
const char * getFamilyName() const override { return "Decimal"; }
const char * getFamilyName() const override { return family_name; }
std::string doGetName() const override;
TypeIndex getTypeId() const override { return TypeId<T>::value; }
bool canBePromoted() const override { return true; }
DataTypePtr promoteNumericType() const override;
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;
Field getDefault() const override;
bool canBePromoted() const override { return true; }
DataTypePtr promoteNumericType() const override;
MutableColumnPtr createColumn() const override;
bool equals(const IDataType & rhs) 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; }
/// Decimal specific
UInt32 getPrecision() const { return precision; }
UInt32 getScale() const { return scale; }
T getScaleMultiplier() const { return T::getScaleMultiplier(scale); }
T wholePart(T x) const
{
if (scale == 0)
return x;
return x / getScaleMultiplier();
}
T fractionalPart(T x) const
{
if (scale == 0)
return 0;
if (x < T(0))
x *= T(-1);
return x % getScaleMultiplier();
}
T maxWholeValue() const { return T::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 <typename U>
T scaleFactorFor(const DataTypeDecimal<U> & x, bool) const
{
if (getScale() < x.getScale())
throw Exception("Decimal result's scale is less then argiment's one", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
UInt32 scale_delta = getScale() - x.getScale(); /// scale_delta >= 0
return T::getScaleMultiplier(scale_delta);
}
template <typename U>
T scaleFactorFor(const DataTypeNumber<U> & , bool is_multiply_or_divisor) const
{
if (is_multiply_or_divisor)
return 1;
return getScaleMultiplier();
}
T parseFromString(const String & str) const;
void readText(T & x, ReadBuffer & istr, bool csv = false) const { readText(x, istr, this->precision, this->scale, csv); }
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);
private:
const UInt32 precision;
const UInt32 scale;
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_);
};
template <typename T, typename U>
typename std::enable_if_t<(sizeof(T) >= sizeof(U)), const DataTypeDecimal<T>>
decimalResultType(const DataTypeDecimal<T> & tx, const DataTypeDecimal<U> & 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 DataTypeDecimal<T>(maxDecimalPrecision<T>(), scale);
}
template <typename T, typename U>
typename std::enable_if_t<(sizeof(T) < sizeof(U)), const DataTypeDecimal<U>>
decimalResultType(const DataTypeDecimal<T> & tx, const DataTypeDecimal<U> & 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 DataTypeDecimal<U>(maxDecimalPrecision<U>(), scale);
}
template <typename T, typename U>
const DataTypeDecimal<T> decimalResultType(const DataTypeDecimal<T> & tx, const DataTypeNumber<U> &, bool, bool)
{
return DataTypeDecimal<T>(maxDecimalPrecision<T>(), tx.getScale());
}
template <typename T, typename U>
const DataTypeDecimal<U> decimalResultType(const DataTypeNumber<T> &, const DataTypeDecimal<U> & ty, bool, bool)
{
return DataTypeDecimal<U>(maxDecimalPrecision<U>(), ty.getScale());
}
template <typename T>
inline const DataTypeDecimal<T> * checkDecimal(const IDataType & data_type)
{
@ -242,14 +76,11 @@ inline UInt32 getDecimalScale(const IDataType & data_type, UInt32 default_value
return default_value;
}
///
template <typename DataType> constexpr bool IsDataTypeDecimal = false;
template <> inline constexpr bool IsDataTypeDecimal<DataTypeDecimal<Decimal32>> = true;
template <> inline constexpr bool IsDataTypeDecimal<DataTypeDecimal<Decimal64>> = true;
template <> inline constexpr bool IsDataTypeDecimal<DataTypeDecimal<Decimal128>> = true;
template <typename DataType> constexpr bool IsDataTypeDecimalOrNumber = IsDataTypeDecimal<DataType> || IsDataTypeNumber<DataType>;
template <typename T>
inline UInt32 getDecimalScale(const DataTypeDecimal<T> & data_type)
{
return data_type.getScale();
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>, typename ToDataType::FieldType>
@ -265,7 +96,8 @@ convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_fro
{
converted_value = MaxFieldType::getScaleMultiplier(scale_to - scale_from);
if (common::mulOverflow(static_cast<MaxNativeType>(value), converted_value, converted_value))
throw Exception("Decimal convert overflow", ErrorCodes::DECIMAL_OVERFLOW);
throw Exception(std::string(ToDataType::family_name) + " convert overflow",
ErrorCodes::DECIMAL_OVERFLOW);
}
else
converted_value = value / MaxFieldType::getScaleMultiplier(scale_from - scale_to);
@ -274,14 +106,15 @@ convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_fro
{
if (converted_value < std::numeric_limits<typename ToFieldType::NativeType>::min() ||
converted_value > std::numeric_limits<typename ToFieldType::NativeType>::max())
throw Exception("Decimal convert overflow", ErrorCodes::DECIMAL_OVERFLOW);
throw Exception(std::string(ToDataType::family_name) + " convert overflow",
ErrorCodes::DECIMAL_OVERFLOW);
}
return converted_value;
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && IsDataTypeNumber<ToDataType>, typename ToDataType::FieldType>
inline std::enable_if_t<IsDataTypeDecimal<FromDataType> && IsNumber<typename ToDataType::FieldType>, typename ToDataType::FieldType>
convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
{
using FromFieldType = typename FromDataType::FieldType;
@ -299,7 +132,8 @@ convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
{
if (converted_value < std::numeric_limits<ToFieldType>::min() ||
converted_value > std::numeric_limits<ToFieldType>::max())
throw Exception("Decimal convert overflow", ErrorCodes::DECIMAL_OVERFLOW);
throw Exception(std::string(FromDataType::family_name) + " convert overflow",
ErrorCodes::DECIMAL_OVERFLOW);
}
else
{
@ -307,7 +141,8 @@ convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
if (converted_value < 0 ||
converted_value > static_cast<CastIntType>(std::numeric_limits<ToFieldType>::max()))
throw Exception("Decimal convert overflow", ErrorCodes::DECIMAL_OVERFLOW);
throw Exception(std::string(FromDataType::family_name) + " convert overflow",
ErrorCodes::DECIMAL_OVERFLOW);
}
}
return converted_value;
@ -315,7 +150,7 @@ convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
}
template <typename FromDataType, typename ToDataType>
inline std::enable_if_t<IsDataTypeNumber<FromDataType> && IsDataTypeDecimal<ToDataType>, typename ToDataType::FieldType>
inline std::enable_if_t<IsNumber<typename FromDataType::FieldType> && IsDataTypeDecimal<ToDataType>, typename ToDataType::FieldType>
convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
{
using FromFieldType = typename FromDataType::FieldType;
@ -325,7 +160,8 @@ convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
if constexpr (std::is_floating_point_v<FromFieldType>)
{
if (!std::isfinite(value))
throw Exception("Decimal convert overflow. Cannot convert infinity or NaN to decimal", ErrorCodes::DECIMAL_OVERFLOW);
throw Exception(std::string(ToDataType::family_name) + " convert overflow. Cannot convert infinity or NaN to decimal",
ErrorCodes::DECIMAL_OVERFLOW);
auto out = value * ToFieldType::getScaleMultiplier(scale);
if constexpr (std::is_same_v<ToNativeType, Int128>)
@ -333,12 +169,14 @@ convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
static constexpr __int128 min_int128 = __int128(0x8000000000000000ll) << 64;
static constexpr __int128 max_int128 = (__int128(0x7fffffffffffffffll) << 64) + 0xffffffffffffffffll;
if (out <= static_cast<ToNativeType>(min_int128) || out >= static_cast<ToNativeType>(max_int128))
throw Exception("Decimal convert overflow. Float is out of Decimal range", ErrorCodes::DECIMAL_OVERFLOW);
throw Exception(std::string(ToDataType::family_name) + " convert overflow. Float is out of Decimal range",
ErrorCodes::DECIMAL_OVERFLOW);
}
else
{
if (out <= std::numeric_limits<ToNativeType>::min() || out >= std::numeric_limits<ToNativeType>::max())
throw Exception("Decimal convert overflow. Float is out of Decimal range", ErrorCodes::DECIMAL_OVERFLOW);
throw Exception(std::string(ToDataType::family_name) + " convert overflow. Float is out of Decimal range",
ErrorCodes::DECIMAL_OVERFLOW);
}
return out;
}
@ -347,6 +185,7 @@ convertToDecimal(const typename FromDataType::FieldType & value, UInt32 scale)
if constexpr (std::is_same_v<FromFieldType, UInt64>)
if (value > static_cast<UInt64>(std::numeric_limits<Int64>::max()))
return convertDecimals<DataTypeDecimal<Decimal128>, ToDataType>(value, 0, scale);
return convertDecimals<DataTypeDecimal<Decimal64>, ToDataType>(value, 0, scale);
}
}

View File

@ -38,16 +38,4 @@ using DataTypeInt64 = DataTypeNumber<Int64>;
using DataTypeFloat32 = DataTypeNumber<Float32>;
using DataTypeFloat64 = DataTypeNumber<Float64>;
template <typename DataType> constexpr bool IsDataTypeNumber = false;
template <> inline constexpr bool IsDataTypeNumber<DataTypeNumber<UInt8>> = true;
template <> inline constexpr bool IsDataTypeNumber<DataTypeNumber<UInt16>> = true;
template <> inline constexpr bool IsDataTypeNumber<DataTypeNumber<UInt32>> = true;
template <> inline constexpr bool IsDataTypeNumber<DataTypeNumber<UInt64>> = true;
template <> inline constexpr bool IsDataTypeNumber<DataTypeNumber<Int8>> = true;
template <> inline constexpr bool IsDataTypeNumber<DataTypeNumber<Int16>> = true;
template <> inline constexpr bool IsDataTypeNumber<DataTypeNumber<Int32>> = true;
template <> inline constexpr bool IsDataTypeNumber<DataTypeNumber<Int64>> = true;
template <> inline constexpr bool IsDataTypeNumber<DataTypeNumber<Float32>> = true;
template <> inline constexpr bool IsDataTypeNumber<DataTypeNumber<Float64>> = true;
}

View File

@ -511,7 +511,8 @@ struct WhichDataType
bool isDate() const { return idx == TypeIndex::Date; }
bool isDateTime() const { return idx == TypeIndex::DateTime; }
bool isDateOrDateTime() const { return isDate() || isDateTime(); }
bool isDateTime64() const { return idx == TypeIndex::DateTime64; }
bool isDateOrDateTime() const { return isDate() || isDateTime() || isDateTime64(); }
bool isString() const { return idx == TypeIndex::String; }
bool isFixedString() const { return idx == TypeIndex::FixedString; }
@ -533,6 +534,7 @@ struct WhichDataType
inline bool isDate(const DataTypePtr & data_type) { return WhichDataType(data_type).isDate(); }
inline bool isDateOrDateTime(const DataTypePtr & data_type) { return WhichDataType(data_type).isDateOrDateTime(); }
inline bool isDateTime64(const DataTypePtr & data_type) { return WhichDataType(data_type).isDateTime64(); }
inline bool isEnum(const DataTypePtr & data_type) { return WhichDataType(data_type).isEnum(); }
inline bool isDecimal(const DataTypePtr & data_type) { return WhichDataType(data_type).isDecimal(); }
inline bool isTuple(const DataTypePtr & data_type) { return WhichDataType(data_type).isTuple(); }
@ -564,6 +566,14 @@ inline bool isFloat(const T & data_type)
return which.isFloat();
}
template <typename T>
inline bool isNativeInteger(const T & data_type)
{
WhichDataType which(data_type);
return which.isNativeInt() || which.isNativeUInt();
}
template <typename T>
inline bool isNativeNumber(const T & data_type)
{
@ -585,6 +595,13 @@ inline bool isColumnedAsNumber(const T & data_type)
return which.isInt() || which.isUInt() || which.isFloat() || which.isDateOrDateTime() || which.isUUID();
}
template <typename T>
inline bool isColumnedAsDecimal(const T & data_type)
{
WhichDataType which(data_type);
return which.isDecimal() || which.isDateTime64();
}
template <typename T>
inline bool isString(const T & data_type)
{
@ -615,6 +632,30 @@ inline bool isCompilableType(const DataTypePtr & data_type)
return data_type->isValueRepresentedByNumber() && !isDecimal(data_type);
}
template <typename DataType> constexpr bool IsDataTypeDecimal = false;
template <typename DataType> constexpr bool IsDataTypeNumber = false;
template <typename DataType> constexpr bool IsDataTypeDateOrDateTime = false;
template <typename DataType> constexpr bool IsDataTypeDecimalOrNumber = IsDataTypeDecimal<DataType> || IsDataTypeNumber<DataType>;
template <typename T>
class DataTypeDecimal;
template <typename T>
class DataTypeNumber;
class DataTypeDate;
class DataTypeDateTime;
class DataTypeDateTime64;
template <typename T> constexpr bool IsDataTypeDecimal<DataTypeDecimal<T>> = true;
template <> constexpr bool IsDataTypeDecimal<DataTypeDateTime64> = true;
template <typename T> constexpr bool IsDataTypeNumber<DataTypeNumber<T>> = true;
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDate> = true;
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime> = true;
template <> inline constexpr bool IsDataTypeDateOrDateTime<DataTypeDateTime64> = true;
}

View File

@ -12,6 +12,7 @@
#include <DataTypes/DataTypeNothing.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
@ -212,14 +213,46 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
{
UInt32 have_date = type_ids.count(TypeIndex::Date);
UInt32 have_datetime = type_ids.count(TypeIndex::DateTime);
UInt32 have_datetime64 = type_ids.count(TypeIndex::DateTime64);
if (have_date || have_datetime)
if (have_date || have_datetime || have_datetime64)
{
bool all_date_or_datetime = type_ids.size() == (have_date + have_datetime);
bool all_date_or_datetime = type_ids.size() == (have_date + have_datetime + have_datetime64);
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<DataTypeDateTime>();
if (have_datetime64 == 0)
{
return std::make_shared<DataTypeDateTime>();
}
// When DateTime64 involved, make sure that supertype has whole-part precision
// big enough to hold max whole-value of any type from `types`.
// That would sacrifice scale when comparing DateTime64 of different scales.
UInt32 max_datetime64_whole_precision = 0;
for (const auto & t : types)
{
if (auto dt64 = typeid_cast<const DataTypeDateTime64 *>(t.get()))
{
const auto whole_precision = dt64->getPrecision() - dt64->getScale();
max_datetime64_whole_precision = std::max(whole_precision, max_datetime64_whole_precision);
}
}
UInt32 least_decimal_precision = 0;
if (have_datetime)
{
least_decimal_precision = leastDecimalPrecisionFor(TypeIndex::UInt32);
}
else if (have_date)
{
least_decimal_precision = leastDecimalPrecisionFor(TypeIndex::UInt16);
}
max_datetime64_whole_precision = std::max(least_decimal_precision, max_datetime64_whole_precision);
const UInt32 scale = DataTypeDateTime64::maxPrecision() - max_datetime64_whole_precision;
return std::make_shared<DataTypeDateTime64>(scale);
}
}

View File

@ -6,117 +6,209 @@
#include <gtest/gtest.h>
namespace DB
{
bool operator==(const IDataType & left, const IDataType & right)
{
return left.equals(right);
}
std::ostream & operator<<(std::ostream & ostr, const IDataType & dt)
{
return ostr << dt.getName();
}
}
using namespace DB;
TEST(data_type, data_type_get_common_type_Test)
auto typeFromString(const std::string & str)
{
auto & data_type_factory = DataTypeFactory::instance();
auto typeFromString = [& data_type_factory](const std::string & str)
{
return data_type_factory.get(str);
};
return data_type_factory.get(str);
};
auto typesFromString = [& typeFromString](const std::string & str)
{
std::istringstream data_types_stream(str);
DataTypes data_types;
std::string data_type;
while (data_types_stream >> data_type)
data_types.push_back(typeFromString(data_type));
auto typesFromString(const std::string & str)
{
std::istringstream data_types_stream(str);
DataTypes data_types;
std::string data_type;
while (data_types_stream >> data_type)
data_types.push_back(typeFromString(data_type));
return data_types;
};
return data_types;
};
ASSERT_TRUE(getLeastSupertype(typesFromString(""))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nothing"))->equals(*typeFromString("Nothing")));
struct TypesTestCase
{
const char * from_types;
const char * expected_type = nullptr;
};
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Int8 Int8"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 Int8"))->equals(*typeFromString("Int16")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 Int16"))->equals(*typeFromString("Int16")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 UInt32 UInt64"))->equals(*typeFromString("UInt64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Int8 Int32 Int64"))->equals(*typeFromString("Int64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("UInt8 UInt32 Int64"))->equals(*typeFromString("Int64")));
std::ostream & operator<<(std::ostream & ostr, const TypesTestCase & test_case)
{
ostr << "TypesTestCase{\"" << test_case.from_types << "\", ";
if (test_case.expected_type)
ostr << "\"" << test_case.expected_type << "\"";
else
ostr << "nullptr";
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 Float64"))->equals(*typeFromString("Float64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 UInt16 Int16"))->equals(*typeFromString("Float32")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 UInt16 Int32"))->equals(*typeFromString("Float64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Float32 Int16 UInt32"))->equals(*typeFromString("Float64")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Date Date"))->equals(*typeFromString("Date")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Date DateTime"))->equals(*typeFromString("DateTime")));
ASSERT_TRUE(getLeastSupertype(typesFromString("String FixedString(32) FixedString(8)"))->equals(*typeFromString("String")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(UInt8) Array(UInt8)"))->equals(*typeFromString("Array(UInt8)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(UInt8) Array(Int8)"))->equals(*typeFromString("Array(Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Float32) Array(Int16) Array(UInt32)"))->equals(*typeFromString("Array(Float64)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Array(UInt8)) Array(Array(UInt8))"))->equals(*typeFromString("Array(Array(UInt8))")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Array(UInt8)) Array(Array(Int8))"))->equals(*typeFromString("Array(Array(Int16))")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(Date) Array(DateTime)"))->equals(*typeFromString("Array(DateTime)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Array(String) Array(FixedString(32))"))->equals(*typeFromString("Array(String)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nullable(Nothing) Nothing"))->equals(*typeFromString("Nullable(Nothing)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nullable(UInt8) Int8"))->equals(*typeFromString("Nullable(Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Nullable(Nothing) UInt8 Int8"))->equals(*typeFromString("Nullable(Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Tuple(Int8,UInt8) Tuple(UInt8,Int8)"))->equals(*typeFromString("Tuple(Int16,Int16)")));
ASSERT_TRUE(getLeastSupertype(typesFromString("Tuple(Nullable(Nothing)) Tuple(Nullable(UInt8))"))->equals(*typeFromString("Tuple(Nullable(UInt8))")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Int8 String")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Int64 UInt64")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Float32 UInt64")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Float64 Int64")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Tuple(Int64) Tuple(UInt64)")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Tuple(Int64, Int8) Tuple(UInt64)")));
EXPECT_ANY_THROW(getLeastSupertype(typesFromString("Array(Int64) Array(String)")));
ASSERT_TRUE(getMostSubtype(typesFromString(""))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nothing"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 UInt8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Int8 Int8"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 Int8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Int8 UInt16"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 UInt32 UInt64"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Int8 Int32 Int64"))->equals(*typeFromString("Int8")));
ASSERT_TRUE(getMostSubtype(typesFromString("UInt8 Int64 UInt64"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 Float64"))->equals(*typeFromString("Float32")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 UInt16 Int16"))->equals(*typeFromString("UInt16")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 UInt16 Int32"))->equals(*typeFromString("UInt16")));
ASSERT_TRUE(getMostSubtype(typesFromString("Float32 Int16 UInt32"))->equals(*typeFromString("Int16")));
ASSERT_TRUE(getMostSubtype(typesFromString("DateTime DateTime"))->equals(*typeFromString("DateTime")));
ASSERT_TRUE(getMostSubtype(typesFromString("Date DateTime"))->equals(*typeFromString("Date")));
ASSERT_TRUE(getMostSubtype(typesFromString("String FixedString(8)"))->equals(*typeFromString("FixedString(8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("FixedString(16) FixedString(8)"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(UInt8) Array(UInt8)"))->equals(*typeFromString("Array(UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(UInt8) Array(Int8)"))->equals(*typeFromString("Array(UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Float32) Array(Int16) Array(UInt32)"))->equals(*typeFromString("Array(Int16)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Array(UInt8)) Array(Array(UInt8))"))->equals(*typeFromString("Array(Array(UInt8))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Array(UInt8)) Array(Array(Int8))"))->equals(*typeFromString("Array(Array(UInt8))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(Date) Array(DateTime)"))->equals(*typeFromString("Array(Date)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(String) Array(FixedString(32))"))->equals(*typeFromString("Array(FixedString(32))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Array(String) Array(FixedString(32))"))->equals(*typeFromString("Array(FixedString(32))")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(Nothing) Nothing"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(UInt8) Int8"))->equals(*typeFromString("UInt8")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(Nothing) UInt8 Int8"))->equals(*typeFromString("Nothing")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(UInt8) Nullable(Int8)"))->equals(*typeFromString("Nullable(UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Nullable(Nothing) Nullable(Int8)"))->equals(*typeFromString("Nullable(Nothing)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Tuple(Int8,UInt8) Tuple(UInt8,Int8)"))->equals(*typeFromString("Tuple(UInt8,UInt8)")));
ASSERT_TRUE(getMostSubtype(typesFromString("Tuple(Nullable(Nothing)) Tuple(Nullable(UInt8))"))->equals(*typeFromString("Tuple(Nullable(Nothing))")));
EXPECT_ANY_THROW(getMostSubtype(typesFromString("Int8 String"), true));
EXPECT_ANY_THROW(getMostSubtype(typesFromString("Nothing"), true));
EXPECT_ANY_THROW(getMostSubtype(typesFromString("FixedString(16) FixedString(8) String"), true));
return ostr << "}";
}
class TypeTest : public ::testing::TestWithParam<TypesTestCase>
{
public:
void SetUp()
{
const auto & p = GetParam();
from_types = typesFromString(p.from_types);
if (p.expected_type)
expected_type = typeFromString(p.expected_type);
else
expected_type.reset();
}
public:
DataTypes from_types;
DataTypePtr expected_type;
};
class LeastSuperTypeTest : public TypeTest {};
TEST_P(LeastSuperTypeTest, getLeastSupertype)
{
if (this->expected_type)
{
ASSERT_EQ(*this->expected_type, *getLeastSupertype(this->from_types));
}
else
{
EXPECT_ANY_THROW(getLeastSupertype(this->from_types));
}
}
class MostSubtypeTest : public TypeTest {};
TEST_P(MostSubtypeTest, getLeastSupertype)
{
if (this->expected_type)
{
ASSERT_EQ(*this->expected_type, *getMostSubtype(this->from_types));
}
else
{
EXPECT_ANY_THROW(getMostSubtype(this->from_types, true));
}
}
INSTANTIATE_TEST_CASE_P(data_type,
LeastSuperTypeTest,
::testing::ValuesIn(
std::initializer_list<TypesTestCase>{
{"", "Nothing"},
{"Nothing", "Nothing"},
{"UInt8", "UInt8"},
{"UInt8 UInt8", "UInt8"},
{"Int8 Int8", "Int8"},
{"UInt8 Int8", "Int16"},
{"UInt8 Int16", "Int16"},
{"UInt8 UInt32 UInt64", "UInt64"},
{"Int8 Int32 Int64", "Int64"},
{"UInt8 UInt32 Int64", "Int64"},
{"Float32 Float64", "Float64"},
{"Float32 UInt16 Int16", "Float32"},
{"Float32 UInt16 Int32", "Float64"},
{"Float32 Int16 UInt32", "Float64"},
{"Date Date", "Date"},
{"Date DateTime", "DateTime"},
{"Date DateTime64(3)", "DateTime64(3)"},
{"DateTime DateTime64(3)", "DateTime64(3)"},
{"DateTime DateTime64(0)", "DateTime64(0)"},
{"DateTime64(9) DateTime64(3)", "DateTime64(3)"},
{"DateTime DateTime64(12)", "DateTime64(8)"},
{"Date DateTime64(15)", "DateTime64(13)"},
{"String FixedString(32) FixedString(8)", "String"},
{"Array(UInt8) Array(UInt8)", "Array(UInt8)"},
{"Array(UInt8) Array(Int8)", "Array(Int16)"},
{"Array(Float32) Array(Int16) Array(UInt32)", "Array(Float64)"},
{"Array(Array(UInt8)) Array(Array(UInt8))", "Array(Array(UInt8))"},
{"Array(Array(UInt8)) Array(Array(Int8))", "Array(Array(Int16))"},
{"Array(Date) Array(DateTime)", "Array(DateTime)"},
{"Array(String) Array(FixedString(32))", "Array(String)"},
{"Nullable(Nothing) Nothing", "Nullable(Nothing)"},
{"Nullable(UInt8) Int8", "Nullable(Int16)"},
{"Nullable(Nothing) UInt8 Int8", "Nullable(Int16)"},
{"Tuple(Int8,UInt8) Tuple(UInt8,Int8)", "Tuple(Int16,Int16)"},
{"Tuple(Nullable(Nothing)) Tuple(Nullable(UInt8))", "Tuple(Nullable(UInt8))"},
{"Int8 String", nullptr},
{"Int64 UInt64", nullptr},
{"Float32 UInt64", nullptr},
{"Float64 Int64", nullptr},
{"Tuple(Int64) Tuple(UInt64)", nullptr},
{"Tuple(Int64,Int8) Tuple(UInt64)", nullptr},
{"Array(Int64) Array(String)", nullptr},
}
),
);
INSTANTIATE_TEST_CASE_P(data_type,
MostSubtypeTest,
::testing::ValuesIn(
std::initializer_list<TypesTestCase>{
{"", "Nothing"},
{"Nothing", "Nothing"},
{"UInt8", "UInt8"},
{"UInt8 UInt8", "UInt8"},
{"Int8 Int8", "Int8"},
{"UInt8 Int8", "UInt8"},
{"Int8 UInt16", "Int8"},
{"UInt8 UInt32 UInt64", "UInt8"},
{"Int8 Int32 Int64", "Int8"},
{"UInt8 Int64 UInt64", "UInt8"},
{"Float32 Float64", "Float32"},
{"Float32 UInt16 Int16", "UInt16"},
{"Float32 UInt16 Int32", "UInt16"},
{"Float32 Int16 UInt32", "Int16"},
{"DateTime DateTime", "DateTime"},
{"Date DateTime", "Date"},
{"String FixedString(8)", "FixedString(8)"},
{"FixedString(16) FixedString(8)", "Nothing"},
{"Array(UInt8) Array(UInt8)", "Array(UInt8)"},
{"Array(UInt8) Array(Int8)", "Array(UInt8)"},
{"Array(Float32) Array(Int16) Array(UInt32)", "Array(Int16)"},
{"Array(Array(UInt8)) Array(Array(UInt8))", "Array(Array(UInt8))"},
{"Array(Array(UInt8)) Array(Array(Int8))", "Array(Array(UInt8))"},
{"Array(Date) Array(DateTime)", "Array(Date)"},
{"Array(String) Array(FixedString(32))", "Array(FixedString(32))"},
{"Array(String) Array(FixedString(32))", "Array(FixedString(32))"},
{"Nullable(Nothing) Nothing", "Nothing"},
{"Nullable(UInt8) Int8", "UInt8"},
{"Nullable(Nothing) UInt8 Int8", "Nothing"},
{"Nullable(UInt8) Nullable(Int8)", "Nullable(UInt8)"},
{"Nullable(Nothing) Nullable(Int8)", "Nullable(Nothing)"},
{"Tuple(Int8,UInt8) Tuple(UInt8,Int8)", "Tuple(UInt8,UInt8)"},
{"Tuple(Nullable(Nothing)) Tuple(Nullable(UInt8))", "Tuple(Nullable(Nothing))"},
{"Int8 String", nullptr},
{"Nothing", nullptr},
{"FixedString(16) FixedString(8) String", nullptr},
}),
);

View File

@ -474,6 +474,11 @@ public:
cannotConvertType("DateTime");
}
bool readDateTime64(DateTime64 &, UInt32) override
{
cannotConvertType("DateTime64");
}
bool readDecimal32(Decimal32 &, UInt32, UInt32) override
{
cannotConvertType("Decimal32");
@ -606,6 +611,15 @@ public:
return true;
}
bool readDateTime64(DateTime64 & date_time, UInt32 scale) override
{
if (!readTempString())
return false;
ReadBufferFromString buf(temp_string);
readDateTime64Text(date_time, scale, buf);
return true;
}
bool readDecimal32(Decimal32 & decimal, UInt32 precision, UInt32 scale) override { return readDecimal(decimal, precision, scale); }
bool readDecimal64(Decimal64 & decimal, UInt32 precision, UInt32 scale) override { return readDecimal(decimal, precision, scale); }
bool readDecimal128(Decimal128 & decimal, UInt32 precision, UInt32 scale) override { return readDecimal(decimal, precision, scale); }
@ -741,6 +755,11 @@ public:
return true;
}
bool readDateTime64(DateTime64 & date_time, UInt32 scale) override
{
return readDecimal(date_time, scale);
}
bool readDecimal32(Decimal32 & decimal, UInt32, UInt32 scale) override { return readDecimal(decimal, scale); }
bool readDecimal64(Decimal64 & decimal, UInt32, UInt32 scale) override { return readDecimal(decimal, scale); }
bool readDecimal128(Decimal128 & decimal, UInt32, UInt32 scale) override { return readDecimal(decimal, scale); }

View File

@ -72,6 +72,7 @@ public:
bool readUUID(UUID & uuid) { return current_converter->readUUID(uuid); }
bool readDate(DayNum & date) { return current_converter->readDate(date); }
bool readDateTime(time_t & tm) { return current_converter->readDateTime(tm); }
bool readDateTime64(DateTime64 & tm, UInt32 scale) { return current_converter->readDateTime64(tm, scale); }
bool readDecimal(Decimal32 & decimal, UInt32 precision, UInt32 scale) { return current_converter->readDecimal32(decimal, precision, scale); }
bool readDecimal(Decimal64 & decimal, UInt32 precision, UInt32 scale) { return current_converter->readDecimal64(decimal, precision, scale); }
@ -152,6 +153,7 @@ private:
virtual bool readUUID(UUID &) = 0;
virtual bool readDate(DayNum &) = 0;
virtual bool readDateTime(time_t &) = 0;
virtual bool readDateTime64(DateTime64 &, UInt32) = 0;
virtual bool readDecimal32(Decimal32 &, UInt32, UInt32) = 0;
virtual bool readDecimal64(Decimal64 &, UInt32, UInt32) = 0;
virtual bool readDecimal128(Decimal128 &, UInt32, UInt32) = 0;
@ -227,6 +229,7 @@ public:
bool readUUID(UUID &) { return false; }
bool readDate(DayNum &) { return false; }
bool readDateTime(time_t &) { return false; }
bool readDateTime64(DateTime64 & /*tm*/, UInt32 /*scale*/) { return false; }
bool readDecimal(Decimal32 &, UInt32, UInt32) { return false; }
bool readDecimal(Decimal64 &, UInt32, UInt32) { return false; }
bool readDecimal(Decimal128 &, UInt32, UInt32) { return false; }

View File

@ -329,6 +329,7 @@ public:
virtual void writeUUID(const UUID &) override { cannotConvertType("UUID"); }
virtual void writeDate(DayNum) override { cannotConvertType("Date"); }
virtual void writeDateTime(time_t) override { cannotConvertType("DateTime"); }
virtual void writeDateTime64(DateTime64, UInt32) override { cannotConvertType("DateTime64"); }
virtual void writeDecimal32(Decimal32, UInt32) override { cannotConvertType("Decimal32"); }
virtual void writeDecimal64(Decimal64, UInt32) override { cannotConvertType("Decimal64"); }
virtual void writeDecimal128(const Decimal128 &, UInt32) override { cannotConvertType("Decimal128"); }
@ -436,6 +437,13 @@ public:
text_buffer.restart();
}
void writeDateTime64(DateTime64 date_time, UInt32 scale) override
{
writeDateTimeText(date_time, scale, text_buffer);
writeField(text_buffer.stringRef());
text_buffer.restart();
}
void writeDecimal32(Decimal32 decimal, UInt32 scale) override { writeDecimal(decimal, scale); }
void writeDecimal64(Decimal64 decimal, UInt32 scale) override { writeDecimal(decimal, scale); }
void writeDecimal128(const Decimal128 & decimal, UInt32 scale) override { writeDecimal(decimal, scale); }
@ -533,7 +541,7 @@ public:
void writeDate(DayNum date) override { castNumericAndWriteField(static_cast<UInt16>(date)); }
void writeDateTime(time_t tm) override { castNumericAndWriteField(tm); }
void writeDateTime64(DateTime64 date_time, UInt32 scale) override { writeDecimal(date_time, scale); }
void writeDecimal32(Decimal32 decimal, UInt32 scale) override { writeDecimal(decimal, scale); }
void writeDecimal64(Decimal64 decimal, UInt32 scale) override { writeDecimal(decimal, scale); }
void writeDecimal128(const Decimal128 & decimal, UInt32 scale) override { writeDecimal(decimal, scale); }

View File

@ -5,6 +5,9 @@
#include <common/DayNum.h>
#include "config_formats.h"
#include <memory>
#if USE_PROTOBUF
#include "ProtobufColumnMatcher.h"
@ -70,6 +73,7 @@ public:
bool writeUUID(const UUID & uuid) { return writeValueIfPossible(&IConverter::writeUUID, uuid); }
bool writeDate(DayNum date) { return writeValueIfPossible(&IConverter::writeDate, date); }
bool writeDateTime(time_t tm) { return writeValueIfPossible(&IConverter::writeDateTime, tm); }
bool writeDateTime64(DateTime64 tm, UInt32 scale) { return writeValueIfPossible(&IConverter::writeDateTime64, tm, scale); }
bool writeDecimal(Decimal32 decimal, UInt32 scale) { return writeValueIfPossible(&IConverter::writeDecimal32, decimal, scale); }
bool writeDecimal(Decimal64 decimal, UInt32 scale) { return writeValueIfPossible(&IConverter::writeDecimal64, decimal, scale); }
bool writeDecimal(const Decimal128 & decimal, UInt32 scale) { return writeValueIfPossible(&IConverter::writeDecimal128, decimal, scale); }
@ -153,6 +157,7 @@ private:
virtual void writeUUID(const UUID &) = 0;
virtual void writeDate(DayNum) = 0;
virtual void writeDateTime(time_t) = 0;
virtual void writeDateTime64(DateTime64, UInt32 scale) = 0;
virtual void writeDecimal32(Decimal32, UInt32) = 0;
virtual void writeDecimal64(Decimal64, UInt32) = 0;
virtual void writeDecimal128(const Decimal128 &, UInt32) = 0;
@ -257,6 +262,7 @@ public:
bool writeUUID(const UUID & /* value */) { return false; }
bool writeDate(DayNum /* date */) { return false; }
bool writeDateTime(time_t /* tm */) { return false; }
bool writeDateTime64(DateTime64 /*tm*/, UInt32 /*scale*/) { return false; }
bool writeDecimal(Decimal32 /* decimal */, UInt32 /* scale */) { return false; }
bool writeDecimal(Decimal64 /* decimal */, UInt32 /* scale */) { return false; }
bool writeDecimal(const Decimal128 & /* decimal */, UInt32 /* scale */) { return false; }

View File

@ -3,6 +3,7 @@
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
#include <Core/Types.h>
#include <Core/DecimalFunctions.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <Common/Exception.h>
@ -90,24 +91,33 @@ struct ToStartOfWeekImpl
template <typename FromType, typename ToType, typename Transform>
struct Transformer
{
static void
vector(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, UInt8 week_mode, const DateLUTImpl & time_zone)
Transformer(Transform transform_)
: transform(std::move(transform_))
{}
template <typename FromVectorType, typename ToVectorType>
void
vector(const FromVectorType & vec_from, ToVectorType & vec_to, UInt8 week_mode, const DateLUTImpl & time_zone) const
{
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(vec_from[i], week_mode, time_zone);
vec_to[i] = transform.execute(vec_from[i], week_mode, time_zone);
}
private:
const Transform transform;
};
template <typename FromType, typename ToType, typename Transform>
template <typename FromDataType, typename ToDataType>
struct CustomWeekTransformImpl
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
template <typename Transform>
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/, Transform transform = {})
{
using Op = Transformer<FromType, ToType, Transform>;
const auto op = Transformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
UInt8 week_mode = DEFAULT_WEEK_MODE;
if (arguments.size() > 1)
@ -118,10 +128,10 @@ struct CustomWeekTransformImpl
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
if (const auto * sources = checkAndGetColumn<typename FromDataType::ColumnType>(source_col.get()))
{
auto col_to = ColumnVector<ToType>::create();
Op::vector(sources->getData(), col_to->getData(), week_mode, time_zone);
auto col_to = ToDataType::ColumnType::create();
op.vector(sources->getData(), col_to->getData(), week_mode, time_zone);
block.getByPosition(result).column = std::move(col_to);
}
else

View File

@ -1,11 +1,14 @@
#pragma once
#include <Core/Types.h>
#include <Core/DecimalFunctions.h>
#include <Common/Exception.h>
#include <common/DateLUTImpl.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
namespace DB
{
@ -629,32 +632,36 @@ struct ToYYYYMMDDhhmmssImpl
template <typename FromType, typename ToType, typename Transform>
struct Transformer
{
static void vector(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, const DateLUTImpl & time_zone)
template <typename FromTypeVector, typename ToTypeVector>
static void vector(const FromTypeVector & vec_from, ToTypeVector & vec_to, const DateLUTImpl & time_zone, const Transform & transform)
{
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(vec_from[i], time_zone);
vec_to[i] = transform.execute(vec_from[i], time_zone);
}
};
template <typename FromType, typename ToType, typename Transform>
template <typename FromDataType, typename ToDataType, typename Transform>
struct DateTimeTransformImpl
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/, const Transform & transform = {})
{
using Op = Transformer<FromType, ToType, Transform>;
using Op = Transformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>;
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
if (const auto * sources = checkAndGetColumn<typename FromDataType::ColumnType>(source_col.get()))
{
auto col_to = ColumnVector<ToType>::create();
Op::vector(sources->getData(), col_to->getData(), time_zone);
block.getByPosition(result).column = std::move(col_to);
auto mutable_result_col = block.getByPosition(result).type->createColumn();
auto * col_to = assert_cast<typename ToDataType::ColumnType *>(mutable_result_col.get());
Op::vector(sources->getData(), col_to->getData(), time_zone, transform);
block.getByPosition(result).column = std::move(mutable_result_col);
}
else
{

View File

@ -9,6 +9,7 @@
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeInterval.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/Native.h>

View File

@ -1,5 +1,6 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Functions/CustomWeekTransforms.h>
#include <Functions/IFunction.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
@ -100,11 +101,17 @@ public:
WhichDataType which(from_type);
if (which.isDate())
CustomWeekTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(
block, arguments, result, input_rows_count);
CustomWeekTransformImpl<DataTypeDate, ToDataType>::execute(
block, arguments, result, input_rows_count, Transform{});
else if (which.isDateTime())
CustomWeekTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(
block, arguments, result, input_rows_count);
CustomWeekTransformImpl<DataTypeDateTime, ToDataType>::execute(
block, arguments, result, input_rows_count, Transform{});
else if (which.isDateTime64())
{
CustomWeekTransformImpl<DataTypeDateTime64, ToDataType>::execute(
block, arguments, result, input_rows_count,
DateTime64BasicTransformWrapper<Transform>{assert_cast<const DataTypeDateTime64 *>(from_type)->getScale()});
}
else
throw Exception(
"Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),

View File

@ -2,6 +2,7 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Columns/ColumnVector.h>
@ -22,9 +23,47 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
}
struct AddSecondsImpl
/// AddOnDateTime64DefaultImpl provides default implementation of add-X functionality for DateTime64.
///
/// Default implementation is not to change fractional part, but only modify whole part as if it was DateTime.
/// That means large whole values (for scale less than 9) might not fit into UInt32-range,
/// and hence default implementation will produce incorrect results.
template <typename T>
struct AddOnDateTime64DefaultImpl
{
/*explicit*/ AddOnDateTime64DefaultImpl(UInt32 scale_ = 0)
: scale_multiplier(DecimalUtils::scaleMultiplier<DateTime64::NativeType>(scale_))
{}
// Default implementation for add/sub on DateTime64: do math on whole part (the same way as for DateTime), leave fractional as it is.
inline DateTime64 execute(const DateTime64 & t, Int64 delta, const DateLUTImpl & time_zone) const
{
const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier);
const auto whole = static_cast<const T*>(this)->execute(static_cast<UInt32>(components.whole), delta, time_zone);
return DecimalUtils::decimalFromComponentsWithMultiplier<DateTime64>(static_cast<DateTime64::NativeType>(whole), components.fractional, scale_multiplier);
}
UInt32 scale_multiplier = 1;
};
/// Type of first argument of 'execute' function overload defines what INPUT DataType it is used for.
/// Return type defines what is the OUTPUT (return) type of the CH function.
/// Corresponding types:
/// - UInt16 => DataTypeDate
/// - UInt32 => DataTypeDateTime
/// - DateTime64 => DataTypeDateTime64
/// Please note that INPUT and OUTPUT types may differ, e.g.:
/// - 'AddSecondsImpl::execute(UInt32, ...) -> UInt32' is available to the ClickHouse users as 'addSeconds(DateTime, ...) -> DateTime'
/// - 'AddSecondsImpl::execute(UInt16, ...) -> UInt32' is available to the ClickHouse users as 'addSeconds(Date, ...) -> DateTime'
struct AddSecondsImpl : public AddOnDateTime64DefaultImpl<AddSecondsImpl>
{
using Base = AddOnDateTime64DefaultImpl<AddSecondsImpl>;
using Base::Base;
using Base::execute;
static constexpr auto name = "addSeconds";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &)
@ -38,8 +77,12 @@ struct AddSecondsImpl
}
};
struct AddMinutesImpl
struct AddMinutesImpl : public AddOnDateTime64DefaultImpl<AddMinutesImpl>
{
using Base = AddOnDateTime64DefaultImpl<AddMinutesImpl>;
using Base::Base;
using Base::execute;
static constexpr auto name = "addMinutes";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &)
@ -53,8 +96,12 @@ struct AddMinutesImpl
}
};
struct AddHoursImpl
struct AddHoursImpl : public AddOnDateTime64DefaultImpl<AddHoursImpl>
{
using Base = AddOnDateTime64DefaultImpl<AddHoursImpl>;
using Base::Base;
using Base::execute;
static constexpr auto name = "addHours";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &)
@ -68,10 +115,20 @@ struct AddHoursImpl
}
};
struct AddDaysImpl
struct AddDaysImpl : public AddOnDateTime64DefaultImpl<AddDaysImpl>
{
using Base = AddOnDateTime64DefaultImpl<AddDaysImpl>;
using Base::Base;
using Base::execute;
static constexpr auto name = "addDays";
// static inline UInt32 execute(UInt64 t, Int64 delta, const DateLUTImpl & time_zone)
// {
// // TODO (nemkov): LUT does not support out-of range date values for now.
// return time_zone.addDays(t, delta);
// }
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addDays(t, delta);
@ -83,8 +140,12 @@ struct AddDaysImpl
}
};
struct AddWeeksImpl
struct AddWeeksImpl : public AddOnDateTime64DefaultImpl<AddWeeksImpl>
{
using Base = AddOnDateTime64DefaultImpl<AddWeeksImpl>;
using Base::Base;
using Base::execute;
static constexpr auto name = "addWeeks";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
@ -98,8 +159,12 @@ struct AddWeeksImpl
}
};
struct AddMonthsImpl
struct AddMonthsImpl : public AddOnDateTime64DefaultImpl<AddMonthsImpl>
{
using Base = AddOnDateTime64DefaultImpl<AddMonthsImpl>;
using Base::Base;
using Base::execute;
static constexpr auto name = "addMonths";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
@ -113,8 +178,12 @@ struct AddMonthsImpl
}
};
struct AddQuartersImpl
struct AddQuartersImpl : public AddOnDateTime64DefaultImpl<AddQuartersImpl>
{
using Base = AddOnDateTime64DefaultImpl<AddQuartersImpl>;
using Base::Base;
using Base::execute;
static constexpr auto name = "addQuarters";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
@ -128,8 +197,12 @@ struct AddQuartersImpl
}
};
struct AddYearsImpl
struct AddYearsImpl : public AddOnDateTime64DefaultImpl<AddYearsImpl>
{
using Base = AddOnDateTime64DefaultImpl<AddYearsImpl>;
using Base::Base;
using Base::execute;
static constexpr auto name = "addYears";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
@ -143,19 +216,16 @@ struct AddYearsImpl
}
};
template <typename Transform>
struct SubtractIntervalImpl
struct SubtractIntervalImpl : public Transform
{
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
using Transform::Transform;
template <typename T>
inline auto execute(T t, Int64 delta, const DateLUTImpl & time_zone) const
{
return Transform::execute(t, -delta, time_zone);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return Transform::execute(d, -delta, time_zone);
}
};
struct SubtractSecondsImpl : SubtractIntervalImpl<AddSecondsImpl> { static constexpr auto name = "subtractSeconds"; };
@ -168,68 +238,77 @@ struct SubtractQuartersImpl : SubtractIntervalImpl<AddQuartersImpl> { static con
struct SubtractYearsImpl : SubtractIntervalImpl<AddYearsImpl> { static constexpr auto name = "subtractYears"; };
template <typename FromType, typename ToType, typename Transform>
template <typename Transform>
struct Adder
{
static void vector_vector(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, const IColumn & delta, const DateLUTImpl & time_zone)
const Transform transform;
explicit Adder(Transform transform_)
: transform(std::move(transform_))
{}
template <typename FromVectorType, typename ToVectorType>
void vector_vector(const FromVectorType & vec_from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone) const
{
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(vec_from[i], delta.getInt(i), time_zone);
vec_to[i] = transform.execute(vec_from[i], delta.getInt(i), time_zone);
}
static void vector_constant(const PaddedPODArray<FromType> & vec_from, PaddedPODArray<ToType> & vec_to, Int64 delta, const DateLUTImpl & time_zone)
template <typename FromVectorType, typename ToVectorType>
void vector_constant(const FromVectorType & vec_from, ToVectorType & vec_to, Int64 delta, const DateLUTImpl & time_zone) const
{
size_t size = vec_from.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(vec_from[i], delta, time_zone);
vec_to[i] = transform.execute(vec_from[i], delta, time_zone);
}
static void constant_vector(const FromType & from, PaddedPODArray<ToType> & vec_to, const IColumn & delta, const DateLUTImpl & time_zone)
template <typename FromType, typename ToVectorType>
void constant_vector(const FromType & from, ToVectorType & vec_to, const IColumn & delta, const DateLUTImpl & time_zone) const
{
size_t size = delta.size();
vec_to.resize(size);
for (size_t i = 0; i < size; ++i)
vec_to[i] = Transform::execute(from, delta.getInt(i), time_zone);
vec_to[i] = transform.execute(from, delta.getInt(i), time_zone);
}
};
template <typename FromType, typename Transform>
template <typename FromDataType, typename ToDataType, typename Transform>
struct DateTimeAddIntervalImpl
{
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
static void execute(Transform transform, Block & block, const ColumnNumbers & arguments, size_t result)
{
using ToType = decltype(Transform::execute(FromType(), 0, std::declval<DateLUTImpl>()));
using Op = Adder<FromType, ToType, Transform>;
using FromValueType = typename FromDataType::FieldType;
using FromColumnType = typename FromDataType::ColumnType;
using ToColumnType = typename ToDataType::ColumnType;
auto op = Adder<Transform>{std::move(transform)};
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
const ColumnPtr source_col = block.getByPosition(arguments[0]).column;
if (const auto * sources = checkAndGetColumn<ColumnVector<FromType>>(source_col.get()))
{
auto col_to = ColumnVector<ToType>::create();
auto result_col = block.getByPosition(result).type->createColumn();
auto col_to = assert_cast<ToColumnType *>(result_col.get());
if (const auto * sources = checkAndGetColumn<FromColumnType>(source_col.get()))
{
const IColumn & delta_column = *block.getByPosition(arguments[1]).column;
if (const auto * delta_const_column = typeid_cast<const ColumnConst *>(&delta_column))
Op::vector_constant(sources->getData(), col_to->getData(), delta_const_column->getField().get<Int64>(), time_zone);
op.vector_constant(sources->getData(), col_to->getData(), delta_const_column->getField().get<Int64>(), time_zone);
else
Op::vector_vector(sources->getData(), col_to->getData(), delta_column, time_zone);
block.getByPosition(result).column = std::move(col_to);
op.vector_vector(sources->getData(), col_to->getData(), delta_column, time_zone);
}
else if (const auto * sources_const = checkAndGetColumnConst<ColumnVector<FromType>>(source_col.get()))
else if (const auto * sources_const = checkAndGetColumnConst<FromColumnType>(source_col.get()))
{
auto col_to = ColumnVector<ToType>::create();
Op::constant_vector(sources_const->template getValue<FromType>(), col_to->getData(), *block.getByPosition(arguments[1]).column, time_zone);
block.getByPosition(result).column = std::move(col_to);
op.constant_vector(sources_const->template getValue<FromValueType>(), col_to->getData(), *block.getByPosition(arguments[1]).column, time_zone);
}
else
{
@ -237,9 +316,21 @@ struct DateTimeAddIntervalImpl
+ " of first argument of function " + Transform::name,
ErrorCodes::ILLEGAL_COLUMN);
}
block.getByPosition(result).column = std::move(result_col);
}
};
namespace date_and_time_type_details
{
// Compile-time mapping of value (DataType::FieldType) types to corresponding DataType
template <typename FieldType> struct ResultDataTypeMap {};
template <> struct ResultDataTypeMap<UInt16> { using ResultDataType = DataTypeDate; };
template <> struct ResultDataTypeMap<Int16> { using ResultDataType = DataTypeDate; };
template <> struct ResultDataTypeMap<UInt32> { using ResultDataType = DataTypeDateTime; };
template <> struct ResultDataTypeMap<Int32> { using ResultDataType = DataTypeDateTime; };
template <> struct ResultDataTypeMap<DateTime64> { using ResultDataType = DataTypeDateTime64; };
}
template <typename Transform>
class FunctionDateOrDateTimeAddInterval : public IFunction
@ -286,20 +377,58 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
if (WhichDataType(arguments[0].type).isDate())
switch (arguments[0].type->getTypeId())
{
if (std::is_same_v<decltype(Transform::execute(DataTypeDate::FieldType(), 0, std::declval<DateLUTImpl>())), UInt16>)
return std::make_shared<DataTypeDate>();
else
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
case TypeIndex::Date:
return resolveReturnType<DataTypeDate>(arguments);
case TypeIndex::DateTime:
return resolveReturnType<DataTypeDateTime>(arguments);
case TypeIndex::DateTime64:
return resolveReturnType<DataTypeDateTime64>(arguments);
default:
{
throw Exception("Invalid type of 1st argument of function " + getName() + ": "
+ arguments[0].type->getName() + ", expected: Date, DateTime or DateTime64.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
}
// Helper templates to deduce return type based on argument type, since some overloads may promote or denote types, e.g. addSeconds(Date, 1) => DateTime
template <typename FieldType>
using TransformExecuteReturnType = decltype(std::declval<Transform>().execute(FieldType(), 0, std::declval<DateLUTImpl>()));
// Deduces RETURN DataType from INTPUT DataType, based on return type of Transform{}.execute(INPUT_TYPE, UInt64, DateLUTImpl).
// e.g. for Transform-type that has execute()-overload with 'UInt16' input and 'UInt32' return,
// argument type is expected to be 'Date', and result type is deduced to be 'DateTime'.
template <typename FromDataType>
using TransformResultDataType = typename date_and_time_type_details::ResultDataTypeMap<TransformExecuteReturnType<typename FromDataType::FieldType>>::ResultDataType;
template <typename FromDataType>
DataTypePtr resolveReturnType(const ColumnsWithTypeAndName & arguments) const
{
using ResultDataType = TransformResultDataType<FromDataType>;
if constexpr (std::is_same_v<ResultDataType, DataTypeDate>)
return std::make_shared<DataTypeDate>();
else if constexpr (std::is_same_v<ResultDataType, DataTypeDateTime>)
{
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
}
else if constexpr (std::is_same_v<ResultDataType, DataTypeDateTime64>)
{
// TODO (vnemkov): what if there is an overload of Transform::execute() that returns DateTime64 from DateTime or Date ?
// Shall we use the default scale or one from optional argument ?
const auto & datetime64_type = assert_cast<const DataTypeDateTime64 &>(*arguments[0].type);
return std::make_shared<DataTypeDateTime64>(datetime64_type.getScale(), extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
}
else
{
if (std::is_same_v<decltype(Transform::execute(DataTypeDateTime::FieldType(), 0, std::declval<DateLUTImpl>())), UInt16>)
return std::make_shared<DataTypeDate>();
else
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
static_assert("Failed to resolve return type.");
}
//to make PVS and GCC happy.
return nullptr;
}
bool useDefaultImplementationForConstants() const override { return true; }
@ -311,9 +440,17 @@ public:
WhichDataType which(from_type);
if (which.isDate())
DateTimeAddIntervalImpl<DataTypeDate::FieldType, Transform>::execute(block, arguments, result);
{
DateTimeAddIntervalImpl<DataTypeDate, TransformResultDataType<DataTypeDate>, Transform>::execute(Transform{}, block, arguments, result);
}
else if (which.isDateTime())
DateTimeAddIntervalImpl<DataTypeDateTime::FieldType, Transform>::execute(block, arguments, result);
{
DateTimeAddIntervalImpl<DataTypeDateTime, TransformResultDataType<DataTypeDateTime>, Transform>::execute(Transform{}, block, arguments, result);
}
else if (const auto * datetime64_type = assert_cast<const DataTypeDateTime64 *>(from_type))
{
DateTimeAddIntervalImpl<DataTypeDateTime64, TransformResultDataType<DataTypeDateTime64>, Transform>::execute(Transform{datetime64_type->getScale()}, block, arguments, result);
}
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -1,5 +1,6 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Functions/IFunction.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <Functions/DateTimeTransforms.h>
@ -15,6 +16,25 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
template<class Transform>
struct WithDateTime64Converter : public Transform
{
UInt8 scale;
Transform transform;
explicit WithDateTime64Converter(UInt8 scale_, Transform transform_ = {})
: scale(scale_),
transform(std::move(transform_))
{}
inline auto execute(DataTypeDateTime64::FieldType t, const DateLUTImpl & time_zone) const
{
auto x = DateTime64(t);
auto res = transform.execute(static_cast<UInt32>(DecimalUtils::getWholePart(x, scale)), time_zone);
return res;
}
};
/// See DateTimeTransforms.h
template <typename ToDataType, typename Transform>
@ -67,8 +87,10 @@ public:
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
/// For DateTime, if time zone is specified, attach it to type.
if (std::is_same_v<ToDataType, DataTypeDateTime>)
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
return std::make_shared<ToDataType>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
return std::make_shared<ToDataType>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
else
return std::make_shared<ToDataType>();
}
@ -82,9 +104,16 @@ public:
WhichDataType which(from_type);
if (which.isDate())
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count);
DateTimeTransformImpl<DataTypeDate, ToDataType, Transform>::execute(block, arguments, result, input_rows_count);
else if (which.isDateTime())
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count);
DateTimeTransformImpl<DataTypeDateTime, ToDataType, Transform>::execute(block, arguments, result, input_rows_count);
else if (which.isDateTime64())
{
const auto scale = static_cast<const DataTypeDateTime64 *>(from_type)->getScale();
WithDateTime64Converter<Transform> transformer(scale);
DateTimeTransformImpl<DataTypeDateTime64, ToDataType, WithDateTime64Converter<Transform>>::execute(block, arguments, result, input_rows_count, transformer);
}
else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -103,8 +103,8 @@ Block createBlockWithNestedColumns(const Block & block, const ColumnNumbers & ar
}
void validateArgumentType(const IFunction & func, const DataTypes & arguments,
size_t argument_index, bool (* validator_func)(const IDataType &),
const char * expected_type_description)
size_t argument_index, bool (* validator_func)(const IDataType &),
const char * expected_type_description)
{
if (arguments.size() <= argument_index)
throw Exception("Incorrect number of arguments of function " + func.getName(),
@ -119,6 +119,75 @@ void validateArgumentType(const IFunction & func, const DataTypes & arguments,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
namespace
{
void validateArgumentsImpl(const IFunction & func,
const ColumnsWithTypeAndName & arguments,
size_t argument_offset,
const FunctionArgumentTypeValidators & validators)
{
for (size_t i = 0; i < validators.size(); ++i)
{
const auto argument_index = i + argument_offset;
if (argument_index >= arguments.size())
{
break;
}
const auto & arg = arguments[i + argument_offset];
const auto validator = validators[i];
if (!validator.validator_func(*arg.type))
throw Exception("Illegal type " + arg.type->getName() +
" of " + std::to_string(i) +
" argument of function " + func.getName() +
" expected " + validator.expected_type_description,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
}
void validateFunctionArgumentTypes(const IFunction & func,
const ColumnsWithTypeAndName & arguments,
const FunctionArgumentTypeValidators & mandatory_args,
const FunctionArgumentTypeValidators & optional_args)
{
if (arguments.size() < mandatory_args.size())
{
auto joinArgumentTypes = [](const auto & args, const String sep = ", ") -> String
{
String result;
for (const auto & a : args)
{
using A = std::decay_t<decltype(a)>;
if constexpr (std::is_same_v<A, FunctionArgumentTypeValidator>)
result += a.expected_type_description;
else if constexpr (std::is_same_v<A, ColumnWithTypeAndName>)
result += a.type->getName();
result += sep;
}
if (args.size() != 0)
result.erase(result.end() - sep.length(), result.end());
return result;
};
throw Exception("Incorrect number of arguments of function " + func.getName()
+ " provided " + std::to_string(arguments.size()) + " (" + joinArgumentTypes(arguments) + ")"
+ " expected " + std::to_string(mandatory_args.size()) + (optional_args.size() ? " or " + std::to_string(mandatory_args.size() + optional_args.size()) : "")
+ " (" + joinArgumentTypes(mandatory_args) + (optional_args.size() ? ", [" + joinArgumentTypes(mandatory_args) + "]" : "") + ")",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
validateArgumentsImpl(func, arguments, 0, mandatory_args);
if (optional_args.size())
{
validateArgumentsImpl(func, arguments, mandatory_args.size(), optional_args);
}
}
std::pair<std::vector<const IColumn *>, const ColumnArray::Offset *>
checkAndGetNestedArrayOffset(const IColumn ** columns, size_t num_arguments)
{

View File

@ -90,6 +90,35 @@ void validateArgumentType(const IFunction & func, const DataTypes & arguments,
size_t argument_index, bool (* validator_func)(const IDataType &),
const char * expected_type_description);
// Simple validator that is used in conjunction with validateFunctionArgumentTypes() to check if function arguments are as expected.
struct FunctionArgumentTypeValidator
{
bool (* validator_func)(const IDataType &);
const char * expected_type_description;
};
using FunctionArgumentTypeValidators = std::vector<FunctionArgumentTypeValidator>;
/** Validate that function arguments match specification.
*
* Designed to simplify argument validation
* for functions with variable arguments (e.g. depending on result type or other trait).
* first, checks that mandatory args present and have valid type.
* second, checks optional arguents types, skipping ones that are missing.
*
* Please note that if you have several optional arguments, like f([a, b, c]),
* only these calls are considered valid:
* f(a)
* f(a, b)
* f(a, b, c)
*
* But NOT these: f(a, c), f(b, c)
* In other words you can't skip
*
* If any mandatory arg is missing, throw an exception, with explicit description of expected arguments.
*/
void validateFunctionArgumentTypes(const IFunction & func, const ColumnsWithTypeAndName & arguments, const FunctionArgumentTypeValidators & mandatory_args, const FunctionArgumentTypeValidators & optional_args = {});
/// Checks if a list of array columns have equal offsets. Return a pair of nested columns and offsets if true, otherwise throw.
std::pair<std::vector<const IColumn *>, const ColumnArray::Offset *>
checkAndGetNestedArrayOffset(const IColumn ** columns, size_t num_arguments);

View File

@ -13,6 +13,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeUUID.h>
@ -684,7 +685,7 @@ private:
return true;
};
if (!callOnBasicTypes<true, false, true, false>(left_number, right_number, call))
if (!callOnBasicTypes<true, false, true, true>(left_number, right_number, call))
throw Exception("Wrong call for " + getName() + " with " + col_left.type->getName() + " and " + col_right.type->getName(),
ErrorCodes::LOGICAL_ERROR);
}
@ -1192,9 +1193,10 @@ public:
{
executeTuple(block, result, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count);
}
else if (isDecimal(left_type) || isDecimal(right_type))
else if (isColumnedAsDecimal(left_type) || isColumnedAsDecimal(right_type))
{
if (!allowDecimalComparison(left_type, right_type))
// compare
if (!allowDecimalComparison(left_type, right_type) && !date_and_datetime)
throw Exception("No operation " + getName() + " between " + left_type->getName() + " and " + right_type->getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -46,6 +46,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionToDate>();
factory.registerFunction<FunctionToDateTime>();
factory.registerFunction<FunctionToDateTime64>();
factory.registerFunction<FunctionToUUID>();
factory.registerFunction<FunctionToString>();
factory.registerFunction<FunctionToFixedString>();
@ -65,6 +66,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionToFloat64OrZero>();
factory.registerFunction<FunctionToDateOrZero>();
factory.registerFunction<FunctionToDateTimeOrZero>();
factory.registerFunction<FunctionToDateTime64OrZero>();
factory.registerFunction<FunctionToDecimal32OrZero>();
factory.registerFunction<FunctionToDecimal64OrZero>();
@ -82,6 +84,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionToFloat64OrNull>();
factory.registerFunction<FunctionToDateOrNull>();
factory.registerFunction<FunctionToDateTimeOrNull>();
factory.registerFunction<FunctionToDateTime64OrNull>();
factory.registerFunction<FunctionToDecimal32OrNull>();
factory.registerFunction<FunctionToDecimal64OrNull>();
@ -90,6 +93,9 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionParseDateTimeBestEffort>();
factory.registerFunction<FunctionParseDateTimeBestEffortOrZero>();
factory.registerFunction<FunctionParseDateTimeBestEffortOrNull>();
factory.registerFunction<FunctionParseDateTime64BestEffort>();
factory.registerFunction<FunctionParseDateTime64BestEffortOrZero>();
factory.registerFunction<FunctionParseDateTime64BestEffortOrNull>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalSecond, PositiveMonotonicity>>();
factory.registerFunction<FunctionConvert<DataTypeInterval, NameToIntervalMinute, PositiveMonotonicity>>();

View File

@ -16,6 +16,7 @@
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
@ -106,14 +107,17 @@ struct ConvertImpl
{
const ColumnWithTypeAndName & named_from = block.getByPosition(arguments[0]);
using ColVecFrom = std::conditional_t<IsDecimalNumber<FromFieldType>, ColumnDecimal<FromFieldType>, ColumnVector<FromFieldType>>;
using ColVecTo = std::conditional_t<IsDecimalNumber<ToFieldType>, ColumnDecimal<ToFieldType>, ColumnVector<ToFieldType>>;
using ColVecFrom = typename FromDataType::ColumnType;
using ColVecTo = typename ToDataType::ColumnType;
if constexpr (IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>)
if constexpr ((IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>)
&& !(std::is_same_v<DataTypeDateTime64, FromDataType> || std::is_same_v<DataTypeDateTime64, ToDataType>))
{
if constexpr (!IsDataTypeDecimalOrNumber<FromDataType> || !IsDataTypeDecimalOrNumber<ToDataType>)
{
throw Exception("Illegal column " + named_from.column->getName() + " of first argument of function " + Name::name,
ErrorCodes::ILLEGAL_COLUMN);
}
}
if (const ColVecFrom * col_from = checkAndGetColumn<ColVecFrom>(named_from.column.get()))
@ -155,6 +159,11 @@ struct ConvertImpl
}
};
/** Conversion of DateTime to Date: throw off time component.
*/
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
: DateTimeTransformImpl<DataTypeDateTime, DataTypeDate, ToDateImpl> {};
/** Conversion of Date to DateTime: adding 00:00:00 time component.
*/
@ -166,11 +175,16 @@ struct ToDateTimeImpl
{
return time_zone.fromDayNum(DayNum(d));
}
// no-op conversion from DateTime to DateTime, used in DateTime64 to DateTime conversion.
static inline UInt32 execute(UInt32 d, const DateLUTImpl & /*time_zone*/)
{
return d;
}
};
template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
: DateTimeTransformImpl<UInt16, UInt32, ToDateTimeImpl> {};
: DateTimeTransformImpl<DataTypeDate, DataTypeDateTime, ToDateTimeImpl> {};
/// Implementation of toDate function.
@ -185,11 +199,6 @@ struct ToDateTransform32Or64
}
};
/** Conversion of DateTime to Date: throw off time component.
*/
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
: DateTimeTransformImpl<UInt32, UInt16, ToDateImpl> {};
/** Special case of converting (U)Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to Date.
* If number is less than 65536, then it is treated as DayNum, and if greater or equals, then as unix timestamp.
* It's a bit illogical, as we actually have two functions in one.
@ -198,17 +207,72 @@ template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name
* (otherwise such usage would be frequent mistake).
*/
template <typename Name> struct ConvertImpl<DataTypeUInt32, DataTypeDate, Name>
: DateTimeTransformImpl<UInt32, UInt16, ToDateTransform32Or64<UInt32, UInt16>> {};
: DateTimeTransformImpl<DataTypeUInt32, DataTypeDate, ToDateTransform32Or64<UInt32, UInt16>> {};
template <typename Name> struct ConvertImpl<DataTypeUInt64, DataTypeDate, Name>
: DateTimeTransformImpl<UInt64, UInt16, ToDateTransform32Or64<UInt64, UInt16>> {};
: DateTimeTransformImpl<DataTypeUInt64, DataTypeDate, ToDateTransform32Or64<UInt64, UInt16>> {};
template <typename Name> struct ConvertImpl<DataTypeInt32, DataTypeDate, Name>
: DateTimeTransformImpl<Int32, UInt16, ToDateTransform32Or64<Int32, UInt16>> {};
: DateTimeTransformImpl<DataTypeInt32, DataTypeDate, ToDateTransform32Or64<Int32, UInt16>> {};
template <typename Name> struct ConvertImpl<DataTypeInt64, DataTypeDate, Name>
: DateTimeTransformImpl<Int64, UInt16, ToDateTransform32Or64<Int64, UInt16>> {};
: DateTimeTransformImpl<DataTypeInt64, DataTypeDate, ToDateTransform32Or64<Int64, UInt16>> {};
template <typename Name> struct ConvertImpl<DataTypeFloat32, DataTypeDate, Name>
: DateTimeTransformImpl<Float32, UInt16, ToDateTransform32Or64<Float32, UInt16>> {};
: DateTimeTransformImpl<DataTypeFloat32, DataTypeDate, ToDateTransform32Or64<Float32, UInt16>> {};
template <typename Name> struct ConvertImpl<DataTypeFloat64, DataTypeDate, Name>
: DateTimeTransformImpl<Float64, UInt16, ToDateTransform32Or64<Float64, UInt16>> {};
: DateTimeTransformImpl<DataTypeFloat64, DataTypeDate, ToDateTransform32Or64<Float64, UInt16>> {};
/** Conversion of Date or DateTime to DateTime64: add zero sub-second part.
*/
struct ToDateTime64Transform
{
static constexpr auto name = "toDateTime64";
const DateTime64::NativeType scale_multiplier = 1;
ToDateTime64Transform(UInt32 scale = 0)
: scale_multiplier(DecimalUtils::scaleMultiplier<DateTime64::NativeType>(scale))
{}
inline DateTime64::NativeType execute(UInt16 d, const DateLUTImpl & time_zone) const
{
const auto dt = ToDateTimeImpl::execute(d, time_zone);
return execute(dt, time_zone);
}
inline DateTime64::NativeType execute(UInt32 dt, const DateLUTImpl & /*time_zone*/) const
{
return DecimalUtils::decimalFromComponentsWithMultiplier<DateTime64>(dt, 0, scale_multiplier);
}
};
template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime64, Name>
: DateTimeTransformImpl<DataTypeDate, DataTypeDateTime64, ToDateTime64Transform> {};
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDateTime64, Name>
: DateTimeTransformImpl<DataTypeDateTime, DataTypeDateTime64, ToDateTime64Transform> {};
/** Conversion of DateTime64 to Date or DateTime: discards fractional part.
*/
template <typename Transform>
struct FromDateTime64Transform
{
static constexpr auto name = Transform::name;
const DateTime64::NativeType scale_multiplier = 1;
FromDateTime64Transform(UInt32 scale)
: scale_multiplier(DecimalUtils::scaleMultiplier<DateTime64::NativeType>(scale))
{}
inline auto execute(DateTime64::NativeType dt, const DateLUTImpl & time_zone) const
{
const auto c = DecimalUtils::splitWithScaleMultiplier(DateTime64(dt), scale_multiplier);
return Transform::execute(static_cast<UInt32>(c.whole), time_zone);
}
};
template <typename Name> struct ConvertImpl<DataTypeDateTime64, DataTypeDate, Name>
: DateTimeTransformImpl<DataTypeDateTime64, DataTypeDate, FromDateTime64Transform<ToDateImpl>> {};
template <typename Name> struct ConvertImpl<DataTypeDateTime64, DataTypeDateTime, Name>
: DateTimeTransformImpl<DataTypeDateTime64, DataTypeDateTime, FromDateTime64Transform<ToDateTimeImpl>> {};
/** Transformation of numbers, dates, datetimes to strings: through formatting.
@ -240,6 +304,16 @@ struct FormatImpl<DataTypeDateTime>
}
};
template <>
struct FormatImpl<DataTypeDateTime64>
{
static void execute(const DataTypeDateTime64::FieldType x, WriteBuffer & wb, const DataTypeDateTime64 * type, const DateLUTImpl * time_zone)
{
writeDateTimeText(DateTime64(x), type->getScale(), wb, *time_zone);
}
};
template <typename FieldType>
struct FormatImpl<DataTypeEnum<FieldType>>
{
@ -284,7 +358,7 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
const DateLUTImpl * time_zone = nullptr;
/// For argument of DateTime type, second argument with time zone could be specified.
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime> || std::is_same_v<FromDataType, DataTypeDateTime64>)
time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
if (const auto col_from = checkAndGetColumn<ColVecType>(col_with_type_and_name.column.get()))
@ -300,6 +374,8 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1));
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + vec_from.getScale() + 1));
else
data_to.resize(size * 3); /// Arbitary
@ -448,6 +524,8 @@ struct ConvertThroughParsing
static_assert(std::is_same_v<FromDataType, DataTypeString> || std::is_same_v<FromDataType, DataTypeFixedString>,
"ConvertThroughParsing is only applicable for String or FixedString data types");
static constexpr bool to_datetime64 = std::is_same_v<ToDataType, DataTypeDateTime64>;
using ToFieldType = typename ToDataType::FieldType;
static bool isAllRead(ReadBuffer & in)
@ -471,15 +549,22 @@ struct ConvertThroughParsing
static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count,
Additions additions [[maybe_unused]] = Additions())
{
using ColVecTo = std::conditional_t<IsDecimalNumber<ToFieldType>, ColumnDecimal<ToFieldType>, ColumnVector<ToFieldType>>;
using ColVecTo = typename ToDataType::ColumnType;
const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr;
const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr;
/// For conversion to DateTime type, second argument with time zone could be specified.
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime> || to_datetime64)
{
local_time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
const auto result_type = removeNullable(block.getByPosition(result).type);
// Time zone is already figured out during result type resultion, no need to do it here.
if (const auto dt_col = checkAndGetDataType<ToDataType>(result_type.get()))
local_time_zone = &dt_col->getTimeZone();
else
{
local_time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
}
if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort)
utc_time_zone = &DateLUT::instance("UTC");
@ -505,7 +590,14 @@ struct ConvertThroughParsing
if constexpr (IsDataTypeDecimal<ToDataType>)
{
UInt32 scale = additions;
ToDataType check_bounds_in_ctor(ToDataType::maxPrecision(), scale);
if constexpr (to_datetime64)
{
ToDataType check_bounds_in_ctor(scale, local_time_zone ? local_time_zone->getTimeZone() : String{});
}
else
{
ToDataType check_bounds_in_ctor(ToDataType::maxPrecision(), scale);
}
col_to = ColVecTo::create(size, scale);
}
else
@ -549,13 +641,28 @@ struct ConvertThroughParsing
{
if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort)
{
time_t res;
parseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone);
vec_to[i] = res;
if constexpr (to_datetime64)
{
DateTime64 res = 0;
parseDateTime64BestEffort(res, vec_to.getScale(), read_buffer, *local_time_zone, *utc_time_zone);
vec_to[i] = res;
}
else
{
time_t res;
parseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone);
vec_to[i] = res;
}
}
else
{
if constexpr (IsDataTypeDecimal<ToDataType>)
if constexpr (to_datetime64)
{
DateTime64 value = 0;
readDateTime64Text(value, vec_to.getScale(), read_buffer, *local_time_zone);
vec_to[i] = value;
}
else if constexpr (IsDataTypeDecimal<ToDataType>)
ToDataType::readText(vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
else
parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
@ -570,13 +677,28 @@ struct ConvertThroughParsing
if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort)
{
time_t res;
parsed = tryParseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone);
vec_to[i] = res;
if constexpr (to_datetime64)
{
DateTime64 res = 0;
parsed = tryParseDateTime64BestEffort(res, vec_to.getScale(), read_buffer, *local_time_zone, *utc_time_zone);
vec_to[i] = res;
}
else
{
time_t res;
parsed = tryParseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone);
vec_to[i] = res;
}
}
else
{
if constexpr (IsDataTypeDecimal<ToDataType>)
if constexpr (to_datetime64)
{
DateTime64 value = 0;
parsed = tryReadDateTime64Text(value, vec_to.getScale(), read_buffer, *local_time_zone);
vec_to[i] = value;
}
else if constexpr (IsDataTypeDecimal<ToDataType>)
parsed = ToDataType::tryReadText(vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
else
parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
@ -725,6 +847,7 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
/// Declared early because used below.
struct NameToDate { static constexpr auto name = "toDate"; };
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
struct NameToDateTime64 { static constexpr auto name = "toDateTime64"; };
struct NameToString { static constexpr auto name = "toString"; };
struct NameToDecimal32 { static constexpr auto name = "toDecimal32"; };
struct NameToDecimal64 { static constexpr auto name = "toDecimal64"; };
@ -760,6 +883,8 @@ public:
static constexpr bool to_decimal =
std::is_same_v<Name, NameToDecimal32> || std::is_same_v<Name, NameToDecimal64> || std::is_same_v<Name, NameToDecimal128>;
static constexpr bool to_datetime64 = std::is_same_v<ToDataType, DataTypeDateTime64>;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvert>(); }
String getName() const override
@ -773,16 +898,19 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (to_decimal && arguments.size() != 2)
FunctionArgumentTypeValidators mandatory_args = {{[](const auto &) {return true;}, "ANY TYPE"}};
FunctionArgumentTypeValidators optional_args;
if constexpr (to_decimal || to_datetime64)
{
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
mandatory_args.push_back(FunctionArgumentTypeValidator{&isNativeInteger, "Integer"}); // scale
}
else if (arguments.size() != 1 && arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 1 or 2. Second argument (time zone) is optional only make sense for DateTime.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
else
{
optional_args.push_back(FunctionArgumentTypeValidator{&isString, "String"}); // timezone
}
validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
if constexpr (std::is_same_v<ToDataType, DataTypeInterval>)
{
@ -796,42 +924,31 @@ public:
UInt64 scale = extractToDecimalScale(arguments[1]);
if constexpr (std::is_same_v<Name, NameToDecimal32>)
return createDecimal(9, scale);
return createDecimal<DataTypeDecimal>(9, scale);
else if constexpr (std::is_same_v<Name, NameToDecimal64>)
return createDecimal(18, scale);
return createDecimal<DataTypeDecimal>(18, scale);
else if constexpr (std::is_same_v<Name, NameToDecimal128>)
return createDecimal(38, scale);
return createDecimal<DataTypeDecimal>(38, scale);
throw Exception("Someting wrong with toDecimalNN()", ErrorCodes::LOGICAL_ERROR);
}
else
{
/** Optional second argument with time zone is supported:
* - for functions toDateTime, toUnixTimestamp, toDate;
* - for function toString of DateTime argument.
*/
// Optional second argument with time zone for DateTime.
UInt8 timezone_arg_position = 1;
UInt32 scale [[maybe_unused]] = DataTypeDateTime64::default_scale;
if (arguments.size() == 2)
// DateTime64 requires more arguments: scale and timezone. Since timezone is optional, scale should be first.
if constexpr (to_datetime64)
{
if (!checkAndGetDataType<DataTypeString>(arguments[1].type.get()))
throw Exception("Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
static constexpr bool to_date_or_time = std::is_same_v<Name, NameToDateTime>
|| std::is_same_v<Name, NameToDate>
|| std::is_same_v<Name, NameToUnixTimestamp>;
if (!(to_date_or_time
|| (std::is_same_v<Name, NameToString> && WhichDataType(arguments[0].type).isDateTime())))
{
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
timezone_arg_position += 1;
scale = static_cast<UInt32>(arguments[1].column->get64(0));
}
if (std::is_same_v<ToDataType, DataTypeDateTime>)
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0));
else if constexpr (to_datetime64)
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0));
else
return std::make_shared<ToDataType>();
}
@ -902,17 +1019,32 @@ private:
if constexpr (IsDataTypeDecimal<RightDataType>)
{
if (arguments.size() != 2)
if constexpr (std::is_same_v<RightDataType, DataTypeDateTime64>)
{
// account for optional timezone argument
if (arguments.size() != 2 && arguments.size() != 3)
throw Exception{"Function " + getName() + " expects 2 or 3 arguments for DataTypeDateTime64.",
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
}
else if (arguments.size() != 2)
{
throw Exception{"Function " + getName() + " expects 2 arguments for Decimal.",
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
}
const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
UInt32 scale = extractToDecimalScale(scale_column);
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count, scale);
}
else if constexpr (IsDataTypeDateOrDateTime<RightDataType> && std::is_same_v<LeftDataType, DataTypeDateTime64>)
{
const auto * dt64 = assert_cast<const DataTypeDateTime64 *>(block.getByPosition(arguments[0]).type.get());
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count, dt64->getScale());
}
else
ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count);
return true;
};
@ -1015,15 +1147,23 @@ public:
UInt64 scale = extractToDecimalScale(arguments[1]);
if constexpr (std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>>)
res = createDecimal(9, scale);
res = createDecimal<DataTypeDecimal>(9, scale);
else if constexpr (std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>>)
res = createDecimal(18, scale);
res = createDecimal<DataTypeDecimal>(18, scale);
else if constexpr (std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>)
res = createDecimal(38, scale);
res = createDecimal<DataTypeDecimal>(38, scale);
if (!res)
throw Exception("Someting wrong with toDecimalNNOrZero() or toDecimalNNOrNull()", ErrorCodes::LOGICAL_ERROR);
}
else if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
{
UInt64 scale = DataTypeDateTime64::default_scale;
if (arguments.size() > 1)
scale = extractToDecimalScale(arguments[1]);
const auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0);
res = std::make_shared<DataTypeDateTime64>(scale, timezone);
}
else
res = std::make_shared<ToDataType>();
@ -1038,7 +1178,7 @@ public:
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
bool ok = true;
if constexpr (to_decimal)
if constexpr (to_decimal || std::is_same_v<ToDataType, DataTypeDateTime64>)
{
if (arguments.size() != 2)
throw Exception{"Function " + getName() + " expects 2 arguments for Decimal.", ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
@ -1380,6 +1520,7 @@ using FunctionToFloat32 = FunctionConvert<DataTypeFloat32, NameToFloat32, ToNumb
using FunctionToFloat64 = FunctionConvert<DataTypeFloat64, NameToFloat64, ToNumberMonotonicity<Float64>>;
using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, ToNumberMonotonicity<UInt16>>;
using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToNumberMonotonicity<UInt32>>;
using FunctionToDateTime64 = FunctionConvert<DataTypeDateTime64, NameToDateTime64, UnknownMonotonicity>;
using FunctionToUUID = FunctionConvert<DataTypeUUID, NameToUUID, ToNumberMonotonicity<UInt128>>;
using FunctionToString = FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity>;
using FunctionToUnixTimestamp = FunctionConvert<DataTypeUInt32, NameToUnixTimestamp, ToNumberMonotonicity<UInt32>>;
@ -1402,6 +1543,7 @@ template <> struct FunctionTo<DataTypeFloat32> { using Type = FunctionToFloat32;
template <> struct FunctionTo<DataTypeFloat64> { using Type = FunctionToFloat64; };
template <> struct FunctionTo<DataTypeDate> { using Type = FunctionToDate; };
template <> struct FunctionTo<DataTypeDateTime> { using Type = FunctionToDateTime; };
template <> struct FunctionTo<DataTypeDateTime64> { using Type = FunctionToDateTime64; };
template <> struct FunctionTo<DataTypeUUID> { using Type = FunctionToUUID; };
template <> struct FunctionTo<DataTypeString> { using Type = FunctionToString; };
template <> struct FunctionTo<DataTypeFixedString> { using Type = FunctionToFixedString; };
@ -1426,6 +1568,7 @@ struct NameToFloat32OrZero { static constexpr auto name = "toFloat32OrZero"; };
struct NameToFloat64OrZero { static constexpr auto name = "toFloat64OrZero"; };
struct NameToDateOrZero { static constexpr auto name = "toDateOrZero"; };
struct NameToDateTimeOrZero { static constexpr auto name = "toDateTimeOrZero"; };
struct NameToDateTime64OrZero { static constexpr auto name = "toDateTime64OrZero"; };
struct NameToDecimal32OrZero { static constexpr auto name = "toDecimal32OrZero"; };
struct NameToDecimal64OrZero { static constexpr auto name = "toDecimal64OrZero"; };
struct NameToDecimal128OrZero { static constexpr auto name = "toDecimal128OrZero"; };
@ -1442,6 +1585,7 @@ using FunctionToFloat32OrZero = FunctionConvertFromString<DataTypeFloat32, NameT
using FunctionToFloat64OrZero = FunctionConvertFromString<DataTypeFloat64, NameToFloat64OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToDateOrZero = FunctionConvertFromString<DataTypeDate, NameToDateOrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToDateTimeOrZero = FunctionConvertFromString<DataTypeDateTime, NameToDateTimeOrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToDateTime64OrZero = FunctionConvertFromString<DataTypeDateTime64, NameToDateTime64OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToDecimal32OrZero = FunctionConvertFromString<DataTypeDecimal<Decimal32>, NameToDecimal32OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToDecimal64OrZero = FunctionConvertFromString<DataTypeDecimal<Decimal64>, NameToDecimal64OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToDecimal128OrZero = FunctionConvertFromString<DataTypeDecimal<Decimal128>, NameToDecimal128OrZero, ConvertFromStringExceptionMode::Zero>;
@ -1458,6 +1602,7 @@ struct NameToFloat32OrNull { static constexpr auto name = "toFloat32OrNull"; };
struct NameToFloat64OrNull { static constexpr auto name = "toFloat64OrNull"; };
struct NameToDateOrNull { static constexpr auto name = "toDateOrNull"; };
struct NameToDateTimeOrNull { static constexpr auto name = "toDateTimeOrNull"; };
struct NameToDateTime64OrNull { static constexpr auto name = "toDateTime64OrNull"; };
struct NameToDecimal32OrNull { static constexpr auto name = "toDecimal32OrNull"; };
struct NameToDecimal64OrNull { static constexpr auto name = "toDecimal64OrNull"; };
struct NameToDecimal128OrNull { static constexpr auto name = "toDecimal128OrNull"; };
@ -1474,6 +1619,7 @@ using FunctionToFloat32OrNull = FunctionConvertFromString<DataTypeFloat32, NameT
using FunctionToFloat64OrNull = FunctionConvertFromString<DataTypeFloat64, NameToFloat64OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToDateOrNull = FunctionConvertFromString<DataTypeDate, NameToDateOrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToDateTimeOrNull = FunctionConvertFromString<DataTypeDateTime, NameToDateTimeOrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToDateTime64OrNull = FunctionConvertFromString<DataTypeDateTime64, NameToDateTime64OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToDecimal32OrNull = FunctionConvertFromString<DataTypeDecimal<Decimal32>, NameToDecimal32OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToDecimal64OrNull = FunctionConvertFromString<DataTypeDecimal<Decimal64>, NameToDecimal64OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToDecimal128OrNull = FunctionConvertFromString<DataTypeDecimal<Decimal128>, NameToDecimal128OrNull, ConvertFromStringExceptionMode::Null>;
@ -1481,6 +1627,10 @@ using FunctionToDecimal128OrNull = FunctionConvertFromString<DataTypeDecimal<Dec
struct NameParseDateTimeBestEffort { static constexpr auto name = "parseDateTimeBestEffort"; };
struct NameParseDateTimeBestEffortOrZero { static constexpr auto name = "parseDateTimeBestEffortOrZero"; };
struct NameParseDateTimeBestEffortOrNull { static constexpr auto name = "parseDateTimeBestEffortOrNull"; };
struct NameParseDateTime64BestEffort { static constexpr auto name = "parseDateTime64BestEffort"; };
struct NameParseDateTime64BestEffortOrZero { static constexpr auto name = "parseDateTime64BestEffortOrZero"; };
struct NameParseDateTime64BestEffortOrNull { static constexpr auto name = "parseDateTime64BestEffortOrNull"; };
using FunctionParseDateTimeBestEffort = FunctionConvertFromString<
DataTypeDateTime, NameParseDateTimeBestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>;
@ -1489,6 +1639,12 @@ using FunctionParseDateTimeBestEffortOrZero = FunctionConvertFromString<
using FunctionParseDateTimeBestEffortOrNull = FunctionConvertFromString<
DataTypeDateTime, NameParseDateTimeBestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTime64BestEffort = FunctionConvertFromString<
DataTypeDateTime64, NameParseDateTime64BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTime64BestEffortOrZero = FunctionConvertFromString<
DataTypeDateTime64, NameParseDateTime64BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTime64BestEffortOrNull = FunctionConvertFromString<
DataTypeDateTime64, NameParseDateTime64BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>;
class PreparedFunctionCast : public PreparedFunctionImpl
{
@ -1639,13 +1795,11 @@ private:
};
}
template <typename FieldType>
WrapperType createDecimalWrapper(const DataTypePtr & from_type, const DataTypeDecimal<FieldType> * to_type) const
template <typename ToDataType>
std::enable_if_t<IsDataTypeDecimal<ToDataType>, WrapperType>
createDecimalWrapper(const DataTypePtr & from_type, const ToDataType * to_type) const
{
using ToDataType = DataTypeDecimal<FieldType>;
TypeIndex type_index = from_type->getTypeId();
UInt32 precision = to_type->getPrecision();
UInt32 scale = to_type->getScale();
WhichDataType which(type_index);
@ -1659,7 +1813,7 @@ private:
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
ErrorCodes::CANNOT_CONVERT_TYPE};
return [type_index, precision, scale] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
return [type_index, scale, to_type] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
auto res = callOnIndexAndDataType<ToDataType>(type_index, [&](const auto & types) -> bool
{
@ -1674,8 +1828,7 @@ private:
/// Additionally check if callOnIndexAndDataType wasn't called at all.
if (!res)
{
auto to = DataTypeDecimal<FieldType>(precision, scale);
throw Exception{"Conversion from " + std::string(getTypeName(type_index)) + " to " + to.getName() +
throw Exception{"Conversion from " + std::string(getTypeName(type_index)) + " to " + to_type->getName() +
" is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
}
};
@ -2140,7 +2293,8 @@ private:
if constexpr (
std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>> ||
std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> ||
std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>)
std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>> ||
std::is_same_v<ToDataType, DataTypeDateTime64>)
{
ret = createDecimalWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()));
return true;

View File

@ -5,10 +5,12 @@
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Common/typeid_cast.h>
#include <Common/memcpySmall.h>
#include <Functions/IFunction.h>
@ -147,7 +149,6 @@ public:
}
};
template <typename ToDataType, typename Name>
class FunctionReinterpretStringAs : public IFunction
{
@ -156,6 +157,7 @@ public:
static FunctionPtr create(const Context &) { return std::make_shared<FunctionReinterpretStringAs>(); }
using ToFieldType = typename ToDataType::FieldType;
using ColumnType = typename ToDataType::ColumnType;
String getName() const override
{
@ -179,12 +181,12 @@ public:
{
if (const ColumnString * col_from = typeid_cast<const ColumnString *>(block.getByPosition(arguments[0]).column.get()))
{
auto col_res = ColumnVector<ToFieldType>::create();
auto col_res = ColumnType::create();
const ColumnString::Chars & data_from = col_from->getChars();
const ColumnString::Offsets & offsets_from = col_from->getOffsets();
size_t size = offsets_from.size();
typename ColumnVector<ToFieldType>::Container & vec_res = col_res->getData();
typename ColumnType::Container & vec_res = col_res->getData();
vec_res.resize(size);
size_t offset = 0;

View File

@ -6,6 +6,7 @@
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Columns/ColumnVector.h>
#include <Interpreters/castColumn.h>
#include "IFunction.h"

View File

@ -36,7 +36,7 @@ struct ArrayCumSumImpl
if (which.isDecimal())
{
UInt32 scale = getDecimalScale(*expression_return);
DataTypePtr nested = std::make_shared<DataTypeDecimal<Decimal128>>(maxDecimalPrecision<Decimal128>(), scale);
DataTypePtr nested = std::make_shared<DataTypeDecimal<Decimal128>>(DecimalUtils::maxPrecision<Decimal128>(), scale);
return std::make_shared<DataTypeArray>(nested);
}

View File

@ -39,7 +39,7 @@ struct ArrayCumSumNonNegativeImpl
if (which.isDecimal())
{
UInt32 scale = getDecimalScale(*expression_return);
DataTypePtr nested = std::make_shared<DataTypeDecimal<Decimal128>>(maxDecimalPrecision<Decimal128>(), scale);
DataTypePtr nested = std::make_shared<DataTypeDecimal<Decimal128>>(DecimalUtils::maxPrecision<Decimal128>(), scale);
return std::make_shared<DataTypeArray>(nested);
}

View File

@ -7,6 +7,7 @@
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/getMostSubtype.h>

View File

@ -36,7 +36,7 @@ struct ArraySumImpl
if (which.isDecimal())
{
UInt32 scale = getDecimalScale(*expression_return);
return std::make_shared<DataTypeDecimal<Decimal128>>(maxDecimalPrecision<Decimal128>(), scale);
return std::make_shared<DataTypeDecimal<Decimal128>>(DecimalUtils::maxPrecision<Decimal128>(), scale);
}
throw Exception("arraySum cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -4,6 +4,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h>

View File

@ -1,4 +1,5 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>

View File

@ -2,6 +2,7 @@
#include <Functions/FunctionHelpers.h>
#include <Core/Block.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Columns/ColumnString.h>
#include <common/DateLUT.h>
@ -43,6 +44,8 @@ std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndNam
/// If time zone is attached to an argument of type DateTime.
if (const DataTypeDateTime * type = checkAndGetDataType<DataTypeDateTime>(arguments[datetime_arg_num].type.get()))
return type->getTimeZone().getTimeZone();
if (const DataTypeDateTime64 * type = checkAndGetDataType<DataTypeDateTime64>(arguments[datetime_arg_num].type.get()))
return type->getTimeZone().getTimeZone();
return {};
}

View File

@ -1,4 +1,7 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Columns/ColumnString.h>
#include <Functions/IFunction.h>
@ -11,6 +14,7 @@
#include <common/DateLUTImpl.h>
#include <common/find_symbols.h>
#include <Core/DecimalFunctions.h>
#include <type_traits>
@ -27,6 +31,16 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
namespace
{
// in private namespace to avoid GCC 9 error: "explicit specialization in non-namespace scope"
template <typename DataType> struct ActionaValueTypeMap {};
template <> struct ActionaValueTypeMap<DataTypeDate> { using ActionValueType = UInt16; };
template <> struct ActionaValueTypeMap<DataTypeDateTime> { using ActionValueType = UInt32; };
// TODO(vnemkov): once there is support for Int64 in LUT, make that Int64.
// TODO(vnemkov): to add sub-second format instruction, make that DateTime64 and do some math in Action<T>.
template <> struct ActionaValueTypeMap<DataTypeDateTime64> { using ActionValueType = UInt32; };
}
/** formatDateTime(time, 'pattern')
* Performs formatting of time, according to provided pattern.
@ -270,86 +284,105 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (!executeType<UInt32>(block, arguments, result)
&& !executeType<UInt16>(block, arguments, result))
if (!executeType<DataTypeDate>(block, arguments, result)
&& !executeType<DataTypeDateTime>(block, arguments, result)
&& !executeType<DataTypeDateTime64>(block, arguments, result))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of function " + getName() + ", must be Date or DateTime",
ErrorCodes::ILLEGAL_COLUMN);
}
template <typename T>
template <typename DataType>
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (auto * times = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
auto * times = checkAndGetColumn<typename DataType::ColumnType>(block.getByPosition(arguments[0]).column.get());
if (!times)
return false;
const ColumnConst * pattern_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!pattern_column)
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
+ " of second ('format') argument of function " + getName()
+ ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
String pattern = pattern_column->getValue<String>();
using T = typename ActionaValueTypeMap<DataType>::ActionValueType;
std::vector<Action<T>> instructions;
String pattern_to_fill = parsePattern(pattern, instructions);
size_t result_size = pattern_to_fill.size();
const DateLUTImpl * time_zone_tmp = nullptr;
if (arguments.size() == 3)
time_zone_tmp = &extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
else
time_zone_tmp = &DateLUT::instance();
const DateLUTImpl & time_zone = *time_zone_tmp;
const auto & vec = times->getData();
UInt32 scale [[maybe_unused]] = 0;
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
{
const ColumnConst * pattern_column = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
scale = vec.getScale();
}
if (!pattern_column)
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
+ " of second ('format') argument of function " + getName()
+ ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
auto col_res = ColumnString::create();
auto & dst_data = col_res->getChars();
auto & dst_offsets = col_res->getOffsets();
dst_data.resize(vec.size() * (result_size + 1));
dst_offsets.resize(vec.size());
String pattern = pattern_column->getValue<String>();
/// Fill result with literals.
{
UInt8 * begin = dst_data.data();
UInt8 * end = begin + dst_data.size();
UInt8 * pos = begin;
std::vector<Action<T>> instructions;
String pattern_to_fill = parsePattern(pattern, instructions);
size_t result_size = pattern_to_fill.size();
const DateLUTImpl * time_zone_tmp = nullptr;
if (arguments.size() == 3)
time_zone_tmp = &extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
else
time_zone_tmp = &DateLUT::instance();
const DateLUTImpl & time_zone = *time_zone_tmp;
const typename ColumnVector<T>::Container & vec = times->getData();
auto col_res = ColumnString::create();
auto & dst_data = col_res->getChars();
auto & dst_offsets = col_res->getOffsets();
dst_data.resize(vec.size() * (result_size + 1));
dst_offsets.resize(vec.size());
/// Fill result with literals.
if (pos < end)
{
UInt8 * begin = dst_data.data();
UInt8 * end = begin + dst_data.size();
UInt8 * pos = begin;
if (pos < end)
{
memcpy(pos, pattern_to_fill.data(), result_size + 1); /// With zero terminator.
pos += result_size + 1;
}
/// Fill by copying exponential growing ranges.
while (pos < end)
{
size_t bytes_to_copy = std::min(pos - begin, end - pos);
memcpy(pos, begin, bytes_to_copy);
pos += bytes_to_copy;
}
memcpy(pos, pattern_to_fill.data(), result_size + 1); /// With zero terminator.
pos += result_size + 1;
}
auto begin = reinterpret_cast<char *>(dst_data.data());
auto pos = begin;
/// Fill by copying exponential growing ranges.
while (pos < end)
{
size_t bytes_to_copy = std::min(pos - begin, end - pos);
memcpy(pos, begin, bytes_to_copy);
pos += bytes_to_copy;
}
}
for (size_t i = 0; i < vec.size(); ++i)
auto begin = reinterpret_cast<char *>(dst_data.data());
auto pos = begin;
for (size_t i = 0; i < vec.size(); ++i)
{
if constexpr (std::is_same_v<DataType, DataTypeDateTime64>)
{
for (auto & instruction : instructions)
{
// since right now LUT does not support Int64-values and not format instructions for subsecond parts,
// treat DatTime64 values just as DateTime values by ignoring fractional and casting to UInt32.
const auto c = DecimalUtils::split(vec[i], scale);
instruction.perform(pos, static_cast<UInt32>(c.whole), time_zone);
}
}
else
{
for (auto & instruction : instructions)
instruction.perform(pos, vec[i], time_zone);
dst_offsets[i] = pos - begin;
}
dst_data.resize(pos - begin);
block.getByPosition(result).column = std::move(col_res);
return true;
dst_offsets[i] = pos - begin;
}
return false;
dst_data.resize(pos - begin);
block.getByPosition(result).column = std::move(col_res);
return true;
}
template <typename T>

View File

@ -1,9 +1,12 @@
#include <DataTypes/DataTypeDateTime.h>
#include <Core/DecimalFunctions.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Core/Field.h>
#include <time.h>
namespace DB

View File

@ -0,0 +1,98 @@
#include <DataTypes/DataTypeDateTime64.h>
#include <Core/DecimalFunctions.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Common/assert_cast.h>
#include <time.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
DateTime64::NativeType nowSubsecond(UInt32 scale)
{
const Int32 fractional_scale = 9;
timespec spec;
clock_gettime(CLOCK_REALTIME, &spec);
DecimalUtils::DecimalComponents<DateTime64::NativeType> components{spec.tv_sec, spec.tv_nsec};
// clock_gettime produces subsecond part in nanoseconds, but decimalFromComponents fractional is scale-dependent.
// Andjust fractional to scale, e.g. for 123456789 nanoseconds:
// if scale is 6 (miscoseconds) => divide by 9 - 6 = 3 to get 123456 microseconds
// if scale is 12 (picoseconds) => multiply by abs(9 - 12) = 3 to get 123456789000 picoseconds
const auto adjust_scale = fractional_scale - static_cast<Int32>(scale);
if (adjust_scale < 0)
components.fractional *= intExp10(std::abs(adjust_scale));
else if (adjust_scale > 0)
components.fractional /= intExp10(adjust_scale);
return DecimalUtils::decimalFromComponents<DateTime64>(components, scale).value;
}
class FunctionNow64 : public IFunction
{
public:
static constexpr auto name = "now64";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionNow64>(); }
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return ColumnNumbers{0}; }
bool isDeterministic() const override { return false; }
// Return type depends on argument value.
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
UInt32 scale = DataTypeDateTime64::default_scale;
// Type check is similar to the validateArgumentType, trying to keep error codes and messages as close to the said function as possible.
if (arguments.size() >= 1)
{
const auto & argument = arguments[0];
if (!isInteger(argument.type) || !isColumnConst(*argument.column))
throw Exception("Illegal type " + argument.type->getName() +
" of 0" +
" argument of function " + getName() +
". Expected const integer.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
scale = argument.column->get64(0);
}
return std::make_shared<DataTypeDateTime64>(scale);
}
void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result, size_t input_rows_count) override
{
auto & result_col = block.getByPosition(result);
UInt32 scale = DataTypeDateTime64::default_scale;
if (const auto * dt64 = assert_cast<const DataTypeDateTime64 *>(result_col.type.get()))
{
scale = dt64->getScale();
}
result_col.column = DataTypeDateTime64(scale).createColumnConst(input_rows_count, nowSubsecond(scale));
}
};
void registerFunctionNow64(FunctionFactory & factory)
{
factory.registerFunction<FunctionNow64>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -37,6 +37,7 @@ void registerFunctionToRelativeMinuteNum(FunctionFactory &);
void registerFunctionToRelativeSecondNum(FunctionFactory &);
void registerFunctionToTime(FunctionFactory &);
void registerFunctionNow(FunctionFactory &);
void registerFunctionNow64(FunctionFactory &);
void registerFunctionToday(FunctionFactory &);
void registerFunctionYesterday(FunctionFactory &);
void registerFunctionTimeSlot(FunctionFactory &);
@ -101,6 +102,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionToRelativeSecondNum(factory);
registerFunctionToTime(factory);
registerFunctionNow(factory);
registerFunctionNow64(factory);
registerFunctionToday(factory);
registerFunctionYesterday(factory);
registerFunctionTimeSlot(factory);

View File

@ -5,6 +5,7 @@
#include <Common/assert_cast.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/NumberTraits.h>
#include <DataTypes/DataTypeNullable.h>

View File

@ -1,5 +1,6 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>

View File

@ -1,6 +1,8 @@
#include <common/DateLUTImpl.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeInterval.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionFactory.h>
@ -125,7 +127,6 @@ namespace
return time_zone.toStartOfSecondInterval(t, seconds);
}
};
}
@ -233,26 +234,34 @@ private:
ColumnPtr dispatchForColumns(
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DateLUTImpl & time_zone)
{
if (WhichDataType(time_column.type.get()).isDateTime())
const auto & from_datatype = *time_column.type.get();
const auto which_type = WhichDataType(from_datatype);
if (which_type.isDateTime())
{
const auto * time_column_vec = checkAndGetColumn<ColumnUInt32>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(*time_column_vec, interval_column, time_zone);
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime&>(from_datatype), *time_column_vec, interval_column, time_zone);
}
if (WhichDataType(time_column.type.get()).isDate())
if (which_type.isDate())
{
const auto * time_column_vec = checkAndGetColumn<ColumnUInt16>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(*time_column_vec, interval_column, time_zone);
return dispatchForIntervalColumn(assert_cast<const DataTypeDate&>(from_datatype), *time_column_vec, interval_column, time_zone);
}
if (which_type.isDateTime64())
{
const auto * time_column_vec = checkAndGetColumn<DataTypeDateTime64::ColumnType>(time_column.column.get());
if (time_column_vec)
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime64&>(from_datatype), *time_column_vec, interval_column, time_zone);
}
throw Exception(
"Illegal column for first argument of function " + getName() + ". Must contain dates or dates with time",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
template <typename FromType>
template <typename ColumnType, typename FromDataType>
ColumnPtr dispatchForIntervalColumn(
const ColumnVector<FromType> & time_column, const ColumnWithTypeAndName & interval_column, const DateLUTImpl & time_zone)
const FromDataType & from, const ColumnType & time_column, const ColumnWithTypeAndName & interval_column, const DateLUTImpl & time_zone)
{
const auto * interval_type = checkAndGetDataType<DataTypeInterval>(interval_column.type.get());
if (!interval_type)
@ -270,36 +279,47 @@ private:
switch (interval_type->getKind())
{
case IntervalKind::Second:
return execute<FromType, UInt32, IntervalKind::Second>(time_column, num_units, time_zone);
return execute<FromDataType, UInt32, IntervalKind::Second>(from, time_column, num_units, time_zone);
case IntervalKind::Minute:
return execute<FromType, UInt32, IntervalKind::Minute>(time_column, num_units, time_zone);
return execute<FromDataType, UInt32, IntervalKind::Minute>(from, time_column, num_units, time_zone);
case IntervalKind::Hour:
return execute<FromType, UInt32, IntervalKind::Hour>(time_column, num_units, time_zone);
return execute<FromDataType, UInt32, IntervalKind::Hour>(from, time_column, num_units, time_zone);
case IntervalKind::Day:
return execute<FromType, UInt32, IntervalKind::Day>(time_column, num_units, time_zone);
return execute<FromDataType, UInt32, IntervalKind::Day>(from, time_column, num_units, time_zone);
case IntervalKind::Week:
return execute<FromType, UInt16, IntervalKind::Week>(time_column, num_units, time_zone);
return execute<FromDataType, UInt16, IntervalKind::Week>(from, time_column, num_units, time_zone);
case IntervalKind::Month:
return execute<FromType, UInt16, IntervalKind::Month>(time_column, num_units, time_zone);
return execute<FromDataType, UInt16, IntervalKind::Month>(from, time_column, num_units, time_zone);
case IntervalKind::Quarter:
return execute<FromType, UInt16, IntervalKind::Quarter>(time_column, num_units, time_zone);
return execute<FromDataType, UInt16, IntervalKind::Quarter>(from, time_column, num_units, time_zone);
case IntervalKind::Year:
return execute<FromType, UInt16, IntervalKind::Year>(time_column, num_units, time_zone);
return execute<FromDataType, UInt16, IntervalKind::Year>(from, time_column, num_units, time_zone);
}
__builtin_unreachable();
}
template <typename FromType, typename ToType, IntervalKind::Kind unit>
ColumnPtr execute(const ColumnVector<FromType> & time_column, UInt64 num_units, const DateLUTImpl & time_zone)
template <typename FromDataType, typename ToType, IntervalKind::Kind unit, typename ColumnType>
ColumnPtr execute(const FromDataType & from_datatype, const ColumnType & time_column, UInt64 num_units, const DateLUTImpl & time_zone)
{
const auto & time_data = time_column.getData();
size_t size = time_column.size();
auto result = ColumnVector<ToType>::create();
auto & result_data = result->getData();
result_data.resize(size);
for (size_t i = 0; i != size; ++i)
result_data[i] = Transform<unit>::execute(time_data[i], num_units, time_zone);
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
{
const auto transform = DateTime64BasicTransformWrapper<Transform<unit>>{from_datatype.getScale()};
for (size_t i = 0; i != size; ++i)
result_data[i] = transform.execute(time_data[i], num_units, time_zone);
}
else
{
for (size_t i = 0; i != size; ++i)
result_data[i] = Transform<unit>::execute(time_data[i], num_units, time_zone);
}
return result;
}
};

View File

@ -1,4 +1,5 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <Core/Field.h>
@ -7,6 +8,7 @@
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <IO/WriteHelpers.h>
#include <Common/assert_cast.h>
namespace DB
@ -39,12 +41,17 @@ public:
+ toString(arguments.size()) + ", should be 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!WhichDataType(arguments[0].type).isDateTime())
const auto which_type = WhichDataType(arguments[0].type);
if (!which_type.isDateTime() && !which_type.isDateTime64())
throw Exception{"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() +
". Should be DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
". Should be DateTime or DateTime64", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
String time_zone_name = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0);
return std::make_shared<DataTypeDateTime>(time_zone_name);
if (which_type.isDateTime())
return std::make_shared<DataTypeDateTime>(time_zone_name);
const auto * date_time64 = assert_cast<const DataTypeDateTime64 *>(arguments[0].type.get());
return std::make_shared<DataTypeDateTime64>(date_time64->getScale(), time_zone_name);
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override

View File

@ -12,14 +12,17 @@
#include <common/LocalDate.h>
#include <common/LocalDateTime.h>
#include <common/StringRef.h>
#include <common/arithmeticOverflow.h>
#include <Core/Types.h>
#include <Core/DecimalFunctions.h>
#include <Core/UUID.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Arena.h>
#include <Common/UInt128.h>
#include <Common/intExp.h>
#include <Formats/FormatSettings.h>
@ -29,6 +32,8 @@
#include <IO/VarInt.h>
#include <IO/ZlibInflatingReadBuffer.h>
#include <DataTypes/DataTypeDateTime.h>
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wdouble-promotion"
@ -252,7 +257,13 @@ inline void readBoolTextWord(bool & x, ReadBuffer & buf)
}
}
template <typename T, typename ReturnType = void>
enum class ReadIntTextCheckOverflow
{
DO_NOT_CHECK_OVERFLOW,
CHECK_OVERFLOW,
};
template <typename T, typename ReturnType = void, ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::DO_NOT_CHECK_OVERFLOW>
ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
@ -267,6 +278,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
return ReturnType(false);
}
const size_t initial_pos = buf.count();
while (!buf.eof())
{
switch (*buf.position())
@ -274,7 +286,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
case '+':
break;
case '-':
if (is_signed_v<T>)
if constexpr (is_signed_v<T>)
negative = true;
else
{
@ -294,30 +306,48 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
case '7': [[fallthrough]];
case '8': [[fallthrough]];
case '9':
if constexpr (check_overflow == ReadIntTextCheckOverflow::CHECK_OVERFLOW)
{
// perform relativelly slow overflow check only when number of decimal digits so far is close to the max for given type.
if (buf.count() - initial_pos >= std::numeric_limits<T>::max_digits10)
{
if (common::mulOverflow(res, static_cast<decltype(res)>(10), res)
|| common::addOverflow(res, static_cast<decltype(res)>(*buf.position() - '0'), res))
return ReturnType(false);
break;
}
}
res *= 10;
res += *buf.position() - '0';
break;
default:
x = negative ? -res : res;
return ReturnType(true);
goto end;
}
++buf.position();
}
end:
x = negative ? -res : res;
return ReturnType(true);
}
template <typename T>
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::DO_NOT_CHECK_OVERFLOW, typename T>
void readIntText(T & x, ReadBuffer & buf)
{
readIntTextImpl<T, void>(x, buf);
readIntTextImpl<T, void, check_overflow>(x, buf);
}
template <typename T>
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::CHECK_OVERFLOW, typename T>
bool tryReadIntText(T & x, ReadBuffer & buf)
{
return readIntTextImpl<T, bool>(x, buf);
return readIntTextImpl<T, bool, check_overflow>(x, buf);
}
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::DO_NOT_CHECK_OVERFLOW, typename T>
void readIntText(Decimal<T> & x, ReadBuffer & buf)
{
readIntText<check_overflow>(x.value, buf);
}
/** More efficient variant (about 1.5 times on real dataset).
@ -617,22 +647,74 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons
}
else
/// Why not readIntTextUnsafe? Because for needs of AdFox, parsing of unix timestamp with leading zeros is supported: 000...NNNN.
return readIntTextImpl<time_t, ReturnType>(datetime, buf);
return readIntTextImpl<time_t, ReturnType, ReadIntTextCheckOverflow::CHECK_OVERFLOW>(datetime, buf);
}
else
return readDateTimeTextFallback<ReturnType>(datetime, buf, date_lut);
}
template <typename ReturnType>
inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut)
{
time_t whole;
if (!readDateTimeTextImpl<bool>(whole, buf, date_lut))
{
return ReturnType(false);
}
DB::DecimalUtils::DecimalComponents<DateTime64::NativeType> c{static_cast<DateTime64::NativeType>(whole), 0};
if (!buf.eof() && *buf.position() == '.')
{
buf.ignore(1); // skip separator
const auto pos_before_fractional = buf.count();
if (!tryReadIntText<ReadIntTextCheckOverflow::CHECK_OVERFLOW>(c.fractional, buf))
{
return ReturnType(false);
}
// Adjust fractional part to the scale, since decimalFromComponents knows nothing
// about convention of ommiting trailing zero on fractional part
// and assumes that fractional part value is less than 10^scale.
// If scale is 3, but we read '12', promote fractional part to '120'.
// And vice versa: if we read '1234', denote it to '123'.
const auto fractional_length = static_cast<Int32>(buf.count() - pos_before_fractional);
if (const auto adjust_scale = static_cast<Int32>(scale) - fractional_length; adjust_scale > 0)
{
c.fractional *= common::exp10_i64(adjust_scale);
}
else if (adjust_scale < 0)
{
c.fractional /= common::exp10_i64(-1 * adjust_scale);
}
}
datetime64 = DecimalUtils::decimalFromComponents<DateTime64>(c, scale);
return ReturnType(true);
}
inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
{
readDateTimeTextImpl<void>(datetime, buf, date_lut);
}
inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
{
readDateTimeTextImpl<void>(datetime64, scale, buf, date_lut);
}
inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
{
return readDateTimeTextImpl<bool>(datetime, buf, date_lut);
}
inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
{
return readDateTimeTextImpl<bool>(datetime64, scale, buf, date_lut);
}
inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf)
{
char s[19];
@ -858,11 +940,11 @@ void readAndThrowException(ReadBuffer & buf, const String & additional_message =
/** Helper function for implementation.
*/
template <typename T>
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::CHECK_OVERFLOW, typename T>
static inline const char * tryReadIntText(T & x, const char * pos, const char * end)
{
ReadBufferFromMemory in(pos, end - pos);
tryReadIntText(x, in);
tryReadIntText<check_overflow>(x, in);
return pos + in.count();
}

View File

@ -12,6 +12,7 @@
#include <common/find_symbols.h>
#include <common/StringRef.h>
#include <Core/DecimalFunctions.h>
#include <Core/Types.h>
#include <Core/UUID.h>
@ -29,7 +30,6 @@
#include <Formats/FormatSettings.h>
namespace DB
{
@ -694,6 +694,46 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTI
date_lut.toHour(datetime), date_lut.toMinute(datetime), date_lut.toSecond(datetime)), buf);
}
/// In the format YYYY-MM-DD HH:MM:SS.NNNNNNNNN, according to the specified time zone.
template <char date_delimeter = '-', char time_delimeter = ':', char between_date_time_delimiter = ' ', char fractional_time_delimiter = '.'>
inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
{
static constexpr UInt32 MaxScale = DecimalUtils::maxPrecision<DateTime64>();
scale = scale > MaxScale ? MaxScale : scale;
if (unlikely(!datetime64))
{
static const char s[] =
{
'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0',
between_date_time_delimiter,
'0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0',
fractional_time_delimiter,
// Exactly MaxScale zeroes
'0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0'
};
buf.write(s, sizeof(s) - (MaxScale - scale));
return;
}
auto c = DecimalUtils::split(datetime64, scale);
const auto & values = date_lut.getValues(c.whole);
writeDateTimeText<date_delimeter, time_delimeter, between_date_time_delimiter>(
LocalDateTime(values.year, values.month, values.day_of_month,
date_lut.toHour(c.whole), date_lut.toMinute(c.whole), date_lut.toSecond(c.whole)), buf);
if (scale > 0)
{
buf.write(fractional_time_delimiter);
char data[20] = {'0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0', '0'};
static_assert(sizeof(data) >= MaxScale);
auto fractional = c.fractional;
for (Int32 pos = scale - 1; pos >= 0 && fractional; --pos, fractional /= DateTime64(10))
data[pos] += fractional % DateTime64(10);
writeString(&data[0], static_cast<size_t>(scale), buf);
}
}
/// In the RFC 1123 format: "Tue, 03 Dec 2019 00:11:50 GMT". You must provide GMT DateLUT.
/// This is needed for HTTP requests.
@ -772,9 +812,7 @@ void writeText(Decimal<T> value, UInt32 scale, WriteBuffer & ostr)
writeChar('-', ostr); /// avoid crop leading minus when whole part is zero
}
T whole_part = value;
if (scale)
whole_part = value / Decimal<T>::getScaleMultiplier(scale);
const T whole_part = DecimalUtils::getWholePart(value, scale);
writeIntText(whole_part, ostr);
if (scale)

View File

@ -6,6 +6,7 @@
#include <IO/WriteHelpers.h>
#include <IO/parseDateTimeBestEffort.h>
#include <limits>
namespace DB
{
@ -64,9 +65,32 @@ inline void readDecimalNumber(T & res, const char * src)
readDecimalNumberImpl<num_digits - 1, 1>(res, src);
}
template <typename T>
inline void readDecimalNumber(T & res, size_t num_digits, const char * src)
{
#define READ_DECIMAL_NUMBER(N) res *= common::exp10_i32(N); readDecimalNumber<N>(res, src); src += N; num_digits -= N; break
while (num_digits)
{
switch (num_digits)
{
case 3: READ_DECIMAL_NUMBER(3); break;
case 2: READ_DECIMAL_NUMBER(2); break;
case 1: READ_DECIMAL_NUMBER(1); break;
default: READ_DECIMAL_NUMBER(4);
}
}
#undef DECIMAL_NUMBER_CASE
}
struct DateTimeSubsecondPart
{
Int64 value;
UInt8 digits;
};
template <typename ReturnType>
ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone)
ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone, DateTimeSubsecondPart * fractional = nullptr)
{
auto on_error = [](const std::string & message [[maybe_unused]], int code [[maybe_unused]])
{
@ -113,7 +137,7 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date
while (!in.eof())
{
char digits[14];
char digits[std::numeric_limits<UInt64>::digits10];
size_t num_digits = 0;
@ -358,9 +382,18 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date
return on_error("Cannot read DateTime: unexpected point symbol", ErrorCodes::CANNOT_PARSE_DATETIME);
++in.position();
num_digits = readDigits(digits, sizeof(digits), in);
if (fractional)
{
using FractionalType = typename std::decay<decltype(fractional->value)>::type;
// Reading more decimal digits than fits into FractionalType would case an
// overflow, so it is better to skip all digits from the right side that do not
// fit into result type. To provide less precise value rather than bogus one.
num_digits = std::min(static_cast<size_t>(std::numeric_limits<FractionalType>::digits10), num_digits);
/// Just ignore fractional part of second.
readDigits(digits, sizeof(digits), in);
fractional->digits = num_digits;
readDecimalNumber(fractional->value, num_digits, digits);
}
}
else if (c == '+' || c == '-')
{
@ -517,6 +550,28 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date
return ReturnType(true);
}
template <typename ReturnType>
ReturnType parseDateTime64BestEffortImpl(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone)
{
time_t whole;
DateTimeSubsecondPart subsecond = {0, 0}; // needs to be explicitly initialized sine it could be missing from input string
if (!parseDateTimeBestEffortImpl<bool>(whole, in, local_time_zone, utc_time_zone, &subsecond))
return ReturnType(false);
DateTime64::NativeType fractional = subsecond.value;
if (scale < subsecond.digits)
{
fractional /= common::exp10_i64(subsecond.digits - scale);
}
else if (scale > subsecond.digits)
{
fractional *= common::exp10_i64(scale - subsecond.digits);
}
res = DecimalUtils::decimalFromComponents<DateTime64>(whole, fractional, scale);
return ReturnType(true);
}
}
#if defined(__PPC__)
@ -535,4 +590,14 @@ bool tryParseDateTimeBestEffort(time_t & res, ReadBuffer & in, const DateLUTImpl
return parseDateTimeBestEffortImpl<bool>(res, in, local_time_zone, utc_time_zone);
}
void parseDateTime64BestEffort(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone)
{
return parseDateTime64BestEffortImpl<void>(res, scale, in, local_time_zone, utc_time_zone);
}
bool tryParseDateTime64BestEffort(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone)
{
return parseDateTime64BestEffortImpl<bool>(res, scale, in, local_time_zone, utc_time_zone);
}
}

View File

@ -1,6 +1,8 @@
#include <stddef.h>
#include <time.h>
#include <Core/Types.h>
class DateLUTImpl;
namespace DB
@ -55,5 +57,7 @@ class ReadBuffer;
void parseDateTimeBestEffort(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);
bool tryParseDateTimeBestEffort(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);
void parseDateTime64BestEffort(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);
bool tryParseDateTime64BestEffort(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);
}

View File

@ -0,0 +1,185 @@
#include <gtest/gtest.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/parseDateTimeBestEffort.h>
#include <Common/PODArray.h>
/** Test formatting and parsing predefined DateTime64 values to/from string
*/
using namespace DB;
struct DateTime64StringsTestParam
{
const std::string_view comment;
const std::string_view string;
DateTime64 dt64;
UInt32 scale;
const DateLUTImpl & timezone = DateLUT::instance();
};
std::ostream & operator << (std::ostream & ostr, const DateTime64StringsTestParam & param)
{
return ostr << param.comment;
}
class DateTime64StringsTest : public ::testing::TestWithParam<DateTime64StringsTestParam> {};
class DateTime64StringParseTest : public DateTime64StringsTest{};
class DateTime64StringParseBestEffortTest : public DateTime64StringsTest{};
class DateTime64StringWriteTest : public DateTime64StringsTest {};
TEST_P(DateTime64StringParseTest, readDateTime64Text)
{
const auto & param = GetParam();
ReadBufferFromMemory read_buffer(param.string.data(), param.string.size());
DateTime64 actual;
EXPECT_TRUE(tryReadDateTime64Text(actual, param.scale, read_buffer));
EXPECT_EQ(param.dt64, actual);
}
TEST_P(DateTime64StringParseTest, parseDateTime64BestEffort)
{
const auto & param = GetParam();
ReadBufferFromMemory read_buffer(param.string.data(), param.string.size());
DateTime64 actual;
EXPECT_TRUE(tryParseDateTime64BestEffort(actual, param.scale, read_buffer, param.timezone, DateLUT::instance("UTC")));
EXPECT_EQ(param.dt64, actual);
}
TEST_P(DateTime64StringWriteTest, WriteText)
{
const auto & param = GetParam();
PaddedPODArray<char> actual_string(param.string.size() * 2, '\0'); // TODO: detect overflows
WriteBuffer write_buffer(actual_string.data(), actual_string.size());
EXPECT_NO_THROW(writeDateTimeText(param.dt64, param.scale, write_buffer));
EXPECT_STREQ(param.string.data(), actual_string.data());
}
TEST_P(DateTime64StringParseBestEffortTest, parse)
{
const auto & param = GetParam();
ReadBufferFromMemory read_buffer(param.string.data(), param.string.size());
DateTime64 actual;
EXPECT_TRUE(tryParseDateTime64BestEffort(actual, param.scale, read_buffer, param.timezone, DateLUT::instance("UTC")));
EXPECT_EQ(param.dt64, actual);
}
// YYYY-MM-DD HH:MM:SS.NNNNNNNNN
INSTANTIATE_TEST_CASE_P(Basic,
DateTime64StringParseTest,
::testing::ValuesIn(std::initializer_list<DateTime64StringsTestParam>{
{
"When subsecond part is missing from string it is set to zero.",
"2019-09-16 19:20:17",
1568650817'000,
3
},
{
"When subsecond part is present in string, but it is zero, it is set to zero.",
"2019-09-16 19:20:17.0",
1568650817'000,
3
},
{
"When scale is 0, subsecond part is not set.",
"2019-09-16 19:20:17",
1568650817ULL,
0
},
{
"When scale is 0, subsecond part is 0 despite beeing present in string.",
"2019-09-16 19:20:17.123",
1568650817ULL,
0
},
{
"When subsecond part is present in string, it is set correctly to DateTime64 value of scale 3.",
"2019-09-16 19:20:17.123",
1568650817'123,
3
},
{
"When subsecond part is present in string (and begins with 0), it is set correctly to DateTime64 value of scale 3.",
"2019-09-16 19:20:17.012",
1568650817'012,
3
},
{
"When subsecond part scale is smaller than DateTime64 scale, subsecond part is properly adjusted (as if padded from right with zeroes).",
"2019-09-16 19:20:17.123",
1568650817'12300ULL,
5
},
{
"When subsecond part scale is larger than DateTime64 scale, subsecond part is truncated.",
"2019-09-16 19:20:17.123",
1568650817'1ULL,
1
}
}),
);
INSTANTIATE_TEST_CASE_P(BestEffort,
DateTime64StringParseBestEffortTest,
::testing::ValuesIn(std::initializer_list<DateTime64StringsTestParam>{
{
"When subsecond part is unreasonably large, it fals to parse",
"2019-09-16 19:20:17.12345678910111213141516171819202122233435363738393031323334353637383940414243444546474849505152535455565758596061626364",
1568650817'123456ULL,
6
}
}),
);
// TODO: add negative test cases for invalid strings, verifying that error is reported properly
INSTANTIATE_TEST_CASE_P(Basic,
DateTime64StringWriteTest,
::testing::ValuesIn(std::initializer_list<DateTime64StringsTestParam>{
{
"non-zero subsecond part on DateTime64 with scale of 3",
"2019-09-16 19:20:17.123",
1568650817'123,
3
},
{
"non-zero subsecond part on DateTime64 with scale of 5",
"2019-09-16 19:20:17.12345",
1568650817'12345ULL,
5
},
{
"Zero subsecond part is written to string",
"2019-09-16 19:20:17.000",
1568650817'000ULL,
3
},
{
"When scale is 0, subsecond part (and separtor) is missing from string",
"2019-09-16 19:20:17",
1568650817ULL,
0
},
{
"Subsecond part with leading zeroes is written to string correctly",
"2019-09-16 19:20:17.001",
1568650817'001ULL,
3
}
}),
);

View File

@ -1508,7 +1508,18 @@ BackgroundProcessingPool & Context::getBackgroundMovePool()
{
auto lock = getLock();
if (!shared->background_move_pool)
shared->background_move_pool.emplace(settings.background_move_pool_size, "BackgroundMovePool", "BgMoveProcPool");
{
BackgroundProcessingPool::PoolSettings pool_settings;
auto & config = getConfigRef();
pool_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10);
pool_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0);
pool_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1);
pool_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10);
pool_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600);
pool_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1);
pool_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0);
shared->background_move_pool.emplace(settings.background_move_pool_size, pool_settings, "BackgroundMovePool", "BgMoveProcPool");
}
return *shared->background_move_pool;
}

View File

@ -333,7 +333,7 @@ void Join::setSampleBlock(const Block & block)
asof_type = AsofRowRefs::getTypeSize(asof_column, asof_size);
if (!asof_type)
{
std::string msg = "ASOF join not supported for type";
std::string msg = "ASOF join not supported for type: ";
msg += asof_column->getFamilyName();
throw Exception(msg, ErrorCodes::BAD_TYPE_OF_FIELD);
}

View File

@ -1,9 +1,12 @@
#include <Interpreters/RowRefs.h>
#include <Core/Block.h>
#include <Core/Types.h>
#include <Common/typeid_cast.h>
#include <Common/ColumnsHashing.h>
#include <Core/Block.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
namespace DB
@ -18,10 +21,15 @@ void callWithType(AsofRowRefs::Type which, F && f)
{
switch (which)
{
case AsofRowRefs::Type::key32: return f(UInt32());
case AsofRowRefs::Type::key64: return f(UInt64());
case AsofRowRefs::Type::keyu32: return f(UInt32());
case AsofRowRefs::Type::keyu64: return f(UInt64());
case AsofRowRefs::Type::keyi32: return f(Int32());
case AsofRowRefs::Type::keyi64: return f(Int64());
case AsofRowRefs::Type::keyf32: return f(Float32());
case AsofRowRefs::Type::keyf64: return f(Float64());
case AsofRowRefs::Type::keyDecimal32: return f(Decimal32());
case AsofRowRefs::Type::keyDecimal64: return f(Decimal64());
case AsofRowRefs::Type::keyDecimal128: return f(Decimal128());
}
__builtin_unreachable();
@ -49,7 +57,9 @@ void AsofRowRefs::insert(Type type, const IColumn * asof_column, const Block * b
using T = std::decay_t<decltype(t)>;
using LookupPtr = typename Entry<T>::LookupPtr;
auto * column = typeid_cast<const ColumnVector<T> *>(asof_column);
using ColumnType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
auto * column = typeid_cast<const ColumnType *>(asof_column);
T key = column->getElement(row_num);
auto entry = Entry<T>(key, RowRef(block, row_num));
std::get<LookupPtr>(lookups)->insert(entry);
@ -71,7 +81,8 @@ const RowRef * AsofRowRefs::findAsof(Type type, ASOF::Inequality inequality, con
using EntryType = Entry<T>;
using LookupPtr = typename EntryType::LookupPtr;
auto * column = typeid_cast<const ColumnVector<T> *>(asof_column);
using ColumnType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
auto * column = typeid_cast<const ColumnType *>(asof_column);
T key = column->getElement(row_num);
auto & typed_lookup = std::get<LookupPtr>(lookups);
@ -90,12 +101,22 @@ std::optional<AsofRowRefs::Type> AsofRowRefs::getTypeSize(const IColumn * asof_c
if (typeid_cast<const ColumnVector<UInt32> *>(asof_column))
{
size = sizeof(UInt32);
return Type::key32;
return Type::keyu32;
}
else if (typeid_cast<const ColumnVector<UInt64> *>(asof_column))
{
size = sizeof(UInt64);
return Type::key64;
return Type::keyu64;
}
else if (typeid_cast<const ColumnVector<Int32> *>(asof_column))
{
size = sizeof(Int32);
return Type::keyi32;
}
else if (typeid_cast<const ColumnVector<Int64> *>(asof_column))
{
size = sizeof(Int64);
return Type::keyi64;
}
else if (typeid_cast<const ColumnVector<Float32> *>(asof_column))
{
@ -107,6 +128,21 @@ std::optional<AsofRowRefs::Type> AsofRowRefs::getTypeSize(const IColumn * asof_c
size = sizeof(Float64);
return Type::keyf64;
}
else if (typeid_cast<const ColumnDecimal<Decimal32> *>(asof_column))
{
size = sizeof(Decimal32);
return Type::keyDecimal32;
}
else if (typeid_cast<const ColumnDecimal<Decimal64> *>(asof_column))
{
size = sizeof(Decimal64);
return Type::keyDecimal64;
}
else if (typeid_cast<const ColumnDecimal<Decimal128> *>(asof_column))
{
size = sizeof(Decimal128);
return Type::keyDecimal128;
}
size = 0;
return {};

View File

@ -216,15 +216,25 @@ public:
using Lookups = std::variant<
Entry<UInt32>::LookupPtr,
Entry<UInt64>::LookupPtr,
Entry<Int32>::LookupPtr,
Entry<Int64>::LookupPtr,
Entry<Float32>::LookupPtr,
Entry<Float64>::LookupPtr>;
Entry<Float64>::LookupPtr,
Entry<Decimal32>::LookupPtr,
Entry<Decimal64>::LookupPtr,
Entry<Decimal128>::LookupPtr>;
enum class Type
{
key32,
key64,
keyu32,
keyu64,
keyi32,
keyi64,
keyf32,
keyf64,
keyDecimal32,
keyDecimal64,
keyDecimal128,
};
AsofRowRefs() {}

View File

@ -11,6 +11,7 @@
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeNullable.h>
@ -146,13 +147,33 @@ UInt64 stringToDateTime(const String & s)
return UInt64(date_time);
}
DateTime64::NativeType stringToDateTime64(const String & s, UInt32 scale)
{
ReadBufferFromString in(s);
DateTime64 datetime64 {0};
readDateTime64Text(datetime64, scale, in);
if (!in.eof())
throw Exception("String is too long for DateTime64: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE);
return datetime64.value;
}
Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const IDataType * from_type_hint)
{
WhichDataType which_type(type);
WhichDataType which_from_type;
if (from_type_hint)
{
which_from_type = WhichDataType(*from_type_hint);
// This was added to mitigate converting DateTime64-Field (a typedef to a Decimal64) to DataTypeDate64-compatitable type.
if (from_type_hint && from_type_hint->equals(type))
{
return src;
}
}
/// Conversion between Date and DateTime and vice versa.
if (which_type.isDate() && which_from_type.isDateTime())
{
@ -187,11 +208,12 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
return dynamic_cast<const IDataTypeEnum &>(type).castToValue(src);
}
if (which_type.isDateOrDateTime() && src.getType() == Field::Types::UInt64)
if (which_type.isDateOrDateTime() && !which_type.isDateTime64() && src.getType() == Field::Types::UInt64)
{
/// We don't need any conversion UInt64 is under type of Date and DateTime
return src;
}
// TODO (vnemkov): extra cases for DateTime64: converting from integer, converting from Decimal
if (src.getType() == Field::Types::String)
{
@ -205,6 +227,12 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
/// Convert 'YYYY-MM-DD hh:mm:ss' Strings to DateTime
return stringToDateTime(src.get<const String &>());
}
else if (which_type.isDateTime64())
{
const auto date_time64 = typeid_cast<const DataTypeDateTime64 *>(&type);
/// Convert 'YYYY-MM-DD hh:mm:ss.NNNNNNNNN' Strings to DateTime
return stringToDateTime64(src.get<const String &>(), date_time64->getScale());
}
else if (which_type.isUUID())
{
return stringToUUID(src.get<const String &>());

View File

@ -176,12 +176,14 @@ void ASTAlterCommand::formatImpl(
settings.ostr << " TO ";
switch (move_destination_type)
{
case MoveDestinationType::DISK:
case PartDestinationType::DISK:
settings.ostr << "DISK ";
break;
case MoveDestinationType::VOLUME:
case PartDestinationType::VOLUME:
settings.ostr << "VOLUME ";
break;
default:
break;
}
settings.ostr << quoteString(move_destination_name);
}

View File

@ -3,6 +3,7 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Parsers/ASTTTLElement.h>
namespace DB
@ -128,15 +129,9 @@ public:
bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
enum MoveDestinationType
{
DISK,
VOLUME,
};
PartDestinationType move_destination_type; /// option for MOVE PART/PARTITION
MoveDestinationType move_destination_type;
String move_destination_name;
String move_destination_name; /// option for MOVE PART/PARTITION
/** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition.
*/

View File

@ -0,0 +1,27 @@
#include <Columns/Collator.h>
#include <Common/quoteString.h>
#include <Parsers/ASTTTLElement.h>
namespace DB
{
void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
children.front()->formatImpl(settings, state, frame);
if (destination_type == PartDestinationType::DISK)
{
settings.ostr << " TO DISK " << quoteString(destination_name);
}
else if (destination_type == PartDestinationType::VOLUME)
{
settings.ostr << " TO VOLUME " << quoteString(destination_name);
}
else if (destination_type == PartDestinationType::DELETE)
{
/// It would be better to output "DELETE" here but that will break compatibility with earlier versions.
}
}
}

View File

@ -0,0 +1,36 @@
#pragma once
#include <Parsers/IAST.h>
#include <Storages/MergeTree/PartDestinationType.h>
namespace DB
{
/** Element of TTL expression.
*/
class ASTTTLElement : public IAST
{
public:
PartDestinationType destination_type;
String destination_name;
ASTTTLElement(PartDestinationType destination_type_, const String & destination_name_)
: destination_type(destination_type_)
, destination_name(destination_name_)
{
}
String getID(char) const override { return "TTLElement"; }
ASTPtr clone() const override
{
auto clone = std::make_shared<ASTTTLElement>(*this);
clone->cloneChildren();
return clone;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -16,6 +16,7 @@
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTQueryParameter.h>
#include <Parsers/ASTTTLElement.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
@ -1414,6 +1415,42 @@ bool ParserFunctionWithKeyValueArguments::parseImpl(Pos & pos, ASTPtr & node, Ex
return true;
}
bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_to_disk("TO DISK");
ParserKeyword s_to_volume("TO VOLUME");
ParserKeyword s_delete("DELETE");
ParserStringLiteral parser_string_literal;
ParserExpression parser_exp;
ASTPtr expr_elem;
if (!parser_exp.parse(pos, expr_elem, expected))
return false;
PartDestinationType destination_type = PartDestinationType::DELETE;
String destination_name;
if (s_to_disk.ignore(pos))
destination_type = PartDestinationType::DISK;
else if (s_to_volume.ignore(pos))
destination_type = PartDestinationType::VOLUME;
else
s_delete.ignore(pos);
if (destination_type == PartDestinationType::DISK || destination_type == PartDestinationType::VOLUME)
{
ASTPtr ast_space_name;
if (!parser_string_literal.parse(pos, ast_space_name, expected))
return false;
destination_name = ast_space_name->as<ASTLiteral &>().value.get<const String &>();
}
node = std::make_shared<ASTTTLElement>(destination_type, destination_name);
node->children.push_back(expr_elem);
return true;
}
bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier non_parametric;

View File

@ -320,4 +320,14 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
/** Element of TTL expression - same as expression element, but in addition,
* TO DISK 'xxx' | TO VOLUME 'xxx' | DELETE could be specified
*/
class ParserTTLElement : public IParserBase
{
protected:
const char * getName() const { return "element of TTL expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
}

View File

@ -557,6 +557,13 @@ bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected &
}
bool ParserTTLExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserTTLElement>(), std::make_unique<ParserToken>(TokenType::Comma), false)
.parse(pos, node, expected);
}
bool ParserNullityChecking::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ASTPtr node_comp;

View File

@ -386,6 +386,7 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Parser for list of key-value pairs.
class ParserKeyValuePairsList : public IParserBase
{
@ -394,4 +395,12 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserTTLExpressionList : public IParserBase
{
protected:
const char * getName() const { return "ttl expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
}

View File

@ -87,6 +87,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
/* allow_empty = */ false);
ParserSetQuery parser_settings(true);
ParserNameList values_p;
ParserTTLExpressionList parser_ttl_list;
if (is_live_view)
{
@ -236,9 +237,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
command->part = true;
if (s_to_disk.ignore(pos))
command->move_destination_type = ASTAlterCommand::MoveDestinationType::DISK;
command->move_destination_type = PartDestinationType::DISK;
else if (s_to_volume.ignore(pos))
command->move_destination_type = ASTAlterCommand::MoveDestinationType::VOLUME;
command->move_destination_type = PartDestinationType::VOLUME;
else
return false;
@ -256,9 +257,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
command->type = ASTAlterCommand::MOVE_PARTITION;
if (s_to_disk.ignore(pos))
command->move_destination_type = ASTAlterCommand::MoveDestinationType::DISK;
command->move_destination_type = PartDestinationType::DISK;
else if (s_to_volume.ignore(pos))
command->move_destination_type = ASTAlterCommand::MoveDestinationType::VOLUME;
command->move_destination_type = PartDestinationType::VOLUME;
else
return false;
@ -431,7 +432,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
}
else if (s_modify_ttl.ignore(pos, expected))
{
if (!parser_exp_elem.parse(pos, command->ttl, expected))
if (!parser_ttl_list.parse(pos, command->ttl, expected))
return false;
command->type = ASTAlterCommand::MODIFY_TTL;
}

View File

@ -250,6 +250,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
ParserExpression expression_p;
ParserSetQuery settings_p(/* parse_only_internals_ = */ true);
ParserTTLExpressionList parser_ttl_list;
ASTPtr engine;
ASTPtr partition_by;
@ -303,7 +304,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!ttl_table && s_ttl.ignore(pos, expected))
{
if (expression_p.parse(pos, ttl_table, expected))
if (parser_ttl_list.parse(pos, ttl_table, expected))
continue;
else
return false;

View File

@ -23,17 +23,6 @@ namespace CurrentMetrics
namespace DB
{
static constexpr double thread_sleep_seconds = 10;
static constexpr double thread_sleep_seconds_random_part = 1.0;
static constexpr double thread_sleep_seconds_if_nothing_to_do = 0.1;
/// For exponential backoff.
static constexpr double task_sleep_seconds_when_no_work_min = 10;
static constexpr double task_sleep_seconds_when_no_work_max = 600;
static constexpr double task_sleep_seconds_when_no_work_multiplier = 1.1;
static constexpr double task_sleep_seconds_when_no_work_random_part = 1.0;
void BackgroundProcessingPoolTaskInfo::wake()
{
Poco::Timestamp current_time;
@ -61,9 +50,13 @@ void BackgroundProcessingPoolTaskInfo::wake()
}
BackgroundProcessingPool::BackgroundProcessingPool(int size_, const char * log_name, const char * thread_name_)
BackgroundProcessingPool::BackgroundProcessingPool(int size_,
const PoolSettings & pool_settings,
const char * log_name,
const char * thread_name_)
: size(size_)
, thread_name(thread_name_)
, settings(pool_settings)
{
logger = &Logger::get(log_name);
LOG_INFO(logger, "Create " << log_name << " with " << size << " threads");
@ -147,7 +140,7 @@ void BackgroundProcessingPool::threadFunction()
memory_tracker->setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool);
pcg64 rng(randomSeed());
std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, thread_sleep_seconds_random_part)(rng)));
std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, settings.thread_sleep_seconds_random_part)(rng)));
while (!shutdown)
{
@ -182,8 +175,8 @@ void BackgroundProcessingPool::threadFunction()
{
std::unique_lock lock(tasks_mutex);
wake_event.wait_for(lock,
std::chrono::duration<double>(thread_sleep_seconds
+ std::uniform_real_distribution<double>(0, thread_sleep_seconds_random_part)(rng)));
std::chrono::duration<double>(settings.thread_sleep_seconds
+ std::uniform_real_distribution<double>(0, settings.thread_sleep_seconds_random_part)(rng)));
continue;
}
@ -193,7 +186,7 @@ void BackgroundProcessingPool::threadFunction()
{
std::unique_lock lock(tasks_mutex);
wake_event.wait_for(lock, std::chrono::microseconds(
min_time - current_time + std::uniform_int_distribution<uint64_t>(0, thread_sleep_seconds_random_part * 1000000)(rng)));
min_time - current_time + std::uniform_int_distribution<uint64_t>(0, settings.thread_sleep_seconds_random_part * 1000000)(rng)));
}
std::shared_lock rlock(task->rwlock);
@ -231,11 +224,11 @@ void BackgroundProcessingPool::threadFunction()
Poco::Timestamp next_time_to_execute; /// current time
if (task_result == TaskResult::ERROR)
next_time_to_execute += 1000000 * (std::min(
task_sleep_seconds_when_no_work_max,
task_sleep_seconds_when_no_work_min * std::pow(task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done))
+ std::uniform_real_distribution<double>(0, task_sleep_seconds_when_no_work_random_part)(rng));
settings.task_sleep_seconds_when_no_work_max,
settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done))
+ std::uniform_real_distribution<double>(0, settings.task_sleep_seconds_when_no_work_random_part)(rng));
else if (task_result == TaskResult::NOTHING_TO_DO)
next_time_to_execute += 1000000 * thread_sleep_seconds_if_nothing_to_do;
next_time_to_execute += 1000000 * settings.thread_sleep_seconds_if_nothing_to_do;
tasks.erase(task->iterator);
task->iterator = tasks.emplace(next_time_to_execute, task);

View File

@ -14,6 +14,7 @@
#include <Core/Types.h>
#include <Common/CurrentThread.h>
#include <Common/ThreadPool.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
@ -46,7 +47,23 @@ public:
using TaskHandle = std::shared_ptr<TaskInfo>;
struct PoolSettings
{
double thread_sleep_seconds = 10;
double thread_sleep_seconds_random_part = 1.0;
double thread_sleep_seconds_if_nothing_to_do = 0.1;
/// For exponential backoff.
double task_sleep_seconds_when_no_work_min = 10;
double task_sleep_seconds_when_no_work_max = 600;
double task_sleep_seconds_when_no_work_multiplier = 1.1;
double task_sleep_seconds_when_no_work_random_part = 1.0;
PoolSettings() noexcept {}
};
BackgroundProcessingPool(int size_,
const PoolSettings & pool_settings = {},
const char * log_name = "BackgroundProcessingPool",
const char * thread_name_ = "BackgrProcPool");
@ -84,6 +101,9 @@ protected:
ThreadGroupStatusPtr thread_group;
void threadFunction();
private:
PoolSettings settings;
};

View File

@ -38,6 +38,7 @@
#include <Common/Increment.h>
#include <Common/SimpleIncrement.h>
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Stopwatch.h>
#include <Common/typeid_cast.h>
@ -70,6 +71,12 @@ namespace CurrentMetrics
}
namespace
{
constexpr UInt64 RESERVATION_MIN_ESTIMATION_SIZE = 1u * 1024u * 1024u; /// 1MB
}
namespace DB
{
@ -124,7 +131,6 @@ MergeTreeData::MergeTreeData(
, merging_params(merging_params_)
, partition_by_ast(partition_by_ast_)
, sample_by_ast(sample_by_ast_)
, ttl_table_ast(ttl_table_ast_)
, require_part_metadata(require_part_metadata_)
, database_name(database_)
, table_name(table_)
@ -566,15 +572,17 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin
void MergeTreeData::setTTLExpressions(const ColumnsDescription::ColumnTTLs & new_column_ttls,
const ASTPtr & new_ttl_table_ast, bool only_check)
{
auto create_ttl_entry = [this](ASTPtr ttl_ast) -> TTLEntry
auto create_ttl_entry = [this](ASTPtr ttl_ast)
{
TTLEntry result;
auto syntax_result = SyntaxAnalyzer(global_context).analyze(ttl_ast, getColumns().getAllPhysical());
auto expr = ExpressionAnalyzer(ttl_ast, syntax_result, global_context).getActions(false);
result.expression = ExpressionAnalyzer(ttl_ast, syntax_result, global_context).getActions(false);
result.destination_type = PartDestinationType::DELETE;
result.result_column = ttl_ast->getColumnName();
String result_column = ttl_ast->getColumnName();
checkTTLExpression(expr, result_column);
return {expr, result_column};
checkTTLExpression(result.expression, result.result_column);
return result;
};
if (!new_column_ttls.empty())
@ -592,23 +600,49 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription::ColumnTTLs & new
for (const auto & [name, ast] : new_column_ttls)
{
if (columns_ttl_forbidden.count(name))
throw Exception("Trying to set ttl for key column " + name, ErrorCodes::ILLEGAL_COLUMN);
throw Exception("Trying to set TTL for key column " + name, ErrorCodes::ILLEGAL_COLUMN);
else
{
auto new_ttl_entry = create_ttl_entry(ast);
if (!only_check)
ttl_entries_by_name.emplace(name, new_ttl_entry);
column_ttl_entries_by_name.emplace(name, new_ttl_entry);
}
}
}
if (new_ttl_table_ast)
{
auto new_ttl_table_entry = create_ttl_entry(new_ttl_table_ast);
if (!only_check)
bool seen_delete_ttl = false;
for (auto ttl_element_ptr : new_ttl_table_ast->children)
{
ttl_table_ast = new_ttl_table_ast;
ttl_table_entry = new_ttl_table_entry;
ASTTTLElement & ttl_element = static_cast<ASTTTLElement &>(*ttl_element_ptr);
if (ttl_element.destination_type == PartDestinationType::DELETE)
{
if (seen_delete_ttl)
{
throw Exception("More than one DELETE TTL expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION);
}
auto new_ttl_table_entry = create_ttl_entry(ttl_element.children[0]);
if (!only_check)
{
ttl_table_ast = ttl_element.children[0];
ttl_table_entry = new_ttl_table_entry;
}
seen_delete_ttl = true;
}
else
{
auto new_ttl_entry = create_ttl_entry(ttl_element.children[0]);
if (!only_check)
{
new_ttl_entry.entry_ast = ttl_element_ptr;
new_ttl_entry.destination_type = ttl_element.destination_type;
new_ttl_entry.destination_name = ttl_element.destination_name;
move_ttl_entries.emplace_back(std::move(new_ttl_entry));
}
}
}
}
}
@ -3096,20 +3130,138 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const
return loaded_parts;
}
DiskSpace::ReservationPtr MergeTreeData::reserveSpace(UInt64 expected_size)
namespace
{
constexpr UInt64 RESERVATION_MIN_ESTIMATION_SIZE = 1u * 1024u * 1024u; /// 1MB
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
auto reservation = storage_policy->reserve(expected_size);
inline DiskSpace::ReservationPtr checkAndReturnReservation(UInt64 expected_size, DiskSpace::ReservationPtr reservation)
{
if (reservation)
return reservation;
throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(expected_size) + ", not enough space.",
throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(expected_size) + ", not enough space",
ErrorCodes::NOT_ENOUGH_SPACE);
}
}
DiskSpace::ReservationPtr MergeTreeData::reserveSpace(UInt64 expected_size) const
{
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
auto reservation = storage_policy->reserve(expected_size);
return checkAndReturnReservation(expected_size, std::move(reservation));
}
DiskSpace::ReservationPtr MergeTreeData::reserveSpace(UInt64 expected_size, DiskSpace::SpacePtr space) const
{
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
auto reservation = tryReserveSpace(expected_size, space);
return checkAndReturnReservation(expected_size, std::move(reservation));
}
DiskSpace::ReservationPtr MergeTreeData::tryReserveSpace(UInt64 expected_size, DiskSpace::SpacePtr space) const
{
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
return space->reserve(expected_size);
}
DiskSpace::ReservationPtr MergeTreeData::reserveSpacePreferringTTLRules(UInt64 expected_size,
const MergeTreeDataPart::TTLInfos & ttl_infos,
time_t time_of_move) const
{
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
DiskSpace::ReservationPtr reservation = tryReserveSpacePreferringTTLRules(expected_size, ttl_infos, time_of_move);
return checkAndReturnReservation(expected_size, std::move(reservation));
}
DiskSpace::ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_size,
const MergeTreeDataPart::TTLInfos & ttl_infos,
time_t time_of_move) const
{
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
DiskSpace::ReservationPtr reservation;
auto ttl_entry = selectTTLEntryForTTLInfos(ttl_infos, time_of_move);
if (ttl_entry != nullptr)
{
DiskSpace::SpacePtr destination_ptr = ttl_entry->getDestination(storage_policy);
if (!destination_ptr)
{
if (ttl_entry->destination_type == PartDestinationType::VOLUME)
LOG_WARNING(log, "Would like to reserve space on volume '"
<< ttl_entry->destination_name << "' by TTL rule of table '"
<< log_name << "' but volume was not found");
else if (ttl_entry->destination_type == PartDestinationType::DISK)
LOG_WARNING(log, "Would like to reserve space on disk '"
<< ttl_entry->destination_name << "' by TTL rule of table '"
<< log_name << "' but disk was not found");
}
else
{
reservation = destination_ptr->reserve(expected_size);
if (reservation)
return reservation;
}
}
reservation = storage_policy->reserve(expected_size);
return reservation;
}
DiskSpace::SpacePtr MergeTreeData::TTLEntry::getDestination(const DiskSpace::StoragePolicyPtr & policy) const
{
if (destination_type == PartDestinationType::VOLUME)
return policy->getVolumeByName(destination_name);
else if (destination_type == PartDestinationType::DISK)
return policy->getDiskByName(destination_name);
else
return {};
}
bool MergeTreeData::TTLEntry::isPartInDestination(const DiskSpace::StoragePolicyPtr & policy, const MergeTreeDataPart & part) const
{
if (destination_type == PartDestinationType::VOLUME)
{
for (const auto & disk : policy->getVolumeByName(destination_name)->disks)
if (disk->getName() == part.disk->getName())
return true;
}
else if (destination_type == PartDestinationType::DISK)
return policy->getDiskByName(destination_name)->getName() == part.disk->getName();
return false;
}
const MergeTreeData::TTLEntry * MergeTreeData::selectTTLEntryForTTLInfos(
const MergeTreeDataPart::TTLInfos & ttl_infos,
time_t time_of_move) const
{
const MergeTreeData::TTLEntry * result = nullptr;
/// Prefer TTL rule which went into action last.
time_t max_max_ttl = 0;
for (const auto & ttl_entry : move_ttl_entries)
{
auto ttl_info_it = ttl_infos.moves_ttl.find(ttl_entry.result_column);
if (ttl_info_it != ttl_infos.moves_ttl.end()
&& ttl_info_it->second.max <= time_of_move
&& max_max_ttl <= ttl_info_it->second.max)
{
result = &ttl_entry;
max_max_ttl = ttl_info_it->second.max;
}
}
return result;
}
MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const
{
DataParts res;
@ -3289,12 +3441,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(
String dst_part_name = src_part->getNewName(dst_part_info);
String tmp_dst_part_name = tmp_part_prefix + dst_part_name;
auto reservation = src_part->disk->reserve(src_part->bytes_on_disk);
if (!reservation)
{
throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(src_part->bytes_on_disk) + ", not enough space",
ErrorCodes::NOT_ENOUGH_SPACE);
}
auto reservation = reserveSpace(src_part->bytes_on_disk, src_part->disk);
String dst_part_path = getFullPathOnDisk(reservation->getDisk());
Poco::Path dst_part_absolute_path = Poco::Path(dst_part_path + tmp_dst_part_name).absolute();
Poco::Path src_part_absolute_path = Poco::Path(src_part->getFullPath()).absolute();

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/SimpleIncrement.h>
#include <Common/DiskSpaceMonitor.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Storages/IStorage.h>
@ -9,6 +10,7 @@
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <Storages/MergeTree/MergeTreeMutationStatus.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/PartDestinationType.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFile.h>
@ -19,7 +21,6 @@
#include <Storages/IndicesDescription.h>
#include <Storages/MergeTree/MergeTreePartsMover.h>
#include <Interpreters/PartLog.h>
#include <Common/DiskSpaceMonitor.h>
#include <boost/multi_index_container.hpp>
#include <boost/multi_index/ordered_index.hpp>
@ -565,7 +566,7 @@ public:
/// All MergeTreeData children have settings.
void checkSettingCanBeChanged(const String & setting_name) const override;
/// Remove columns, that have been markedd as empty after zeroing values with expired ttl
/// Remove columns, that have been marked as empty after zeroing values with expired ttl
void removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part);
/// Freezes all parts.
@ -587,7 +588,7 @@ public:
bool hasPrimaryKey() const { return !primary_key_columns.empty(); }
bool hasSkipIndices() const { return !skip_indices.empty(); }
bool hasTableTTL() const { return ttl_table_ast != nullptr; }
bool hasAnyColumnTTL() const { return !ttl_entries_by_name.empty(); }
bool hasAnyColumnTTL() const { return !column_ttl_entries_by_name.empty(); }
/// Check that the part is not broken and calculate the checksums for it if they are not present.
MutableDataPartPtr loadPartAndFixMetadata(const DiskSpace::DiskPtr & disk, const String & relative_path);
@ -673,9 +674,20 @@ public:
using PathsWithDisks = std::vector<PathWithDisk>;
PathsWithDisks getDataPathsWithDisks() const;
/// Reserves space at least 1MB
DiskSpace::ReservationPtr reserveSpace(UInt64 expected_size);
/// Reserves space at least 1MB.
DiskSpace::ReservationPtr reserveSpace(UInt64 expected_size) const;
/// Reserves space at least 1MB on specific disk or volume.
DiskSpace::ReservationPtr reserveSpace(UInt64 expected_size, DiskSpace::SpacePtr space) const;
DiskSpace::ReservationPtr tryReserveSpace(UInt64 expected_size, DiskSpace::SpacePtr space) const;
/// Reserves space at least 1MB preferring best destination according to `ttl_infos`.
DiskSpace::ReservationPtr reserveSpacePreferringTTLRules(UInt64 expected_size,
const MergeTreeDataPart::TTLInfos & ttl_infos,
time_t time_of_move) const;
DiskSpace::ReservationPtr tryReserveSpacePreferringTTLRules(UInt64 expected_size,
const MergeTreeDataPart::TTLInfos & ttl_infos,
time_t time_of_move) const;
/// Choose disk with max available free space
/// Reserves 0 bytes
DiskSpace::ReservationPtr makeEmptyReservationOnLargestDisk() { return storage_policy->makeEmptyReservationOnLargestDisk(); }
@ -719,12 +731,27 @@ public:
{
ExpressionActionsPtr expression;
String result_column;
/// Name and type of a destination are only valid in table-level context.
PartDestinationType destination_type;
String destination_name;
ASTPtr entry_ast;
/// Returns destination disk or volume for this rule.
DiskSpace::SpacePtr getDestination(const DiskSpace::StoragePolicyPtr & policy) const;
/// Checks if given part already belongs destination disk or volume for this rule.
bool isPartInDestination(const DiskSpace::StoragePolicyPtr & policy, const MergeTreeDataPart & part) const;
};
const TTLEntry * selectTTLEntryForTTLInfos(const MergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const;
using TTLEntriesByName = std::unordered_map<String, TTLEntry>;
TTLEntriesByName ttl_entries_by_name;
TTLEntriesByName column_ttl_entries_by_name;
TTLEntry ttl_table_entry;
std::vector<TTLEntry> move_ttl_entries;
String sampling_expr_column_name;
Names columns_required_for_sampling;

View File

@ -1,6 +1,7 @@
#include <Storages/MergeTree/MergeTreeDataPartTTLInfo.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/quoteString.h>
#include <common/JSON.h>
@ -15,10 +16,16 @@ void MergeTreeDataPartTTLInfos::update(const MergeTreeDataPartTTLInfos & other_i
updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
}
for (const auto & [expression, ttl_info] : other_infos.moves_ttl)
{
moves_ttl[expression].update(ttl_info);
}
table_ttl.update(other_infos.table_ttl);
updatePartMinMaxTTL(table_ttl.min, table_ttl.max);
}
void MergeTreeDataPartTTLInfos::read(ReadBuffer & in)
{
String json_str;
@ -28,7 +35,7 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in)
JSON json(json_str);
if (json.has("columns"))
{
JSON columns = json["columns"];
const JSON & columns = json["columns"];
for (auto col : columns)
{
MergeTreeDataPartTTLInfo ttl_info;
@ -42,14 +49,27 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in)
}
if (json.has("table"))
{
JSON table = json["table"];
const JSON & table = json["table"];
table_ttl.min = table["min"].getUInt();
table_ttl.max = table["max"].getUInt();
updatePartMinMaxTTL(table_ttl.min, table_ttl.max);
}
if (json.has("moves"))
{
const JSON & moves = json["moves"];
for (auto move : moves)
{
MergeTreeDataPartTTLInfo ttl_info;
ttl_info.min = move["min"].getUInt();
ttl_info.max = move["max"].getUInt();
String expression = move["expression"].getString();
moves_ttl.emplace(expression, ttl_info);
}
}
}
void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const
{
writeString("ttl format version: 1\n", out);
@ -62,9 +82,9 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const
if (it != columns_ttl.begin())
writeString(",", out);
writeString("{\"name\":\"", out);
writeString(it->first, out);
writeString("\",\"min\":", out);
writeString("{\"name\":", out);
writeString(doubleQuoteString(it->first), out);
writeString(",\"min\":", out);
writeIntText(it->second.min, out);
writeString(",\"max\":", out);
writeIntText(it->second.max, out);
@ -82,6 +102,26 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const
writeIntText(table_ttl.max, out);
writeString("}", out);
}
if (!moves_ttl.empty())
{
if (!columns_ttl.empty() || table_ttl.min)
writeString(",", out);
writeString("\"moves\":[", out);
for (auto it = moves_ttl.begin(); it != moves_ttl.end(); ++it)
{
if (it != moves_ttl.begin())
writeString(",", out);
writeString("{\"expression\":", out);
writeString(doubleQuoteString(it->first), out);
writeString(",\"min\":", out);
writeIntText(it->second.min, out);
writeString(",\"max\":", out);
writeIntText(it->second.max, out);
writeString("}", out);
}
writeString("]", out);
}
writeString("}", out);
}

View File

@ -35,9 +35,14 @@ struct MergeTreeDataPartTTLInfos
{
std::unordered_map<String, MergeTreeDataPartTTLInfo> columns_ttl;
MergeTreeDataPartTTLInfo table_ttl;
/// `part_min_ttl` and `part_max_ttl` are TTLs which are used for selecting parts
/// to merge in order to remove expired rows.
time_t part_min_ttl = 0;
time_t part_max_ttl = 0;
std::unordered_map<String, MergeTreeDataPartTTLInfo> moves_ttl;
void read(ReadBuffer & in);
void write(WriteBuffer & out) const;
void update(const MergeTreeDataPartTTLInfos & other_infos);
@ -50,6 +55,11 @@ struct MergeTreeDataPartTTLInfos
if (time_max && (!part_max_ttl || time_max > part_max_ttl))
part_max_ttl = time_max;
}
bool empty()
{
return !part_min_ttl && moves_ttl.empty();
}
};
}

View File

@ -75,12 +75,17 @@ void buildScatterSelector(
}
/// Computes ttls and updates ttl infos
void updateTTL(const MergeTreeData::TTLEntry & ttl_entry, MergeTreeDataPart::TTLInfos & ttl_infos, Block & block, const String & column_name)
void updateTTL(const MergeTreeData::TTLEntry & ttl_entry,
MergeTreeDataPart::TTLInfos & ttl_infos,
DB::MergeTreeDataPartTTLInfo & ttl_info,
Block & block, bool update_part_min_max_ttls)
{
bool remove_column = false;
if (!block.has(ttl_entry.result_column))
{
ttl_entry.expression->execute(block);
auto & ttl_info = (column_name.empty() ? ttl_infos.table_ttl : ttl_infos.columns_ttl[column_name]);
remove_column = true;
}
const auto & current = block.getByName(ttl_entry.result_column);
@ -113,7 +118,11 @@ void updateTTL(const MergeTreeData::TTLEntry & ttl_entry, MergeTreeDataPart::TTL
else
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
ttl_infos.updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
if (update_part_min_max_ttls)
ttl_infos.updatePartMinMaxTTL(ttl_info.min, ttl_info.max);
if (remove_column)
block.erase(ttl_entry.result_column);
}
}
@ -212,10 +221,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
else
part_name = new_part_info.getPartName();
/// Size of part would not be grater than block.bytes() + epsilon
/// Size of part would not be greater than block.bytes() + epsilon
size_t expected_size = block.bytes();
auto reservation = data.reserveSpace(expected_size);
DB::MergeTreeDataPart::TTLInfos move_ttl_infos;
for (const auto & ttl_entry : data.move_ttl_entries)
updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false);
DiskSpace::ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr));
MergeTreeData::MutableDataPartPtr new_data_part =
std::make_shared<MergeTreeData::DataPart>(data, reservation->getDisk(), part_name, new_part_info);
@ -251,7 +264,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocks);
/// Sort.
/// Sort
IColumn::Permutation * perm_ptr = nullptr;
IColumn::Permutation perm;
if (!sort_description.empty())
@ -266,10 +279,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
}
if (data.hasTableTTL())
updateTTL(data.ttl_table_entry, new_data_part->ttl_infos, block, "");
updateTTL(data.ttl_table_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true);
for (const auto & [name, ttl_entry] : data.ttl_entries_by_name)
updateTTL(ttl_entry, new_data_part->ttl_infos, block, name);
for (const auto & [name, ttl_entry] : data.column_ttl_entries_by_name)
updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true);
new_data_part->ttl_infos.update(move_ttl_infos);
/// This effectively chooses minimal compression method:
/// either default lz4 or compression method with zero thresholds on absolute and relative part size.

View File

@ -52,11 +52,14 @@ public:
elems.emplace(part);
current_size_sum += part->bytes_on_disk;
while (!elems.empty() && (current_size_sum - (*elems.begin())->bytes_on_disk >= required_size_sum))
{
current_size_sum -= (*elems.begin())->bytes_on_disk;
elems.erase(elems.begin());
}
removeRedundantElements();
}
/// Weaken requirements on size
void decreaseRequiredSizeAndRemoveRedundantParts(UInt64 size_decrease)
{
required_size_sum -= std::min(size_decrease, required_size_sum);
removeRedundantElements();
}
/// Returns parts ordered by size
@ -67,6 +70,16 @@ public:
res.push_back(elem);
return res;
}
private:
void removeRedundantElements()
{
while (!elems.empty() && (current_size_sum - (*elems.begin())->bytes_on_disk >= required_size_sum))
{
current_size_sum -= (*elems.begin())->bytes_on_disk;
elems.erase(elems.begin());
}
}
};
}
@ -85,46 +98,70 @@ bool MergeTreePartsMover::selectPartsForMove(
const auto & policy = data->getStoragePolicy();
const auto & volumes = policy->getVolumes();
/// Do not check if policy has one volume
if (volumes.size() == 1)
return false;
/// Do not check last volume
for (size_t i = 0; i != volumes.size() - 1; ++i)
if (volumes.size() > 0)
{
for (const auto & disk : volumes[i]->disks)
/// Do not check last volume
for (size_t i = 0; i != volumes.size() - 1; ++i)
{
UInt64 required_available_space = disk->getTotalSpace() * policy->getMoveFactor();
UInt64 unreserved_space = disk->getUnreservedSpace();
for (const auto & disk : volumes[i]->disks)
{
UInt64 required_maximum_available_space = disk->getTotalSpace() * policy->getMoveFactor();
UInt64 unreserved_space = disk->getUnreservedSpace();
if (required_available_space > unreserved_space)
need_to_move.emplace(disk, required_available_space - unreserved_space);
if (unreserved_space < required_maximum_available_space)
need_to_move.emplace(disk, required_maximum_available_space - unreserved_space);
}
}
}
time_t time_of_move = time(nullptr);
for (const auto & part : data_parts)
{
String reason;
/// Don't report message to log, because logging is excessive
/// Don't report message to log, because logging is excessive.
if (!can_move(part, &reason))
continue;
const MergeTreeData::TTLEntry * ttl_entry_ptr = part->storage.selectTTLEntryForTTLInfos(part->ttl_infos, time_of_move);
auto to_insert = need_to_move.find(part->disk);
if (to_insert != need_to_move.end())
to_insert->second.add(part);
DiskSpace::ReservationPtr reservation;
if (ttl_entry_ptr)
{
auto destination = ttl_entry_ptr->getDestination(policy);
if (destination && !ttl_entry_ptr->isPartInDestination(policy, *part))
reservation = part->storage.tryReserveSpace(part->bytes_on_disk, ttl_entry_ptr->getDestination(policy));
}
if (reservation) /// Found reservation by TTL rule.
{
parts_to_move.emplace_back(part, std::move(reservation));
/// If table TTL rule satisfies on this part, won't apply policy rules on it.
/// In order to not over-move, we need to "release" required space on this disk,
/// possibly to zero.
if (to_insert != need_to_move.end())
{
to_insert->second.decreaseRequiredSizeAndRemoveRedundantParts(part->bytes_on_disk);
}
}
else
{
if (to_insert != need_to_move.end())
to_insert->second.add(part);
}
}
for (auto && move : need_to_move)
{
auto min_volume_priority = policy->getVolumeIndexByDisk(move.first) + 1;
auto min_volume_index = policy->getVolumeIndexByDisk(move.first) + 1;
for (auto && part : move.second.getAccumulatedParts())
{
auto reservation = policy->reserve(part->bytes_on_disk, min_volume_priority);
auto reservation = policy->reserve(part->bytes_on_disk, min_volume_index);
if (!reservation)
{
/// Next parts to move from this disk has greater size and same min volume priority
/// There are no space for them
/// But it can be possible to move data from other disks
/// Next parts to move from this disk has greater size and same min volume index.
/// There are no space for them.
/// But it can be possible to move data from other disks.
break;
}
parts_to_move.emplace_back(part, std::move(reservation));

View File

@ -186,7 +186,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
}
if (new_part->ttl_infos.part_min_ttl)
if (!new_part->ttl_infos.empty())
{
/// Write a file with ttl infos in json format.
WriteBufferFromFile out(part_path + "ttl.txt", 4096);

View File

@ -0,0 +1,14 @@
#pragma once
namespace DB
{
enum class PartDestinationType
{
DISK,
VOLUME,
DELETE,
};
}

View File

@ -5,6 +5,7 @@
#include <Parsers/ExpressionListParsers.h>
#include <IO/Operators.h>
namespace DB
{
@ -47,6 +48,16 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
partition_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.partition_by_ast));
ttl_table = formattedAST(data.ttl_table_ast);
std::ostringstream ttl_move_stream;
for (const auto & ttl_entry : data.move_ttl_entries)
{
if (ttl_move_stream.tellp() > 0)
ttl_move_stream << ", ";
ttl_move_stream << formattedAST(ttl_entry.entry_ast);
}
ttl_move = ttl_move_stream.str();
skip_indices = data.getIndices().toString();
if (data.canUseAdaptiveGranularity())
index_granularity_bytes = data_settings->index_granularity_bytes;
@ -78,6 +89,9 @@ void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const
if (!ttl_table.empty())
out << "ttl: " << ttl_table << "\n";
if (!ttl_move.empty())
out << "move ttl: " << ttl_move << "\n";
if (!skip_indices.empty())
out << "indices: " << skip_indices << "\n";
@ -119,6 +133,9 @@ void ReplicatedMergeTreeTableMetadata::read(ReadBuffer & in)
if (checkString("ttl: ", in))
in >> ttl_table >> "\n";
if (checkString("move ttl: ", in))
in >> ttl_move >> "\n";
if (checkString("indices: ", in))
in >> skip_indices >> "\n";
@ -223,12 +240,27 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl
}
else
throw Exception(
"Existing table metadata in ZooKeeper differs in ttl."
"Existing table metadata in ZooKeeper differs in TTL."
" Stored in ZooKeeper: " + from_zk.ttl_table +
", local: " + ttl_table,
ErrorCodes::METADATA_MISMATCH);
}
if (ttl_move != from_zk.ttl_move)
{
if (allow_alter)
{
diff.ttl_move_changed = true;
diff.new_ttl_move = from_zk.ttl_move;
}
else
throw Exception(
"Existing table metadata in ZooKeeper differs in move TTL."
" Stored in ZooKeeper: " + from_zk.ttl_move +
", local: " + ttl_move,
ErrorCodes::METADATA_MISMATCH);
}
if (skip_indices != from_zk.skip_indices)
{
if (allow_alter)

View File

@ -28,6 +28,7 @@ struct ReplicatedMergeTreeTableMetadata
String skip_indices;
String constraints;
String ttl_table;
String ttl_move;
UInt64 index_granularity_bytes;
ReplicatedMergeTreeTableMetadata() = default;
@ -53,9 +54,12 @@ struct ReplicatedMergeTreeTableMetadata
bool ttl_table_changed = false;
String new_ttl_table;
bool ttl_move_changed = false;
String new_ttl_move;
bool empty() const
{
return !sorting_key_changed && !skip_indices_changed && !ttl_table_changed && !constraints_changed;
return !sorting_key_changed && !skip_indices_changed && !ttl_table_changed && !constraints_changed && !ttl_move_changed;
}
};

View File

@ -1,5 +1,6 @@
#include <Storages/PartitionCommands.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/PartDestinationType.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTIdentifier.h>
@ -47,12 +48,14 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
res.part = command_ast->part;
switch (command_ast->move_destination_type)
{
case ASTAlterCommand::MoveDestinationType::DISK:
case PartDestinationType::DISK:
res.move_destination_type = PartitionCommand::MoveDestinationType::DISK;
break;
case ASTAlterCommand::MoveDestinationType::VOLUME:
case PartDestinationType::VOLUME:
res.move_destination_type = PartitionCommand::MoveDestinationType::VOLUME;
break;
default:
break;
}
res.move_destination_name = command_ast->move_destination_name;
return res;

View File

@ -350,9 +350,15 @@ public:
/// if we mutate part, than we should reserve space on the same disk, because mutations possible can create hardlinks
if (is_mutation)
reserved_space = future_part_.parts[0]->disk->reserve(total_size);
reserved_space = storage.tryReserveSpace(total_size, future_part_.parts[0]->disk);
else
reserved_space = storage.reserveSpace(total_size);
{
MergeTreeDataPart::TTLInfos ttl_infos;
for (auto & part_ptr : future_part_.parts)
ttl_infos.update(part_ptr->ttl_infos);
reserved_space = storage.tryReserveSpacePreferringTTLRules(total_size, ttl_infos, time(nullptr));
}
if (!reserved_space)
{
if (is_mutation)

Some files were not shown because too many files have changed in this diff Show More