moved getting server TZ DateLUT to separate place, upd tests and fix

This commit is contained in:
zvonand 2023-04-12 12:47:05 +02:00
parent 5d18343fb8
commit a9499eed79
23 changed files with 59 additions and 50 deletions

View File

@ -43,7 +43,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self)
time_t timestamp = Poco::Timestamp().epochTime(); time_t timestamp = Poco::Timestamp().epochTime();
auto curr_pid = Poco::Process::id(); auto curr_pid = Poco::Process::id();
process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); process_id = std::to_string(DateLUT::serverTimezoneInstance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid);
host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id;
process_path = fs::weakly_canonical(fs::path(base_dir) / ("clickhouse-copier_" + process_id)); process_path = fs::weakly_canonical(fs::path(base_dir) / ("clickhouse-copier_" + process_id));
fs::create_directories(process_path); fs::create_directories(process_path);

View File

@ -399,8 +399,8 @@ try
/// Initialize DateLUT early, to not interfere with running time of first query. /// Initialize DateLUT early, to not interfere with running time of first query.
LOG_DEBUG(log, "Initializing DateLUT."); LOG_DEBUG(log, "Initializing DateLUT.");
DateLUT::instance(); DateLUT::serverTimezoneInstance();
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::serverTimezoneInstance().getTimeZone());
/// Don't want to use DNS cache /// Don't want to use DNS cache
DNSResolver::instance().setDisableCacheFlag(); DNSResolver::instance().setDisableCacheFlag();

View File

@ -492,7 +492,7 @@ private:
const DateLUTImpl & date_lut; const DateLUTImpl & date_lut;
public: public:
explicit DateTimeModel(UInt64 seed_) : seed(seed_), date_lut(DateLUT::instance()) {} explicit DateTimeModel(UInt64 seed_) : seed(seed_), date_lut(DateLUT::serverTimezoneInstance()) {}
void train(const IColumn &) override {} void train(const IColumn &) override {}
void finalize() override {} void finalize() override {}

View File

@ -1041,8 +1041,8 @@ try
/// Initialize DateLUT early, to not interfere with running time of first query. /// Initialize DateLUT early, to not interfere with running time of first query.
LOG_DEBUG(log, "Initializing DateLUT."); LOG_DEBUG(log, "Initializing DateLUT.");
DateLUT::instance(); DateLUT::serverTimezoneInstance();
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::serverTimezoneInstance().getTimeZone());
/// Storage with temporary data for processing of heavy queries. /// Storage with temporary data for processing of heavy queries.
if (!server_settings.tmp_policy.value.empty()) if (!server_settings.tmp_policy.value.empty())

View File

@ -84,18 +84,18 @@ using namespace std::literals;
namespace namespace
{ {
using namespace DB; using namespace DB;
using ContetGetterFunc = std::function<ContextPtr(void)> const; //using ContetGetterFunc = std::function<ContextPtr(void)> const;
const void* getContextPtrOrNull(ContetGetterFunc contextFunc) //const void* getContextPtrOrNull(ContetGetterFunc contextFunc)
{ //{
try // try
{ // {
return contextFunc().get(); // return contextFunc().get();
} // }
catch(...) // catch(...)
{ // {
} // }
return nullptr; // return nullptr;
} //}
//void LogContextes(const std::string_view scope, const ContextPtr global_context) //void LogContextes(const std::string_view scope, const ContextPtr global_context)
//{ //{
@ -1087,7 +1087,7 @@ void ClientBase::onProgress(const Progress & value)
void ClientBase::onTimezoneUpdate(const String & tz) void ClientBase::onTimezoneUpdate(const String & tz)
{ {
std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; // std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl;
Settings settings; Settings settings;
settings.timezone = tz; settings.timezone = tz;

View File

@ -17,6 +17,13 @@
class DateLUT : private boost::noncopyable class DateLUT : private boost::noncopyable
{ {
public: public:
/// Return singleton DateLUTImpl instance for server's (native) time zone.
static ALWAYS_INLINE const DateLUTImpl & serverTimezoneInstance()
{
const auto & date_lut = getInstance();
return *date_lut.default_impl.load(std::memory_order_acquire);
}
/// Return singleton DateLUTImpl instance for timezone set by `timezone` setting for current session is used. /// Return singleton DateLUTImpl instance for timezone set by `timezone` setting for current session is used.
/// If it is not set, server's timezone (the one which server has) is being used. /// If it is not set, server's timezone (the one which server has) is being used.
static ALWAYS_INLINE const DateLUTImpl & instance() static ALWAYS_INLINE const DateLUTImpl & instance()

View File

@ -1003,7 +1003,7 @@ void BaseDaemon::shouldSetupWatchdog(char * argv0_)
void BaseDaemon::setupWatchdog() void BaseDaemon::setupWatchdog()
{ {
/// Initialize in advance to avoid double initialization in forked processes. /// Initialize in advance to avoid double initialization in forked processes.
DateLUT::instance(); DateLUT::serverTimezoneInstance();
std::string original_process_name; std::string original_process_name;
if (argv0) if (argv0)

View File

@ -75,7 +75,7 @@ namespace
public: public:
static constexpr auto name = "serverTimezone"; static constexpr auto name = "serverTimezone";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionServerTimezone>(context); } static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionServerTimezone>(context); }
explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance("").getTimeZone()}, context->isDistributed()) {} explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::serverTimezoneInstance().getTimeZone()}, context->isDistributed()) {}
}; };

