提交 671395e8 编写于 作者: A Alexey Milovidov

Most likely improve performance

上级 845f4afb
......@@ -944,16 +944,70 @@ public:
}
struct DateComponents
{
uint16_t year;
uint8_t month;
uint8_t day;
};
struct TimeComponents
{
uint8_t hour;
uint8_t minute;
uint8_t second;
};
struct DateTimeComponents
{
DateComponents date;
TimeComponents time;
};
inline DateComponents toDateComponents(time_t t) const
{
const Values & values = getValues(t);
return { values.year, values.month, values.day_of_month };
}
inline DateTimeComponents toDateTimeComponents(time_t t) const
{
const LUTIndex index = findIndex(t);
const Values & values = lut[index];
DateTimeComponents res;
res.date.year = values.year;
res.date.month = values.month;
res.date.day = values.day_of_month;
time_t time = t - values.date;
if (time >= values.time_at_offset_change())
time += values.amount_of_offset_change();
res.time.second = time % 60;
res.time.minute = time / 60 % 60;
res.time.hour = time / 3600;
/// In case time was changed backwards at the start of next day, we will repeat the hour 23.
if (unlikely(res.time.hour > 23))
res.time.hour = 23;
return res;
}
inline UInt64 toNumYYYYMMDDhhmmss(time_t t) const
{
const Values & values = find(t);
DateTimeComponents components = toDateTimeComponents(t);
return
toSecond(t)
+ toMinute(t) * 100
+ toHour(t) * 10000
+ UInt64(values.day_of_month) * 1000000
+ UInt64(values.month) * 100000000
+ UInt64(values.year) * 10000000000;
components.time.second
+ components.time.minute * 100
+ components.time.hour * 10000
+ UInt64(components.date.day) * 1000000
+ UInt64(components.date.month) * 100000000
+ UInt64(components.date.year) * 10000000000;
}
inline time_t YYYYMMDDhhmmssToTime(UInt64 num) const
......@@ -972,16 +1026,19 @@ public:
inline NO_SANITIZE_UNDEFINED time_t addDays(time_t t, Int64 delta) const
{
LUTIndex index = findIndex(t);
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
const LUTIndex index = findIndex(t);
const Values & values = lut[index];
index += delta;
index &= date_lut_mask;
time_t time = t - values.date;
if (time >= values.time_at_offset_change())
time += values.amount_of_offset_change();
if (time_offset >= lut[index].time_at_offset_change())
time_offset -= lut[index].amount_of_offset_change();
const LUTIndex new_index = index + delta;
return lut[index].date + time_offset;
if (time >= lut[new_index].time_at_offset_change())
time -= lut[new_index].amount_of_offset_change();
return lut[new_index].date + time;
}
inline NO_SANITIZE_UNDEFINED time_t addWeeks(time_t t, Int64 delta) const
......@@ -1033,12 +1090,17 @@ public:
{
const auto result_day = addMonthsIndex(t, delta);
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
const LUTIndex index = findIndex(t);
const Values & values = lut[index];
time_t time = t - values.date;
if (time >= values.time_at_offset_change())
time += values.amount_of_offset_change();
if (time_offset >= lut[result_day].time_at_offset_change())
time_offset -= lut[result_day].amount_of_offset_change();
if (time >= lut[result_day].time_at_offset_change())
time -= lut[result_day].amount_of_offset_change();
return lut[result_day].date + time_offset;
return lut[result_day].date + time;
}
inline ExtendedDayNum NO_SANITIZE_UNDEFINED addMonths(ExtendedDayNum d, Int64 delta) const
......@@ -1077,12 +1139,17 @@ public:
{
auto result_day = addYearsIndex(t, delta);
time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t);
const LUTIndex index = findIndex(t);
const Values & values = lut[index];
time_t time = t - values.date;
if (time >= values.time_at_offset_change())
time += values.amount_of_offset_change();
if (time_offset >= lut[result_day].time_at_offset_change())
time_offset -= lut[result_day].amount_of_offset_change();
if (time >= lut[result_day].time_at_offset_change())
time -= lut[result_day].amount_of_offset_change();
return lut[result_day].date + time_offset;
return lut[result_day].date + time;
}
inline ExtendedDayNum addYears(ExtendedDayNum d, Int64 delta) const
......@@ -1093,29 +1160,25 @@ public:
inline std::string timeToString(time_t t) const
{
const Values & values = getValues(t);
DateTimeComponents components = toDateTimeComponents(t);
std::string s {"0000-00-00 00:00:00"};
s[0] += values.year / 1000;
s[1] += (values.year / 100) % 10;
s[2] += (values.year / 10) % 10;
s[3] += values.year % 10;
s[5] += values.month / 10;
s[6] += values.month % 10;
s[8] += values.day_of_month / 10;
s[9] += values.day_of_month % 10;
auto hour = toHour(t);
auto minute = toMinute(t);
auto second = toSecond(t);
s[11] += hour / 10;
s[12] += hour % 10;
s[14] += minute / 10;
s[15] += minute % 10;
s[17] += second / 10;
s[18] += second % 10;
s[0] += components.date.year / 1000;
s[1] += (components.date.year / 100) % 10;
s[2] += (components.date.year / 10) % 10;
s[3] += components.date.year % 10;
s[5] += components.date.month / 10;
s[6] += components.date.month % 10;
s[8] += components.date.day / 10;
s[9] += components.date.day % 10;
s[11] += components.time.hour / 10;
s[12] += components.time.hour % 10;
s[14] += components.time.minute / 10;
s[15] += components.time.minute % 10;
s[17] += components.time.second / 10;
s[18] += components.time.second % 10;
return s;
}
......
......@@ -92,17 +92,6 @@ public:
LocalDate(const LocalDate &) noexcept = default;
LocalDate & operator= (const LocalDate &) noexcept = default;
LocalDate & operator= (time_t time)
{
init(time);
return *this;
}
operator time_t() const
{
return DateLUT::instance().makeDate(m_year, m_month, m_day);
}
DayNum getDayNum() const
{
const auto & lut = DateLUT::instance();
......@@ -167,12 +156,3 @@ public:
};
static_assert(sizeof(LocalDate) == 4);
namespace std
{
inline string to_string(const LocalDate & date)
{
return date.toString();
}
}
......@@ -29,29 +29,16 @@ private:
/// NOTE We may use attribute packed instead, but it is less portable.
unsigned char pad = 0;
void init(time_t time)
void init(time_t time, const DateLUTImpl & time_zone)
{
if (unlikely(time > DATE_LUT_MAX || time == 0))
{
m_year = 0;
m_month = 0;
m_day = 0;
m_hour = 0;
m_minute = 0;
m_second = 0;
return;
}
const auto & date_lut = DateLUT::instance();
const auto & values = date_lut.getValues(time);
m_year = values.year;
m_month = values.month;
m_day = values.day_of_month;
m_hour = date_lut.toHour(time);
m_minute = date_lut.toMinute(time);
m_second = date_lut.toSecond(time);
DateLUTImpl::DateTimeComponents components = time_zone.toDateTimeComponents(time);
m_year = components.date.year;
m_month = components.date.month;
m_day = components.date.day;
m_hour = components.time.hour;
m_minute = components.time.minute;
m_second = components.time.second;
(void)pad; /// Suppress unused private field warning.
}
......@@ -73,9 +60,9 @@ private:
}
public:
explicit LocalDateTime(time_t time)
explicit LocalDateTime(time_t time, const DateLUTImpl & time_zone = DateLUT::instance())
{
init(time);
init(time, time_zone);
}
LocalDateTime(unsigned short year_, unsigned char month_, unsigned char day_,
......@@ -104,19 +91,6 @@ public:
LocalDateTime(const LocalDateTime &) noexcept = default;
LocalDateTime & operator= (const LocalDateTime &) noexcept = default;
LocalDateTime & operator= (time_t time)
{
init(time);
return *this;
}
operator time_t() const
{
return m_year == 0
? 0
: DateLUT::instance().makeDateTime(m_year, m_month, m_day, m_hour, m_minute, m_second);
}
unsigned short year() const { return m_year; }
unsigned char month() const { return m_month; }
unsigned char day() const { return m_day; }
......@@ -167,14 +141,3 @@ public:
};
static_assert(sizeof(LocalDateTime) == 8);
namespace std
{
inline string to_string(const LocalDateTime & datetime)
{
stringstream str;
str << datetime;
return str.str();
}
}
......@@ -1064,7 +1064,7 @@ void processCommit(
time_t commit_time;
readText(commit_time, in);
commit.time = commit_time;
commit.time = LocalDateTime(commit_time);
assertChar('\0', in);
readNullTerminated(commit.author, in);
std::string parent_hash;
......
......@@ -160,8 +160,13 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view
assert_cast<ColumnUInt16 &>(column).insertValue(UInt16{LocalDate{std::string(value)}.getDayNum()});
break;
case ValueType::vtDateTime:
assert_cast<ColumnUInt32 &>(column).insertValue(time_t{LocalDateTime{std::string(value)}});
{
ReadBufferFromString in(value);
time_t time = 0;
readDateTimeText(time, in);
assert_cast<ColumnUInt32 &>(column).insertValue(time);
break;
}
case ValueType::vtDateTime64:[[fallthrough]];
case ValueType::vtDecimal32: [[fallthrough]];
case ValueType::vtDecimal64: [[fallthrough]];
......@@ -257,7 +262,13 @@ void PostgreSQLBlockInputStream::prepareArrayInfo(size_t column_idx, const DataT
else if (which.isDate())
parser = [](std::string & field) -> Field { return UInt16{LocalDate{field}.getDayNum()}; };
else if (which.isDateTime())
parser = [](std::string & field) -> Field { return time_t{LocalDateTime{field}}; };
parser = [](std::string & field) -> Field
{
ReadBufferFromString in(field);
time_t time = 0;
readDateTimeText(time, in);
return time;
};
else if (which.isDecimal32())
parser = [nested](std::string & field) -> Field
{
......
......@@ -118,10 +118,9 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate()
{
if (update_time != std::chrono::system_clock::from_time_t(0))
{
auto tmp_time = update_time;
time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - 1;
std::string str_time = DateLUT::instance().timeToString(hr_time);
update_time = std::chrono::system_clock::now();
time_t hr_time = std::chrono::system_clock::to_time_t(tmp_time) - 1;
std::string str_time = std::to_string(LocalDateTime(hr_time));
return query_builder.composeUpdateQuery(update_field, str_time);
}
else
......
......@@ -107,10 +107,9 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate()
{
if (update_time != std::chrono::system_clock::from_time_t(0))
{
auto tmp_time = update_time;
time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - 1;
std::string str_time = DateLUT::instance().timeToString(hr_time);
update_time = std::chrono::system_clock::now();
time_t hr_time = std::chrono::system_clock::to_time_t(tmp_time) - 1;
std::string str_time = std::to_string(LocalDateTime(hr_time));
return query_builder.composeUpdateQuery(update_field, str_time);
}
else
......@@ -262,7 +261,7 @@ LocalDateTime MySQLDictionarySource::getLastModification(mysqlxx::Pool::Entry &
if (!update_time_value.isNull())
{
modification_time = update_time_value.getDateTime();
LOG_TRACE(log, "Got modification time: {}", modification_time);
LOG_TRACE(log, "Got modification time: {}", update_time_value.getString());
}
/// fetch remaining rows to avoid "commands out of sync" error
......
......@@ -127,10 +127,9 @@ std::string PostgreSQLDictionarySource::getUpdateFieldAndDate()
{
if (update_time != std::chrono::system_clock::from_time_t(0))
{
auto tmp_time = update_time;
time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - 1;
std::string str_time = DateLUT::instance().timeToString(hr_time);
update_time = std::chrono::system_clock::now();
time_t hr_time = std::chrono::system_clock::to_time_t(tmp_time) - 1;
std::string str_time = std::to_string(LocalDateTime(hr_time));
return query_builder.composeUpdateQuery(update_field, str_time);
}
else
......
......@@ -99,8 +99,13 @@ namespace DB
assert_cast<ColumnUInt16 &>(column).insertValue(parse<LocalDate>(string_value).getDayNum());
break;
case ValueType::vtDateTime:
assert_cast<ColumnUInt32 &>(column).insertValue(static_cast<UInt32>(parse<LocalDateTime>(string_value)));
{
ReadBufferFromString in(string_value);
time_t time = 0;
readDateTimeText(time, in);
assert_cast<ColumnUInt32 &>(column).insertValue(time);
break;
}
case ValueType::vtUUID:
assert_cast<ColumnUInt128 &>(column).insertValue(parse<UUID>(string_value));
break;
......
......@@ -156,10 +156,9 @@ std::string XDBCDictionarySource::getUpdateFieldAndDate()
{
if (update_time != std::chrono::system_clock::from_time_t(0))
{
auto tmp_time = update_time;
time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - 1;
std::string str_time = DateLUT::instance().timeToString(hr_time);
update_time = std::chrono::system_clock::now();
time_t hr_time = std::chrono::system_clock::to_time_t(tmp_time) - 1;
std::string str_time = std::to_string(LocalDateTime(hr_time));
return query_builder.composeUpdateQuery(update_field, str_time);
}
else
......
......@@ -11,6 +11,7 @@
# include <Columns/ColumnFixedString.h>
# include <DataTypes/IDataType.h>
# include <DataTypes/DataTypeNullable.h>
# include <IO/ReadBufferFromString.h>
# include <IO/ReadHelpers.h>
# include <IO/WriteHelpers.h>
# include <IO/Operators.h>
......@@ -97,8 +98,13 @@ namespace
assert_cast<ColumnUInt16 &>(column).insertValue(UInt16(value.getDate().getDayNum()));
break;
case ValueType::vtDateTime:
assert_cast<ColumnUInt32 &>(column).insertValue(UInt32(value.getDateTime()));
{
ReadBufferFromString in(value);
time_t time = 0;
readDateTimeText(time, in);
assert_cast<ColumnUInt32 &>(column).insertValue(time);
break;
}
case ValueType::vtUUID:
assert_cast<ColumnUInt128 &>(column).insert(parse<UUID>(value.data(), value.size()));
break;
......
......@@ -821,24 +821,18 @@ inline void writeDateTimeText(const LocalDateTime & datetime, WriteBuffer & buf)
template <char date_delimeter = '-', char time_delimeter = ':', char between_date_time_delimiter = ' '>
inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance())
{
const auto & values = time_zone.getValues(datetime);
writeDateTimeText<date_delimeter, time_delimeter, between_date_time_delimiter>(
LocalDateTime(values.year, values.month, values.day_of_month,
time_zone.toHour(datetime), time_zone.toMinute(datetime), time_zone.toSecond(datetime)), buf);
writeDateTimeText<date_delimeter, time_delimeter, between_date_time_delimiter>(LocalDateTime(datetime, time_zone), buf);
}
/// In the format YYYY-MM-DD HH:MM:SS.NNNNNNNNN, according to the specified time zone.
template <char date_delimeter = '-', char time_delimeter = ':', char between_date_time_delimiter = ' ', char fractional_time_delimiter = '.'>
inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance())
{
static constexpr UInt32 MaxScale = DecimalUtils::max_precision<DateTime64>;
scale = scale > MaxScale ? MaxScale : scale;
auto components = DecimalUtils::split(datetime64, scale);
const auto & values = date_lut.getValues(components.whole);
writeDateTimeText<date_delimeter, time_delimeter, between_date_time_delimiter>(
LocalDateTime(values.year, values.month, values.day_of_month,
date_lut.toHour(components.whole), date_lut.toMinute(components.whole), date_lut.toSecond(components.whole)), buf);
writeDateTimeText<date_delimeter, time_delimeter, between_date_time_delimiter>(LocalDateTime(components.whole, time_zone), buf);
if (scale > 0)
{
......
......@@ -75,7 +75,9 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat
LocalDateTime create_time_dt;
*buf >> "create time: " >> create_time_dt >> "\n";
create_time = create_time_dt;
create_time = DateLUT::instance().makeDateTime(
create_time_dt.year(), create_time_dt.month(), create_time_dt.day(),
create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second());
*buf >> "commands: ";
commands.readText(*buf);
......
......@@ -162,7 +162,9 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in)
{
LocalDateTime create_time_dt;
in >> "create_time: " >> create_time_dt >> "\n";
create_time = create_time_dt;
create_time = DateLUT::instance().makeDateTime(
create_time_dt.year(), create_time_dt.month(), create_time_dt.day(),
create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second());
}
in >> "source replica: " >> source_replica >> "\n";
......
......@@ -37,7 +37,9 @@ void ReplicatedMergeTreeMutationEntry::readText(ReadBuffer & in)
LocalDateTime create_time_dt;
in >> "create time: " >> create_time_dt >> "\n";
create_time = create_time_dt;
create_time = DateLUT::instance().makeDateTime(
create_time_dt.year(), create_time_dt.month(), create_time_dt.day(),
create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second());
in >> "source replica: " >> source_replica >> "\n";
......
......@@ -1450,7 +1450,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
{
LOG_INFO(log, "Will try to fetch part {} until '{}' because this part assigned to recompression merge. "
"Source replica {} will try to merge this part first", entry.new_part_name,
LocalDateTime(entry.create_time + storage_settings_ptr->try_fetch_recompressed_part_timeout.totalSeconds()), entry.source_replica);
DateLUT::instance().timeToString(entry.create_time + storage_settings_ptr->try_fetch_recompressed_part_timeout.totalSeconds()), entry.source_replica);
return false;
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册