Merge 'support_timezone_dev' into 'cnch-dev'

feat(clickhousech@m-4692103199): [TO CNCH-dev] Support session level timezone

See merge request: !23632
This commit is contained in:
fredwang 2024-09-24 03:18:47 +00:00
parent b0ecc2f697
commit 47af6a390f
116 changed files with 650 additions and 270 deletions

View File

@ -39,7 +39,7 @@ std::string to_string(const std::chrono::time_point<Clock, Duration> & tp)
// Don't use DateLUT because it shows weird characters for
// TimePoint::max(). I wish we could use C++20 format, but it's not
// there yet.
// return DateLUT::instance().timeToString(std::chrono::system_clock::to_time_t(tp));
// return DateLUT::serverTimezoneInstance().timeToString(std::chrono::system_clock::to_time_t(tp));
auto in_time_t = std::chrono::system_clock::to_time_t(tp);
return to_string(in_time_t);

View File

@ -1128,7 +1128,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)

View File

@ -19,7 +19,9 @@
* All Bytedance's Modifications are Copyright (2023) Bytedance Ltd. and/or its affiliates.
*/
#include "Common/CurrentThread.h"
#include "ConnectionParameters.h"
#include "Core/Protocol.h"
#include "QueryFuzzer.h"
#include "Storages/HDFS/HDFSCommon.h"
#include "Suggest.h"
@ -496,7 +498,7 @@ private:
if (current_time % 3 != 0)
return false;
auto days = DateLUT::instance().toDayNum(current_time).toUnderType();
auto days = DateLUT::sessionInstance().toDayNum(current_time).toUnderType();
for (auto d : chineseNewYearIndicators)
{
/// Let's celebrate until Lantern Festival
@ -591,12 +593,21 @@ private:
int mainImpl()
{
UseSSL use_ssl;
MainThreadStatus::getInstance();
setupSignalHandler();
registerFormats();
registerFunctions();
registerAggregateFunctions();
{
// All that just to set DB::CurrentThread::get().getGlobalContext()
// which is required for client timezone (pushed from server) to work.
auto thread_group = std::make_shared<ThreadGroupStatus>();
const_cast<ContextWeakPtr&>(thread_group->global_context) = context;
CurrentThread::attachTo(thread_group);
}
/// Batch mode is enabled if one of the following is true:
/// - -e (--query) command line option is present.
/// The value of the option is used as the text of query (or of multiple queries).
@ -652,7 +663,7 @@ private:
connect();
/// Initialize DateLUT here to avoid counting time spent here as query execution time.
const auto local_tz = DateLUT::instance().getTimeZone();
const auto local_tz = DateLUT::sessionInstance().getTimeZone();
if (is_interactive)
{
@ -1726,12 +1737,28 @@ private:
context->applySettingsChanges(settings_ast.as<ASTSetQuery>()->changes);
};
const auto * insert = parsed_query->as<ASTInsertQuery>();
if (insert && insert->settings_ast)
if (const auto * select = parsed_query->as<ASTSelectQuery>(); select && select->settings())
apply_query_settings(*select->settings());
else if (const auto * select_with_union = parsed_query->as<ASTSelectWithUnionQuery>())
{
const ASTs & children = select_with_union->list_of_selects->children;
if (!children.empty())
{
// On the client it is enough to apply settings only for the
// last SELECT, since the only thing that is important to apply
// on the client is format settings.
const auto * last_select = children.back()->as<ASTSelectQuery>();
if (last_select && last_select->settings())
{
apply_query_settings(*last_select->settings());
}
}
}
else if (const auto * query_with_output = parsed_query->as<ASTQueryWithOutput>();
query_with_output && query_with_output->settings_ast)
apply_query_settings(*query_with_output->settings_ast);
else if (insert && insert->settings_ast)
apply_query_settings(*insert->settings_ast);
/// FIXME: try to prettify this cast using `as<>()`
const auto * with_output = dynamic_cast<const ASTQueryWithOutput *>(parsed_query.get());
if (with_output && with_output->settings_ast)
apply_query_settings(*with_output->settings_ast);
if (!connection->checkConnected())
connect();
@ -2206,6 +2233,10 @@ private:
case Protocol::Server::QueryMetrics:
return true;
case Protocol::Server::TimezoneUpdate:
onTimezoneUpdate(packet.server_timezone);
return true;
default:
throw Exception(
ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}", packet.type, connection->getDescription());
@ -2238,9 +2269,13 @@ private:
columns_description = ColumnsDescription::parse(packet.multistring_message[1]);
return receiveSampleBlock(out, columns_description);
case Protocol::Server::TimezoneUpdate:
onTimezoneUpdate(packet.server_timezone);
break;
default:
throw NetException(
"Unexpected packet from server (expected Data, Exception or Log, got "
"Unexpected packet from server (expected Data, Exception or Log or TimezoneUpdate , got "
+ String(Protocol::Server::toString(packet.type)) + ")",
ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
}
@ -2269,6 +2304,10 @@ private:
onLogData(packet.block);
break;
case Protocol::Server::TimezoneUpdate:
onTimezoneUpdate(packet.server_timezone);
break;
default:
throw NetException(
"Unexpected packet from server (expected Exception, EndOfStream or Log, got "
@ -2283,7 +2322,7 @@ private:
{
auto packet_type = connection->checkPacket();
while (packet_type && *packet_type == Protocol::Server::Log)
while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::TimezoneUpdate))
{
receiveAndProcessPacket(false);
packet_type = connection->checkPacket();
@ -2551,6 +2590,11 @@ private:
}
}
void onTimezoneUpdate(const String & tz)
{
context->setSetting("session_timezone", tz);
}
static void showClientVersion()
{
#define RESET_ "\033[0m"

View File

@ -39,7 +39,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);

View File

@ -379,8 +379,8 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
/// 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();

View File

@ -461,7 +461,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 {}

View File

@ -909,8 +909,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// 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.
{

View File

@ -35,7 +35,7 @@ AggregateFunctionPtr createAggregateFunctionGenArrayMonth(const std::string & na
String date_start = parameters[1].safeGet<String>();
// use local timezone on default
String timezone = DateLUT::instance().getTimeZone();
String timezone = DateLUT::sessionInstance().getTimeZone();
if (parameters.size() == 3) {
timezone = parameters[2].safeGet<String>();
}

View File

@ -88,12 +88,12 @@ namespace
UInt64 field_time = timestamp.safeGet<UInt64>();
if (field_time > mills_test)
{
String date = DateLUT::instance().dateToString(field_time / 1000);
String date = DateLUT::serverTimezoneInstance().dateToString(field_time / 1000);
return std::make_shared<ASTLiteral>(Field(date));
}
else
{
String date = DateLUT::instance().dateToString(field_time);
String date = DateLUT::serverTimezoneInstance().dateToString(field_time);
return std::make_shared<ASTLiteral>(Field(date));
}
}

View File

@ -1344,7 +1344,6 @@ namespace Catalog
throw Exception("Table not found: " + database + "." + name, ErrorCodes::UNKNOWN_TABLE);
}
auto cache_manager = context.getPartCacheManager();
bool is_host_server = false;
const auto host_server = context.getCnchTopologyMaster()->getTargetServer(table_id->uuid(), getServerVwNameFrom(*table_id), true);
@ -1352,7 +1351,7 @@ namespace Catalog
if (!host_server.empty())
is_host_server = isLocalServer(host_server.getRPCAddress(), std::to_string(context.getRPCPort()));
if (is_host_server && cache_manager)
if (is_host_server && cache_manager && !query_context.hasSessionTimeZone())
{
auto cached_storage = cache_manager->getStorageFromCache(UUIDHelpers::toUUID(table_id->uuid()), host_server.topology_version);
if (cached_storage && cached_storage->commit_time <= ts && cached_storage->getStorageID().database_name == database && cached_storage->getStorageID().table_name == name)

View File

@ -1040,6 +1040,11 @@ Packet Connection::receivePacket()
case Protocol::Server::ReadTaskRequest:
return res;
case Protocol::Server::TimezoneUpdate:
readStringBinary(server_timezone, *in);
res.server_timezone = server_timezone;
return res;
default:
/// In unknown state, disconnect - to not leave unsynchronised connection.

View File

@ -96,6 +96,8 @@ struct Packet
BlockStreamProfileInfo profile_info;
std::vector<UUID> part_uuids;
std::string server_timezone;
Packet() : type(Protocol::Server::Hello) {}
};

View File

@ -262,6 +262,7 @@ Packet HedgedConnections::drain()
case Protocol::Server::Totals:
case Protocol::Server::Extremes:
case Protocol::Server::EndOfStream:
case Protocol::Server::TimezoneUpdate:
break;
case Protocol::Server::Exception:

View File

@ -305,6 +305,7 @@ Packet MultiplexedConnections::drain()
case Protocol::Server::Totals:
case Protocol::Server::Extremes:
case Protocol::Server::EndOfStream:
case Protocol::Server::TimezoneUpdate:
break;
case Protocol::Server::ProfileInfo:
@ -383,6 +384,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac
case Protocol::Server::Totals:
case Protocol::Server::Extremes:
case Protocol::Server::Log:
case Protocol::Server::TimezoneUpdate:
break;
case Protocol::Server::EndOfStream:

View File

@ -376,6 +376,8 @@ brpc::CallId CnchWorkerClient::sendResources(
/// so it should be larger than max_execution_time to make sure the session is not to be destroyed in advance.
UInt64 recycle_timeout = max_execution_time > 0 ? max_execution_time + 60UL : 3600;
request.set_timeout(recycle_timeout);
if (!settings.session_timezone.value.empty())
request.set_session_timezone(settings.session_timezone.value);
bool require_worker_info = false;
for (const auto & resource: resources_to_send)

View File

@ -88,7 +88,7 @@ void CnchWorkerResource::executeCacheableCreateQuery(
static auto * log = &Poco::Logger::get("WorkerResource");
std::shared_ptr<StorageCloudMergeTree> cached;
if (auto cache = context->tryGetCloudTableDefinitionCache())
if (auto cache = context->tryGetCloudTableDefinitionCache(); cache && !context->hasSessionTimeZone())
{
auto load = [&]() -> std::shared_ptr<StorageCloudMergeTree>
{

View File

@ -713,6 +713,10 @@ void CnchWorkerServiceImpl::sendResources(
auto session = rpc_context->acquireNamedCnchSession(request->txn_id(), request->timeout(), false);
auto query_context = session->context;
query_context->setTemporaryTransaction(request->txn_id(), request->primary_txn_id());
if (request->has_session_timezone())
query_context->setSetting("session_timezone", request->session_timezone());
CurrentThread::QueryScope query_scope(query_context);
auto worker_resource = query_context->getCnchWorkerResource();
/// store cloud tables in cnch_session_resource.

View File

@ -1,5 +1,9 @@
#include <Common/DateLUT.h>
#include <Interpreters/Context.h>
#include <Common/CurrentThread.h>
#include <Common/filesystemHelpers.h>
#include <Poco/DigestStream.h>
#include <Poco/Exception.h>
#include <Poco/SHA1Engine.h>
@ -29,12 +33,12 @@ std::string determineDefaultTimeZone()
{
namespace fs = std::filesystem;
const char * tzdir_env_var = std::getenv("TZDIR");
const char * tzdir_env_var = std::getenv("TZDIR"); // NOLINT(concurrency-mt-unsafe) // ok, because it does not run concurrently with other getenv calls
fs::path tz_database_path = tzdir_env_var ? tzdir_env_var : "/usr/share/zoneinfo/";
fs::path tz_file_path;
std::string error_prefix;
const char * tz_env_var = std::getenv("TZ");
const char * tz_env_var = std::getenv("TZ"); // NOLINT(concurrency-mt-unsafe) // ok, because it does not run concurrently with other getenv calls
/// In recent tzdata packages some files now are symlinks and canonical path resolution
/// may give wrong timezone names - store the name as it is, if possible.
@ -138,6 +142,38 @@ std::string determineDefaultTimeZone()
}
const DateLUTImpl & DateLUT::sessionInstance()
{
const auto & date_lut = getInstance();
if (DB::CurrentThread::isInitialized())
{
std::string timezone_from_context;
const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext();
if (query_context)
{
timezone_from_context = extractTimezoneFromContext(query_context);
if (!timezone_from_context.empty())
return date_lut.getImplementation(timezone_from_context);
}
/// On the server side, timezone is passed in query_context,
/// but on CH-client side we have no query context,
/// and each time we modify client's global context
const DB::ContextPtr global_context = DB::CurrentThread::get().getGlobalContext();
if (global_context)
{
timezone_from_context = extractTimezoneFromContext(global_context);
if (!timezone_from_context.empty())
return date_lut.getImplementation(timezone_from_context);
}
}
return serverTimezoneInstance();
}
DateLUT::DateLUT()
{
/// Initialize the pointer to the default DateLUTImpl.
@ -148,7 +184,7 @@ DateLUT::DateLUT()
const DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const
{
std::lock_guard<std::mutex> lock(mutex);
std::lock_guard lock(mutex);
auto it = impls.emplace(time_zone, nullptr).first;
if (!it->second)
@ -162,3 +198,8 @@ DateLUT & DateLUT::getInstance()
static DateLUT ret;
return ret;
}
std::string DateLUT::extractTimezoneFromContext(DB::ContextPtr query_context)
{
return query_context->getSettingsRef().session_timezone.value;
}

View File

@ -3,6 +3,7 @@
#include <Common/DateLUTImpl.h>
#include <common/defines.h>
#include <common/types.h>
#include <boost/noncopyable.hpp>
@ -11,13 +12,22 @@
#include <mutex>
#include <unordered_map>
namespace DB
{
class Context;
using ContextPtr = std::shared_ptr<const Context>;
}
class DateLUTImpl;
/// This class provides lazy initialization and lookup of singleton DateLUTImpl objects for a given timezone.
class DateLUT : private boost::noncopyable
{
public:
/// Return singleton DateLUTImpl instance for the default time zone.
static ALWAYS_INLINE const DateLUTImpl & instance() // -V1071
/// The default instance will return singleton DateLUTImpl for the server time zone.
/// It may be set using 'timezone' server setting.
static ALWAYS_INLINE const DateLUTImpl & serverTimezoneInstance()
{
const auto & date_lut = getInstance();
return *date_lut.default_impl.load(std::memory_order_acquire);
@ -26,12 +36,18 @@ public:
/// Return singleton DateLUTImpl instance for a given time zone.
static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone)
{
const auto & date_lut = getInstance();
if (time_zone.empty())
return *date_lut.default_impl.load(std::memory_order_acquire);
return sessionInstance();
const auto & date_lut = getInstance();
return date_lut.getImplementation(time_zone);
}
/// Return DateLUTImpl instance for session timezone.
/// session_timezone is a session-level setting.
/// If setting is not set, returns the server timezone.
static const DateLUTImpl & sessionInstance();
static void setDefaultTimezone(const std::string & time_zone)
{
auto & date_lut = getInstance();
@ -45,6 +61,8 @@ protected:
private:
static DateLUT & getInstance();
static std::string extractTimezoneFromContext(DB::ContextPtr query_context);
const DateLUTImpl & getImplementation(const std::string & time_zone) const;
using DateLUTImplPtr = std::unique_ptr<DateLUTImpl>;

View File

@ -46,9 +46,8 @@ private:
unsigned char m_month;
unsigned char m_day;
void init(time_t time)
void init(time_t time, const DateLUTImpl & date_lut)
{
const auto & date_lut = DateLUT::instance();
const auto & values = date_lut.getValues(time);
m_year = values.year;
@ -78,22 +77,19 @@ private:
}
public:
explicit LocalDate(time_t time)
{
init(time);
}
explicit LocalDate(time_t time, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance()) { init(time, time_zone); }
LocalDate(DayNum day_num)
LocalDate(DayNum day_num, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance()) /// NOLINT
{
const auto & values = DateLUT::instance().getValues(day_num);
const auto & values = time_zone.getValues(day_num);
m_year = values.year;
m_month = values.month;
m_day = values.day_of_month;
}
explicit LocalDate(ExtendedDayNum day_num)
explicit LocalDate(ExtendedDayNum day_num, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance())
{
const auto & values = DateLUT::instance().getValues(day_num);
const auto & values = time_zone.getValues(day_num);
m_year = values.year;
m_month = values.month;
m_day = values.day_of_month;
@ -121,16 +117,14 @@ public:
LocalDate(const LocalDate &) noexcept = default;
LocalDate & operator= (const LocalDate &) noexcept = default;
DayNum getDayNum() const
DayNum getDayNum(const DateLUTImpl & lut = DateLUT::serverTimezoneInstance()) const
{
const auto & lut = DateLUT::instance();
return DayNum(lut.makeDayNum(m_year, m_month, m_day).toUnderType());
}
ExtendedDayNum getExtendedDayNum() const
ExtendedDayNum getExtendedDayNum(const DateLUTImpl & lut = DateLUT::serverTimezoneInstance()) const
{
const auto & lut = DateLUT::instance();
return ExtendedDayNum (lut.makeDayNum(m_year, m_month, m_day).toUnderType());
return ExtendedDayNum(lut.makeDayNum(m_year, m_month, m_day).toUnderType());
}
operator DayNum() const
@ -138,10 +132,7 @@ public:
return getDayNum();
}
operator time_t() const
{
return DateLUT::instance().makeDate(m_year, m_month, m_day);
}
operator time_t() const { return DateLUT::serverTimezoneInstance().makeDate(m_year, m_month, m_day); }
unsigned short year() const { return m_year; }
unsigned char month() const { return m_month; }

View File

@ -81,10 +81,7 @@ private:
}
public:
explicit LocalDateTime(time_t time, const DateLUTImpl & time_zone = DateLUT::instance())
{
init(time, time_zone);
}
explicit LocalDateTime(time_t time, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance()) { init(time, time_zone); }
LocalDateTime(unsigned short year_, unsigned char month_, unsigned char day_,
unsigned char hour_, unsigned char minute_, unsigned char second_)
@ -115,9 +112,7 @@ public:
operator time_t() const
{
return m_year == 0
? 0
: DateLUT::instance().makeDateTime(m_year, m_month, m_day, m_hour, m_minute, m_second);
return m_year == 0 ? 0 : DateLUT::serverTimezoneInstance().makeDateTime(m_year, m_month, m_day, m_hour, m_minute, m_second);
}
unsigned short year() const { return m_year; }

View File

@ -313,6 +313,11 @@ public:
return query_context.lock();
}
ContextPtr getGlobalContext() const
{
return global_context.lock();
}
/// Starts new query and create new thread group for it, current thread becomes master thread of the query
void initializeQuery(MemoryTracker * memory_tracker_ = nullptr);

View File

@ -430,8 +430,11 @@ namespace MySQLReplication
UInt32 i24 = 0;
payload.readStrict(reinterpret_cast<char *>(&i24), 3);
const DayNum date_day_number(DateLUT::instance().makeDayNum(
static_cast<int>((i24 >> 9) & 0x7fff), static_cast<int>((i24 >> 5) & 0xf), static_cast<int>(i24 & 0x1f)).toUnderType());
const DayNum date_day_number(
DateLUT::serverTimezoneInstance()
.makeDayNum(
static_cast<int>((i24 >> 9) & 0x7fff), static_cast<int>((i24 >> 5) & 0xf), static_cast<int>(i24 & 0x1f))
.toUnderType());
row.push_back(Field(date_day_number.toUnderType()));
break;
@ -536,10 +539,13 @@ namespace MySQLReplication
readTimeFractionalPart(payload, fsp, meta);
UInt32 year_month = readBits(val, 1, 17, 40);
time_t date_time = DateLUT::instance().makeDateTime(
year_month / 13, year_month % 13, readBits(val, 18, 5, 40)
, readBits(val, 23, 5, 40), readBits(val, 28, 6, 40), readBits(val, 34, 6, 40)
);
time_t date_time = DateLUT::serverTimezoneInstance().makeDateTime(
year_month / 13,
year_month % 13,
readBits(val, 18, 5, 40),
readBits(val, 23, 5, 40),
readBits(val, 28, 6, 40),
readBits(val, 34, 6, 40));
if (!meta)
// The max value of the 64 bit int flagged here exceeds the year value that is

View File

@ -101,7 +101,8 @@ namespace Protocol
/// This is such an inverted logic, where server sends requests
/// And client returns back response
QueryMetrics = 14, /// Query metrics in cnch worker side
MAX = QueryMetrics,
TimezoneUpdate = 15,
MAX = TimezoneUpdate,
};
@ -126,7 +127,8 @@ namespace Protocol
"TableColumns",
"PartUUIDs",
"ReadTaskRequest",
"QueryMetrics"
"QueryMetrics",
"TimezoneUpdate"
};
return packet <= MAX
? data[packet]

View File

@ -59,9 +59,11 @@
/// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION,
/// later is just a number for server version (one number instead of commit SHA)
/// for simplicity (sometimes it may be more convenient in some use cases).
#define DBMS_TCP_PROTOCOL_VERSION 54450
#define DBMS_TCP_PROTOCOL_VERSION 54451
#define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449
#define DBMS_MIN_REVISION_WITH_QUERY_METRICS 54450
static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES = 54451;

View File

@ -459,46 +459,19 @@ enum PreloadLevelSettings : UInt64
M(String, count_distinct_implementation, "uniqExact", "What aggregate function to use for implementation of count(DISTINCT ...)", 0) \
\
M(Bool, add_http_cors_header, false, "Write add http CORS header.", 0) \
\
M(UInt64, \
max_http_get_redirects, \
0, \
"Max number of http GET redirects hops allowed. Make sure additional security measures are in place to prevent a malicious server " \
"to redirect your requests to unexpected services.", \
0) \
\
M(Bool, \
use_client_time_zone, \
false, \
"Use client timezone for interpreting DateTime string values, instead of adopting server timezone.", \
0) \
\
M(Bool, \
send_progress_in_http_headers, \
false, \
"Send progress notifications using X-ClickHouse-Progress headers. Some clients do not support high amount of HTTP headers (Python " \
"requests in particular), so it is disabled by default.", \
0) \
\
M(UInt64, \
http_headers_progress_interval_ms, \
100, \
"Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.", \
0) \
\
M(Bool, \
fsync_metadata, \
1, \
"Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with " \
"high load of DDL queries and high load of disk subsystem.", \
0) \
\
M(Bool, \
join_use_nulls, \
1, \
"Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding " \
"columns data type.", \
IMPORTANT) \
\
M(UInt64, max_http_get_redirects, 0, "Max number of http GET redirects hops allowed. Make sure additional security measures are in place to prevent a malicious server to redirect your requests to unexpected services.", 0) \
\
M(Bool, use_client_time_zone, false, "Use client timezone for interpreting DateTime string values, instead of adopting server timezone.", 0) \
M(Timezone, session_timezone, "", "The default timezone for current session or query. The default value is server default timezone if empty.", 0) \
\
M(Bool, send_progress_in_http_headers, false, "Send progress notifications using X-ClickHouse-Progress headers. Some clients do not support high amount of HTTP headers (Python requests in particular), so it is disabled by default.", 0) \
\
M(UInt64, http_headers_progress_interval_ms, 100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.", 0) \
\
M(Bool, fsync_metadata, 1, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \
\
M(Bool, join_use_nulls, 1, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \
M(Bool, join_using_null_safe, 0, "Force null safe equal comparison for USING keys except the last key of ASOF join", 0) \
\
M(Bool, allow_return_nullable_array, 1, "For array related functions, if true, will return nullable(array)", 0) \

View File

@ -31,6 +31,7 @@
#include <IO/Operators.h>
#include <boost/algorithm/string/predicate.hpp>
#include <regex>
#include <cctz/time_zone.h>
namespace DB
@ -327,6 +328,24 @@ String SettingFieldEnumHelpers::readBinary(ReadBuffer & in)
return str;
}
void SettingFieldTimezone::writeBinary(WriteBuffer & out) const
{
writeStringBinary(value, out);
}
void SettingFieldTimezone::readBinary(ReadBuffer & in)
{
String str;
readStringBinary(str, in);
*this = std::move(str);
}
void SettingFieldTimezone::validateTimezone(const std::string & tz_str)
{
cctz::time_zone validated_tz;
if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz))
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str);
}
String SettingFieldCustom::toString() const
{

View File

@ -462,6 +462,37 @@ void SettingFieldMultiEnum<EnumT, Traits>::readBinary(ReadBuffer & in)
return std::initializer_list<std::pair<const char*, NEW_NAME>> __VA_ARGS__ .size();\
}
/// Setting field for specifying user-defined timezone. It is basically a string, but it needs validation.
struct SettingFieldTimezone
{
String value;
bool changed = false;
explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; }
explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; }
explicit SettingFieldTimezone(String && str) { validateTimezone(str); value = std::move(str); }
explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; }
explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet<const String &>(); validateTimezone(str); value = str; }
SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; }
SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; }
SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; }
SettingFieldTimezone & operator =(const char * str) { *this = std::string_view{str}; return *this; }
SettingFieldTimezone & operator =(const Field & f) { *this = f.safeGet<const String &>(); return *this; }
operator const String &() const { return value; } /// NOLINT
explicit operator Field() const { return value; }
const String & toString() const { return value; }
void parseFromString(const String & str) { *this = str; }
void writeBinary(WriteBuffer & out) const;
void readBinary(ReadBuffer & in);
private:
void validateTimezone(const std::string & tz_str);
};
/// Can keep a value of any type. Used for user-defined settings.
struct SettingFieldCustom
{

View File

@ -10,13 +10,12 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
ITTLAlgorithm::ITTLAlgorithm(
const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
ITTLAlgorithm::ITTLAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_)
: description(description_)
, old_ttl_info(old_ttl_info_)
, current_time(current_time_)
, force(force_)
, date_lut(DateLUT::instance())
, date_lut(DateLUT::serverTimezoneInstance())
{
}

View File

@ -270,7 +270,7 @@ namespace
throw Exception{"Type mismatch, expected Timestamp, got type id = " + toString(value.type()) + " for column " + name,
ErrorCodes::TYPE_MISMATCH};
assert_cast<ColumnUInt16 &>(column).getData().push_back(static_cast<UInt16>(DateLUT::instance().toDayNum(
assert_cast<ColumnUInt16 &>(column).getData().push_back(static_cast<UInt16>(DateLUT::serverTimezoneInstance().toDayNum(
static_cast<const Poco::MongoDB::ConcreteElement<Poco::Timestamp> &>(value).value().epochTime())));
break;
}

View File

@ -28,10 +28,7 @@ public:
TypeIndex getTypeId() const override { return TypeIndex::Date32; }
const char * getFamilyName() const override { return family_name; }
Field getDefault() const override
{
return -static_cast<Int64>(DateLUT::instance().getDayNumOffsetEpoch());
}
Field getDefault() const override { return -static_cast<Int64>(DateLUT::serverTimezoneInstance().getDayNumOffsetEpoch()); }
bool canBeUsedAsVersion() const override { return true; }
bool canBeInsideNullable() const override { return true; }

View File

@ -43,7 +43,9 @@ protected:
* all types with different time zones are equivalent and may be used interchangingly.
* Time zone only affects parsing and displaying in text formats.
*
* If time zone is not specified (example: DateTime without parameter), then default time zone is used.
* If time zone is not specified (example: DateTime without parameter),
* then `session_timezone` setting value is used.
* If `session_timezone` is not set (or empty string), server default time zone is used.
* Default time zone is server time zone, if server is doing transformations
* and if client is doing transformations, unless 'use_client_time_zone' setting is passed to client;
* Server time zone is the time zone specified in 'timezone' parameter in configuration file,

View File

@ -33,7 +33,7 @@ void SerializationDate::checkDataOverflow(const FormatSettings & settings)
void SerializationDate::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
writeDateText(DayNum(assert_cast<const ColumnUInt16 &>(column).getData()[row_num]), ostr);
writeDateText(DayNum(assert_cast<const ColumnUInt16 &>(column).getData()[row_num]), ostr, time_zone);
}
void SerializationDate::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
@ -44,7 +44,7 @@ void SerializationDate::deserializeWholeText(IColumn & column, ReadBuffer & istr
void SerializationDate::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
DayNum x;
readDateText(x, istr);
readDateText(x, istr, time_zone);
checkDataOverflow(settings);
assert_cast<ColumnUInt16 &>(column).getData().push_back(x);
}
@ -65,7 +65,7 @@ void SerializationDate::deserializeTextQuoted(IColumn & column, ReadBuffer & ist
{
DayNum x;
assertChar('\'', istr);
readDateText(x, istr);
readDateText(x, istr, time_zone);
assertChar('\'', istr);
checkDataOverflow(settings);
assert_cast<ColumnUInt16 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
@ -82,7 +82,7 @@ void SerializationDate::deserializeTextJSON(IColumn & column, ReadBuffer & istr,
{
DayNum x;
assertChar('"', istr);
readDateText(x, istr);
readDateText(x, istr, time_zone);
assertChar('"', istr);
checkDataOverflow(settings);
assert_cast<ColumnUInt16 &>(column).getData().push_back(x);

View File

@ -1,6 +1,7 @@
#pragma once
#include <DataTypes/Serializations/SerializationNumber.h>
#include <Common/DateLUT.h>
namespace DB
{
@ -8,6 +9,8 @@ namespace DB
class SerializationDate final : public SerializationNumber<UInt16>
{
public:
explicit SerializationDate(const DateLUTImpl & time_zone_ = DateLUT::sessionInstance()): time_zone(time_zone_) {}
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;
@ -19,6 +22,9 @@ public:
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;
protected:
const DateLUTImpl & time_zone;
private:
static void checkDataOverflow(const FormatSettings & settings);
};

View File

@ -47,7 +47,7 @@ void SerializationDate32::checkDataOverflow(const FormatSettings & settings)
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);
writeDateText(ExtendedDayNum(assert_cast<const ColumnInt32 &>(column).getData()[row_num]), ostr, time_zone);
}
void SerializationDate32::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
@ -60,7 +60,7 @@ void SerializationDate32::deserializeWholeText(IColumn & column, ReadBuffer & is
void SerializationDate32::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
ExtendedDayNum x;
readDateText(x, istr);
readDateText(x, istr, time_zone);
checkDataOverflow(settings);
assert_cast<ColumnInt32 &>(column).getData().push_back(x);
}
@ -81,7 +81,7 @@ void SerializationDate32::deserializeTextQuoted(IColumn & column, ReadBuffer & i
{
ExtendedDayNum x;
assertChar('\'', istr);
readDateText(x, istr);
readDateText(x, istr, time_zone);
assertChar('\'', istr);
checkDataOverflow(settings);
assert_cast<ColumnInt32 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
@ -98,7 +98,7 @@ void SerializationDate32::deserializeTextJSON(IColumn & column, ReadBuffer & ist
{
ExtendedDayNum x;
assertChar('"', istr);
readDateText(x, istr);
readDateText(x, istr, time_zone);
assertChar('"', istr);
checkDataOverflow(settings);
assert_cast<ColumnInt32 &>(column).getData().push_back(x);

View File

@ -16,12 +16,15 @@
#pragma once
#include <DataTypes/Serializations/SerializationNumber.h>
#include <Common/DateLUT.h>
namespace DB
{
class SerializationDate32 final : public SerializationNumber<Int32>
{
public:
explicit SerializationDate32(const DateLUTImpl & time_zone_ = DateLUT::sessionInstance()): time_zone(time_zone_) {}
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;
@ -33,6 +36,9 @@ public:
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;
protected:
const DateLUTImpl & time_zone;
private:
static void checkDataOverflow(const FormatSettings & settings);
};

View File

@ -157,7 +157,7 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate()
if (update_time != std::chrono::system_clock::from_time_t(0))
{
time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - configuration.update_lag;
std::string str_time = DateLUT::instance().timeToString(hr_time);
std::string str_time = DateLUT::serverTimezoneInstance().timeToString(hr_time);
update_time = std::chrono::system_clock::now();
return query_builder.composeUpdateQuery(configuration.update_field, str_time);
}

View File

@ -120,7 +120,7 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate()
if (update_time != std::chrono::system_clock::from_time_t(0))
{
time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - configuration.update_lag;
std::string str_time = DateLUT::instance().timeToString(hr_time);
std::string str_time = DateLUT::serverTimezoneInstance().timeToString(hr_time);
update_time = std::chrono::system_clock::now();
return query_builder.composeUpdateQuery(configuration.update_field, str_time);
}

View File

@ -145,7 +145,7 @@ std::string PostgreSQLDictionarySource::getUpdateFieldAndDate()
if (update_time != std::chrono::system_clock::from_time_t(0))
{
time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - configuration.update_lag;
std::string str_time = DateLUT::instance().timeToString(hr_time);
std::string str_time = DateLUT::serverTimezoneInstance().timeToString(hr_time);
update_time = std::chrono::system_clock::now();
return query_builder.composeUpdateQuery(configuration.update_field, str_time);
}

View File

@ -143,7 +143,7 @@ std::string XDBCDictionarySource::getUpdateFieldAndDate()
if (update_time != std::chrono::system_clock::from_time_t(0))
{
time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - configuration.update_lag;
std::string str_time = DateLUT::instance().timeToString(hr_time);
std::string str_time = DateLUT::serverTimezoneInstance().timeToString(hr_time);
update_time = std::chrono::system_clock::now();
return query_builder.composeUpdateQuery(configuration.update_field, str_time);
}

View File

@ -238,12 +238,12 @@ public:
switch (base_type->getTypeId())
{
case TypeIndex::Date: {
const auto & time_zone = DateLUT::instance();
const auto & time_zone = DateLUT::sessionInstance();
executeInternal<ColumnDate, ColumnDateTime64>(base_col, delta_arg, result_col.get(), time_zone, 0);
break;
}
case TypeIndex::Date32: {
const auto & time_zone = DateLUT::instance();
const auto & time_zone = DateLUT::sessionInstance();
executeInternal<ColumnDate32, ColumnDateTime64>(base_col, delta_arg, result_col.get(), time_zone, 0);
break;
}
@ -260,7 +260,7 @@ public:
}
case TypeIndex::Time: {
const auto & t = assert_cast<const DataTypeTime &>(*arguments[0].type);
const auto & time_zone = DateLUT::instance();
const auto & time_zone = DateLUT::sessionInstance();
executeInternal<ColumnTime, ColumnTime>(base_col, delta_arg, result_col.get(), time_zone, t.getScale());
break;
}

View File

@ -211,7 +211,7 @@ public:
}
/// This method is called only if the function has one argument. Therefore, we do not care about the non-local time zone.
const DateLUTImpl & date_lut = DateLUT::instance();
const DateLUTImpl & date_lut = DateLUT::sessionInstance();
if (left.isNull() || right.isNull())
return is_not_monotonic;

View File

@ -180,7 +180,7 @@ public:
}
/// This method is called only if the function has one argument. Therefore, we do not care about the non-local time zone.
const DateLUTImpl * date_lut = &DateLUT::instance();
const DateLUTImpl * date_lut = &DateLUT::sessionInstance();
if (const auto * timezone = dynamic_cast<const TimezoneMixin *>(&type))
date_lut = &timezone->getTimeZone();
if (left.isNull() || right.isNull())

View File

@ -499,9 +499,9 @@ struct ConvertImpl<std::enable_if_t<IsDataTypeDateOrDateTime<ToDataType> && !std
{
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, UInt16 = 0)
{
const auto & time_zone = DateLUT::instance();
const auto & time_zone = DateLUT::sessionInstance();
const auto today = time_zone.toDayNum(time(nullptr));
auto date_time = DateLUT::instance().fromDayNum(today);
auto date_time = DateLUT::sessionInstance().fromDayNum(today);
if constexpr (std::is_same_v<ToDataType, DataTypeDate> || std::is_same_v<ToDataType, DataTypeDate32>)
{
@ -709,7 +709,7 @@ struct ToDate32Transform32Or64Signed
static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone)
{
static const Int32 daynum_min_offset = -static_cast<Int32>(DateLUT::instance().getDayNumOffsetEpoch());
static const Int32 daynum_min_offset = -static_cast<Int32>(time_zone.getDayNumOffsetEpoch());
if (from < daynum_min_offset)
return daynum_min_offset;
return (from < DATE_LUT_MAX_EXTEND_DAY_NUM)
@ -1020,18 +1020,18 @@ struct FormatImpl
template <>
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 * time_zone)
{
writeDateText(DayNum(x), wb);
writeDateText(DayNum(x), wb, *time_zone);
}
};
template <>
struct FormatImpl<DataTypeDate32>
{
static void execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl *)
static void execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl * time_zone)
{
writeDateText(ExtendedDayNum(x), wb);
writeDateText(ExtendedDayNum(x), wb, *time_zone);
}
};
@ -1109,7 +1109,9 @@ struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType,
const auto & column = col_with_type_and_name.column->convertToFullColumnIfConst();
const DateLUTImpl * time_zone = nullptr;
/// For argument of DateTime type, second argument with time zone could be specified.
if constexpr (std::is_same_v<FromDataType, DataTypeDate> || std::is_same_v<FromDataType, DataTypeDate32>)
time_zone = &DateLUT::sessionInstance();
/// For argument of Date or DateTime type, second argument with time zone could be specified.
if constexpr (std::is_same_v<FromDataType, DataTypeDateTime> || std::is_same_v<FromDataType, DataTypeDateTime64>)
time_zone = &extractTimeZoneFromFunctionArguments(arguments, 1, 0);
@ -1264,18 +1266,18 @@ void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTI
}
template <>
inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
{
DayNum tmp(0);
readDateText(tmp, rb);
readDateText(tmp, rb, *time_zone);
x = tmp;
}
template <>
inline void parseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
inline void parseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
{
ExtendedDayNum tmp(0);
readDateText(tmp, rb);
readDateText(tmp, rb, *time_zone);
x = tmp;
}
@ -1322,20 +1324,20 @@ bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateL
}
template <>
inline bool tryParseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
inline bool tryParseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
{
DayNum tmp(0);
if (!tryReadDateText(tmp, rb))
if (!tryReadDateText(tmp, rb, *time_zone))
return false;
x = tmp;
return true;
}
template <>
inline bool tryParseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
inline bool tryParseImpl<DataTypeDate32>(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
{
ExtendedDayNum tmp(0);
if (!tryReadDateText(tmp, rb))
if (!tryReadDateText(tmp, rb, *time_zone))
return false;
// ExtendedDayNum is int32 and DataTypeData32::FieldType is also int32
// coverity[store_truncates_time_t]
@ -1549,7 +1551,7 @@ struct ConvertThroughParsing
const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr;
const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr;
/// For conversion to DateTime type, second argument with time zone could be specified.
/// For conversion to Date or DateTime type, second argument with time zone could be specified.
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime> || to_datetime64)
{
const auto result_type = removeNullable(res_type);
@ -1564,6 +1566,12 @@ struct ConvertThroughParsing
if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort || parsing_mode == ConvertFromStringParsingMode::BestEffortUS)
utc_time_zone = &DateLUT::instance("UTC");
}
else if constexpr (std::is_same_v<ToDataType, DataTypeDate> || std::is_same_v<ToDataType, DataTypeDate32>)
{
// Timezone is more or less dummy when parsing Date/Date32 from string.
local_time_zone = &DateLUT::sessionInstance();
utc_time_zone = &DateLUT::instance("UTC");
}
const IColumn * col_from = arguments[0].column.get();
const ColumnString * col_from_string = checkAndGetColumn<ColumnString>(col_from);
@ -1807,7 +1815,7 @@ struct ConvertThroughParsing
{
if constexpr (std::is_same_v<ToDataType, DataTypeDate32>)
{
vec_to[i] = -static_cast<Int32>(DateLUT::instance().getDayNumOffsetEpoch());
vec_to[i] = -static_cast<Int32>(DateLUT::sessionInstance().getDayNumOffsetEpoch());
}
else
{
@ -2230,7 +2238,7 @@ public:
|| std::is_same_v<Name, NameToUnixTimestamp>
// toDate(value[, timezone : String])
|| std::is_same_v<ToDataType, DataTypeDate> // TODO: shall we allow timestamp argument for toDate? DateTime knows nothing about timezones and this argument is ignored below.
// toDate(value[, timezone : String])
// toDate32(value[, timezone : String])
|| std::is_same_v<ToDataType, DataTypeDate32>
// toDateTime(value[, timezone: String])
|| std::is_same_v<ToDataType, DataTypeDateTime>

View File

@ -55,7 +55,7 @@ public:
const IFunction::Monotonicity is_not_monotonic;
/// This method is called only if the function has one argument. Therefore, we do not care about the non-local time zone.
const DateLUTImpl & date_lut = DateLUT::instance();
const DateLUTImpl & date_lut = DateLUT::sessionInstance();
if (left.isNull() || right.isNull())
return {};

View File

@ -63,7 +63,7 @@ public:
const IFunction::Monotonicity is_monotonic{.is_monotonic = true, .is_positive = true, .is_always_monotonic = false};
const IFunction::Monotonicity is_not_monotonic;
const DateLUTImpl * date_lut = &DateLUT::instance();
const DateLUTImpl * date_lut = &DateLUT::sessionInstance();
if (const auto * timezone = dynamic_cast<const TimezoneMixin *>(&type))
date_lut = &timezone->getTimeZone();

View File

@ -130,7 +130,7 @@ public:
}
DateTime64 dt64 = DB::nowSubsecondDt64(scale);
ToTimeTransform transformer(scale, scale);
Decimal64::NativeType t = transformer.execute(dt64, intExp10(scale), DateLUT::instance());
Decimal64::NativeType t = transformer.execute(dt64, intExp10(scale), DateLUT::sessionInstance());
return std::make_unique<FunctionBaseCurrentTime>(t, scale, std::make_shared<DataTypeTime>(scale));
}
};

View File

@ -444,7 +444,7 @@ public:
*/
// Note: It is impossible to take the civil-time diff of 2 different timezones to mysql's timestampdiff.
// Because, mysql will convert both absolute times to the same timezone given by the session time_zone variable.
const DateLUTImpl & date_lut = DateLUT::instance();
const DateLUTImpl & date_lut = DateLUT::sessionInstance();
bool should_swap = seconds_x > seconds_y;
struct DateTimeComponents {
const DateLUTImpl::Values & values;
@ -803,13 +803,13 @@ public:
{
auto res = ColumnTime::create(rows, 0);
impl.dispatchForColumns<Second>(x, y, DateLUT::instance(), DateLUT::instance(), res->getData());
impl.dispatchForColumns<Second>(x, y, DateLUT::sessionInstance(), DateLUT::sessionInstance(), res->getData());
return res;
} else
{
auto res = ColumnInt64::create(rows);
impl.dispatchForColumns<Second>(x, y, DateLUT::instance(), DateLUT::instance(), res->getData());
impl.dispatchForColumns<Second>(x, y, DateLUT::sessionInstance(), DateLUT::sessionInstance(), res->getData());
return res;
}
}

View File

@ -146,7 +146,7 @@ public:
if (std::is_same_v<DataType, DataTypeDateTime64> || std::is_same_v<DataType, DataTypeDateTime>)
time_zone_tmp = &extractTimeZoneFromFunctionArguments(arguments, 2, 1);
else
time_zone_tmp = &DateLUT::instance();
time_zone_tmp = &DateLUT::sessionInstance();
const auto & times_data = times->getData();
const DateLUTImpl & time_zone = *time_zone_tmp;

View File

@ -66,7 +66,7 @@ const DateLUTImpl & extractTimeZoneFromFunctionArguments(const ColumnsWithTypeAn
else
{
if (arguments.size() <= datetime_arg_num)
return DateLUT::instance();
return DateLUT::sessionInstance();
const auto & dt_arg = arguments[datetime_arg_num].type.get();
/// If time zone is attached to an argument of type DateTime.
@ -75,7 +75,7 @@ const DateLUTImpl & extractTimeZoneFromFunctionArguments(const ColumnsWithTypeAn
if (const auto * type = checkAndGetDataType<DataTypeDateTime64>(dt_arg))
return type->getTimeZone();
return DateLUT::instance();
return DateLUT::sessionInstance();
}
}

View File

@ -1320,7 +1320,7 @@ namespace
else if (std::is_same_v<DataType, DataTypeDateTime64> || std::is_same_v<DataType, DataTypeDateTime>)
time_zone_tmp = &extractTimeZoneFromFunctionArguments(arguments, 2, 0);
else
time_zone_tmp = &DateLUT::instance();
time_zone_tmp = &DateLUT::sessionInstance();
const DateLUTImpl & time_zone = *time_zone_tmp;
const auto & vec = times->getData();

View File

@ -109,7 +109,7 @@ namespace
{
const auto * col_from = checkAndGetColumn<ColVecType>(column);
static const Int32 daynum_min_offset = -static_cast<Int32>(DateLUT::instance().getDayNumOffsetEpoch());
static const Int32 daynum_min_offset = -static_cast<Int32>(DateLUT::sessionInstance().getDayNumOffsetEpoch());
MutableColumnPtr res = DataTypeDate32().createColumn();
auto & res_data = dynamic_cast<ColumnVector<Int32> *>(res.get())->getData();
@ -239,7 +239,7 @@ namespace
if (col == nullptr)
throw Exception("Column type does not match to the data type", ErrorCodes::ILLEGAL_COLUMN);
const auto & timezone = DateLUT::instance();
const auto & timezone = DateLUT::sessionInstance();
auto & data = col->getData();
const auto row_size = data.size();
res_data.resize(row_size);

View File

@ -131,7 +131,7 @@ namespace
auto res_column = Traits::ReturnDataType::ColumnType::create(input_rows_count);
auto & result_data = res_column->getData();
const auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::sessionInstance();
const Int32 max_days_since_epoch = date_lut.makeDayNum(Traits::MAX_DATE[0], Traits::MAX_DATE[1], Traits::MAX_DATE[2]);
if (is_year_month_day_variant)
@ -573,7 +573,7 @@ namespace
const auto & minute_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[1]).getData();
const auto & second_data = typeid_cast<const ColumnFloat32 &>(*converted_arguments[2]).getData();
const auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::sessionInstance();
const auto max_fraction = pow(10, precision) - 1;
const auto min_time = minTime(date_lut);
const auto max_time = maxTime(date_lut);

View File

@ -1970,7 +1970,7 @@ namespace
const DateLUTImpl & getTimeZone(const ColumnsWithTypeAndName & arguments) const
{
if (arguments.size() < 3)
return DateLUT::instance();
return DateLUT::sessionInstance();
const auto * col = checkAndGetColumnConst<ColumnString>(arguments[2].column.get());
if (!col)

View File

@ -59,16 +59,26 @@ namespace
explicit FunctionTcpPort(ContextPtr context) : FunctionConstantBase(context->getTCPPort(), context->isDistributed()) {}
};
/// Returns the server time zone.
/// Returns timezone for current session.
class FunctionTimezone : public FunctionConstantBase<FunctionTimezone, String, DataTypeString>
{
public:
static constexpr auto name = "timezone";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionTimezone>(context); }
explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {}
explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(DateLUT::sessionInstance().getTimeZone(), context->isDistributed()) {}
};
/// Returns the server time zone (timezone in which server runs).
class FunctionServerTimezone : public FunctionConstantBase<FunctionServerTimezone, String, DataTypeString>
{
public:
static constexpr auto name = "serverTimezone";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionServerTimezone>(context); }
explicit FunctionServerTimezone(ContextPtr context)
: FunctionConstantBase(DateLUT::serverTimezoneInstance().getTimeZone(), context->isDistributed())
{
}
};
/// Returns server uptime in seconds.
class FunctionUptime : public FunctionConstantBase<FunctionUptime, UInt32, DataTypeUInt32>
@ -146,6 +156,12 @@ REGISTER_FUNCTION(Timezone)
factory.registerAlias("timeZone", "timezone");
}
REGISTER_FUNCTION(ServerTimezone)
{
factory.registerFunction<FunctionServerTimezone>();
factory.registerAlias("serverTimeZone", "serverTimezone");
}
REGISTER_FUNCTION(Uptime)
{
factory.registerFunction<FunctionUptime>();

View File

@ -60,7 +60,7 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
const DateLUTImpl * local_time_zone = &DateLUT::instance();
const DateLUTImpl * local_time_zone = &DateLUT::sessionInstance();
auto col_result = ColumnDateTime64::create(input_rows_count, DATETIME_SCALE);
ColumnDateTime64::Container & vec_result = col_result->getData();

View File

@ -41,7 +41,7 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
{
return DataTypeString().createColumnConst(input_rows_count, DateLUT::instance().getTimeZone());
return DataTypeString().createColumnConst(input_rows_count, DateLUT::sessionInstance().getTimeZone());
}
};

View File

@ -101,7 +101,7 @@ public:
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName &, const DataTypePtr &) const override
{
return std::make_unique<FunctionBaseToday>(DayNum(DateLUT::instance().toDayNum(time(nullptr)).toUnderType()));
return std::make_unique<FunctionBaseToday>(DayNum(DateLUT::sessionInstance().toDayNum(time(nullptr)).toUnderType()));
}
};