View File

@ -31,6 +31,6 @@ public:
virtual void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text) const; virtual void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text) const;
private: private:
const DateLUTImpl & server_timezone = DateLUT::instance(""); const DateLUTImpl & server_timezone = DateLUT::serverTimezoneInstance();
bool color; bool color;
}; };

View File

@ -1030,7 +1030,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex()
DayNum 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::serverTimezoneInstance();
partition = MergeTreePartition(date_lut.toNumYYYYMM(min_date)); partition = MergeTreePartition(date_lut.toNumYYYYMM(min_date));
minmax_idx = std::make_shared<MinMaxIndex>(min_date, max_date); minmax_idx = std::make_shared<MinMaxIndex>(min_date, max_date);
} }

View File

@ -57,7 +57,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare()
{ {
LOG_INFO(log, "Will try to fetch part {} until '{}' because this part assigned to recompression merge. " 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, "Source replica {} will try to merge this part first", entry.new_part_name,
DateLUT::instance().timeToString(entry.create_time + storage_settings_ptr->try_fetch_recompressed_part_timeout.totalSeconds()), entry.source_replica); DateLUT::serverTimezoneInstance().timeToString(entry.create_time + storage_settings_ptr->try_fetch_recompressed_part_timeout.totalSeconds()), entry.source_replica);
/// Waiting other replica to recompress part. No need to check it. /// Waiting other replica to recompress part. No need to check it.
return PrepareResult{ return PrepareResult{
.prepared_successfully = false, .prepared_successfully = false,

View File

@ -115,7 +115,7 @@ void updateTTL(
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(ttl_column.get())) if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(ttl_column.get()))
{ {
const auto & date_lut = DateLUT::instance(); const auto & date_lut = DateLUT::serverTimezoneInstance();
for (const auto & val : column_date->getData()) for (const auto & val : column_date->getData())
ttl_info.update(date_lut.fromDayNum(DayNum(val))); ttl_info.update(date_lut.fromDayNum(DayNum(val)));
} }
@ -128,7 +128,7 @@ void updateTTL(
{ {
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn())) if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
{ {
const auto & date_lut = DateLUT::instance(); const auto & date_lut = DateLUT::serverTimezoneInstance();
ttl_info.update(date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>()))); ttl_info.update(date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>())));
} }
else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn())) else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
@ -369,7 +369,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
DayNum min_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].left.get<UInt64>()); DayNum min_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].left.get<UInt64>());
DayNum max_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].right.get<UInt64>()); DayNum max_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].right.get<UInt64>());
const auto & date_lut = DateLUT::instance(); const auto & date_lut = DateLUT::serverTimezoneInstance();
auto min_month = date_lut.toNumYYYYMM(min_date); auto min_month = date_lut.toNumYYYYMM(min_date);
auto max_month = date_lut.toNumYYYYMM(max_date); auto max_month = date_lut.toNumYYYYMM(max_date);

View File

