提交 13762471 编写于 作者: A Alexey Milovidov

Changed DayNum_t to DayNum as Amos Bird suggested [#CLICKHOUSE-3]

上级 5665709b
...@@ -11,12 +11,12 @@ namespace DB ...@@ -11,12 +11,12 @@ namespace DB
void DataTypeDate::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const void DataTypeDate::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{ {
writeDateText(DayNum_t(static_cast<const ColumnUInt16 &>(column).getData()[row_num]), ostr); writeDateText(DayNum(static_cast<const ColumnUInt16 &>(column).getData()[row_num]), ostr);
} }
static void deserializeText(IColumn & column, ReadBuffer & istr) static void deserializeText(IColumn & column, ReadBuffer & istr)
{ {
DayNum_t x; DayNum x;
readDateText(x, istr); readDateText(x, istr);
static_cast<ColumnUInt16 &>(column).getData().push_back(x); static_cast<ColumnUInt16 &>(column).getData().push_back(x);
} }
...@@ -40,7 +40,7 @@ void DataTypeDate::serializeTextQuoted(const IColumn & column, size_t row_num, W ...@@ -40,7 +40,7 @@ void DataTypeDate::serializeTextQuoted(const IColumn & column, size_t row_num, W
void DataTypeDate::deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const void DataTypeDate::deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const
{ {
DayNum_t x; DayNum x;
assertChar('\'', istr); assertChar('\'', istr);
readDateText(x, istr); readDateText(x, istr);
assertChar('\'', istr); assertChar('\'', istr);
...@@ -56,7 +56,7 @@ void DataTypeDate::serializeTextJSON(const IColumn & column, size_t row_num, Wri ...@@ -56,7 +56,7 @@ void DataTypeDate::serializeTextJSON(const IColumn & column, size_t row_num, Wri
void DataTypeDate::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const void DataTypeDate::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const
{ {
DayNum_t x; DayNum x;
assertChar('"', istr); assertChar('"', istr);
readDateText(x, istr); readDateText(x, istr);
assertChar('"', istr); assertChar('"', istr);
......
...@@ -828,7 +828,7 @@ private: ...@@ -828,7 +828,7 @@ private:
if (is_date) if (is_date)
{ {
DayNum_t date; DayNum date;
ReadBufferFromMemory in(string_value.data, string_value.size); ReadBufferFromMemory in(string_value.data, string_value.size);
readDateText(date, in); readDateText(date, in);
if (!in.eof()) if (!in.eof())
......
...@@ -112,7 +112,7 @@ struct ToDateTimeImpl ...@@ -112,7 +112,7 @@ struct ToDateTimeImpl
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.fromDayNum(DayNum_t(d)); return time_zone.fromDayNum(DayNum(d));
} }
}; };
...@@ -175,7 +175,7 @@ struct FormatImpl<DataTypeDate> ...@@ -175,7 +175,7 @@ struct FormatImpl<DataTypeDate>
{ {
static void execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl *) static void execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl *)
{ {
writeDateText(DayNum_t(x), wb); writeDateText(DayNum(x), wb);
} }
}; };
...@@ -307,7 +307,7 @@ template <typename DataType> void parseImpl(typename DataType::FieldType & x, Re ...@@ -307,7 +307,7 @@ template <typename DataType> void parseImpl(typename DataType::FieldType & x, Re
template <> inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) template <> inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
{ {
DayNum_t tmp(0); DayNum tmp(0);
readDateText(tmp, rb); readDateText(tmp, rb);
x = tmp; x = tmp;
} }
......
...@@ -133,7 +133,7 @@ struct ToMondayImpl ...@@ -133,7 +133,7 @@ struct ToMondayImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toFirstDayNumOfWeek(DayNum_t(d)); return time_zone.toFirstDayNumOfWeek(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -149,7 +149,7 @@ struct ToStartOfMonthImpl ...@@ -149,7 +149,7 @@ struct ToStartOfMonthImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toFirstDayNumOfMonth(DayNum_t(d)); return time_zone.toFirstDayNumOfMonth(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -165,7 +165,7 @@ struct ToStartOfQuarterImpl ...@@ -165,7 +165,7 @@ struct ToStartOfQuarterImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toFirstDayNumOfQuarter(DayNum_t(d)); return time_zone.toFirstDayNumOfQuarter(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -181,7 +181,7 @@ struct ToStartOfYearImpl ...@@ -181,7 +181,7 @@ struct ToStartOfYearImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toFirstDayNumOfYear(DayNum_t(d)); return time_zone.toFirstDayNumOfYear(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -281,7 +281,7 @@ struct ToYearImpl ...@@ -281,7 +281,7 @@ struct ToYearImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toYear(DayNum_t(d)); return time_zone.toYear(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -297,7 +297,7 @@ struct ToQuarterImpl ...@@ -297,7 +297,7 @@ struct ToQuarterImpl
} }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toQuarter(DayNum_t(d)); return time_zone.toQuarter(DayNum(d));
} }
using FactorTransform = ToStartOfYearImpl; using FactorTransform = ToStartOfYearImpl;
...@@ -313,7 +313,7 @@ struct ToMonthImpl ...@@ -313,7 +313,7 @@ struct ToMonthImpl
} }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toMonth(DayNum_t(d)); return time_zone.toMonth(DayNum(d));
} }
using FactorTransform = ToStartOfYearImpl; using FactorTransform = ToStartOfYearImpl;
...@@ -329,7 +329,7 @@ struct ToDayOfMonthImpl ...@@ -329,7 +329,7 @@ struct ToDayOfMonthImpl
} }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toDayOfMonth(DayNum_t(d)); return time_zone.toDayOfMonth(DayNum(d));
} }
using FactorTransform = ToStartOfMonthImpl; using FactorTransform = ToStartOfMonthImpl;
...@@ -345,7 +345,7 @@ struct ToDayOfWeekImpl ...@@ -345,7 +345,7 @@ struct ToDayOfWeekImpl
} }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toDayOfWeek(DayNum_t(d)); return time_zone.toDayOfWeek(DayNum(d));
} }
using FactorTransform = ToMondayImpl; using FactorTransform = ToMondayImpl;
...@@ -410,7 +410,7 @@ struct ToRelativeYearNumImpl ...@@ -410,7 +410,7 @@ struct ToRelativeYearNumImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toYear(DayNum_t(d)); return time_zone.toYear(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -426,7 +426,7 @@ struct ToRelativeQuarterNumImpl ...@@ -426,7 +426,7 @@ struct ToRelativeQuarterNumImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toRelativeQuarterNum(DayNum_t(d)); return time_zone.toRelativeQuarterNum(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -442,7 +442,7 @@ struct ToRelativeMonthNumImpl ...@@ -442,7 +442,7 @@ struct ToRelativeMonthNumImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toRelativeMonthNum(DayNum_t(d)); return time_zone.toRelativeMonthNum(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -458,7 +458,7 @@ struct ToRelativeWeekNumImpl ...@@ -458,7 +458,7 @@ struct ToRelativeWeekNumImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toRelativeWeekNum(DayNum_t(d)); return time_zone.toRelativeWeekNum(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -474,7 +474,7 @@ struct ToRelativeDayNumImpl ...@@ -474,7 +474,7 @@ struct ToRelativeDayNumImpl
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl &) static inline UInt16 execute(UInt16 d, const DateLUTImpl &)
{ {
return static_cast<DayNum_t>(d); return static_cast<DayNum>(d);
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -491,7 +491,7 @@ struct ToRelativeHourNumImpl ...@@ -491,7 +491,7 @@ struct ToRelativeHourNumImpl
} }
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toRelativeHourNum(DayNum_t(d)); return time_zone.toRelativeHourNum(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -507,7 +507,7 @@ struct ToRelativeMinuteNumImpl ...@@ -507,7 +507,7 @@ struct ToRelativeMinuteNumImpl
} }
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toRelativeMinuteNum(DayNum_t(d)); return time_zone.toRelativeMinuteNum(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -523,7 +523,7 @@ struct ToRelativeSecondNumImpl ...@@ -523,7 +523,7 @@ struct ToRelativeSecondNumImpl
} }
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.fromDayNum(DayNum_t(d)); return time_zone.fromDayNum(DayNum(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -539,7 +539,7 @@ struct ToYYYYMMImpl ...@@ -539,7 +539,7 @@ struct ToYYYYMMImpl
} }
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toNumYYYYMM(static_cast<DayNum_t>(d)); return time_zone.toNumYYYYMM(static_cast<DayNum>(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -555,7 +555,7 @@ struct ToYYYYMMDDImpl ...@@ -555,7 +555,7 @@ struct ToYYYYMMDDImpl
} }
static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toNumYYYYMMDD(static_cast<DayNum_t>(d)); return time_zone.toNumYYYYMMDD(static_cast<DayNum>(d));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -571,7 +571,7 @@ struct ToYYYYMMDDhhmmssImpl ...@@ -571,7 +571,7 @@ struct ToYYYYMMDDhhmmssImpl
} }
static inline UInt64 execute(UInt16 d, const DateLUTImpl & time_zone) static inline UInt64 execute(UInt16 d, const DateLUTImpl & time_zone)
{ {
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast<DayNum_t>(d))); return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(static_cast<DayNum>(d)));
} }
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
...@@ -732,7 +732,7 @@ struct AddSecondsImpl ...@@ -732,7 +732,7 @@ struct AddSecondsImpl
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{ {
return time_zone.fromDayNum(DayNum_t(d)) + delta; return time_zone.fromDayNum(DayNum(d)) + delta;
} }
}; };
...@@ -747,7 +747,7 @@ struct AddMinutesImpl ...@@ -747,7 +747,7 @@ struct AddMinutesImpl
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{ {
return time_zone.fromDayNum(DayNum_t(d)) + delta * 60; return time_zone.fromDayNum(DayNum(d)) + delta * 60;
} }
}; };
...@@ -762,7 +762,7 @@ struct AddHoursImpl ...@@ -762,7 +762,7 @@ struct AddHoursImpl
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{ {
return time_zone.fromDayNum(DayNum_t(d)) + delta * 3600; return time_zone.fromDayNum(DayNum(d)) + delta * 3600;
} }
}; };
...@@ -807,7 +807,7 @@ struct AddMonthsImpl ...@@ -807,7 +807,7 @@ struct AddMonthsImpl
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{ {
return time_zone.addMonths(DayNum_t(d), delta); return time_zone.addMonths(DayNum(d), delta);
} }
}; };
...@@ -822,7 +822,7 @@ struct AddYearsImpl ...@@ -822,7 +822,7 @@ struct AddYearsImpl
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{ {
return time_zone.addYears(DayNum_t(d), delta); return time_zone.addYears(DayNum(d), delta);
} }
}; };
......
...@@ -483,7 +483,7 @@ inline void readDateText(LocalDate & date, ReadBuffer & buf) ...@@ -483,7 +483,7 @@ inline void readDateText(LocalDate & date, ReadBuffer & buf)
readDateTextFallback(date, buf); readDateTextFallback(date, buf);
} }
inline void readDateText(DayNum_t & date, ReadBuffer & buf) inline void readDateText(DayNum & date, ReadBuffer & buf)
{ {
LocalDate local_date; LocalDate local_date;
readDateText(local_date, buf); readDateText(local_date, buf);
......
...@@ -537,7 +537,7 @@ inline void writeDateText(const LocalDate & date, WriteBuffer & buf) ...@@ -537,7 +537,7 @@ inline void writeDateText(const LocalDate & date, WriteBuffer & buf)
} }
template <char delimiter = '-'> template <char delimiter = '-'>
inline void writeDateText(DayNum_t date, WriteBuffer & buf) inline void writeDateText(DayNum date, WriteBuffer & buf)
{ {
if (unlikely(!date)) if (unlikely(!date))
{ {
......
...@@ -72,10 +72,10 @@ static Field convertNumericType(const Field & from, const IDataType & type) ...@@ -72,10 +72,10 @@ static Field convertNumericType(const Field & from, const IDataType & type)
} }
DayNum_t stringToDate(const String & s) DayNum stringToDate(const String & s)
{ {
ReadBufferFromString in(s); ReadBufferFromString in(s);
DayNum_t date{}; DayNum date{};
readDateText(date, in); readDateText(date, in);
if (!in.eof()) if (!in.eof())
......
...@@ -93,8 +93,8 @@ void MergeTreeDataMerger::FuturePart::assign(MergeTreeData::DataPartsVector part ...@@ -93,8 +93,8 @@ void MergeTreeDataMerger::FuturePart::assign(MergeTreeData::DataPartsVector part
if (parts.front()->storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) if (parts.front()->storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{ {
DayNum_t min_date = DayNum_t(std::numeric_limits<UInt16>::max()); DayNum min_date = DayNum(std::numeric_limits<UInt16>::max());
DayNum_t max_date = DayNum_t(std::numeric_limits<UInt16>::min()); DayNum max_date = DayNum(std::numeric_limits<UInt16>::min());
for (const auto & part : parts) for (const auto & part : parts)
{ {
min_date = std::min(min_date, part->getMinDate()); min_date = std::min(min_date, part->getMinDate());
......
...@@ -231,21 +231,21 @@ String MergeTreeDataPart::getNameWithPrefix() const ...@@ -231,21 +231,21 @@ String MergeTreeDataPart::getNameWithPrefix() const
} }
DayNum_t MergeTreeDataPart::getMinDate() const DayNum MergeTreeDataPart::getMinDate() const
{ {
if (storage.minmax_idx_date_column_pos != -1) if (storage.minmax_idx_date_column_pos != -1)
return DayNum_t(minmax_idx.min_values[storage.minmax_idx_date_column_pos].get<UInt64>()); return DayNum(minmax_idx.min_values[storage.minmax_idx_date_column_pos].get<UInt64>());
else else
return DayNum_t(); return DayNum();
} }
DayNum_t MergeTreeDataPart::getMaxDate() const DayNum MergeTreeDataPart::getMaxDate() const
{ {
if (storage.minmax_idx_date_column_pos != -1) if (storage.minmax_idx_date_column_pos != -1)
return DayNum_t(minmax_idx.max_values[storage.minmax_idx_date_column_pos].get<UInt64>()); return DayNum(minmax_idx.max_values[storage.minmax_idx_date_column_pos].get<UInt64>());
else else
return DayNum_t(); return DayNum();
} }
...@@ -469,8 +469,8 @@ void MergeTreeDataPart::loadPartitionAndMinMaxIndex() ...@@ -469,8 +469,8 @@ void MergeTreeDataPart::loadPartitionAndMinMaxIndex()
{ {
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{ {
DayNum_t min_date; DayNum min_date;
DayNum_t max_date; DayNum max_date;
MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date); MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date);
const auto & date_lut = DateLUT::instance(); const auto & date_lut = DateLUT::instance();
......
...@@ -65,8 +65,8 @@ struct MergeTreeDataPart ...@@ -65,8 +65,8 @@ struct MergeTreeDataPart
bool contains(const MergeTreeDataPart & other) const { return info.contains(other.info); } bool contains(const MergeTreeDataPart & other) const { return info.contains(other.info); }
/// If the partition key includes date column (a common case), these functions will return min and max values for this column. /// If the partition key includes date column (a common case), these functions will return min and max values for this column.
DayNum_t getMinDate() const; DayNum getMinDate() const;
DayNum_t getMaxDate() const; DayNum getMaxDate() const;
MergeTreeData & storage; MergeTreeData & storage;
...@@ -178,7 +178,7 @@ struct MergeTreeDataPart ...@@ -178,7 +178,7 @@ struct MergeTreeDataPart
MinMaxIndex() = default; MinMaxIndex() = default;
/// For month-based partitioning. /// For month-based partitioning.
MinMaxIndex(DayNum_t min_date, DayNum_t max_date) MinMaxIndex(DayNum min_date, DayNum max_date)
: min_values(1, static_cast<UInt64>(min_date)) : min_values(1, static_cast<UInt64>(min_date))
, max_values(1, static_cast<UInt64>(max_date)) , max_values(1, static_cast<UInt64>(max_date))
, initialized(true) , initialized(true)
......
...@@ -140,13 +140,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa ...@@ -140,13 +140,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
String part_name; String part_name;
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{ {
DayNum_t min_date(minmax_idx.min_values[data.minmax_idx_date_column_pos].get<UInt64>()); DayNum min_date(minmax_idx.min_values[data.minmax_idx_date_column_pos].get<UInt64>());
DayNum_t max_date(minmax_idx.max_values[data.minmax_idx_date_column_pos].get<UInt64>()); DayNum max_date(minmax_idx.max_values[data.minmax_idx_date_column_pos].get<UInt64>());
const auto & date_lut = DateLUT::instance(); const auto & date_lut = DateLUT::instance();
DayNum_t min_month = date_lut.toFirstDayNumOfMonth(DayNum_t(min_date)); DayNum min_month = date_lut.toFirstDayNumOfMonth(DayNum(min_date));
DayNum_t max_month = date_lut.toFirstDayNumOfMonth(DayNum_t(max_date)); DayNum max_month = date_lut.toFirstDayNumOfMonth(DayNum(max_date));
if (min_month != max_month) if (min_month != max_month)
throw Exception("Logical error: part spans more than one month."); throw Exception("Logical error: part spans more than one month.");
......
...@@ -78,7 +78,7 @@ bool MergeTreePartInfo::tryParsePartName(const String & dir_name, MergeTreePartI ...@@ -78,7 +78,7 @@ bool MergeTreePartInfo::tryParsePartName(const String & dir_name, MergeTreePartI
} }
void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & dir_name, DayNum_t & min_date, DayNum_t & max_date) void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & dir_name, DayNum & min_date, DayNum & max_date)
{ {
UInt32 min_yyyymmdd = 0; UInt32 min_yyyymmdd = 0;
UInt32 max_yyyymmdd = 0; UInt32 max_yyyymmdd = 0;
...@@ -97,8 +97,8 @@ void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & dir_name, Da ...@@ -97,8 +97,8 @@ void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & dir_name, Da
min_date = date_lut.YYYYMMDDToDayNum(min_yyyymmdd); min_date = date_lut.YYYYMMDDToDayNum(min_yyyymmdd);
max_date = date_lut.YYYYMMDDToDayNum(max_yyyymmdd); max_date = date_lut.YYYYMMDDToDayNum(max_yyyymmdd);
DayNum_t min_month = date_lut.toFirstDayNumOfMonth(min_date); DayNum min_month = date_lut.toFirstDayNumOfMonth(min_date);
DayNum_t max_month = date_lut.toFirstDayNumOfMonth(max_date); DayNum max_month = date_lut.toFirstDayNumOfMonth(max_date);
if (min_month != max_month) if (min_month != max_month)
throw Exception("Part name " + dir_name + " contains different months", ErrorCodes::BAD_DATA_PART_NAME); throw Exception("Part name " + dir_name + " contains different months", ErrorCodes::BAD_DATA_PART_NAME);
...@@ -129,7 +129,7 @@ String MergeTreePartInfo::getPartName() const ...@@ -129,7 +129,7 @@ String MergeTreePartInfo::getPartName() const
} }
String MergeTreePartInfo::getPartNameV0(DayNum_t left_date, DayNum_t right_date) const String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) const
{ {
const auto & date_lut = DateLUT::instance(); const auto & date_lut = DateLUT::instance();
......
...@@ -52,7 +52,7 @@ struct MergeTreePartInfo ...@@ -52,7 +52,7 @@ struct MergeTreePartInfo
} }
String getPartName() const; String getPartName() const;
String getPartNameV0(DayNum_t left_date, DayNum_t right_date) const; String getPartNameV0(DayNum left_date, DayNum right_date) const;
UInt64 getBlocksCount() const UInt64 getBlocksCount() const
{ {
return static_cast<UInt64>(max_block - min_block + 1); return static_cast<UInt64>(max_block - min_block + 1);
...@@ -62,7 +62,7 @@ struct MergeTreePartInfo ...@@ -62,7 +62,7 @@ struct MergeTreePartInfo
static bool tryParsePartName(const String & dir_name, MergeTreePartInfo * part_info, MergeTreeDataFormatVersion format_version); static bool tryParsePartName(const String & dir_name, MergeTreePartInfo * part_info, MergeTreeDataFormatVersion format_version);
static void parseMinMaxDatesFromPartName(const String & part_name, DayNum_t & min_date, DayNum_t & max_date); static void parseMinMaxDatesFromPartName(const String & part_name, DayNum & min_date, DayNum & max_date);
static bool contains(const String & outer_part_name, const String & inner_part_name, MergeTreeDataFormatVersion format_version); static bool contains(const String & outer_part_name, const String & inner_part_name, MergeTreeDataFormatVersion format_version);
}; };
......
...@@ -52,7 +52,7 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const ...@@ -52,7 +52,7 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const
result += '-'; result += '-';
if (typeid_cast<const DataTypeDate *>(storage.partition_key_sample.getByPosition(i).type.get())) if (typeid_cast<const DataTypeDate *>(storage.partition_key_sample.getByPosition(i).type.get()))
result += toString(DateLUT::instance().toNumYYYYMMDD(DayNum_t(value[i].safeGet<UInt64>()))); result += toString(DateLUT::instance().toNumYYYYMMDD(DayNum(value[i].safeGet<UInt64>())));
else else
result += applyVisitor(to_string_visitor, value[i]); result += applyVisitor(to_string_visitor, value[i]);
......
...@@ -3139,8 +3139,8 @@ static String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, ...@@ -3139,8 +3139,8 @@ static String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version,
/// The date range is all month long. /// The date range is all month long.
const auto & lut = DateLUT::instance(); const auto & lut = DateLUT::instance();
time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(part_info.partition_id + "01")); time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(part_info.partition_id + "01"));
DayNum_t left_date = lut.toDayNum(start_time); DayNum left_date = lut.toDayNum(start_time);
DayNum_t right_date = DayNum_t(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1); DayNum right_date = DayNum(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);
return part_info.getPartNameV0(left_date, right_date); return part_info.getPartNameV0(left_date, right_date);
} }
......
...@@ -5,9 +5,9 @@ ...@@ -5,9 +5,9 @@
int main(int, char **) int main(int, char **)
{ {
DayNum_t today = DateLUT::instance().toDayNum(time(nullptr)); DayNum today = DateLUT::instance().toDayNum(time(nullptr));
for (DayNum_t date = today; DayNum_t(date + 10) > today; --date) for (DayNum date = today; DayNum(date + 10) > today; --date)
{ {
DB::MergeTreePartInfo part_info("partition", 0, 0, 0); DB::MergeTreePartInfo part_info("partition", 0, 0, 0);
std::string name = part_info.getPartNameV0(date, date); std::string name = part_info.getPartNameV0(date, date);
......
...@@ -15,7 +15,7 @@ ...@@ -15,7 +15,7 @@
#define DATE_LUT_YEARS (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table #define DATE_LUT_YEARS (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table
STRONG_TYPEDEF(UInt16, DayNum_t); STRONG_TYPEDEF(UInt16, DayNum);
/** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on. /** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on.
...@@ -57,10 +57,10 @@ private: ...@@ -57,10 +57,10 @@ private:
Values lut[DATE_LUT_SIZE]; Values lut[DATE_LUT_SIZE];
/// Year number after DATE_LUT_MIN_YEAR -> day num for start of year. /// Year number after DATE_LUT_MIN_YEAR -> day num for start of year.
DayNum_t years_lut[DATE_LUT_YEARS]; DayNum years_lut[DATE_LUT_YEARS];
/// Year number after DATE_LUT_MIN_YEAR * month number starting at zero -> day num for first day of month /// Year number after DATE_LUT_MIN_YEAR * month number starting at zero -> day num for first day of month
DayNum_t years_months_lut[DATE_LUT_YEARS * 12]; DayNum years_months_lut[DATE_LUT_YEARS * 12];
/// UTC offset at beginning of the Unix epoch. The same as unix timestamp of 1970-01-01 00:00:00 local time. /// UTC offset at beginning of the Unix epoch. The same as unix timestamp of 1970-01-01 00:00:00 local time.
time_t offset_at_start_of_epoch; time_t offset_at_start_of_epoch;
...@@ -117,12 +117,12 @@ public: ...@@ -117,12 +117,12 @@ public:
return lut[index - (lut[index].day_of_week - 1)].date; return lut[index - (lut[index].day_of_week - 1)].date;
} }
inline DayNum_t toFirstDayNumOfWeek(DayNum_t d) const inline DayNum toFirstDayNumOfWeek(DayNum d) const
{ {
return DayNum_t(d - (lut[d].day_of_week - 1)); return DayNum(d - (lut[d].day_of_week - 1));
} }
inline DayNum_t toFirstDayNumOfWeek(time_t t) const inline DayNum toFirstDayNumOfWeek(time_t t) const
{ {
return toFirstDayNumOfWeek(toDayNum(t)); return toFirstDayNumOfWeek(toDayNum(t));
} }
...@@ -134,18 +134,18 @@ public: ...@@ -134,18 +134,18 @@ public:
return lut[index - (lut[index].day_of_month - 1)].date; return lut[index - (lut[index].day_of_month - 1)].date;
} }
inline DayNum_t toFirstDayNumOfMonth(DayNum_t d) const inline DayNum toFirstDayNumOfMonth(DayNum d) const
{ {
return DayNum_t(d - (lut[d].day_of_month - 1)); return DayNum(d - (lut[d].day_of_month - 1));
} }
inline DayNum_t toFirstDayNumOfMonth(time_t t) const inline DayNum toFirstDayNumOfMonth(time_t t) const
{ {
return toFirstDayNumOfMonth(toDayNum(t)); return toFirstDayNumOfMonth(toDayNum(t));
} }
/// Round down to start of quarter. /// Round down to start of quarter.
inline DayNum_t toFirstDayNumOfQuarter(DayNum_t d) const inline DayNum toFirstDayNumOfQuarter(DayNum d) const
{ {
size_t index = d; size_t index = d;
size_t month_inside_quarter = (lut[index].month - 1) % 3; size_t month_inside_quarter = (lut[index].month - 1) % 3;
...@@ -157,10 +157,10 @@ public: ...@@ -157,10 +157,10 @@ public:
--month_inside_quarter; --month_inside_quarter;
} }
return DayNum_t(index + 1); return DayNum(index + 1);
} }
inline DayNum_t toFirstDayNumOfQuarter(time_t t) const inline DayNum toFirstDayNumOfQuarter(time_t t) const
{ {
return toFirstDayNumOfQuarter(toDayNum(t)); return toFirstDayNumOfQuarter(toDayNum(t));
} }
...@@ -176,12 +176,12 @@ public: ...@@ -176,12 +176,12 @@ public:
return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date; return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date;
} }
inline DayNum_t toFirstDayNumOfYear(DayNum_t d) const inline DayNum toFirstDayNumOfYear(DayNum d) const
{ {
return years_lut[lut[d].year - DATE_LUT_MIN_YEAR]; return years_lut[lut[d].year - DATE_LUT_MIN_YEAR];
} }
inline DayNum_t toFirstDayNumOfYear(time_t t) const inline DayNum toFirstDayNumOfYear(time_t t) const
{ {
return toFirstDayNumOfYear(toDayNum(t)); return toFirstDayNumOfYear(toDayNum(t));
} }
...@@ -200,7 +200,7 @@ public: ...@@ -200,7 +200,7 @@ public:
return lut[index - (lut[index].day_of_month - 1)].date; return lut[index - (lut[index].day_of_month - 1)].date;
} }
inline UInt8 daysInMonth(DayNum_t d) const inline UInt8 daysInMonth(DayNum d) const
{ {
return lut[d].days_in_month; return lut[d].days_in_month;
} }
...@@ -300,20 +300,20 @@ public: ...@@ -300,20 +300,20 @@ public:
* because the same calendar day starts/ends at different timestamps in different time zones) * because the same calendar day starts/ends at different timestamps in different time zones)
*/ */
inline DayNum_t toDayNum(time_t t) const { return static_cast<DayNum_t>(findIndex(t)); } inline DayNum toDayNum(time_t t) const { return static_cast<DayNum>(findIndex(t)); }
inline time_t fromDayNum(DayNum_t d) const { return lut[d].date; } inline time_t fromDayNum(DayNum d) const { return lut[d].date; }
inline time_t toDate(DayNum_t d) const { return lut[d].date; } inline time_t toDate(DayNum d) const { return lut[d].date; }
inline unsigned toMonth(DayNum_t d) const { return lut[d].month; } inline unsigned toMonth(DayNum d) const { return lut[d].month; }
inline unsigned toQuarter(DayNum_t d) const { return (lut[d].month - 1) / 3 + 1; } inline unsigned toQuarter(DayNum d) const { return (lut[d].month - 1) / 3 + 1; }
inline unsigned toYear(DayNum_t d) const { return lut[d].year; } inline unsigned toYear(DayNum d) const { return lut[d].year; }
inline unsigned toDayOfWeek(DayNum_t d) const { return lut[d].day_of_week; } inline unsigned toDayOfWeek(DayNum d) const { return lut[d].day_of_week; }
inline unsigned toDayOfMonth(DayNum_t d) const { return lut[d].day_of_month; } inline unsigned toDayOfMonth(DayNum d) const { return lut[d].day_of_month; }
/// Number of week from some fixed moment in the past. Week begins at monday. /// Number of week from some fixed moment in the past. Week begins at monday.
/// (round down to monday and divide DayNum by 7; we made an assumption, /// (round down to monday and divide DayNum by 7; we made an assumption,
/// that in domain of the function there was no weeks with any other number of days than 7) /// that in domain of the function there was no weeks with any other number of days than 7)
inline unsigned toRelativeWeekNum(DayNum_t d) const inline unsigned toRelativeWeekNum(DayNum d) const
{ {
/// We add 8 to avoid underflow at beginning of unix epoch. /// We add 8 to avoid underflow at beginning of unix epoch.
return (d + 8 - lut[d].day_of_week) / 7; return (d + 8 - lut[d].day_of_week) / 7;
...@@ -325,7 +325,7 @@ public: ...@@ -325,7 +325,7 @@ public:
} }
/// Number of month from some fixed moment in the past (year * 12 + month) /// Number of month from some fixed moment in the past (year * 12 + month)
inline unsigned toRelativeMonthNum(DayNum_t d) const inline unsigned toRelativeMonthNum(DayNum d) const
{ {
return lut[d].year * 12 + lut[d].month; return lut[d].year * 12 + lut[d].month;
} }
...@@ -335,7 +335,7 @@ public: ...@@ -335,7 +335,7 @@ public:
return toRelativeMonthNum(toDayNum(t)); return toRelativeMonthNum(toDayNum(t));
} }
inline unsigned toRelativeQuarterNum(DayNum_t d) const inline unsigned toRelativeQuarterNum(DayNum d) const
{ {
return lut[d].year * 4 + (lut[d].month - 1) / 3; return lut[d].year * 4 + (lut[d].month - 1) / 3;
} }
...@@ -356,7 +356,7 @@ public: ...@@ -356,7 +356,7 @@ public:
return (t + 86400 - offset_at_start_of_epoch) / 3600; return (t + 86400 - offset_at_start_of_epoch) / 3600;
} }
inline time_t toRelativeHourNum(DayNum_t d) const inline time_t toRelativeHourNum(DayNum d) const
{ {
return toRelativeHourNum(lut[d].date); return toRelativeHourNum(lut[d].date);
} }
...@@ -366,18 +366,18 @@ public: ...@@ -366,18 +366,18 @@ public:
return t / 60; return t / 60;
} }
inline time_t toRelativeMinuteNum(DayNum_t d) const inline time_t toRelativeMinuteNum(DayNum d) const
{ {
return toRelativeMinuteNum(lut[d].date); return toRelativeMinuteNum(lut[d].date);
} }
/// Create DayNum_t from year, month, day of month. /// Create DayNum from year, month, day of month.
inline DayNum_t makeDayNum(UInt16 year, UInt8 month, UInt8 day_of_month) const inline DayNum makeDayNum(UInt16 year, UInt8 month, UInt8 day_of_month) const
{ {
if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31)) if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31))
return DayNum_t(0); return DayNum(0);
return DayNum_t(years_months_lut[(year - DATE_LUT_MIN_YEAR) * 12 + month - 1] + day_of_month - 1); return DayNum(years_months_lut[(year - DATE_LUT_MIN_YEAR) * 12 + month - 1] + day_of_month - 1);
} }
inline time_t makeDate(UInt16 year, UInt8 month, UInt8 day_of_month) const inline time_t makeDate(UInt16 year, UInt8 month, UInt8 day_of_month) const
...@@ -398,7 +398,7 @@ public: ...@@ -398,7 +398,7 @@ public:
return lut[index].date + time_offset; return lut[index].date + time_offset;
} }
inline const Values & getValues(DayNum_t d) const { return lut[d]; } inline const Values & getValues(DayNum d) const { return lut[d]; }
inline const Values & getValues(time_t t) const { return lut[findIndex(t)]; } inline const Values & getValues(time_t t) const { return lut[findIndex(t)]; }
inline UInt32 toNumYYYYMM(time_t t) const inline UInt32 toNumYYYYMM(time_t t) const
...@@ -407,7 +407,7 @@ public: ...@@ -407,7 +407,7 @@ public:
return values.year * 100 + values.month; return values.year * 100 + values.month;
} }
inline UInt32 toNumYYYYMM(DayNum_t d) const inline UInt32 toNumYYYYMM(DayNum d) const
{ {
const Values & values = lut[d]; const Values & values = lut[d];
return values.year * 100 + values.month; return values.year * 100 + values.month;
...@@ -419,7 +419,7 @@ public: ...@@ -419,7 +419,7 @@ public:
return values.year * 10000 + values.month * 100 + values.day_of_month; return values.year * 10000 + values.month * 100 + values.day_of_month;
} }
inline UInt32 toNumYYYYMMDD(DayNum_t d) const inline UInt32 toNumYYYYMMDD(DayNum d) const
{ {
const Values & values = lut[d]; const Values & values = lut[d];
return values.year * 10000 + values.month * 100 + values.day_of_month; return values.year * 10000 + values.month * 100 + values.day_of_month;
...@@ -430,7 +430,7 @@ public: ...@@ -430,7 +430,7 @@ public:
return makeDate(num / 10000, num / 100 % 100, num % 100); return makeDate(num / 10000, num / 100 % 100, num % 100);
} }
inline DayNum_t YYYYMMDDToDayNum(UInt32 num) const inline DayNum YYYYMMDDToDayNum(UInt32 num) const
{ {
return makeDayNum(num / 10000, num / 100 % 100, num % 100); return makeDayNum(num / 10000, num / 100 % 100, num % 100);
} }
...@@ -497,7 +497,7 @@ public: ...@@ -497,7 +497,7 @@ public:
/// Example: 31 Aug + 1 month = 30 Sep. /// Example: 31 Aug + 1 month = 30 Sep.
inline time_t addMonths(time_t t, Int64 delta) const inline time_t addMonths(time_t t, Int64 delta) const
{ {
DayNum_t result_day = addMonths(toDayNum(t), delta); DayNum result_day = addMonths(toDayNum(t), delta);
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t); time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
...@@ -507,7 +507,7 @@ public: ...@@ -507,7 +507,7 @@ public:
return lut[result_day].date + time_offset; return lut[result_day].date + time_offset;
} }
inline DayNum_t addMonths(DayNum_t d, Int64 delta) const inline DayNum addMonths(DayNum d, Int64 delta) const
{ {
const Values & values = lut[d]; const Values & values = lut[d];
...@@ -534,7 +534,7 @@ public: ...@@ -534,7 +534,7 @@ public:
/// Saturation can occur if 29 Feb is mapped to non-leap year. /// Saturation can occur if 29 Feb is mapped to non-leap year.
inline time_t addYears(time_t t, Int64 delta) const inline time_t addYears(time_t t, Int64 delta) const
{ {
DayNum_t result_day = addYears(toDayNum(t), delta); DayNum result_day = addYears(toDayNum(t), delta);
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t); time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
...@@ -544,7 +544,7 @@ public: ...@@ -544,7 +544,7 @@ public:
return lut[result_day].date + time_offset; return lut[result_day].date + time_offset;
} }
inline DayNum_t addYears(DayNum_t d, Int64 delta) const inline DayNum addYears(DayNum d, Int64 delta) const
{ {
const Values & values = lut[d]; const Values & values = lut[d];
...@@ -607,7 +607,7 @@ public: ...@@ -607,7 +607,7 @@ public:
return s; return s;
} }
inline std::string dateToString(DayNum_t d) const inline std::string dateToString(DayNum d) const
{ {
const Values & values = lut[d]; const Values & values = lut[d];
......
...@@ -62,7 +62,7 @@ public: ...@@ -62,7 +62,7 @@ public:
init(time); init(time);
} }
LocalDate(DayNum_t day_num) LocalDate(DayNum day_num)
{ {
const auto & values = DateLUT::instance().getValues(day_num); const auto & values = DateLUT::instance().getValues(day_num);
m_year = values.year; m_year = values.year;
...@@ -103,12 +103,12 @@ public: ...@@ -103,12 +103,12 @@ public:
return DateLUT::instance().makeDate(m_year, m_month, m_day); return DateLUT::instance().makeDate(m_year, m_month, m_day);
} }
DayNum_t getDayNum() const DayNum getDayNum() const
{ {
return DateLUT::instance().makeDayNum(m_year, m_month, m_day); return DateLUT::instance().makeDayNum(m_year, m_month, m_day);
} }
operator DayNum_t() const operator DayNum() const
{ {
return getDayNum(); return getDayNum();
} }
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册