View File

@ -78,7 +78,7 @@ public:
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName &, const DataTypePtr &) const override
{
auto day_num = DateLUT::instance().toDayNum(time(nullptr)) - 1;
auto day_num = DateLUT::sessionInstance().toDayNum(time(nullptr)) - 1;
return std::make_unique<FunctionBaseYesterday>(static_cast<DayNum>(day_num));
}
};

View File

@ -738,7 +738,7 @@ inline void convertToDayNum(DayNum & date, ExtendedDayNum & from)
}
template <typename ReturnType = void>
inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf)
inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
@ -749,13 +749,13 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf)
else if (!readDateTextImpl<ReturnType>(local_date, buf))
return false;
ExtendedDayNum ret = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day());
convertToDayNum(date,ret);
ExtendedDayNum ret = date_lut.makeDayNum(local_date.year(), local_date.month(), local_date.day());
convertToDayNum(date, ret);
return ReturnType(true);
}
template <typename ReturnType = void>
inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf)
inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
@ -765,8 +765,8 @@ inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf)
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(), -static_cast<Int32>(DateLUT::instance().getDayNumOffsetEpoch()));
/// When the parameter is out of rule or out of range, Date32 uses 1925-01-01 as the default value (-DateLUT::serverTimezoneInstance().getDayNumOffsetEpoch(), -16436) and Date uses 1970-01-01.
date = date_lut.makeDayNum(local_date.year(), local_date.month(), local_date.day(), -static_cast<Int32>(date_lut.getDayNumOffsetEpoch()));
return ReturnType(true);
}
@ -776,14 +776,14 @@ inline void readDateText(LocalDate & date, ReadBuffer & buf)
readDateTextImpl<void>(date, buf);
}
inline void readDateText(DayNum & date, ReadBuffer & buf)
inline void readDateText(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::serverTimezoneInstance())
{
readDateTextImpl<void>(date, buf);
readDateTextImpl<void>(date, buf, date_lut);
}
inline void readDateText(ExtendedDayNum & date, ReadBuffer & buf)
inline void readDateText(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::serverTimezoneInstance())
{
readDateTextImpl<void>(date, buf);
readDateTextImpl<void>(date, buf, date_lut);
}
inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf)
@ -791,14 +791,14 @@ inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf)
return readDateTextImpl<bool>(date, buf);
}
inline bool tryReadDateText(DayNum & date, ReadBuffer & buf)
inline bool tryReadDateText(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::serverTimezoneInstance())
{
return readDateTextImpl<bool>(date, buf);
return readDateTextImpl<bool>(date, buf, date_lut);
}
inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf)
inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::serverTimezoneInstance())
{
return readDateTextImpl<bool>(date, buf);
return readDateTextImpl<bool>(date, buf, date_lut);
}
template <typename ReturnType = void>
@ -1277,12 +1277,13 @@ inline ReturnType readTimeTextImpl(Decimal64 & time, UInt32 scale, ReadBuffer &
return ReturnType(true);
}
inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance())
inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance())
{
readDateTimeTextImpl<void>(datetime, buf, time_zone);
}
inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
inline void readDateTime64Text(
DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::serverTimezoneInstance())
{
readDateTimeTextImpl<void>(datetime64, scale, buf, date_lut);
}
@ -1297,12 +1298,13 @@ inline bool tryReadTimeText(Decimal64 & time, UInt32 scale, ReadBuffer & buf)
return readTimeTextImpl<bool>(time, scale, buf);
}
inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance())
inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance())
{
return readDateTimeTextImpl<bool>(datetime, buf, time_zone);
}
inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
inline bool tryReadDateTime64Text(
DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::serverTimezoneInstance())
{
return readDateTimeTextImpl<bool>(datetime64, scale, buf, date_lut);
}
@ -1505,8 +1507,14 @@ tryReadText(T & x, ReadBuffer & buf) { return tryReadFloatText(x, buf); }
inline void readText(bool & x, ReadBuffer & buf) { readBoolText(x, buf); }
inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf); }
inline void readText(DayNum & x, ReadBuffer & buf) { readDateText(x, buf); }
inline void readText(ExtendedDayNum & x, ReadBuffer & buf) { readDateText(x, buf); }
inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance())
{
readDateText(x, buf, time_zone);
}
inline void readText(ExtendedDayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance())
{
readDateText(x, buf, time_zone);
}
inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); }
inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); }
inline void readText(UUID & x, ReadBuffer & buf) { readUUIDText(x, buf); }