@ -127,7 +127,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat
LocalDateTime create_time_dt; LocalDateTime create_time_dt;
*buf >> "create time: " >> create_time_dt >> "\n"; *buf >> "create time: " >> create_time_dt >> "\n";
create_time = DateLUT::instance().makeDateTime( create_time = DateLUT::serverTimezoneInstance().makeDateTime(
create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.year(), create_time_dt.month(), create_time_dt.day(),
create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second());

View File

@ -148,7 +148,7 @@ void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & part_name, D
throw Exception(ErrorCodes::BAD_DATA_PART_NAME, "Unexpected part name: {}", part_name); throw Exception(ErrorCodes::BAD_DATA_PART_NAME, "Unexpected part name: {}", part_name);
} }
const auto & date_lut = DateLUT::instance(); const auto & date_lut = DateLUT::serverTimezoneInstance();
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);
@ -219,7 +219,7 @@ String MergeTreePartInfo::getPartNameV1() const
String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) const String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) const
{ {
const auto & date_lut = DateLUT::instance(); const auto & date_lut = DateLUT::serverTimezoneInstance();
/// Directory name for the part has form: `YYYYMMDD_YYYYMMDD_N_N_L`. /// Directory name for the part has form: `YYYYMMDD_YYYYMMDD_N_N_L`.

View File

@ -239,7 +239,7 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const
result += '-'; result += '-';
if (typeid_cast<const DataTypeDate *>(partition_key_sample.getByPosition(i).type.get())) if (typeid_cast<const DataTypeDate *>(partition_key_sample.getByPosition(i).type.get()))
result += toString(DateLUT::instance().toNumYYYYMMDD(DayNum(value[i].safeGet<UInt64>()))); result += toString(DateLUT::serverTimezoneInstance().toNumYYYYMMDD(DayNum(value[i].safeGet<UInt64>())));
else if (typeid_cast<const DataTypeIPv4 *>(partition_key_sample.getByPosition(i).type.get())) else if (typeid_cast<const DataTypeIPv4 *>(partition_key_sample.getByPosition(i).type.get()))
result += toString(value[i].get<IPv4>().toUnderType()); result += toString(value[i].get<IPv4>().toUnderType());
else else
@ -320,7 +320,7 @@ std::optional<Row> MergeTreePartition::tryParseValueFromID(const String & partit
throw Exception( throw Exception(
ErrorCodes::INVALID_PARTITION_VALUE, "Cannot parse partition_id: got unexpected Date: {}", date_yyyymmdd); ErrorCodes::INVALID_PARTITION_VALUE, "Cannot parse partition_id: got unexpected Date: {}", date_yyyymmdd);
UInt32 date = DateLUT::instance().YYYYMMDDToDayNum(date_yyyymmdd); UInt32 date = DateLUT::serverTimezoneInstance().YYYYMMDDToDayNum(date_yyyymmdd);
res.emplace_back(date); res.emplace_back(date);
break; break;
} }

View File

@ -199,7 +199,7 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in, MergeTreeDataFor
{ {
LocalDateTime create_time_dt; LocalDateTime create_time_dt;
in >> "create_time: " >> create_time_dt >> "\n"; in >> "create_time: " >> create_time_dt >> "\n";
create_time = DateLUT::instance().makeDateTime( create_time = DateLUT::serverTimezoneInstance().makeDateTime(
create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.year(), create_time_dt.month(), create_time_dt.day(),
create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second());
} }

View File

@ -38,7 +38,7 @@ void ReplicatedMergeTreeMutationEntry::readText(ReadBuffer & in)
LocalDateTime create_time_dt; LocalDateTime create_time_dt;
in >> "create time: " >> create_time_dt >> "\n"; in >> "create time: " >> create_time_dt >> "\n";
create_time = DateLUT::instance().makeDateTime( create_time = DateLUT::serverTimezoneInstance().makeDateTime(
create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.year(), create_time_dt.month(), create_time_dt.day(),
create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second());

View File

@ -5318,7 +5318,7 @@ String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const
if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{ {
/// The date range is all month long. /// The date range is all month long.
const auto & lut = DateLUT::instance(); const auto & lut = DateLUT::serverTimezoneInstance();
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 left_date = DayNum(lut.toDayNum(start_time).toUnderType()); DayNum left_date = DayNum(lut.toDayNum(start_time).toUnderType());
DayNum right_date = DayNum(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);

View File

@ -1335,7 +1335,7 @@ ASTPtr StorageWindowView::innerQueryParser(const ASTSelectQuery & query)
time_zone = &DateLUT::instance(window_view_timezone); time_zone = &DateLUT::instance(window_view_timezone);
} }
else else
time_zone = &DateLUT::instance(); time_zone = &DateLUT::serverTimezoneInstance();
return result; return result;
} }

View File

@ -1,3 +0,0 @@
1999-12-12 18:23:23.123
1999-12-12 23:23:23.123
1999-12-13 04:23:23.123

View File

@ -1,11 +0,0 @@
SET timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS}
SET timezone = 'Asia/Novosibirsk';
SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich');
SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich';
SET timezone = 'Asia/Manila';
SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk');
SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich';
SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn';

View File

@ -0,0 +1,5 @@
2022-12-12 17:23:23.123
2022-12-12 23:23:23.123
2022-12-12 22:23:23.123
Europe/Zurich Europe/Zurich
Pacific/Pitcairn Pacific/Pitcairn

View File

@ -0,0 +1,11 @@
SET timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS}
SET timezone = 'Asia/Novosibirsk';
SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich');
SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich';
SET timezone = 'Asia/Manila';
SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk');
SELECT timezone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich' FORMAT TSV;
SELECT timezone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn' FORMAT TSV;