mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
moved getting server TZ DateLUT to separate place, upd tests and fix
This commit is contained in:
parent
5d18343fb8
commit
a9499eed79
@ -43,7 +43,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self)
|
||||
time_t timestamp = Poco::Timestamp().epochTime();
|
||||
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;
|
||||
process_path = fs::weakly_canonical(fs::path(base_dir) / ("clickhouse-copier_" + process_id));
|
||||
fs::create_directories(process_path);
|
||||
|
@ -399,8 +399,8 @@ try
|
||||
|
||||
/// Initialize DateLUT early, to not interfere with running time of first query.
|
||||
LOG_DEBUG(log, "Initializing DateLUT.");
|
||||
DateLUT::instance();
|
||||
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone());
|
||||
DateLUT::serverTimezoneInstance();
|
||||
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::serverTimezoneInstance().getTimeZone());
|
||||
|
||||
/// Don't want to use DNS cache
|
||||
DNSResolver::instance().setDisableCacheFlag();
|
||||
|
@ -492,7 +492,7 @@ private:
|
||||
const DateLUTImpl & date_lut;
|
||||
|
||||
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 finalize() override {}
|
||||
|
@ -1041,8 +1041,8 @@ try
|
||||
|
||||
/// Initialize DateLUT early, to not interfere with running time of first query.
|
||||
LOG_DEBUG(log, "Initializing DateLUT.");
|
||||
DateLUT::instance();
|
||||
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone());
|
||||
DateLUT::serverTimezoneInstance();
|
||||
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::serverTimezoneInstance().getTimeZone());
|
||||
|
||||
/// Storage with temporary data for processing of heavy queries.
|
||||
if (!server_settings.tmp_policy.value.empty())
|
||||
|
@ -84,18 +84,18 @@ using namespace std::literals;
|
||||
namespace
|
||||
{
|
||||
using namespace DB;
|
||||
using ContetGetterFunc = std::function<ContextPtr(void)> const;
|
||||
const void* getContextPtrOrNull(ContetGetterFunc contextFunc)
|
||||
{
|
||||
try
|
||||
{
|
||||
return contextFunc().get();
|
||||
}
|
||||
catch(...)
|
||||
{
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
//using ContetGetterFunc = std::function<ContextPtr(void)> const;
|
||||
//const void* getContextPtrOrNull(ContetGetterFunc contextFunc)
|
||||
//{
|
||||
// try
|
||||
// {
|
||||
// return contextFunc().get();
|
||||
// }
|
||||
// catch(...)
|
||||
// {
|
||||
// }
|
||||
// return nullptr;
|
||||
//}
|
||||
|
||||
//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)
|
||||
{
|
||||
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.timezone = tz;
|
||||
|
@ -17,6 +17,13 @@
|
||||
class DateLUT : private boost::noncopyable
|
||||
{
|
||||
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.
|
||||
/// If it is not set, server's timezone (the one which server has) is being used.
|
||||
static ALWAYS_INLINE const DateLUTImpl & instance()
|
||||
|
@ -1003,7 +1003,7 @@ void BaseDaemon::shouldSetupWatchdog(char * argv0_)
|
||||
void BaseDaemon::setupWatchdog()
|
||||
{
|
||||
/// Initialize in advance to avoid double initialization in forked processes.
|
||||
DateLUT::instance();
|
||||
DateLUT::serverTimezoneInstance();
|
||||
|
||||
std::string original_process_name;
|
||||
if (argv0)
|
||||
|
@ -75,7 +75,7 @@ namespace
|
||||
public:
|
||||
static constexpr auto name = "serverTimezone";
|
||||
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()) {}
|
||||
};
|
||||
|
||||
|
||||
|
@ -31,6 +31,6 @@ public:
|
||||
virtual void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text) const;
|
||||
|
||||
private:
|
||||
const DateLUTImpl & server_timezone = DateLUT::instance("");
|
||||
const DateLUTImpl & server_timezone = DateLUT::serverTimezoneInstance();
|
||||
bool color;
|
||||
};
|
||||
|
@ -1030,7 +1030,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex()
|
||||
DayNum 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));
|
||||
minmax_idx = std::make_shared<MinMaxIndex>(min_date, max_date);
|
||||
}
|
||||
|
@ -57,7 +57,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare()
|
||||
{
|
||||
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,
|
||||
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.
|
||||
return PrepareResult{
|
||||
.prepared_successfully = false,
|
||||
|
@ -115,7 +115,7 @@ void updateTTL(
|
||||
|
||||
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())
|
||||
ttl_info.update(date_lut.fromDayNum(DayNum(val)));
|
||||
}
|
||||
@ -128,7 +128,7 @@ void updateTTL(
|
||||
{
|
||||
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>())));
|
||||
}
|
||||
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 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 max_month = date_lut.toNumYYYYMM(max_date);
|
||||
|
@ -127,7 +127,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat
|
||||
|
||||
LocalDateTime create_time_dt;
|
||||
*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.hour(), create_time_dt.minute(), create_time_dt.second());
|
||||
|
||||
|
@ -148,7 +148,7 @@ void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & part_name, D
|
||||
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);
|
||||
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
|
||||
{
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
const auto & date_lut = DateLUT::serverTimezoneInstance();
|
||||
|
||||
/// Directory name for the part has form: `YYYYMMDD_YYYYMMDD_N_N_L`.
|
||||
|
||||
|
@ -239,7 +239,7 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const
|
||||
result += '-';
|
||||
|
||||
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()))
|
||||
result += toString(value[i].get<IPv4>().toUnderType());
|
||||
else
|
||||
@ -320,7 +320,7 @@ std::optional<Row> MergeTreePartition::tryParseValueFromID(const String & partit
|
||||
throw Exception(
|
||||
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);
|
||||
break;
|
||||
}
|
||||
|
@ -199,7 +199,7 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in, MergeTreeDataFor
|
||||
{
|
||||
LocalDateTime create_time_dt;
|
||||
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.hour(), create_time_dt.minute(), create_time_dt.second());
|
||||
}
|
||||
|
@ -38,7 +38,7 @@ void ReplicatedMergeTreeMutationEntry::readText(ReadBuffer & in)
|
||||
|
||||
LocalDateTime create_time_dt;
|
||||
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.hour(), create_time_dt.minute(), create_time_dt.second());
|
||||
|
||||
|
@ -5318,7 +5318,7 @@ String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const
|
||||
if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
/// 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"));
|
||||
DayNum left_date = DayNum(lut.toDayNum(start_time).toUnderType());
|
||||
DayNum right_date = DayNum(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);
|
||||
|
@ -1335,7 +1335,7 @@ ASTPtr StorageWindowView::innerQueryParser(const ASTSelectQuery & query)
|
||||
time_zone = &DateLUT::instance(window_view_timezone);
|
||||
}
|
||||
else
|
||||
time_zone = &DateLUT::instance();
|
||||
time_zone = &DateLUT::serverTimezoneInstance();
|
||||
|
||||
return result;
|
||||
}
|
||||
|
@ -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
|
@ -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';
|
@ -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
|
11
tests/queries/0_stateless/02681_timezone_setting.sql
Normal file
11
tests/queries/0_stateless/02681_timezone_setting.sql
Normal 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;
|
Loading…
Reference in New Issue
Block a user