View File

@ -757,15 +757,15 @@ inline void writeDateText(const LocalDate & date, WriteBuffer & buf)
}
template <char delimiter = '-'>
inline void writeDateText(DayNum date, WriteBuffer & buf)
inline void writeDateText(DayNum date, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance())
{
writeDateText<delimiter>(LocalDate(date), buf);
writeDateText<delimiter>(LocalDate(date, time_zone), buf);
}
template <char delimiter = '-'>
inline void writeDateText(ExtendedDayNum date, WriteBuffer & buf)
inline void writeDateText(ExtendedDayNum date, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance())
{
writeDateText<delimiter>(LocalDate(date), buf);
writeDateText<delimiter>(LocalDate(date, time_zone), buf);
}
/// In the format YYYY-MM-DD HH:MM:SS
@ -818,14 +818,19 @@ inline void writeDateTimeText(const LocalDateTime & datetime, WriteBuffer & buf)
/// In the format YYYY-MM-DD HH:MM:SS, according to the specified time zone.
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())
inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance())
{
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 & time_zone = DateLUT::instance())
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 & time_zone = DateLUT::serverTimezoneInstance())
{
static constexpr UInt32 MaxScale = DecimalUtils::max_precision<DateTime64>;
scale = scale > MaxScale ? MaxScale : scale;
@ -892,7 +897,7 @@ inline void writeTimeText(Decimal64 time, UInt32 scale, WriteBuffer & buf)
/// In the RFC 1123 format: "Tue, 03 Dec 2019 00:11:50 GMT". You must provide GMT DateLUT.
/// This is needed for HTTP requests.
inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance())
inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance())
{
const auto & values = time_zone.getValues(datetime);
@ -996,8 +1001,10 @@ template <> inline void writeText<bool>(const bool & x, WriteBuffer & buf) { wri
/// assumes here that `x` is a null-terminated string.
inline void writeText(const char * x, WriteBuffer & buf) { writeCString(x, buf); }
inline void writeText(const char * x, size_t size, WriteBuffer & buf) { writeString(x, size, buf); }
inline void writeText(const DayNum & x, WriteBuffer & buf) { writeDateText(LocalDate(x), buf); }
inline void writeText(const DayNum & x, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::serverTimezoneInstance())
{
writeDateText(LocalDate(x, time_zone), buf);
}
inline void writeText(const LocalDate & x, WriteBuffer & buf) { writeDateText(x, buf); }
inline void writeText(const LocalDateTime & x, WriteBuffer & buf) { writeDateTimeText(x, buf); }
inline void writeText(const UUID & x, WriteBuffer & buf) { writeUUIDText(x, buf); }

View File

@ -12,7 +12,7 @@ using namespace DB;
int main(int, char **)
try
{
const DateLUTImpl & local_time_zone = DateLUT::instance();
const DateLUTImpl & local_time_zone = DateLUT::serverTimezoneInstance();
const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC");
ReadBufferFromFileDescriptor in(STDIN_FILENO);

View File

@ -44,7 +44,7 @@ void AsynchronousMetricLog::addValues(const AsynchronousMetricValues & values)
const auto now = std::chrono::system_clock::now();
element.event_time = time_in_seconds(now);
element.event_time_microseconds = time_in_microseconds(now);
element.event_date = DateLUT::instance().toDayNum(element.event_time);
element.event_date = DateLUT::serverTimezoneInstance().toDayNum(element.event_time);
for (const auto & [key, value] : values)
{

View File

@ -5961,6 +5961,11 @@ Context::HybridPartAllocator Context::getHybridPartAllocationAlgo() const
}
}
bool Context::hasSessionTimeZone() const
{
return !settings.session_timezone.value.empty();
}
void Context::createPlanNodeIdAllocator(int max_id)
{
id_allocator = std::make_shared<PlanNodeIdAllocator>(max_id);

View File

@ -1711,6 +1711,10 @@ public:
};
HybridPartAllocator getHybridPartAllocationAlgo() const;
// If session timezone is specified, some cache which involves creating table/storage can't be used.
// Because it may use wrong timezone for DateTime column, which leads to incorrect result.
bool hasSessionTimeZone() const;
String getDefaultCnchPolicyName() const;
String getCnchAuxilityPolicyName() const;

View File

@ -40,7 +40,7 @@ void CrashLogElement::appendToBlock(MutableColumns & columns) const
{
size_t i = 0;
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(timestamp_ns);
columns[i++]->insert(signal);

View File

@ -1216,7 +1216,7 @@ DatabasePtr DatabaseCatalog::tryGetDatabaseCnch(const String & database_name, Co
return res;
res = getDatabaseFromCnchOrHiveCatalog(
database_name,
getContext(),
local_context,
txn ? txn->getStartTime() : TxnTimestamp::maxTS(),
local_context->getSettingsRef().enable_three_part_identifier);
if (res && txn)

View File

@ -67,7 +67,7 @@ void KafkaLogElement::appendToBlock(MutableColumns & columns) const
size_t i = 0;
columns[i++]->insert(UInt64(event_type));
columns[i++]->insert(UInt64(DateLUT::instance().toDayNum(event_time)));
columns[i++]->insert(UInt64(DateLUT::serverTimezoneInstance().toDayNum(event_time)));
columns[i++]->insert(UInt64(event_time));
columns[i++]->insert(UInt64(duration_ms));

View File

@ -62,7 +62,7 @@ void MaterializedMySQLLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(nameset_2_array(tables));
columns[i++]->insert(type);
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(resync_table);

View File

@ -42,7 +42,7 @@ void MetricLogElement::appendToBlock(MutableColumns & columns) const
{
size_t column_idx = 0;
columns[column_idx++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[column_idx++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[column_idx++]->insert(event_time);
columns[column_idx++]->insert(event_time_microseconds);
columns[column_idx++]->insert(milliseconds);

View File

@ -56,7 +56,7 @@ void MutationLogElement::appendToBlock(MutableColumns & columns) const
size_t i = 0;
columns[i++]->insert(UInt64(event_type));
columns[i++]->insert(UInt64(DateLUT::instance().toDayNum(event_time)));
columns[i++]->insert(UInt64(DateLUT::serverTimezoneInstance().toDayNum(event_time)));
columns[i++]->insert(UInt64(event_time));
columns[i++]->insert(database_name);

View File

@ -58,7 +58,7 @@ void OpenTelemetrySpanLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(operation_name);
columns[i++]->insert(start_time_us);
columns[i++]->insert(finish_time_us);
columns[i++]->insert(DateLUT::instance().toDayNum(finish_time_us / 1000000).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(finish_time_us / 1000000).toUnderType());
// The user might add some ints values, and we will have Int Field, and the
// insert will fail because the column requires Strings. Convert the fields
// here, because it's hard to remember to convert them in all other places.

View File

@ -78,7 +78,7 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(query_id);
columns[i++]->insert(event_type);
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(start_time);
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);

View File

@ -64,7 +64,7 @@ void PartMergeLogElement::appendToBlock(MutableColumns & columns) const
size_t i = 0;
columns[i++]->insert(UInt64(event_type));
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(UInt64(event_time));
columns[i++]->insert(database);

View File

@ -60,7 +60,7 @@ void ProcessorProfileLogElement::appendToBlock(MutableColumns & columns) const
{
size_t i = 0;
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);

View File

@ -85,7 +85,7 @@ void QueryExchangeLogElement::appendToBlock(MutableColumns & columns) const
{
size_t i = 0;
columns[i++]->insert(initial_query_id);
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insertData(type.data(), type.size());
columns[i++]->insert(exchange_id);

View File

@ -174,7 +174,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const
size_t i = 0;
columns[i++]->insert(type);
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(query_start_time);

View File

@ -109,7 +109,7 @@ void QueryThreadLogElement::appendToBlock(MutableColumns & columns) const
{
size_t i = 0;
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(query_start_time);

View File

@ -35,7 +35,7 @@ void RemoteReadLogElement::appendToBlock(MutableColumns & columns) const
{
size_t i = 0;
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::sessionInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(request_time_microseconds);
columns[i++]->insert(context);

View File

@ -82,7 +82,7 @@ void ServerPartLogElement::appendToBlock(MutableColumns & columns) const
size_t i = 0;
columns[i++]->insert(static_cast<UInt64>(event_type));
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(txn_id);

View File

@ -55,7 +55,7 @@ void TextLogElement::appendToBlock(MutableColumns & columns) const
{
size_t i = 0;
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(microseconds);

View File

@ -42,7 +42,7 @@ void TraceLogElement::appendToBlock(MutableColumns & columns) const
{
size_t i = 0;
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(timestamp_ns);

View File

@ -44,7 +44,7 @@ void UniqueTableLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(table);
columns[i++]->insert(type);
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::sessionInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(txn_id);

View File

@ -172,7 +172,7 @@ void ZooKeeperLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(type);
auto event_time_seconds = event_time / 1000000;
columns[i++]->insert(DateLUT::instance().toDayNum(event_time_seconds).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time_seconds).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insertData(IPv6ToBinary(address.host()).data(), 16);
columns[i++]->insert(address.port());

View File

@ -2142,7 +2142,7 @@ void executeQuery(
if (set_result_details)
set_result_details(
context->getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone(), streams.coordinator);
context->getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::serverTimezoneInstance().getTimeZone(), streams.coordinator);
copyData(
*streams.in, *out, []() { return false; }, [&out](const Block &) { out->flush(); });
@ -2199,7 +2199,7 @@ void executeQuery(
if (set_result_details)
set_result_details(
context->getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone(), streams.coordinator);
context->getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::serverTimezoneInstance().getTimeZone(), streams.coordinator);
pipeline.setOutputFormat(std::move(out));
}
@ -2346,7 +2346,7 @@ void executeHttpQueryInAsyncMode(
query.data(), query.data() + query.size(), ast, context, false, QueryProcessingStage::Complete, has_query_tail, istr);
auto & pipeline = streams.pipeline;
if (set_result_details_cp)
set_result_details_cp(query_id, "text/plain; charset=UTF-8", format_name1_cp, DateLUT::instance().getTimeZone(), streams.coordinator);
set_result_details_cp(query_id, "text/plain; charset=UTF-8", format_name1_cp, DateLUT::serverTimezoneInstance().getTimeZone(), streams.coordinator);
if (streams.in)
{
const auto * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());

View File

@ -2112,7 +2112,7 @@ void MergeTreeMetaBase::filterPartitionByTTL(std::vector<std::shared_ptr<MergeTr
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(column))
{
const auto & date_lut = DateLUT::instance();
const auto & date_lut = DateLUT::serverTimezoneInstance();
for (size_t index = 0; index < column->size(); index++)
{
auto ttl_value = date_lut.fromDayNum(DayNum(column_date->getElement(index)));
@ -2137,7 +2137,7 @@ void MergeTreeMetaBase::filterPartitionByTTL(std::vector<std::shared_ptr<MergeTr
// time_t ttl_value = 0;
// if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(column))
// {
// const auto & date_lut = DateLUT::instance();
// const auto & date_lut = DateLUT::serverTimezoneInstance();
// ttl_value = date_lut.fromDayNum(DayNum(column_date->getElement(index)));
// }
// else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))

View File

@ -184,8 +184,9 @@ void BrpcRemoteBroadcastReceiver::pushReceiveQueue(MultiPathDataPacket packet)
return;
}
throw Exception(
"Push exchange data to receiver for " + getName() + " timeout from " + DateLUT::instance().timeToString(context->getClientInfo().initial_query_start_time) +
" to " + DateLUT::instance().timeToString(context->getQueryExpirationTimeStamp().tv_sec),
"Push exchange data to receiver for " + getName() + " timeout from "
+ DateLUT::serverTimezoneInstance().timeToString(context->getClientInfo().initial_query_start_time) + " to "
+ DateLUT::serverTimezoneInstance().timeToString(context->getQueryExpirationTimeStamp().tv_sec),
ErrorCodes::DISTRIBUTE_STAGE_QUERY_EXCEPTION);
}
}
@ -196,8 +197,9 @@ RecvDataPacket BrpcRemoteBroadcastReceiver::recv(timespec timeout_ts) noexcept
MultiPathDataPacket data_packet;
if (!queue->tryPopUntil(data_packet, timeout_ts))
{
const auto error_msg = "Try pop receive queue for " + getName() + " timeout, from " +
DateLUT::instance().timeToString(context->getClientInfo().initial_query_start_time) + " to " + DateLUT::instance().timeToString(timeout_ts.tv_sec);
const auto error_msg = "Try pop receive queue for " + getName() + " timeout, from "
+ DateLUT::serverTimezoneInstance().timeToString(context->getClientInfo().initial_query_start_time) + " to "
+ DateLUT::serverTimezoneInstance().timeToString(timeout_ts.tv_sec);
BroadcastStatus current_status = finish(BroadcastStatusCode::RECV_TIMEOUT, error_msg);
return std::move(current_status);
}

View File

@ -85,7 +85,7 @@ RecvDataPacket LocalBroadcastChannel::recv(timespec timeout_ts)
BroadcastStatus current_status = finish(
BroadcastStatusCode::RECV_TIMEOUT,
"Receive from channel " + name + " timeout after ms: " + DateLUT::instance().timeToString(timeout_ts.tv_sec));
"Receive from channel " + name + " timeout after ms: " + DateLUT::serverTimezoneInstance().timeToString(timeout_ts.tv_sec));
if (enable_receiver_metrics)
receiver_metrics.recv_time_ms << s.elapsedMilliseconds();
return current_status;

View File

@ -274,7 +274,8 @@ RecvDataPacket MultiPathReceiver::recv(timespec timeout_ts)
{
bool collector_closed = collector->closed();
String error_msg = "Try pop receive collector for " + name;
error_msg.append(collector_closed ? " interrupted" : " timeout at " + DateLUT::instance().timeToString(timeout_ts.tv_sec));
error_msg.append(
collector_closed ? " interrupted" : " timeout at " + DateLUT::serverTimezoneInstance().timeToString(timeout_ts.tv_sec));
BroadcastStatus current_status
= finish(collector_closed ? BroadcastStatusCode::RECV_UNKNOWN_ERROR : BroadcastStatusCode::RECV_TIMEOUT, error_msg);

View File

@ -145,7 +145,7 @@ static time_t roundTimeToPrecision(const DateLUTImpl & date_lut, time_t time, UI
IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge()
{
const DateLUTImpl & date_lut = DateLUT::instance();
const DateLUTImpl & date_lut = DateLUT::serverTimezoneInstance();
/// Take rows in needed order and put them into `merged_data` until we get `max_block_size` rows.
///

View File

@ -512,6 +512,7 @@ message SendResourcesReq
optional WorkerInfo worker_info = 9;
// can coexist with `create_queries'
repeated CacheableTableDefinition cacheable_create_queries = 10;
optional string session_timezone = 11;
}
message SendResourcesResp

View File

@ -348,6 +348,7 @@ void TCPHandler::runImpl()
/// Send block to the client - input storage structure.
state.input_header = metadata_snapshot->getSampleBlock();
sendData(state.input_header);
sendTimezone();
});
query_context->setInputBlocksReaderCallback([&connection_settings, this](ContextPtr context) -> Block {
@ -1051,6 +1052,19 @@ void TCPHandler::sendExtremes(const Block & extremes)
}
}
void TCPHandler::sendTimezone()
{
if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES)
return;
const String & tz = query_context->getSettingsRef().session_timezone.value;
LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz);
writeVarUInt(Protocol::Server::TimezoneUpdate, *out);
writeStringBinary(tz, *out);
out->next();
}
bool TCPHandler::receiveProxyHeader()
{
if (in->eof())
@ -1261,7 +1275,7 @@ void TCPHandler::sendHello()
writeVarUInt(VERSION_MINOR, *out);
writeVarUInt(DBMS_TCP_PROTOCOL_VERSION, *out);
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE)
writeStringBinary(DateLUT::instance().getTimeZone(), *out);
writeStringBinary(DateLUT::serverTimezoneInstance().getTimeZone(), *out);
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME)
writeStringBinary(server_display_name, *out);
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSION_PATCH)

View File

@ -242,6 +242,7 @@ private:
void sendProfileInfo(const BlockStreamProfileInfo & info);
void sendTotals(const Block & totals);
void sendExtremes(const Block & extremes);
void sendTimezone();
/// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled.
void initBlockInput();

View File

@ -88,7 +88,7 @@ TimePoint nowTimePoint()
ExtendedDayNum convertToDate(DateTime64 time)
{
time_t ts = time.value / DecimalUtils::scaleMultiplier<time_t>(DataTypeDateTime64::default_scale);
auto date = DateLUT::instance().toDayNum(ts);
auto date = DateLUT::serverTimezoneInstance().toDayNum(ts);
return date;
}
@ -109,7 +109,7 @@ std::optional<Time> getTimeFromString(const String & text)
{
return std::nullopt;
}
return DateLUT::instance().toTime(DateLUT::instance().makeDateTime(1970, 1, 1, tmp.tm_hour, tmp.tm_min, 0));
return DateLUT::serverTimezoneInstance().toTime(DateLUT::serverTimezoneInstance().makeDateTime(1970, 1, 1, tmp.tm_hour, tmp.tm_min, 0));
}
@ -151,7 +151,7 @@ std::optional<double> calcPriority(const InternalConfig & cfg, UInt64 total_udi,
String serializeToText(DateTime64 time)
{
WriteBufferFromOwnString buffer;
writeDateTimeText(time, DataTypeDateTime64::default_scale, buffer, DateLUT::instance());
writeDateTimeText(time, DataTypeDateTime64::default_scale, buffer, DateLUT::serverTimezoneInstance());
return buffer.str();
}

View File

@ -100,7 +100,7 @@ AutoStatisticsManager::AutoStatisticsManager(ContextPtr context_)
static String timeToString(Time time)
{
auto & lut = DateLUT::instance();
auto & lut = DateLUT::serverTimezoneInstance();
auto t = lut.toDateTimeComponents(time).time;
return fmt::format("{:02d}:{:02d}", t.hour, t.minute);
}
@ -582,7 +582,7 @@ void AutoStatisticsManager::initialize(ContextMutablePtr context_, const Poco::U
bool AutoStatisticsManager::isNowValidTimeRange()
{
auto current_time = time(nullptr);
auto current_time_point = DateLUT::instance().toTime(current_time);
auto current_time_point = DateLUT::serverTimezoneInstance().toTime(current_time);
auto [begin_time, end_time] = parseTimeWindow(internal_config.collect_window());
if (betweenTime(current_time_point, begin_time, end_time))

View File

@ -247,8 +247,11 @@ void CnchTablePartitionMetricsHelper::recalculateOrSnapshotPartitionsMetrics(
LOG_TRACE(log, "recalculateOrSnapshotPartitionsMetrics {} {}", table_meta_ptr->table, partition->partition_id);
auto task = [this, partition, current_time, table_meta_ptr, force]() {
auto thread_group = CurrentThread::getGroup();
auto task = [this, partition, current_time, table_meta_ptr, force, thread_group]() {
CurrentMetrics::Increment metric_increment(CurrentMetrics::SystemCnchPartsInfoRecalculationTasksSize);
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
/// After actually recalculated, update `metrics_last_update_time`.
if (partition->metrics_ptr->recalculate(current_time, getContext(), force))
{

View File

@ -54,7 +54,7 @@ void ViewRefreshTaskLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(view);
columns[i++]->insert(status);
columns[i++]->insert(refresh_type);
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(DateLUT::serverTimezoneInstance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(partition_map);
columns[i++]->insert(query_duration_ms);

View File

@ -1126,7 +1126,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 = MinMaxIndex(min_date, max_date);
}
@ -2303,7 +2303,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex(ReadBuffer & buf)
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 = MinMaxIndex(min_date, max_date);
}
@ -2496,7 +2496,7 @@ void IMergeTreeDataPart::deserializePartitionAndMinMaxIndex(ReadBuffer & buffer)
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 = MinMaxIndex(min_date, max_date);
}

View File

@ -145,7 +145,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)));
}
@ -158,7 +158,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()))
@ -394,7 +394,7 @@ MergeTreeMetaBase::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(
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);

View File

@ -72,9 +72,13 @@ 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_dt.year(), create_time_dt.month(), create_time_dt.day(),
create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second());
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());
*buf >> "commands: ";
commands.readText(*buf);

Some files were not shown because too many files have changed in this diff Show More