mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #26352 from ClickHouse/revert-25774-datatype-date32
Revert "Datatype Date32, support range 1925 to 2283"
This commit is contained in:
commit
94a210650f
@ -18,8 +18,6 @@
|
||||
|
||||
#define DATE_LUT_MAX (0xFFFFFFFFU - 86400)
|
||||
#define DATE_LUT_MAX_DAY_NUM 0xFFFF
|
||||
/// Max int value of Date32, DATE LUT cache size minus daynum_offset_epoch
|
||||
#define DATE_LUT_MAX_EXTEND_DAY_NUM (DATE_LUT_SIZE - 16436)
|
||||
|
||||
/// A constant to add to time_t so every supported time point becomes non-negative and still has the same remainder of division by 3600.
|
||||
/// If we treat "remainder of division" operation in the sense of modular arithmetic (not like in C++).
|
||||
@ -272,8 +270,6 @@ public:
|
||||
auto getOffsetAtStartOfEpoch() const { return offset_at_start_of_epoch; }
|
||||
auto getTimeOffsetAtStartOfLUT() const { return offset_at_start_of_lut; }
|
||||
|
||||
auto getDayNumOffsetEpoch() const { return daynum_offset_epoch; }
|
||||
|
||||
/// All functions below are thread-safe; arguments are not checked.
|
||||
|
||||
inline ExtendedDayNum toDayNum(ExtendedDayNum d) const
|
||||
@ -930,17 +926,15 @@ public:
|
||||
{
|
||||
if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
|
||||
return LUTIndex(0);
|
||||
auto year_lut_index = (year - DATE_LUT_MIN_YEAR) * 12 + month - 1;
|
||||
UInt32 index = years_months_lut[year_lut_index].toUnderType() + day_of_month - 1;
|
||||
/// When date is out of range, default value is DATE_LUT_SIZE - 1 (2283-11-11)
|
||||
return LUTIndex{std::min(index, static_cast<UInt32>(DATE_LUT_SIZE - 1))};
|
||||
|
||||
return LUTIndex{years_months_lut[(year - DATE_LUT_MIN_YEAR) * 12 + month - 1] + day_of_month - 1};
|
||||
}
|
||||
|
||||
/// Create DayNum from year, month, day of month.
|
||||
inline ExtendedDayNum makeDayNum(Int16 year, UInt8 month, UInt8 day_of_month, Int32 default_error_day_num = 0) const
|
||||
inline ExtendedDayNum makeDayNum(Int16 year, UInt8 month, UInt8 day_of_month) const
|
||||
{
|
||||
if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
|
||||
return ExtendedDayNum(default_error_day_num);
|
||||
return ExtendedDayNum(0);
|
||||
|
||||
return toDayNum(makeLUTIndex(year, month, day_of_month));
|
||||
}
|
||||
@ -1097,9 +1091,9 @@ public:
|
||||
return lut[new_index].date + time;
|
||||
}
|
||||
|
||||
inline NO_SANITIZE_UNDEFINED Time addWeeks(Time t, Int32 delta) const
|
||||
inline NO_SANITIZE_UNDEFINED Time addWeeks(Time t, Int64 delta) const
|
||||
{
|
||||
return addDays(t, static_cast<Int64>(delta) * 7);
|
||||
return addDays(t, delta * 7);
|
||||
}
|
||||
|
||||
inline UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const
|
||||
@ -1164,14 +1158,14 @@ public:
|
||||
return toDayNum(addMonthsIndex(d, delta));
|
||||
}
|
||||
|
||||
inline Time NO_SANITIZE_UNDEFINED addQuarters(Time t, Int32 delta) const
|
||||
inline Time NO_SANITIZE_UNDEFINED addQuarters(Time t, Int64 delta) const
|
||||
{
|
||||
return addMonths(t, static_cast<Int64>(delta) * 3);
|
||||
return addMonths(t, delta * 3);
|
||||
}
|
||||
|
||||
inline ExtendedDayNum addQuarters(ExtendedDayNum d, Int32 delta) const
|
||||
inline ExtendedDayNum addQuarters(ExtendedDayNum d, Int64 delta) const
|
||||
{
|
||||
return addMonths(d, static_cast<Int64>(delta) * 3);
|
||||
return addMonths(d, delta * 3);
|
||||
}
|
||||
|
||||
template <typename DateOrTime>
|
||||
|
@ -70,14 +70,6 @@ public:
|
||||
m_day = values.day_of_month;
|
||||
}
|
||||
|
||||
explicit LocalDate(ExtendedDayNum day_num)
|
||||
{
|
||||
const auto & values = DateLUT::instance().getValues(day_num);
|
||||
m_year = values.year;
|
||||
m_month = values.month;
|
||||
m_day = values.day_of_month;
|
||||
}
|
||||
|
||||
LocalDate(unsigned short year_, unsigned char month_, unsigned char day_)
|
||||
: m_year(year_), m_month(month_), m_day(day_)
|
||||
{
|
||||
@ -106,12 +98,6 @@ public:
|
||||
return DayNum(lut.makeDayNum(m_year, m_month, m_day).toUnderType());
|
||||
}
|
||||
|
||||
ExtendedDayNum getExtenedDayNum() const
|
||||
{
|
||||
const auto & lut = DateLUT::instance();
|
||||
return ExtendedDayNum (lut.makeDayNum(m_year, m_month, m_day).toUnderType());
|
||||
}
|
||||
|
||||
operator DayNum() const
|
||||
{
|
||||
return getDayNum();
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <AggregateFunctions/AggregateFunctionSequenceMatch.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
#include <common/range.h>
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
@ -50,8 +49,6 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
|
||||
return res;
|
||||
else if (which.isDate())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDate::FieldType, Data>>(argument_types);
|
||||
else if (which.isDate32())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDate32::FieldType, Data>>(argument_types);
|
||||
else if (which.isDateTime())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDateTime::FieldType, Data>>(argument_types);
|
||||
else if (which.isStringOrFixedString())
|
||||
@ -98,8 +95,6 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
|
||||
return res;
|
||||
else if (which.isDate())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDate::FieldType, Data<DataTypeDate::FieldType>>>(argument_types);
|
||||
else if (which.isDate32())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDate32::FieldType, Data<DataTypeDate32::FieldType>>>(argument_types);
|
||||
else if (which.isDateTime())
|
||||
return std::make_shared<AggregateFunctionUniq<DataTypeDateTime::FieldType, Data<DataTypeDateTime::FieldType>>>(argument_types);
|
||||
else if (which.isStringOrFixedString())
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
#include <functional>
|
||||
@ -52,8 +51,6 @@ namespace
|
||||
return res;
|
||||
else if (which.isDate())
|
||||
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate::FieldType>>(argument_types, params);
|
||||
else if (which.isDate32())
|
||||
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate32::FieldType>>(argument_types, params);
|
||||
else if (which.isDateTime())
|
||||
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDateTime::FieldType>>(argument_types, params);
|
||||
else if (which.isStringOrFixedString())
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <AggregateFunctions/AggregateFunctionUniqUpTo.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
@ -62,8 +61,6 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c
|
||||
return res;
|
||||
else if (which.isDate())
|
||||
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDate::FieldType>>(threshold, argument_types, params);
|
||||
else if (which.isDate32())
|
||||
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDate32::FieldType>>(threshold, argument_types, params);
|
||||
else if (which.isDateTime())
|
||||
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeDateTime::FieldType>>(threshold, argument_types, params);
|
||||
else if (which.isStringOrFixedString())
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
#include <common/range.h>
|
||||
|
@ -62,8 +62,6 @@ void ExternalResultDescription::init(const Block & sample_block_)
|
||||
types.emplace_back(ValueType::vtString, is_nullable);
|
||||
else if (which.isDate())
|
||||
types.emplace_back(ValueType::vtDate, is_nullable);
|
||||
else if (which.isDate32())
|
||||
types.emplace_back(ValueType::vtDate32, is_nullable);
|
||||
else if (which.isDateTime())
|
||||
types.emplace_back(ValueType::vtDateTime, is_nullable);
|
||||
else if (which.isUUID())
|
||||
|
@ -26,7 +26,6 @@ struct ExternalResultDescription
|
||||
vtEnum16,
|
||||
vtString,
|
||||
vtDate,
|
||||
vtDate32,
|
||||
vtDateTime,
|
||||
vtUUID,
|
||||
vtDateTime64,
|
||||
|
@ -90,9 +90,6 @@ void insertPostgreSQLValue(
|
||||
case ExternalResultDescription::ValueType::vtDate:
|
||||
assert_cast<ColumnUInt16 &>(column).insertValue(UInt16{LocalDate{std::string(value)}.getDayNum()});
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtDate32:
|
||||
assert_cast<ColumnInt32 &>(column).insertValue(Int32{LocalDate{std::string(value)}.getExtenedDayNum()});
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtDateTime:
|
||||
{
|
||||
ReadBufferFromString in(value);
|
||||
|
@ -39,7 +39,6 @@ enum class TypeIndex
|
||||
Float32,
|
||||
Float64,
|
||||
Date,
|
||||
Date32,
|
||||
DateTime,
|
||||
DateTime64,
|
||||
String,
|
||||
@ -258,7 +257,6 @@ inline constexpr const char * getTypeName(TypeIndex idx)
|
||||
case TypeIndex::Float32: return "Float32";
|
||||
case TypeIndex::Float64: return "Float64";
|
||||
case TypeIndex::Date: return "Date";
|
||||
case TypeIndex::Date32: return "Date32";
|
||||
case TypeIndex::DateTime: return "DateTime";
|
||||
case TypeIndex::DateTime64: return "DateTime64";
|
||||
case TypeIndex::String: return "String";
|
||||
|
@ -192,7 +192,6 @@ bool callOnIndexAndDataType(TypeIndex number, F && f, ExtraArgs && ... args)
|
||||
case TypeIndex::Decimal256: return f(TypePair<DataTypeDecimal<Decimal256>, T>(), std::forward<ExtraArgs>(args)...);
|
||||
|
||||
case TypeIndex::Date: return f(TypePair<DataTypeDate, T>(), std::forward<ExtraArgs>(args)...);
|
||||
case TypeIndex::Date32: return f(TypePair<DataTypeDate, T>(), std::forward<ExtraArgs>(args)...);
|
||||
case TypeIndex::DateTime: return f(TypePair<DataTypeDateTime, T>(), std::forward<ExtraArgs>(args)...);
|
||||
case TypeIndex::DateTime64: return f(TypePair<DataTypeDateTime64, T>(), std::forward<ExtraArgs>(args)...);
|
||||
|
||||
|
@ -1,23 +0,0 @@
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationDate32.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
bool DataTypeDate32::equals(const IDataType & rhs) const
|
||||
{
|
||||
return typeid(rhs) == typeid(*this);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeDate32::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationDate32>();
|
||||
}
|
||||
|
||||
void registerDataTypeDate32(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType(
|
||||
"Date32", [] { return DataTypePtr(std::make_shared<DataTypeDate32>()); }, DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
@ -1,23 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class DataTypeDate32 final : public DataTypeNumberBase<Int32>
|
||||
{
|
||||
public:
|
||||
static constexpr auto family_name = "Date32";
|
||||
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Date32; }
|
||||
const char * getFamilyName() const override { return family_name; }
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
protected:
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
}
|
@ -194,7 +194,6 @@ DataTypeFactory::DataTypeFactory()
|
||||
registerDataTypeNumbers(*this);
|
||||
registerDataTypeDecimal(*this);
|
||||
registerDataTypeDate(*this);
|
||||
registerDataTypeDate32(*this);
|
||||
registerDataTypeDateTime(*this);
|
||||
registerDataTypeString(*this);
|
||||
registerDataTypeFixedString(*this);
|
||||
|
@ -69,7 +69,6 @@ private:
|
||||
void registerDataTypeNumbers(DataTypeFactory & factory);
|
||||
void registerDataTypeDecimal(DataTypeFactory & factory);
|
||||
void registerDataTypeDate(DataTypeFactory & factory);
|
||||
void registerDataTypeDate32(DataTypeFactory & factory);
|
||||
void registerDataTypeDateTime(DataTypeFactory & factory);
|
||||
void registerDataTypeString(DataTypeFactory & factory);
|
||||
void registerDataTypeFixedString(DataTypeFactory & factory);
|
||||
|
@ -78,8 +78,6 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat
|
||||
return creator(static_cast<ColumnFixedString *>(nullptr));
|
||||
else if (which.isDate())
|
||||
return creator(static_cast<ColumnVector<UInt16> *>(nullptr));
|
||||
else if (which.isDate32())
|
||||
return creator(static_cast<ColumnVector<Int32> *>(nullptr));
|
||||
else if (which.isDateTime())
|
||||
return creator(static_cast<ColumnVector<UInt32> *>(nullptr));
|
||||
else if (which.isUUID())
|
||||
|
@ -322,10 +322,8 @@ struct WhichDataType
|
||||
constexpr bool isEnum() const { return isEnum8() || isEnum16(); }
|
||||
|
||||
constexpr bool isDate() const { return idx == TypeIndex::Date; }
|
||||
constexpr bool isDate32() const { return idx == TypeIndex::Date32; }
|
||||
constexpr bool isDateTime() const { return idx == TypeIndex::DateTime; }
|
||||
constexpr bool isDateTime64() const { return idx == TypeIndex::DateTime64; }
|
||||
constexpr bool isDateOrDate32() const { return isDate() || isDate32(); }
|
||||
|
||||
constexpr bool isString() const { return idx == TypeIndex::String; }
|
||||
constexpr bool isFixedString() const { return idx == TypeIndex::FixedString; }
|
||||
@ -349,10 +347,6 @@ struct WhichDataType
|
||||
template <typename T>
|
||||
inline bool isDate(const T & data_type) { return WhichDataType(data_type).isDate(); }
|
||||
template <typename T>
|
||||
inline bool isDate32(const T & data_type) { return WhichDataType(data_type).isDate32(); }
|
||||
template <typename T>
|
||||
inline bool isDateOrDate32(const T & data_type) { return WhichDataType(data_type).isDateOrDate32(); }
|
||||
template <typename T>
|
||||
inline bool isDateTime(const T & data_type) { return WhichDataType(data_type).isDateTime(); }
|
||||
template <typename T>
|
||||
inline bool isDateTime64(const T & data_type) { return WhichDataType(data_type).isDateTime64(); }
|
||||
|
@ -1,78 +0,0 @@
|
||||
#include <DataTypes/Serializations/SerializationDate32.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
void SerializationDate32::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
writeDateText(ExtendedDayNum(assert_cast<const ColumnInt32 &>(column).getData()[row_num]), ostr);
|
||||
}
|
||||
|
||||
void SerializationDate32::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextEscaped(column, istr, settings);
|
||||
}
|
||||
|
||||
void SerializationDate32::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
ExtendedDayNum x;
|
||||
readDateText(x, istr);
|
||||
assert_cast<ColumnInt32 &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
void SerializationDate32::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationDate32::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 SerializationDate32::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
ExtendedDayNum x;
|
||||
assertChar('\'', istr);
|
||||
readDateText(x, istr);
|
||||
assertChar('\'', istr);
|
||||
assert_cast<ColumnInt32 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
|
||||
}
|
||||
|
||||
void SerializationDate32::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 SerializationDate32::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
ExtendedDayNum x;
|
||||
assertChar('"', istr);
|
||||
readDateText(x, istr);
|
||||
assertChar('"', istr);
|
||||
assert_cast<ColumnInt32 &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
void SerializationDate32::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 SerializationDate32::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
LocalDate value;
|
||||
readCSV(value, istr);
|
||||
assert_cast<ColumnInt32 &>(column).getData().push_back(value.getExtenedDayNum());
|
||||
}
|
||||
}
|
@ -1,21 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class SerializationDate32 final : public SerializationNumber<Int32>
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void 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;
|
||||
};
|
||||
}
|
@ -16,7 +16,6 @@ SRCS(
|
||||
DataTypeCustomIPv4AndIPv6.cpp
|
||||
DataTypeCustomSimpleAggregateFunction.cpp
|
||||
DataTypeDate.cpp
|
||||
DataTypeDate32.cpp
|
||||
DataTypeDateTime.cpp
|
||||
DataTypeDateTime64.cpp
|
||||
DataTypeDecimalBase.cpp
|
||||
@ -46,7 +45,6 @@ SRCS(
|
||||
Serializations/SerializationArray.cpp
|
||||
Serializations/SerializationCustomSimpleText.cpp
|
||||
Serializations/SerializationDate.cpp
|
||||
Serializations/SerializationDate32.cpp
|
||||
Serializations/SerializationDateTime.cpp
|
||||
Serializations/SerializationDateTime64.cpp
|
||||
Serializations/SerializationDecimal.cpp
|
||||
|
@ -42,11 +42,6 @@ struct ToYearWeekImpl
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
|
||||
return yw.first * 100 + yw.second;
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(ExtendedDayNum (d), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
|
||||
return yw.first * 100 + yw.second;
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode | static_cast<UInt32>(WeekModeFlag::YEAR));
|
||||
@ -70,10 +65,6 @@ struct ToStartOfWeekImpl
|
||||
return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t), week_mode);
|
||||
// return time_zone.toFirstDayNumOfWeek(t, week_mode);
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d), week_mode);
|
||||
@ -97,11 +88,6 @@ struct ToWeekImpl
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(ExtendedDayNum(d), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
|
||||
|
@ -46,7 +46,6 @@ struct ZeroTransform
|
||||
{
|
||||
static inline UInt16 execute(Int64, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(UInt32, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(Int32, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(UInt16, const DateLUTImpl &) { return 0; }
|
||||
};
|
||||
|
||||
@ -62,10 +61,6 @@ struct ToDateImpl
|
||||
{
|
||||
return UInt16(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl &)
|
||||
{
|
||||
return d;
|
||||
@ -87,10 +82,6 @@ struct ToStartOfDayImpl
|
||||
{
|
||||
return time_zone.toDate(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDate(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDate(ExtendedDayNum(d));
|
||||
@ -113,10 +104,6 @@ struct ToMondayImpl
|
||||
//return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t));
|
||||
return time_zone.toFirstDayNumOfWeek(t);
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d));
|
||||
@ -137,10 +124,6 @@ struct ToStartOfMonthImpl
|
||||
{
|
||||
return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfMonth(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfMonth(ExtendedDayNum(d));
|
||||
@ -161,10 +144,6 @@ struct ToStartOfQuarterImpl
|
||||
{
|
||||
return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfQuarter(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfQuarter(ExtendedDayNum(d));
|
||||
@ -185,10 +164,6 @@ struct ToStartOfYearImpl
|
||||
{
|
||||
return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfYear(ExtendedDayNum(d));
|
||||
@ -211,10 +186,7 @@ struct ToTimeImpl
|
||||
{
|
||||
return time_zone.toTime(t) + 86400;
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -235,10 +207,6 @@ struct ToStartOfMinuteImpl
|
||||
{
|
||||
return time_zone.toStartOfMinute(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -274,10 +242,6 @@ struct ToStartOfSecondImpl
|
||||
{
|
||||
throw Exception("Illegal type DateTime of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -298,10 +262,6 @@ struct ToStartOfFiveMinuteImpl
|
||||
{
|
||||
return time_zone.toStartOfFiveMinute(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -322,10 +282,6 @@ struct ToStartOfTenMinutesImpl
|
||||
{
|
||||
return time_zone.toStartOfTenMinutes(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -346,10 +302,6 @@ struct ToStartOfFifteenMinutesImpl
|
||||
{
|
||||
return time_zone.toStartOfFifteenMinutes(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -374,11 +326,6 @@ struct TimeSlotImpl
|
||||
return t / 1800 * 1800;
|
||||
}
|
||||
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -401,11 +348,6 @@ struct ToStartOfHourImpl
|
||||
return time_zone.toStartOfHour(t);
|
||||
}
|
||||
|
||||
static inline UInt32 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -426,10 +368,6 @@ struct ToYearImpl
|
||||
{
|
||||
return time_zone.toYear(t);
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(ExtendedDayNum(d));
|
||||
@ -450,10 +388,6 @@ struct ToQuarterImpl
|
||||
{
|
||||
return time_zone.toQuarter(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toQuarter(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toQuarter(ExtendedDayNum(d));
|
||||
@ -474,10 +408,6 @@ struct ToMonthImpl
|
||||
{
|
||||
return time_zone.toMonth(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toMonth(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toMonth(ExtendedDayNum(d));
|
||||
@ -498,10 +428,6 @@ struct ToDayOfMonthImpl
|
||||
{
|
||||
return time_zone.toDayOfMonth(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfMonth(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfMonth(ExtendedDayNum(d));
|
||||
@ -522,10 +448,6 @@ struct ToDayOfWeekImpl
|
||||
{
|
||||
return time_zone.toDayOfWeek(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfWeek(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfWeek(ExtendedDayNum(d));
|
||||
@ -546,10 +468,6 @@ struct ToDayOfYearImpl
|
||||
{
|
||||
return time_zone.toDayOfYear(t);
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toDayOfYear(ExtendedDayNum(d));
|
||||
@ -570,10 +488,7 @@ struct ToHourImpl
|
||||
{
|
||||
return time_zone.toHour(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
|
||||
static inline UInt8 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -596,11 +511,6 @@ struct TimezoneOffsetImpl
|
||||
return time_zone.timezoneOffset(t);
|
||||
}
|
||||
|
||||
static inline time_t execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
|
||||
static inline time_t execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -621,10 +531,6 @@ struct ToMinuteImpl
|
||||
{
|
||||
return time_zone.toMinute(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt8 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -645,10 +551,6 @@ struct ToSecondImpl
|
||||
{
|
||||
return time_zone.toSecond(t);
|
||||
}
|
||||
static inline UInt8 execute(Int32, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
}
|
||||
static inline UInt8 execute(UInt16, const DateLUTImpl &)
|
||||
{
|
||||
return dateIsNotSupported(name);
|
||||
@ -669,10 +571,6 @@ struct ToISOYearImpl
|
||||
{
|
||||
return time_zone.toISOYear(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOYear(ExtendedDayNum(d));
|
||||
@ -693,10 +591,6 @@ struct ToStartOfISOYearImpl
|
||||
{
|
||||
return time_zone.toFirstDayNumOfISOYear(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfISOYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toFirstDayNumOfISOYear(ExtendedDayNum(d));
|
||||
@ -717,10 +611,6 @@ struct ToISOWeekImpl
|
||||
{
|
||||
return time_zone.toISOWeek(time_zone.toDayNum(t));
|
||||
}
|
||||
static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOWeek(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toISOWeek(ExtendedDayNum(d));
|
||||
@ -741,10 +631,6 @@ struct ToRelativeYearNumImpl
|
||||
{
|
||||
return time_zone.toYear(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toYear(ExtendedDayNum(d));
|
||||
@ -765,10 +651,6 @@ struct ToRelativeQuarterNumImpl
|
||||
{
|
||||
return time_zone.toRelativeQuarterNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeQuarterNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeQuarterNum(ExtendedDayNum(d));
|
||||
@ -789,10 +671,6 @@ struct ToRelativeMonthNumImpl
|
||||
{
|
||||
return time_zone.toRelativeMonthNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMonthNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMonthNum(ExtendedDayNum(d));
|
||||
@ -813,10 +691,6 @@ struct ToRelativeWeekNumImpl
|
||||
{
|
||||
return time_zone.toRelativeWeekNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeWeekNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeWeekNum(ExtendedDayNum(d));
|
||||
@ -837,10 +711,6 @@ struct ToRelativeDayNumImpl
|
||||
{
|
||||
return time_zone.toDayNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt16 execute(Int32 d, const DateLUTImpl &)
|
||||
{
|
||||
return static_cast<ExtendedDayNum>(d);
|
||||
}
|
||||
static inline UInt16 execute(UInt16 d, const DateLUTImpl &)
|
||||
{
|
||||
return static_cast<DayNum>(d);
|
||||
@ -862,10 +732,6 @@ struct ToRelativeHourNumImpl
|
||||
{
|
||||
return time_zone.toRelativeHourNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeHourNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeHourNum(ExtendedDayNum(d));
|
||||
@ -886,10 +752,6 @@ struct ToRelativeMinuteNumImpl
|
||||
{
|
||||
return time_zone.toRelativeMinuteNum(static_cast<time_t>(t));
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMinuteNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toRelativeMinuteNum(ExtendedDayNum(d));
|
||||
@ -910,10 +772,6 @@ struct ToRelativeSecondNumImpl
|
||||
{
|
||||
return t;
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d));
|
||||
@ -934,10 +792,6 @@ struct ToYYYYMMImpl
|
||||
{
|
||||
return time_zone.toNumYYYYMM(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMM(static_cast<ExtendedDayNum>(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMM(static_cast<DayNum>(d));
|
||||
@ -958,10 +812,6 @@ struct ToYYYYMMDDImpl
|
||||
{
|
||||
return time_zone.toNumYYYYMMDD(t);
|
||||
}
|
||||
static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDD(static_cast<ExtendedDayNum>(d));
|
||||
}
|
||||
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDD(static_cast<DayNum>(d));
|
||||
@ -982,10 +832,6 @@ struct ToYYYYMMDDhhmmssImpl
|
||||
{
|
||||
return time_zone.toNumYYYYMMDDhhmmss(t);
|
||||
}
|
||||
static inline UInt64 execute(Int32 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast<ExtendedDayNum>(d)));
|
||||
}
|
||||
static inline UInt64 execute(UInt16 d, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast<DayNum>(d)));
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <Functions/CustomWeekTransforms.h>
|
||||
@ -36,7 +35,7 @@ public:
|
||||
{
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
@ -44,7 +43,7 @@ public:
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
@ -60,7 +59,7 @@ public:
|
||||
}
|
||||
else if (arguments.size() == 3)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
@ -106,9 +105,6 @@ public:
|
||||
if (which.isDate())
|
||||
return CustomWeekTransformImpl<DataTypeDate, ToDataType>::execute(
|
||||
arguments, result_type, input_rows_count, Transform{});
|
||||
else if (which.isDate32())
|
||||
return CustomWeekTransformImpl<DataTypeDate32, ToDataType>::execute(
|
||||
arguments, result_type, input_rows_count, Transform{});
|
||||
else if (which.isDateTime())
|
||||
return CustomWeekTransformImpl<DataTypeDateTime, ToDataType>::execute(
|
||||
arguments, result_type, input_rows_count, Transform{});
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <common/DateLUTImpl.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
|
||||
@ -51,11 +50,7 @@ struct AddSecondsImpl
|
||||
{
|
||||
return t + delta;
|
||||
}
|
||||
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// use default datetime64 scale
|
||||
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta) * 1000;
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d)) + delta;
|
||||
@ -76,11 +71,7 @@ struct AddMinutesImpl
|
||||
{
|
||||
return t + delta * 60;
|
||||
}
|
||||
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// use default datetime64 scale
|
||||
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 60) * 1000;
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 60;
|
||||
@ -100,11 +91,7 @@ struct AddHoursImpl
|
||||
{
|
||||
return t + delta * 3600;
|
||||
}
|
||||
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// use default datetime64 scale
|
||||
return (time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 3600) * 1000;
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.fromDayNum(ExtendedDayNum(d)) + delta * 3600;
|
||||
@ -130,11 +117,6 @@ struct AddDaysImpl
|
||||
{
|
||||
return d + delta;
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int64 delta, const DateLUTImpl &)
|
||||
{
|
||||
return d + delta;
|
||||
}
|
||||
};
|
||||
|
||||
struct AddWeeksImpl
|
||||
@ -142,22 +124,17 @@ struct AddWeeksImpl
|
||||
static constexpr auto name = "addWeeks";
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED DecimalUtils::DecimalComponents<DateTime64>
|
||||
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int32 delta, const DateLUTImpl & time_zone)
|
||||
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return {time_zone.addWeeks(t.whole, delta), t.fractional};
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int32 delta, const DateLUTImpl & time_zone)
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addWeeks(t, delta);
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl &)
|
||||
{
|
||||
return d + delta * 7;
|
||||
}
|
||||
|
||||
static inline NO_SANITIZE_UNDEFINED Int32 execute(Int32 d, Int32 delta, const DateLUTImpl &)
|
||||
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
|
||||
{
|
||||
return d + delta * 7;
|
||||
}
|
||||
@ -182,11 +159,6 @@ struct AddMonthsImpl
|
||||
{
|
||||
return time_zone.addMonths(ExtendedDayNum(d), delta);
|
||||
}
|
||||
|
||||
static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addMonths(ExtendedDayNum(d), delta);
|
||||
}
|
||||
};
|
||||
|
||||
struct AddQuartersImpl
|
||||
@ -194,22 +166,17 @@ struct AddQuartersImpl
|
||||
static constexpr auto name = "addQuarters";
|
||||
|
||||
static inline DecimalUtils::DecimalComponents<DateTime64>
|
||||
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int32 delta, const DateLUTImpl & time_zone)
|
||||
execute(DecimalUtils::DecimalComponents<DateTime64> t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return {time_zone.addQuarters(t.whole, delta), t.fractional};
|
||||
}
|
||||
|
||||
static inline UInt32 execute(UInt32 t, Int32 delta, const DateLUTImpl & time_zone)
|
||||
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addQuarters(t, delta);
|
||||
}
|
||||
|
||||
static inline UInt16 execute(UInt16 d, Int32 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addQuarters(ExtendedDayNum(d), delta);
|
||||
}
|
||||
|
||||
static inline Int32 execute(Int32 d, Int32 delta, const DateLUTImpl & time_zone)
|
||||
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addQuarters(ExtendedDayNum(d), delta);
|
||||
}
|
||||
@ -234,11 +201,6 @@ struct AddYearsImpl
|
||||
{
|
||||
return time_zone.addYears(ExtendedDayNum(d), delta);
|
||||
}
|
||||
|
||||
static inline Int32 execute(Int32 d, Int64 delta, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.addYears(ExtendedDayNum(d), delta);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Transform>
|
||||
@ -380,7 +342,7 @@ 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 = DataTypeDate32; };
|
||||
template <> struct ResultDataTypeMap<Int32> { using ResultDataType = DataTypeDateTime; };
|
||||
template <> struct ResultDataTypeMap<DateTime64> { using ResultDataType = DataTypeDateTime64; };
|
||||
template <> struct ResultDataTypeMap<Int64> { using ResultDataType = DataTypeDateTime64; };
|
||||
}
|
||||
@ -413,7 +375,7 @@ public:
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() +
|
||||
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
@ -436,8 +398,6 @@ public:
|
||||
{
|
||||
case TypeIndex::Date:
|
||||
return resolveReturnType<DataTypeDate>(arguments);
|
||||
case TypeIndex::Date32:
|
||||
return resolveReturnType<DataTypeDate32>(arguments);
|
||||
case TypeIndex::DateTime:
|
||||
return resolveReturnType<DataTypeDateTime>(arguments);
|
||||
case TypeIndex::DateTime64:
|
||||
@ -477,23 +437,16 @@ public:
|
||||
|
||||
if constexpr (std::is_same_v<ResultDataType, DataTypeDate>)
|
||||
return std::make_shared<DataTypeDate>();
|
||||
else if constexpr (std::is_same_v<ResultDataType, DataTypeDate32>)
|
||||
return std::make_shared<DataTypeDate32>();
|
||||
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>)
|
||||
{
|
||||
if (typeid_cast<const DataTypeDateTime64 *>(arguments[0].type.get()))
|
||||
{
|
||||
const auto & datetime64_type = assert_cast<const DataTypeDateTime64 &>(*arguments[0].type);
|
||||
return std::make_shared<DataTypeDateTime64>(datetime64_type.getScale(), extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
|
||||
}
|
||||
else
|
||||
{
|
||||
return std::make_shared<DataTypeDateTime64>(DataTypeDateTime64::default_scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
|
||||
}
|
||||
// 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
|
||||
{
|
||||
@ -517,11 +470,6 @@ public:
|
||||
return DateTimeAddIntervalImpl<DataTypeDate, TransformResultDataType<DataTypeDate>, Transform>::execute(
|
||||
Transform{}, arguments, result_type);
|
||||
}
|
||||
else if (which.isDate32())
|
||||
{
|
||||
return DateTimeAddIntervalImpl<DataTypeDate32, TransformResultDataType<DataTypeDate32>, Transform>::execute(
|
||||
Transform{}, arguments, result_type);
|
||||
}
|
||||
else if (which.isDateTime())
|
||||
{
|
||||
return DateTimeAddIntervalImpl<DataTypeDateTime, TransformResultDataType<DataTypeDateTime>, Transform>::execute(
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
@ -39,7 +38,7 @@ public:
|
||||
{
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
@ -47,7 +46,7 @@ public:
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName()
|
||||
+ ". Should be a date or a date with time",
|
||||
@ -58,7 +57,7 @@ public:
|
||||
"must be of type Date or DateTime. The 2nd argument (optional) must be "
|
||||
"a constant string with timezone name",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
if ((isDate(arguments[0].type) || isDate32(arguments[0].type)) && (std::is_same_v<ToDataType, DataTypeDate> || std::is_same_v<ToDataType, DataTypeDate32>))
|
||||
if (isDate(arguments[0].type) && std::is_same_v<ToDataType, DataTypeDate>)
|
||||
throw Exception(
|
||||
"The timezone argument of function " + getName() + " is allowed only when the 1st argument has the type DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -104,8 +103,6 @@ public:
|
||||
|
||||
if (which.isDate())
|
||||
return DateTimeTransformImpl<DataTypeDate, ToDataType, Transform>::execute(arguments, result_type, input_rows_count);
|
||||
else if (which.isDate32())
|
||||
return DateTimeTransformImpl<DataTypeDate32, ToDataType, Transform>::execute(arguments, result_type, input_rows_count);
|
||||
else if (which.isDateTime())
|
||||
return DateTimeTransformImpl<DataTypeDateTime, ToDataType, Transform>::execute(arguments, result_type, input_rows_count);
|
||||
else if (which.isDateTime64())
|
||||
@ -149,12 +146,6 @@ public:
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic : is_not_monotonic;
|
||||
}
|
||||
else if (checkAndGetDataType<DataTypeDate32>(&type))
|
||||
{
|
||||
return Transform::FactorTransform::execute(Int32(left.get<UInt64>()), date_lut)
|
||||
== Transform::FactorTransform::execute(Int32(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic : is_not_monotonic;
|
||||
}
|
||||
else
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), date_lut)
|
||||
|
@ -1081,7 +1081,7 @@ public:
|
||||
const DataTypeTuple * right_tuple = checkAndGetDataType<DataTypeTuple>(arguments[1].get());
|
||||
|
||||
bool both_represented_by_number = arguments[0]->isValueRepresentedByNumber() && arguments[1]->isValueRepresentedByNumber();
|
||||
bool has_date = left.isDateOrDate32() || right.isDateOrDate32();
|
||||
bool has_date = left.isDate() || right.isDate();
|
||||
|
||||
if (!((both_represented_by_number && !has_date) /// Do not allow to compare date and number.
|
||||
|| (left.isStringOrFixedString() || right.isStringOrFixedString()) /// Everything can be compared with string by conversion.
|
||||
|
@ -32,7 +32,7 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToDate>();
|
||||
/// MysQL compatibility alias.
|
||||
factory.registerFunction<FunctionToDate>("DATE", FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<FunctionToDate32>();
|
||||
|
||||
factory.registerFunction<FunctionToDateTime>();
|
||||
factory.registerFunction<FunctionToDateTime32>();
|
||||
factory.registerFunction<FunctionToDateTime64>();
|
||||
@ -62,7 +62,6 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToFloat32OrZero>();
|
||||
factory.registerFunction<FunctionToFloat64OrZero>();
|
||||
factory.registerFunction<FunctionToDateOrZero>();
|
||||
factory.registerFunction<FunctionToDate32OrZero>();
|
||||
factory.registerFunction<FunctionToDateTimeOrZero>();
|
||||
factory.registerFunction<FunctionToDateTime64OrZero>();
|
||||
|
||||
@ -88,7 +87,6 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionToFloat32OrNull>();
|
||||
factory.registerFunction<FunctionToFloat64OrNull>();
|
||||
factory.registerFunction<FunctionToDateOrNull>();
|
||||
factory.registerFunction<FunctionToDate32OrNull>();
|
||||
factory.registerFunction<FunctionToDateTimeOrNull>();
|
||||
factory.registerFunction<FunctionToDateTime64OrNull>();
|
||||
|
||||
|
@ -12,7 +12,6 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
@ -710,8 +709,6 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
|
||||
|
||||
if constexpr (std::is_same_v<FromDataType, DataTypeDate>)
|
||||
data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDate32>)
|
||||
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>)
|
||||
@ -814,14 +811,6 @@ inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb
|
||||
x = tmp;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline void parseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
||||
{
|
||||
ExtendedDayNum tmp(0);
|
||||
readDateText(tmp, rb);
|
||||
x = tmp;
|
||||
}
|
||||
|
||||
// NOTE: no need of extra overload of DateTime64, since readDateTimeText64 has different signature and that case is explicitly handled in the calling code.
|
||||
template <>
|
||||
inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
|
||||
@ -862,16 +851,6 @@ inline bool tryParseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer &
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline bool tryParseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
|
||||
{
|
||||
ExtendedDayNum tmp(0);
|
||||
if (!tryReadDateText(tmp, rb))
|
||||
return false;
|
||||
x = tmp;
|
||||
return true;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
|
||||
{
|
||||
@ -1296,7 +1275,6 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name, ConvertDefaultBeha
|
||||
|
||||
/// Declared early because used below.
|
||||
struct NameToDate { static constexpr auto name = "toDate"; };
|
||||
struct NameToDate32 { static constexpr auto name = "toDate32"; };
|
||||
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
|
||||
struct NameToDateTime32 { static constexpr auto name = "toDateTime32"; };
|
||||
struct NameToDateTime64 { static constexpr auto name = "toDateTime64"; };
|
||||
@ -1979,7 +1957,7 @@ struct ToDateMonotonicity
|
||||
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
|
||||
{
|
||||
auto which = WhichDataType(type);
|
||||
if (which.isDate() || which.isDate32() || which.isDateTime() || which.isDateTime64() || which.isInt8() || which.isInt16() || which.isUInt8() || which.isUInt16())
|
||||
if (which.isDate() || which.isDateTime() || which.isDateTime64() || which.isInt8() || which.isInt16() || which.isUInt8() || which.isUInt16())
|
||||
return {true, true, true};
|
||||
else if (
|
||||
(which.isUInt() && ((left.isNull() || left.get<UInt64>() < 0xFFFF) && (right.isNull() || right.get<UInt64>() >= 0xFFFF)))
|
||||
@ -2080,7 +2058,6 @@ using FunctionToInt256 = FunctionConvert<DataTypeInt256, NameToInt256, ToNumberM
|
||||
using FunctionToFloat32 = FunctionConvert<DataTypeFloat32, NameToFloat32, ToNumberMonotonicity<Float32>>;
|
||||
using FunctionToFloat64 = FunctionConvert<DataTypeFloat64, NameToFloat64, ToNumberMonotonicity<Float64>>;
|
||||
using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, ToDateMonotonicity>;
|
||||
using FunctionToDate32 = FunctionConvert<DataTypeDate32, NameToDate32, ToDateMonotonicity>;
|
||||
using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToDateTimeMonotonicity>;
|
||||
using FunctionToDateTime32 = FunctionConvert<DataTypeDateTime, NameToDateTime32, ToDateTimeMonotonicity>;
|
||||
using FunctionToDateTime64 = FunctionConvert<DataTypeDateTime64, NameToDateTime64, UnknownMonotonicity>;
|
||||
@ -2140,7 +2117,6 @@ struct NameToInt256OrZero { static constexpr auto name = "toInt256OrZero"; };
|
||||
struct NameToFloat32OrZero { static constexpr auto name = "toFloat32OrZero"; };
|
||||
struct NameToFloat64OrZero { static constexpr auto name = "toFloat64OrZero"; };
|
||||
struct NameToDateOrZero { static constexpr auto name = "toDateOrZero"; };
|
||||
struct NameToDate32OrZero { static constexpr auto name = "toDate32OrZero"; };
|
||||
struct NameToDateTimeOrZero { static constexpr auto name = "toDateTimeOrZero"; };
|
||||
struct NameToDateTime64OrZero { static constexpr auto name = "toDateTime64OrZero"; };
|
||||
struct NameToDecimal32OrZero { static constexpr auto name = "toDecimal32OrZero"; };
|
||||
@ -2164,7 +2140,6 @@ using FunctionToInt256OrZero = FunctionConvertFromString<DataTypeInt256, NameToI
|
||||
using FunctionToFloat32OrZero = FunctionConvertFromString<DataTypeFloat32, NameToFloat32OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToFloat64OrZero = FunctionConvertFromString<DataTypeFloat64, NameToFloat64OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDateOrZero = FunctionConvertFromString<DataTypeDate, NameToDateOrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDate32OrZero = FunctionConvertFromString<DataTypeDate32, NameToDate32OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDateTimeOrZero = FunctionConvertFromString<DataTypeDateTime, NameToDateTimeOrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDateTime64OrZero = FunctionConvertFromString<DataTypeDateTime64, NameToDateTime64OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
using FunctionToDecimal32OrZero = FunctionConvertFromString<DataTypeDecimal<Decimal32>, NameToDecimal32OrZero, ConvertFromStringExceptionMode::Zero>;
|
||||
@ -2188,7 +2163,6 @@ struct NameToInt256OrNull { static constexpr auto name = "toInt256OrNull"; };
|
||||
struct NameToFloat32OrNull { static constexpr auto name = "toFloat32OrNull"; };
|
||||
struct NameToFloat64OrNull { static constexpr auto name = "toFloat64OrNull"; };
|
||||
struct NameToDateOrNull { static constexpr auto name = "toDateOrNull"; };
|
||||
struct NameToDate32OrNull { static constexpr auto name = "toDate32OrNull"; };
|
||||
struct NameToDateTimeOrNull { static constexpr auto name = "toDateTimeOrNull"; };
|
||||
struct NameToDateTime64OrNull { static constexpr auto name = "toDateTime64OrNull"; };
|
||||
struct NameToDecimal32OrNull { static constexpr auto name = "toDecimal32OrNull"; };
|
||||
@ -2212,7 +2186,6 @@ using FunctionToInt256OrNull = FunctionConvertFromString<DataTypeInt256, NameToI
|
||||
using FunctionToFloat32OrNull = FunctionConvertFromString<DataTypeFloat32, NameToFloat32OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToFloat64OrNull = FunctionConvertFromString<DataTypeFloat64, NameToFloat64OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDateOrNull = FunctionConvertFromString<DataTypeDate, NameToDateOrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDate32OrNull = FunctionConvertFromString<DataTypeDate32, NameToDate32OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDateTimeOrNull = FunctionConvertFromString<DataTypeDateTime, NameToDateTimeOrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDateTime64OrNull = FunctionConvertFromString<DataTypeDateTime64, NameToDateTime64OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
using FunctionToDecimal32OrNull = FunctionConvertFromString<DataTypeDecimal<Decimal32>, NameToDecimal32OrNull, ConvertFromStringExceptionMode::Null>;
|
||||
|
@ -683,8 +683,6 @@ public:
|
||||
return executeType<Int64>(arguments);
|
||||
else if (which.isDate())
|
||||
return executeType<UInt16>(arguments);
|
||||
else if (which.isDate32())
|
||||
return executeType<Int32>(arguments);
|
||||
else if (which.isDateTime())
|
||||
return executeType<UInt32>(arguments);
|
||||
else if (which.isDecimal32())
|
||||
@ -988,7 +986,6 @@ private:
|
||||
else if (which.isEnum8()) executeIntType<Int8, first>(icolumn, vec_to);
|
||||
else if (which.isEnum16()) executeIntType<Int16, first>(icolumn, vec_to);
|
||||
else if (which.isDate()) executeIntType<UInt16, first>(icolumn, vec_to);
|
||||
else if (which.isDate32()) executeIntType<Int32, first>(icolumn, vec_to);
|
||||
else if (which.isDateTime()) executeIntType<UInt32, first>(icolumn, vec_to);
|
||||
/// TODO: executeIntType() for Decimal32/64 leads to incompatible result
|
||||
else if (which.isDecimal32()) executeBigIntType<Decimal32, first>(icolumn, vec_to);
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
@ -408,9 +407,6 @@ ColumnPtr FunctionArrayIntersect::executeImpl(const ColumnsWithTypeAndName & arg
|
||||
using DateMap = ClearableHashMapWithStackMemory<DataTypeDate::FieldType,
|
||||
size_t, DefaultHash<DataTypeDate::FieldType>, INITIAL_SIZE_DEGREE>;
|
||||
|
||||
using Date32Map = ClearableHashMapWithStackMemory<DataTypeDate32::FieldType,
|
||||
size_t, DefaultHash<DataTypeDate32::FieldType>, INITIAL_SIZE_DEGREE>;
|
||||
|
||||
using DateTimeMap = ClearableHashMapWithStackMemory<
|
||||
DataTypeDateTime::FieldType, size_t,
|
||||
DefaultHash<DataTypeDateTime::FieldType>, INITIAL_SIZE_DEGREE>;
|
||||
@ -425,8 +421,6 @@ ColumnPtr FunctionArrayIntersect::executeImpl(const ColumnsWithTypeAndName & arg
|
||||
|
||||
if (which.isDate())
|
||||
result_column = execute<DateMap, ColumnVector<DataTypeDate::FieldType>, true>(arrays, std::move(column));
|
||||
else if (which.isDate32())
|
||||
result_column = execute<Date32Map, ColumnVector<DataTypeDate32::FieldType>, true>(arrays, std::move(column));
|
||||
else if (which.isDateTime())
|
||||
result_column = execute<DateTimeMap, ColumnVector<DataTypeDateTime::FieldType>, true>(arrays, std::move(column));
|
||||
else if (which.isString())
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
@ -116,8 +115,6 @@ private:
|
||||
f(Float64());
|
||||
else if (which.isDate())
|
||||
f(DataTypeDate::FieldType());
|
||||
else if (which.isDate32())
|
||||
f(DataTypeDate::FieldType());
|
||||
else if (which.isDateTime())
|
||||
f(DataTypeDateTime::FieldType());
|
||||
else
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <common/DateLUTImpl.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeInterval.h>
|
||||
@ -40,11 +39,6 @@ namespace
|
||||
return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years);
|
||||
}
|
||||
|
||||
static UInt16 execute(Int32 d, UInt64 years, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfYearInterval(ExtendedDayNum(d), years);
|
||||
}
|
||||
|
||||
static UInt16 execute(UInt32 t, UInt64 years, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfYearInterval(time_zone.toDayNum(t), years);
|
||||
@ -66,11 +60,6 @@ namespace
|
||||
return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters);
|
||||
}
|
||||
|
||||
static UInt16 execute(Int32 d, UInt64 quarters, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfQuarterInterval(ExtendedDayNum(d), quarters);
|
||||
}
|
||||
|
||||
static UInt16 execute(UInt32 t, UInt64 quarters, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfQuarterInterval(time_zone.toDayNum(t), quarters);
|
||||
@ -92,11 +81,6 @@ namespace
|
||||
return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months);
|
||||
}
|
||||
|
||||
static UInt16 execute(Int32 d, UInt64 months, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfMonthInterval(ExtendedDayNum(d), months);
|
||||
}
|
||||
|
||||
static UInt16 execute(UInt32 t, UInt64 months, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfMonthInterval(time_zone.toDayNum(t), months);
|
||||
@ -118,11 +102,6 @@ namespace
|
||||
return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks);
|
||||
}
|
||||
|
||||
static UInt16 execute(Int32 d, UInt64 weeks, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfWeekInterval(ExtendedDayNum(d), weeks);
|
||||
}
|
||||
|
||||
static UInt16 execute(UInt32 t, UInt64 weeks, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfWeekInterval(time_zone.toDayNum(t), weeks);
|
||||
@ -144,11 +123,6 @@ namespace
|
||||
return time_zone.toStartOfDayInterval(ExtendedDayNum(d), days);
|
||||
}
|
||||
|
||||
static UInt32 execute(Int32 d, UInt64 days, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfDayInterval(ExtendedDayNum(d), days);
|
||||
}
|
||||
|
||||
static UInt32 execute(UInt32 t, UInt64 days, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), days);
|
||||
@ -166,7 +140,6 @@ namespace
|
||||
static constexpr auto name = function_name;
|
||||
|
||||
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
|
||||
static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
|
||||
static UInt32 execute(UInt32 t, UInt64 hours, const DateLUTImpl & time_zone) { return time_zone.toStartOfHourInterval(t, hours); }
|
||||
static UInt32 execute(Int64 t, UInt64 hours, const DateLUTImpl & time_zone) { return time_zone.toStartOfHourInterval(t, hours); }
|
||||
};
|
||||
@ -178,8 +151,6 @@ namespace
|
||||
|
||||
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
|
||||
|
||||
static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
|
||||
|
||||
static UInt32 execute(UInt32 t, UInt64 minutes, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfMinuteInterval(t, minutes);
|
||||
@ -198,8 +169,6 @@ namespace
|
||||
|
||||
static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
|
||||
|
||||
static UInt32 execute(Int32, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); }
|
||||
|
||||
static UInt32 execute(UInt32 t, UInt64 seconds, const DateLUTImpl & time_zone)
|
||||
{
|
||||
return time_zone.toStartOfSecondInterval(t, seconds);
|
||||
@ -330,12 +299,6 @@ private:
|
||||
if (time_column_vec)
|
||||
return dispatchForIntervalColumn(assert_cast<const DataTypeDate&>(from_datatype), *time_column_vec, interval_column, time_zone);
|
||||
}
|
||||
if (which_type.isDate32())
|
||||
{
|
||||
const auto * time_column_vec = checkAndGetColumn<ColumnInt32>(time_column.column.get());
|
||||
if (time_column_vec)
|
||||
return dispatchForIntervalColumn(assert_cast<const DataTypeDate32&>(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());
|
||||
|
@ -632,22 +632,6 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf)
|
||||
return ReturnType(true);
|
||||
}
|
||||
|
||||
template <typename ReturnType = void>
|
||||
inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf)
|
||||
{
|
||||
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
|
||||
|
||||
LocalDate local_date;
|
||||
|
||||
if constexpr (throw_exception)
|
||||
readDateTextImpl<ReturnType>(local_date, buf);
|
||||
else if (!readDateTextImpl<ReturnType>(local_date, buf))
|
||||
return false;
|
||||
/// When the parameter is out of rule or out of range, Date32 uses 1925-01-01 as the default value (-DateLUT::instance().getDayNumOffsetEpoch(), -16436) and Date uses 1970-01-01.
|
||||
date = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day(), -DateLUT::instance().getDayNumOffsetEpoch());
|
||||
return ReturnType(true);
|
||||
}
|
||||
|
||||
|
||||
inline void readDateText(LocalDate & date, ReadBuffer & buf)
|
||||
{
|
||||
@ -659,11 +643,6 @@ inline void readDateText(DayNum & date, ReadBuffer & buf)
|
||||
readDateTextImpl<void>(date, buf);
|
||||
}
|
||||
|
||||
inline void readDateText(ExtendedDayNum & date, ReadBuffer & buf)
|
||||
{
|
||||
readDateTextImpl<void>(date, buf);
|
||||
}
|
||||
|
||||
inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf)
|
||||
{
|
||||
return readDateTextImpl<bool>(date, buf);
|
||||
@ -674,11 +653,6 @@ inline bool tryReadDateText(DayNum & date, ReadBuffer & buf)
|
||||
return readDateTextImpl<bool>(date, buf);
|
||||
}
|
||||
|
||||
inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf)
|
||||
{
|
||||
return readDateTextImpl<bool>(date, buf);
|
||||
}
|
||||
|
||||
template <typename ReturnType = void>
|
||||
inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf)
|
||||
{
|
||||
|
@ -728,11 +728,6 @@ inline void writeDateText(DayNum date, WriteBuffer & buf)
|
||||
writeDateText<delimiter>(LocalDate(date), buf);
|
||||
}
|
||||
|
||||
template <char delimiter = '-'>
|
||||
inline void writeDateText(ExtendedDayNum date, WriteBuffer & buf)
|
||||
{
|
||||
writeDateText<delimiter>(LocalDate(date), buf);
|
||||
}
|
||||
|
||||
/// In the format YYYY-MM-DD HH:MM:SS
|
||||
template <char date_delimeter = '-', char time_delimeter = ':', char between_date_time_delimiter = ' '>
|
||||
|
@ -90,7 +90,6 @@ struct BloomFilterHash
|
||||
else if (which.isEnum8()) return build_hash_column(getNumberTypeHash<Int64, Int8>(field));
|
||||
else if (which.isEnum16()) return build_hash_column(getNumberTypeHash<Int64, Int16>(field));
|
||||
else if (which.isDate()) return build_hash_column(getNumberTypeHash<UInt64, UInt16>(field));
|
||||
else if (which.isDate32()) return build_hash_column(getNumberTypeHash<UInt64, Int32>(field));
|
||||
else if (which.isDateTime()) return build_hash_column(getNumberTypeHash<UInt64, UInt32>(field));
|
||||
else if (which.isFloat32()) return build_hash_column(getNumberTypeHash<Float64, Float64>(field));
|
||||
else if (which.isFloat64()) return build_hash_column(getNumberTypeHash<Float64, Float64>(field));
|
||||
@ -152,7 +151,6 @@ struct BloomFilterHash
|
||||
else if (which.isEnum8()) getNumberTypeHash<Int8, is_first>(column, vec, pos);
|
||||
else if (which.isEnum16()) getNumberTypeHash<Int16, is_first>(column, vec, pos);
|
||||
else if (which.isDate()) getNumberTypeHash<UInt16, is_first>(column, vec, pos);
|
||||
else if (which.isDate32()) getNumberTypeHash<Int32, is_first>(column, vec, pos);
|
||||
else if (which.isDateTime()) getNumberTypeHash<UInt32, is_first>(column, vec, pos);
|
||||
else if (which.isFloat32()) getNumberTypeHash<Float32, is_first>(column, vec, pos);
|
||||
else if (which.isFloat64()) getNumberTypeHash<Float64, is_first>(column, vec, pos);
|
||||
|
@ -159,18 +159,10 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
|
||||
{
|
||||
return static_cast<UInt16>(static_cast<const DataTypeDateTime &>(*from_type_hint).getTimeZone().toDayNum(src.get<UInt64>()).toUnderType());
|
||||
}
|
||||
else if (which_type.isDate32() && which_from_type.isDateTime())
|
||||
{
|
||||
return static_cast<Int32>(static_cast<const DataTypeDateTime &>(*from_type_hint).getTimeZone().toDayNum(src.get<UInt64>()).toUnderType());
|
||||
}
|
||||
else if (which_type.isDateTime() && which_from_type.isDate())
|
||||
{
|
||||
return static_cast<const DataTypeDateTime &>(type).getTimeZone().fromDayNum(DayNum(src.get<UInt64>()));
|
||||
}
|
||||
else if (which_type.isDateTime() && which_from_type.isDate32())
|
||||
{
|
||||
return static_cast<const DataTypeDateTime &>(type).getTimeZone().fromDayNum(DayNum(src.get<Int32>()));
|
||||
}
|
||||
else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String)
|
||||
{
|
||||
if (which_type.isUInt8()) return convertNumericType<UInt8>(src, type);
|
||||
|
@ -53,7 +53,6 @@ namespace DB
|
||||
|
||||
{arrow::Type::BOOL, "UInt8"},
|
||||
{arrow::Type::DATE32, "Date"},
|
||||
{arrow::Type::DATE32, "Date32"},
|
||||
{arrow::Type::DATE64, "DateTime"},
|
||||
{arrow::Type::TIMESTAMP, "DateTime"},
|
||||
|
||||
@ -146,36 +145,9 @@ namespace DB
|
||||
}
|
||||
|
||||
/// Arrow stores Parquet::DATE in Int32, while ClickHouse stores Date in UInt16. Therefore, it should be checked before saving
|
||||
static void fillColumnWithDate32Data(std::shared_ptr<arrow::ChunkedArray> & arrow_column, IColumn & internal_column)
|
||||
{
|
||||
PaddedPODArray<UInt16> & column_data = assert_cast<ColumnVector<UInt16> &>(internal_column).getData();
|
||||
column_data.reserve(arrow_column->length());
|
||||
|
||||
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
|
||||
static void fillColumnWithDate32Data(std::shared_ptr<arrow::ChunkedArray> & arrow_column, IColumn & internal_column)
|
||||
{
|
||||
arrow::Date32Array & chunk = static_cast<arrow::Date32Array &>(*(arrow_column->chunk(chunk_i)));
|
||||
|
||||
for (size_t value_i = 0, length = static_cast<size_t>(chunk.length()); value_i < length; ++value_i)
|
||||
{
|
||||
UInt32 days_num = static_cast<UInt32>(chunk.Value(value_i));
|
||||
if (days_num > DATE_LUT_MAX_DAY_NUM)
|
||||
{
|
||||
// TODO: will it rollback correctly?
|
||||
throw Exception
|
||||
{
|
||||
fmt::format("Input value {} of a column \"{}\" is greater than max allowed Date value, which is {}", days_num, internal_column.getName(), DATE_LUT_MAX_DAY_NUM),
|
||||
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE
|
||||
};
|
||||
}
|
||||
|
||||
column_data.emplace_back(days_num);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static void fillDate32ColumnWithDate32Data(std::shared_ptr<arrow::ChunkedArray> & arrow_column, IColumn & internal_column)
|
||||
{
|
||||
PaddedPODArray<Int32> & column_data = assert_cast<ColumnVector<Int32> &>(internal_column).getData();
|
||||
PaddedPODArray<UInt16> & column_data = assert_cast<ColumnVector<UInt16> &>(internal_column).getData();
|
||||
column_data.reserve(arrow_column->length());
|
||||
|
||||
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
|
||||
@ -184,8 +156,8 @@ static void fillColumnWithDate32Data(std::shared_ptr<arrow::ChunkedArray> & arro
|
||||
|
||||
for (size_t value_i = 0, length = static_cast<size_t>(chunk.length()); value_i < length; ++value_i)
|
||||
{
|
||||
Int32 days_num = static_cast<Int32>(chunk.Value(value_i));
|
||||
if (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM)
|
||||
UInt32 days_num = static_cast<UInt32>(chunk.Value(value_i));
|
||||
if (days_num > DATE_LUT_MAX_DAY_NUM)
|
||||
{
|
||||
// TODO: will it rollback correctly?
|
||||
throw Exception
|
||||
@ -356,14 +328,7 @@ static void fillColumnWithDate32Data(std::shared_ptr<arrow::ChunkedArray> & arro
|
||||
fillColumnWithBooleanData(arrow_column, internal_column);
|
||||
break;
|
||||
case arrow::Type::DATE32:
|
||||
if (WhichDataType(internal_column.getDataType()).isUInt16())
|
||||
{
|
||||
fillColumnWithDate32Data(arrow_column, internal_column);
|
||||
}
|
||||
else
|
||||
{
|
||||
fillDate32ColumnWithDate32Data(arrow_column, internal_column);
|
||||
}
|
||||
fillColumnWithDate32Data(arrow_column, internal_column);
|
||||
break;
|
||||
case arrow::Type::DATE64:
|
||||
fillColumnWithDate64Data(arrow_column, internal_column);
|
||||
@ -555,19 +520,8 @@ static void fillColumnWithDate32Data(std::shared_ptr<arrow::ChunkedArray> & arro
|
||||
);
|
||||
}
|
||||
|
||||
auto filter = [=](auto && elem)
|
||||
{
|
||||
auto which = WhichDataType(column_type);
|
||||
if (arrow_type->id() == arrow::Type::DATE32 && which.isDateOrDate32())
|
||||
{
|
||||
return (strcmp(elem.second, "Date") == 0 && which.isDate()) || (strcmp(elem.second, "Date32") == 0 && which.isDate32());
|
||||
}
|
||||
else
|
||||
{
|
||||
return elem.first == arrow_type->id();
|
||||
}
|
||||
};
|
||||
if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), filter);
|
||||
if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(),
|
||||
[=](auto && elem) { return elem.first == arrow_type->id(); });
|
||||
internal_type_it != arrow_type_to_internal_type.end())
|
||||
{
|
||||
return DataTypeFactory::instance().get(internal_type_it->second);
|
||||
|
@ -47,7 +47,7 @@ FillingTransform::FillingTransform(
|
||||
DataTypePtr to_type;
|
||||
|
||||
/// TODO Wrong results for big integers.
|
||||
if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime())
|
||||
if (isInteger(type) || which.isDate() || which.isDateTime())
|
||||
{
|
||||
max_type = Field::Types::Int64;
|
||||
to_type = std::make_shared<DataTypeInt64>();
|
||||
|
@ -1538,7 +1538,6 @@ void checkVersionColumnTypesConversion(const IDataType * old_type, const IDataTy
|
||||
if ((which_old_type.isInt() && !which_new_type.isInt())
|
||||
|| (which_old_type.isUInt() && !which_new_type.isUInt())
|
||||
|| (which_old_type.isDate() && !which_new_type.isDate())
|
||||
|| (which_old_type.isDate32() && !which_new_type.isDate32())
|
||||
|| (which_old_type.isDateTime() && !which_new_type.isDateTime())
|
||||
|| (which_old_type.isFloat() && !which_new_type.isFloat()))
|
||||
{
|
||||
|
@ -118,7 +118,6 @@ SRCS(
|
||||
MySQL/MySQLSettings.cpp
|
||||
PartitionCommands.cpp
|
||||
ProjectionsDescription.cpp
|
||||
ReadFinalForExternalReplicaStorage.cpp
|
||||
ReadInOrderOptimizer.cpp
|
||||
SelectQueryDescription.cpp
|
||||
SetSettings.cpp
|
||||
|
@ -91,11 +91,6 @@ idx10 ['This','is','a','test']
|
||||
=== Try load data from datapage_v2.snappy.parquet
|
||||
Code: 33. DB::ParsingEx---tion: Error while reading Parquet data: IOError: Not yet implemented: Unsupported encoding.: data for INSERT was parsed from stdin
|
||||
|
||||
=== Try load data from datatype-date32.parquet
|
||||
1925-01-01
|
||||
1949-10-01
|
||||
2021-10-01
|
||||
2282-12-31
|
||||
=== Try load data from dict-page-offset-zero.parquet
|
||||
1552
|
||||
1552
|
||||
|
@ -1,282 +0,0 @@
|
||||
1925-01-01
|
||||
1925-01-01
|
||||
2282-12-31
|
||||
2283-11-11
|
||||
2021-06-22
|
||||
-------toYear---------
|
||||
1925
|
||||
1925
|
||||
2282
|
||||
2283
|
||||
2021
|
||||
-------toMonth---------
|
||||
1
|
||||
1
|
||||
12
|
||||
11
|
||||
6
|
||||
-------toQuarter---------
|
||||
1
|
||||
1
|
||||
4
|
||||
4
|
||||
2
|
||||
-------toDayOfMonth---------
|
||||
1
|
||||
1
|
||||
31
|
||||
11
|
||||
22
|
||||
-------toDayOfWeek---------
|
||||
4
|
||||
4
|
||||
7
|
||||
7
|
||||
2
|
||||
-------toDayOfYear---------
|
||||
1
|
||||
1
|
||||
365
|
||||
315
|
||||
173
|
||||
-------toHour---------
|
||||
-------toMinute---------
|
||||
-------toSecond---------
|
||||
-------toStartOfDay---------
|
||||
2061-02-06 07:28:16
|
||||
2061-02-06 07:28:16
|
||||
2010-10-17 12:03:28
|
||||
2011-08-28 12:03:28
|
||||
2021-06-22 00:00:00
|
||||
-------toMonday---------
|
||||
2104-06-04
|
||||
2104-06-04
|
||||
2103-07-21
|
||||
2104-05-31
|
||||
2021-06-21
|
||||
-------toISOWeek---------
|
||||
1
|
||||
1
|
||||
52
|
||||
45
|
||||
25
|
||||
-------toISOYear---------
|
||||
1925
|
||||
1925
|
||||
2282
|
||||
2283
|
||||
2021
|
||||
-------toWeek---------
|
||||
0
|
||||
0
|
||||
53
|
||||
45
|
||||
25
|
||||
-------toYearWeek---------
|
||||
192452
|
||||
192452
|
||||
228253
|
||||
228345
|
||||
202125
|
||||
-------toStartOfWeek---------
|
||||
2104-06-03
|
||||
2104-06-03
|
||||
2103-07-27
|
||||
2104-06-06
|
||||
2021-06-20
|
||||
-------toStartOfMonth---------
|
||||
2104-06-07
|
||||
2104-06-07
|
||||
2103-06-27
|
||||
2104-05-27
|
||||
2021-06-01
|
||||
-------toStartOfQuarter---------
|
||||
2104-06-07
|
||||
2104-06-07
|
||||
2103-04-27
|
||||
2104-04-26
|
||||
2021-04-01
|
||||
-------toStartOfYear---------
|
||||
2104-06-07
|
||||
2104-06-07
|
||||
2102-07-28
|
||||
2103-07-28
|
||||
2021-01-01
|
||||
-------toStartOfSecond---------
|
||||
-------toStartOfMinute---------
|
||||
-------toStartOfFiveMinute---------
|
||||
-------toStartOfTenMinutes---------
|
||||
-------toStartOfFifteenMinutes---------
|
||||
-------toStartOfHour---------
|
||||
-------toStartOfISOYear---------
|
||||
2104-06-04
|
||||
2104-06-04
|
||||
2102-07-29
|
||||
2103-07-28
|
||||
2021-01-04
|
||||
-------toRelativeYearNum---------
|
||||
1925
|
||||
1925
|
||||
2282
|
||||
2283
|
||||
2021
|
||||
-------toRelativeQuarterNum---------
|
||||
7700
|
||||
7700
|
||||
9131
|
||||
9135
|
||||
8085
|
||||
-------toRelativeMonthNum---------
|
||||
23101
|
||||
23101
|
||||
27396
|
||||
27407
|
||||
24258
|
||||
-------toRelativeWeekNum---------
|
||||
63189
|
||||
63189
|
||||
16331
|
||||
63188
|
||||
2686
|
||||
-------toRelativeDayNum---------
|
||||
49100
|
||||
49100
|
||||
48784
|
||||
49099
|
||||
18800
|
||||
-------toRelativeHourNum---------
|
||||
4294572851
|
||||
4294572851
|
||||
2743677
|
||||
2751237
|
||||
451197
|
||||
-------toRelativeMinuteNum---------
|
||||
4271299336
|
||||
4271299336
|
||||
164620620
|
||||
165074220
|
||||
27071820
|
||||
-------toRelativeSecondNum---------
|
||||
2874889696
|
||||
2874889696
|
||||
1287302608
|
||||
1314518608
|
||||
1624309200
|
||||
-------toTime---------
|
||||
-------toYYYYMM---------
|
||||
192501
|
||||
192501
|
||||
228212
|
||||
228311
|
||||
202106
|
||||
-------toYYYYMMDD---------
|
||||
19250101
|
||||
19250101
|
||||
22821231
|
||||
22831111
|
||||
20210622
|
||||
-------toYYYYMMDDhhmmss---------
|
||||
19250101000000
|
||||
19250101000000
|
||||
22821231000000
|
||||
22831111000000
|
||||
20210622000000
|
||||
-------addSeconds---------
|
||||
1925-01-01 01:00:00.000
|
||||
1925-01-01 01:00:00.000
|
||||
2282-12-31 01:00:00.000
|
||||
2283-11-11 01:00:00.000
|
||||
2021-06-22 01:00:00.000
|
||||
-------addMinutes---------
|
||||
1925-01-01 01:00:00.000
|
||||
1925-01-01 01:00:00.000
|
||||
2282-12-31 01:00:00.000
|
||||
2283-11-11 01:00:00.000
|
||||
2021-06-22 01:00:00.000
|
||||
-------addHours---------
|
||||
1925-01-01 12:00:00.000
|
||||
1925-01-01 12:00:00.000
|
||||
2282-12-31 12:00:00.000
|
||||
2283-11-11 12:00:00.000
|
||||
2021-06-22 12:00:00.000
|
||||
-------addDays---------
|
||||
1925-01-08
|
||||
1925-01-08
|
||||
2283-01-07
|
||||
1925-01-07
|
||||
2021-06-29
|
||||
-------addWeeks---------
|
||||
1925-01-08
|
||||
1925-01-08
|
||||
2283-01-07
|
||||
1925-01-07
|
||||
2021-06-29
|
||||
-------addMonths---------
|
||||
1925-02-01
|
||||
1925-02-01
|
||||
2283-01-31
|
||||
2283-11-11
|
||||
2021-07-22
|
||||
-------addQuarters---------
|
||||
1925-04-01
|
||||
1925-04-01
|
||||
2283-03-31
|
||||
1925-01-01
|
||||
2021-09-22
|
||||
-------addYears---------
|
||||
1926-01-01
|
||||
1926-01-01
|
||||
2283-11-11
|
||||
1925-01-01
|
||||
2022-06-22
|
||||
-------subtractSeconds---------
|
||||
1925-01-01 00:00:00.000
|
||||
1925-01-01 00:00:00.000
|
||||
2282-12-30 23:00:00.000
|
||||
2283-11-10 23:00:00.000
|
||||
2021-06-21 23:00:00.000
|
||||
-------subtractMinutes---------
|
||||
1925-01-01 00:00:00.000
|
||||
1925-01-01 00:00:00.000
|
||||
2282-12-30 23:00:00.000
|
||||
2283-11-10 23:00:00.000
|
||||
2021-06-21 23:00:00.000
|
||||
-------subtractHours---------
|
||||
1925-01-01 00:00:00.000
|
||||
1925-01-01 00:00:00.000
|
||||
2282-12-30 23:00:00.000
|
||||
2283-11-10 23:00:00.000
|
||||
2021-06-21 23:00:00.000
|
||||
-------subtractDays---------
|
||||
2283-11-05
|
||||
2283-11-05
|
||||
2282-12-24
|
||||
2283-11-04
|
||||
2021-06-15
|
||||
-------subtractWeeks---------
|
||||
2283-11-05
|
||||
2283-11-05
|
||||
2282-12-24
|
||||
2283-11-04
|
||||
2021-06-15
|
||||
-------subtractMonths---------
|
||||
1925-01-01
|
||||
1925-01-01
|
||||
2282-11-30
|
||||
2283-10-11
|
||||
2021-05-22
|
||||
-------subtractQuarters---------
|
||||
1925-01-01
|
||||
1925-01-01
|
||||
2282-09-30
|
||||
2283-08-11
|
||||
2021-03-22
|
||||
-------subtractYears---------
|
||||
1925-01-01
|
||||
1925-01-01
|
||||
2281-12-31
|
||||
2282-11-11
|
||||
2020-06-22
|
||||
-------toDate32---------
|
||||
1925-01-01 2000-01-01
|
||||
1925-01-01 1925-01-01
|
@ -1,118 +0,0 @@
|
||||
drop table if exists t1;
|
||||
create table t1(x1 Date32) engine Memory;
|
||||
|
||||
insert into t1 values ('1925-01-01'),('1924-01-01'),('2282-12-31'),('2283-12-31'),('2021-06-22');
|
||||
|
||||
select x1 from t1;
|
||||
select '-------toYear---------';
|
||||
select toYear(x1) from t1;
|
||||
select '-------toMonth---------';
|
||||
select toMonth(x1) from t1;
|
||||
select '-------toQuarter---------';
|
||||
select toQuarter(x1) from t1;
|
||||
select '-------toDayOfMonth---------';
|
||||
select toDayOfMonth(x1) from t1;
|
||||
select '-------toDayOfWeek---------';
|
||||
select toDayOfWeek(x1) from t1;
|
||||
select '-------toDayOfYear---------';
|
||||
select toDayOfYear(x1) from t1;
|
||||
select '-------toHour---------';
|
||||
select toHour(x1) from t1; -- { serverError 43 }
|
||||
select '-------toMinute---------';
|
||||
select toMinute(x1) from t1; -- { serverError 43 }
|
||||
select '-------toSecond---------';
|
||||
select toSecond(x1) from t1; -- { serverError 43 }
|
||||
select '-------toStartOfDay---------';
|
||||
select toStartOfDay(x1) from t1;
|
||||
select '-------toMonday---------';
|
||||
select toMonday(x1) from t1;
|
||||
select '-------toISOWeek---------';
|
||||
select toISOWeek(x1) from t1;
|
||||
select '-------toISOYear---------';
|
||||
select toISOYear(x1) from t1;
|
||||
select '-------toWeek---------';
|
||||
select toWeek(x1) from t1;
|
||||
select '-------toYearWeek---------';
|
||||
select toYearWeek(x1) from t1;
|
||||
select '-------toStartOfWeek---------';
|
||||
select toStartOfWeek(x1) from t1;
|
||||
select '-------toStartOfMonth---------';
|
||||
select toStartOfMonth(x1) from t1;
|
||||
select '-------toStartOfQuarter---------';
|
||||
select toStartOfQuarter(x1) from t1;
|
||||
select '-------toStartOfYear---------';
|
||||
select toStartOfYear(x1) from t1;
|
||||
select '-------toStartOfSecond---------';
|
||||
select toStartOfSecond(x1) from t1; -- { serverError 43 }
|
||||
select '-------toStartOfMinute---------';
|
||||
select toStartOfMinute(x1) from t1; -- { serverError 43 }
|
||||
select '-------toStartOfFiveMinute---------';
|
||||
select toStartOfFiveMinute(x1) from t1; -- { serverError 43 }
|
||||
select '-------toStartOfTenMinutes---------';
|
||||
select toStartOfTenMinutes(x1) from t1; -- { serverError 43 }
|
||||
select '-------toStartOfFifteenMinutes---------';
|
||||
select toStartOfFifteenMinutes(x1) from t1; -- { serverError 43 }
|
||||
select '-------toStartOfHour---------';
|
||||
select toStartOfHour(x1) from t1; -- { serverError 43 }
|
||||
select '-------toStartOfISOYear---------';
|
||||
select toStartOfISOYear(x1) from t1;
|
||||
select '-------toRelativeYearNum---------';
|
||||
select toRelativeYearNum(x1) from t1;
|
||||
select '-------toRelativeQuarterNum---------';
|
||||
select toRelativeQuarterNum(x1) from t1;
|
||||
select '-------toRelativeMonthNum---------';
|
||||
select toRelativeMonthNum(x1) from t1;
|
||||
select '-------toRelativeWeekNum---------';
|
||||
select toRelativeWeekNum(x1) from t1;
|
||||
select '-------toRelativeDayNum---------';
|
||||
select toRelativeDayNum(x1) from t1;
|
||||
select '-------toRelativeHourNum---------';
|
||||
select toRelativeHourNum(x1) from t1;
|
||||
select '-------toRelativeMinuteNum---------';
|
||||
select toRelativeMinuteNum(x1) from t1;
|
||||
select '-------toRelativeSecondNum---------';
|
||||
select toRelativeSecondNum(x1) from t1;
|
||||
select '-------toTime---------';
|
||||
select toTime(x1) from t1; -- { serverError 43 }
|
||||
select '-------toYYYYMM---------';
|
||||
select toYYYYMM(x1) from t1;
|
||||
select '-------toYYYYMMDD---------';
|
||||
select toYYYYMMDD(x1) from t1;
|
||||
select '-------toYYYYMMDDhhmmss---------';
|
||||
select toYYYYMMDDhhmmss(x1) from t1;
|
||||
select '-------addSeconds---------';
|
||||
select addSeconds(x1, 3600) from t1;
|
||||
select '-------addMinutes---------';
|
||||
select addMinutes(x1, 60) from t1;
|
||||
select '-------addHours---------';
|
||||
select addHours(x1, 12) from t1;
|
||||
select '-------addDays---------';
|
||||
select addDays(x1, 7) from t1;
|
||||
select '-------addWeeks---------';
|
||||
select addWeeks(x1, 1) from t1;
|
||||
select '-------addMonths---------';
|
||||
select addMonths(x1, 1) from t1;
|
||||
select '-------addQuarters---------';
|
||||
select addQuarters(x1, 1) from t1;
|
||||
select '-------addYears---------';
|
||||
select addYears(x1, 1) from t1;
|
||||
select '-------subtractSeconds---------';
|
||||
select subtractSeconds(x1, 3600) from t1;
|
||||
select '-------subtractMinutes---------';
|
||||
select subtractMinutes(x1, 60) from t1;
|
||||
select '-------subtractHours---------';
|
||||
select subtractHours(x1, 1) from t1;
|
||||
select '-------subtractDays---------';
|
||||
select subtractDays(x1, 7) from t1;
|
||||
select '-------subtractWeeks---------';
|
||||
select subtractWeeks(x1, 1) from t1;
|
||||
select '-------subtractMonths---------';
|
||||
select subtractMonths(x1, 1) from t1;
|
||||
select '-------subtractQuarters---------';
|
||||
select subtractQuarters(x1, 1) from t1;
|
||||
select '-------subtractYears---------';
|
||||
select subtractYears(x1, 1) from t1;
|
||||
select '-------toDate32---------';
|
||||
select toDate32('1925-01-01'), toDate32(toDate('2000-01-01'));
|
||||
select toDate32OrZero('1924-01-01'), toDate32OrNull('1924-01-01');
|
||||
|
Binary file not shown.
@ -1 +0,0 @@
|
||||
`date32` Date32
|
Loading…
Reference in New Issue
Block a user