mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Hackish way of setting up timezone on the client
Warning: lots of debug logging
This commit is contained in:
parent
a871c36d38
commit
e36addb96a
@ -4,8 +4,10 @@
|
||||
#include <map>
|
||||
#include <iostream>
|
||||
#include <iomanip>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string_view>
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <boost/program_options.hpp>
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
@ -307,7 +309,7 @@ int Client::main(const std::vector<std::string> & /*args*/)
|
||||
try
|
||||
{
|
||||
UseSSL use_ssl;
|
||||
MainThreadStatus::getInstance();
|
||||
auto & thread_status = MainThreadStatus::getInstance();
|
||||
setupSignalHandler();
|
||||
|
||||
std::cout << std::fixed << std::setprecision(3);
|
||||
@ -320,6 +322,14 @@ try
|
||||
processConfig();
|
||||
initTtyBuffer(toProgressOption(config().getString("progress", "default")));
|
||||
|
||||
{
|
||||
// All that just to set DB::CurrentThread::get().getGlobalContext()
|
||||
// which is required for client timezone (pushed as from server) to work.
|
||||
auto thread_group = std::make_shared<ThreadGroupStatus>();
|
||||
thread_group->global_context = global_context;
|
||||
thread_status.attachQuery(thread_group, false);
|
||||
}
|
||||
|
||||
/// Includes delayed_interactive.
|
||||
if (is_interactive)
|
||||
{
|
||||
|
@ -65,6 +65,7 @@
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
|
||||
#include <boost/algorithm/string/case_conv.hpp>
|
||||
#include <Poco/Logger.h>
|
||||
#include <iostream>
|
||||
#include <filesystem>
|
||||
#include <map>
|
||||
@ -73,11 +74,44 @@
|
||||
#include "config_version.h"
|
||||
#include "config.h"
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
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;
|
||||
}
|
||||
|
||||
void LogContextes(const std::string_view scope, const ContextPtr global_context)
|
||||
{
|
||||
const auto * context = global_context.get();
|
||||
std::cerr << scope << " contextes"
|
||||
<< "\n\tglobal: " << reinterpret_cast<const void*>(context)
|
||||
<< "\n\tsession: " << getContextPtrOrNull([&]() { return context ? context->getSessionContext() : nullptr; })
|
||||
<< "\n\tquery: " << getContextPtrOrNull([&]() { return context ? context->getQueryContext() : nullptr; })
|
||||
<< "\n\tcurrent T query: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getQueryContext(); })
|
||||
<< "\n\tcurrent T global: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getGlobalContext(); })
|
||||
// << "\n\tbuffer: " << getContextPtrOrNull(context, &Context::getBufferContext)
|
||||
<< std::endl;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric MemoryTracking;
|
||||
@ -438,7 +472,12 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query)
|
||||
/// output_format, do not output it.
|
||||
/// Also do not output too much data if we're fuzzing.
|
||||
if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100))
|
||||
{
|
||||
LogContextes("ClientBase::onData header", global_context);
|
||||
return;
|
||||
}
|
||||
|
||||
LogContextes("ClientBase::onData DATA block", global_context);
|
||||
|
||||
/// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker.
|
||||
if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout))
|
||||
@ -1048,7 +1087,15 @@ void ClientBase::onProgress(const Progress & value)
|
||||
|
||||
void ClientBase::onTimezoneUpdate(const String & tz)
|
||||
{
|
||||
std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl;
|
||||
DateLUT::setDefaultTimezone(tz);
|
||||
|
||||
Settings settings;
|
||||
settings.timezone = tz;
|
||||
global_context->applySettingsChanges(settings.changes());
|
||||
// DB::CurrentThread::get().getQueryContext()->applySettingsChanges(settings.changes());
|
||||
|
||||
LogContextes("ClientBase::onTimezoneUpdate", global_context);
|
||||
}
|
||||
|
||||
|
||||
|
@ -5,7 +5,11 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Common/DateLUTImpl.h>
|
||||
#include <Core/Field.h>
|
||||
#include "Poco/Logger.h"
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -52,6 +56,15 @@ public:
|
||||
{
|
||||
DataTypePtr type_no_nullable = removeNullable(arguments[0].type);
|
||||
|
||||
{
|
||||
const auto query_context = DB::CurrentThread::get().getQueryContext();
|
||||
|
||||
LOG_DEBUG(&Poco::Logger::get("Function timezoneOf"), "query context: {}, timezone: {} ({})",
|
||||
reinterpret_cast<const void*>(query_context.get()),
|
||||
query_context->getSettingsRef().timezone.toString(),
|
||||
(query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED"));
|
||||
}
|
||||
|
||||
return DataTypeString().createColumnConst(input_rows_count,
|
||||
dynamic_cast<const TimezoneMixin &>(*type_no_nullable).getTimeZone().getTimeZone());
|
||||
}
|
||||
|
@ -118,6 +118,8 @@
|
||||
#include <rocksdb/table.h>
|
||||
#endif
|
||||
|
||||
#include <ranges>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -1543,6 +1545,13 @@ void Context::applySettingChange(const SettingChange & change)
|
||||
void Context::applySettingsChanges(const SettingsChanges & changes)
|
||||
{
|
||||
auto lock = getLock();
|
||||
LOG_DEBUG(shared->log, "Context::applySettingsChanges {} applying settings changes: {}", reinterpret_cast<const void*>(this),
|
||||
fmt::join(std::ranges::transform_view(changes,
|
||||
[](const SettingChange & change)
|
||||
{
|
||||
return change.name + ": " + change.value.dump();
|
||||
}), ", "));
|
||||
|
||||
for (const SettingChange & change : changes)
|
||||
applySettingChange(change);
|
||||
applySettingsQuirks(settings);
|
||||
|
@ -345,6 +345,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([this] (ContextPtr context) -> Block
|
||||
@ -452,9 +453,7 @@ void TCPHandler::runImpl()
|
||||
if (isQueryCancelled())
|
||||
return true;
|
||||
|
||||
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES
|
||||
&& client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE)
|
||||
sendTimezone();
|
||||
// sendTimezone();
|
||||
sendProgress();
|
||||
sendSelectProfileEvents();
|
||||
sendLogs();
|
||||
@ -496,9 +495,7 @@ void TCPHandler::runImpl()
|
||||
|
||||
{
|
||||
std::lock_guard lock(task_callback_mutex);
|
||||
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES
|
||||
&& client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE)
|
||||
sendTimezone();
|
||||
// sendTimezone();
|
||||
sendLogs();
|
||||
sendEndOfStream();
|
||||
}
|
||||
@ -764,7 +761,7 @@ void TCPHandler::processInsertQuery()
|
||||
|
||||
/// Send block to the client - table structure.
|
||||
sendData(executor.getHeader());
|
||||
|
||||
sendTimezone();
|
||||
sendLogs();
|
||||
|
||||
while (readDataNext())
|
||||
@ -809,6 +806,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
|
||||
{
|
||||
std::lock_guard lock(task_callback_mutex);
|
||||
sendData(header);
|
||||
sendTimezone();
|
||||
}
|
||||
}
|
||||
|
||||
@ -1061,7 +1059,16 @@ void TCPHandler::sendInsertProfileEvents()
|
||||
|
||||
void TCPHandler::sendTimezone()
|
||||
{
|
||||
// if (client_tcp_protocol_version <= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES
|
||||
// || client_tcp_protocol_version <= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE)
|
||||
// return;
|
||||
|
||||
// const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString();
|
||||
LOG_DEBUG(log, "TCPHandler::sendTimezone() query context: {}, timezone: {} ({})",
|
||||
reinterpret_cast<const void*>(query_context.get()),
|
||||
query_context->getSettingsRef().timezone.toString(),
|
||||
(query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED"));
|
||||
|
||||
const String & tz = query_context->getSettingsRef().timezone.toString();
|
||||
if (!tz.empty())
|
||||
{
|
||||
|
@ -1,9 +1,11 @@
|
||||
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 = 'Europe/Zurich';
|
||||
SET timezone = 'Asia/Manila';
|
||||
SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk');
|
||||
|
||||
SET timezone = 'Абырвалг';
|
||||
select now(); -- { serverError POCO_EXCEPTION }
|
||||
SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich';
|
||||
SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn';
|
||||
|
Loading…
Reference in New Issue
Block a user