mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Unify query elapsed time measurements (#43455)
* Unify query elapsed time reporting * add-test: Make shell tests executable * Add some tests around query elapsed time * Style and ubsan
This commit is contained in:
parent
4fce10940e
commit
5de11979ce
@ -1034,7 +1034,13 @@ void ClientBase::onEndOfStream()
|
||||
progress_indication.clearProgressOutput(*tty_buf);
|
||||
|
||||
if (output_format)
|
||||
{
|
||||
/// Do our best to estimate the start of the query so the output format matches the one reported by the server
|
||||
bool is_running = false;
|
||||
output_format->setStartTime(
|
||||
clock_gettime_ns(CLOCK_MONOTONIC) - static_cast<UInt64>(progress_indication.elapsedSeconds() * 1000000000), is_running);
|
||||
output_format->finalize();
|
||||
}
|
||||
|
||||
resetOutput();
|
||||
|
||||
|
@ -40,6 +40,10 @@ public:
|
||||
* Pass CLOCK_MONOTONIC_COARSE, if you need better performance with acceptable cost of several milliseconds of inaccuracy.
|
||||
*/
|
||||
explicit Stopwatch(clockid_t clock_type_ = CLOCK_MONOTONIC) : clock_type(clock_type_) { start(); }
|
||||
explicit Stopwatch(clockid_t clock_type_, UInt64 start_nanoseconds, bool is_running_)
|
||||
: start_ns(start_nanoseconds), clock_type(clock_type_), is_running(is_running_)
|
||||
{
|
||||
}
|
||||
|
||||
void start() { start_ns = nanoseconds(); is_running = true; }
|
||||
void stop() { stop_ns = nanoseconds(); is_running = false; }
|
||||
@ -51,6 +55,8 @@ public:
|
||||
UInt64 elapsedMilliseconds() const { return elapsedNanoseconds() / 1000000UL; }
|
||||
double elapsedSeconds() const { return static_cast<double>(elapsedNanoseconds()) / 1000000000ULL; }
|
||||
|
||||
UInt64 getStart() { return start_ns; }
|
||||
|
||||
private:
|
||||
UInt64 start_ns = 0;
|
||||
UInt64 stop_ns = 0;
|
||||
|
@ -317,6 +317,9 @@ static void addExistingProgressToOutputFormat(OutputFormatPtr format, ContextPtr
|
||||
auto current_progress = element_id->getProgressIn();
|
||||
Progress read_progress{current_progress.read_rows, current_progress.read_bytes, current_progress.total_rows_to_read};
|
||||
format->onProgress(read_progress);
|
||||
|
||||
/// Update the start of the statistics to use the start of the query, and not the creation of the format class
|
||||
format->setStartTime(element_id->getQueryCPUStartTime(), true);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -69,7 +69,8 @@ static bool isUnlimitedQuery(const IAST * ast)
|
||||
}
|
||||
|
||||
|
||||
ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr query_context)
|
||||
ProcessList::EntryPtr
|
||||
ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr query_context, UInt64 watch_start_nanoseconds)
|
||||
{
|
||||
EntryPtr res;
|
||||
|
||||
@ -243,13 +244,16 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as
|
||||
/// since allocation and deallocation could happen in different threads
|
||||
}
|
||||
|
||||
auto process_it = processes.emplace(processes.end(), std::make_shared<QueryStatus>(
|
||||
query_context,
|
||||
query_,
|
||||
client_info,
|
||||
priorities.insert(static_cast<int>(settings.priority)),
|
||||
std::move(thread_group),
|
||||
query_kind));
|
||||
auto process_it = processes.emplace(
|
||||
processes.end(),
|
||||
std::make_shared<QueryStatus>(
|
||||
query_context,
|
||||
query_,
|
||||
client_info,
|
||||
priorities.insert(static_cast<int>(settings.priority)),
|
||||
std::move(thread_group),
|
||||
query_kind,
|
||||
watch_start_nanoseconds));
|
||||
|
||||
increaseQueryKindAmount(query_kind);
|
||||
|
||||
@ -344,11 +348,13 @@ QueryStatus::QueryStatus(
|
||||
const ClientInfo & client_info_,
|
||||
QueryPriorities::Handle && priority_handle_,
|
||||
ThreadGroupStatusPtr && thread_group_,
|
||||
IAST::QueryKind query_kind_)
|
||||
IAST::QueryKind query_kind_,
|
||||
UInt64 watch_start_nanoseconds)
|
||||
: WithContext(context_)
|
||||
, query(query_)
|
||||
, client_info(client_info_)
|
||||
, thread_group(std::move(thread_group_))
|
||||
, watch(CLOCK_MONOTONIC, watch_start_nanoseconds, true)
|
||||
, priority_handle(std::move(priority_handle_))
|
||||
, global_overcommit_tracker(context_->getGlobalOvercommitTracker())
|
||||
, query_kind(query_kind_)
|
||||
@ -522,7 +528,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even
|
||||
|
||||
res.query = query;
|
||||
res.client_info = client_info;
|
||||
res.elapsed_seconds = watch.elapsedSeconds();
|
||||
res.elapsed_microseconds = watch.elapsedMicroseconds();
|
||||
res.is_cancelled = is_killed.load(std::memory_order_relaxed);
|
||||
res.is_all_data_sent = is_all_data_sent.load(std::memory_order_relaxed);
|
||||
res.read_rows = progress_in.read_rows;
|
||||
|
@ -53,7 +53,7 @@ class ProcessListEntry;
|
||||
struct QueryStatusInfo
|
||||
{
|
||||
String query;
|
||||
double elapsed_seconds;
|
||||
UInt64 elapsed_microseconds;
|
||||
size_t read_rows;
|
||||
size_t read_bytes;
|
||||
size_t total_rows;
|
||||
@ -142,15 +142,14 @@ protected:
|
||||
CurrentMetrics::Increment num_queries_increment;
|
||||
|
||||
public:
|
||||
|
||||
QueryStatus(
|
||||
ContextPtr context_,
|
||||
const String & query_,
|
||||
const ClientInfo & client_info_,
|
||||
QueryPriorities::Handle && priority_handle_,
|
||||
ThreadGroupStatusPtr && thread_group_,
|
||||
IAST::QueryKind query_kind_
|
||||
);
|
||||
IAST::QueryKind query_kind_,
|
||||
UInt64 watch_start_nanoseconds);
|
||||
|
||||
~QueryStatus();
|
||||
|
||||
@ -221,6 +220,9 @@ public:
|
||||
bool checkTimeLimit();
|
||||
/// Same as checkTimeLimit but it never throws
|
||||
[[nodiscard]] bool checkTimeLimitSoft();
|
||||
|
||||
/// Get the reference for the start of the query. Used to synchronize with other Stopwatches
|
||||
UInt64 getQueryCPUStartTime() { return watch.getStart(); }
|
||||
};
|
||||
|
||||
using QueryStatusPtr = std::shared_ptr<QueryStatus>;
|
||||
@ -382,7 +384,7 @@ public:
|
||||
* If timeout is passed - throw an exception.
|
||||
* Don't count KILL QUERY queries.
|
||||
*/
|
||||
EntryPtr insert(const String & query_, const IAST * ast, ContextMutablePtr query_context);
|
||||
EntryPtr insert(const String & query_, const IAST * ast, ContextMutablePtr query_context, UInt64 watch_start_nanoseconds);
|
||||
|
||||
/// Number of currently executing queries.
|
||||
size_t size() const { return processes.size(); }
|
||||
|
@ -201,26 +201,32 @@ static void logException(ContextPtr context, QueryLogElement & elem)
|
||||
elem.stack_trace);
|
||||
}
|
||||
|
||||
static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr context, UInt64 current_time_us, ASTPtr ast, const std::shared_ptr<OpenTelemetry::SpanHolder> & query_span)
|
||||
static void onExceptionBeforeStart(
|
||||
const String & query_for_logging,
|
||||
ContextPtr context,
|
||||
ASTPtr ast,
|
||||
const std::shared_ptr<OpenTelemetry::SpanHolder> & query_span,
|
||||
UInt64 elapsed_millliseconds)
|
||||
{
|
||||
auto query_end_time = std::chrono::system_clock::now();
|
||||
|
||||
/// Exception before the query execution.
|
||||
if (auto quota = context->getQuota())
|
||||
quota->used(QuotaType::ERRORS, 1, /* check_exceeded = */ false);
|
||||
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
const auto & client_info = context->getClientInfo();
|
||||
|
||||
/// Log the start of query execution into the table if necessary.
|
||||
QueryLogElement elem;
|
||||
|
||||
elem.type = QueryLogElementType::EXCEPTION_BEFORE_START;
|
||||
|
||||
// all callers to onExceptionBeforeStart method construct the timespec for event_time and
|
||||
// event_time_microseconds from the same time point. So, it can be assumed that both of these
|
||||
// times are equal up to the precision of a second.
|
||||
elem.event_time = current_time_us / 1000000;
|
||||
elem.event_time_microseconds = current_time_us;
|
||||
elem.query_start_time = current_time_us / 1000000;
|
||||
elem.query_start_time_microseconds = current_time_us;
|
||||
elem.event_time = timeInSeconds(query_end_time);
|
||||
elem.event_time_microseconds = timeInMicroseconds(query_end_time);
|
||||
elem.query_start_time = client_info.initial_query_start_time;
|
||||
elem.query_start_time_microseconds = client_info.initial_query_start_time_microseconds;
|
||||
elem.query_duration_ms = elapsed_millliseconds;
|
||||
|
||||
elem.current_database = context->getCurrentDatabase();
|
||||
elem.query = query_for_logging;
|
||||
@ -324,19 +330,32 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
/// we still have enough span logs for the execution of external queries.
|
||||
std::shared_ptr<OpenTelemetry::SpanHolder> query_span = internal ? nullptr : std::make_shared<OpenTelemetry::SpanHolder>("query");
|
||||
|
||||
const auto current_time = std::chrono::system_clock::now();
|
||||
auto query_start_time = std::chrono::system_clock::now();
|
||||
|
||||
/// Used to set the watch in QueryStatus and the output formats. It is not based on query_start_time as that might be based on
|
||||
/// the value passed by the client
|
||||
Stopwatch start_watch{CLOCK_MONOTONIC};
|
||||
|
||||
auto & client_info = context->getClientInfo();
|
||||
|
||||
// If it's not an internal query and we don't see an initial_query_start_time yet, initialize it
|
||||
// to current time. Internal queries are those executed without an independent client context,
|
||||
// thus should not set initial_query_start_time, because it might introduce data race. It's also
|
||||
// possible to have unset initial_query_start_time for non-internal and non-initial queries. For
|
||||
// example, the query is from an initiator that is running an old version of clickhouse.
|
||||
if (!internal && client_info.initial_query_start_time == 0)
|
||||
if (!internal)
|
||||
{
|
||||
client_info.initial_query_start_time = timeInSeconds(current_time);
|
||||
client_info.initial_query_start_time_microseconds = timeInMicroseconds(current_time);
|
||||
// If it's not an internal query and we don't see an initial_query_start_time yet, initialize it
|
||||
// to current time. Internal queries are those executed without an independent client context,
|
||||
// thus should not set initial_query_start_time, because it might introduce data race. It's also
|
||||
// possible to have unset initial_query_start_time for non-internal and non-initial queries. For
|
||||
// example, the query is from an initiator that is running an old version of clickhouse.
|
||||
// On the other hand, if it's initialized then take it as the start of the query
|
||||
if (client_info.initial_query_start_time == 0)
|
||||
{
|
||||
client_info.initial_query_start_time = timeInSeconds(query_start_time);
|
||||
client_info.initial_query_start_time_microseconds = timeInMicroseconds(query_start_time);
|
||||
}
|
||||
else
|
||||
{
|
||||
query_start_time = std::chrono::time_point<std::chrono::system_clock>(
|
||||
std::chrono::microseconds{client_info.initial_query_start_time_microseconds});
|
||||
}
|
||||
}
|
||||
|
||||
assert(internal || CurrentThread::get().getQueryContext());
|
||||
@ -413,7 +432,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
logQuery(query_for_logging, context, internal, stage);
|
||||
|
||||
if (!internal)
|
||||
onExceptionBeforeStart(query_for_logging, context, timeInMicroseconds(current_time), ast, query_span);
|
||||
onExceptionBeforeStart(query_for_logging, context, ast, query_span, start_watch.elapsedMilliseconds());
|
||||
throw;
|
||||
}
|
||||
|
||||
@ -514,7 +533,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
if (!internal && !ast->as<ASTShowProcesslistQuery>())
|
||||
{
|
||||
/// processlist also has query masked now, to avoid secrets leaks though SHOW PROCESSLIST by other users.
|
||||
process_list_entry = context->getProcessList().insert(query_for_logging, ast.get(), context);
|
||||
process_list_entry = context->getProcessList().insert(query_for_logging, ast.get(), context, start_watch.getStart());
|
||||
context->setProcessListElement(process_list_entry->getQueryStatus());
|
||||
}
|
||||
|
||||
@ -720,10 +739,10 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
|
||||
elem.type = QueryLogElementType::QUERY_START; //-V1048
|
||||
|
||||
elem.event_time = timeInSeconds(current_time);
|
||||
elem.event_time_microseconds = timeInMicroseconds(current_time);
|
||||
elem.query_start_time = timeInSeconds(current_time);
|
||||
elem.query_start_time_microseconds = timeInMicroseconds(current_time);
|
||||
elem.event_time = timeInSeconds(query_start_time);
|
||||
elem.event_time_microseconds = timeInMicroseconds(query_start_time);
|
||||
elem.query_start_time = timeInSeconds(query_start_time);
|
||||
elem.query_start_time_microseconds = timeInMicroseconds(query_start_time);
|
||||
|
||||
elem.current_database = context->getCurrentDatabase();
|
||||
elem.query = query_for_logging;
|
||||
@ -772,25 +791,29 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
}
|
||||
|
||||
/// Common code for finish and exception callbacks
|
||||
auto status_info_to_query_log = [](QueryLogElement & element, const QueryStatusInfo & info, const ASTPtr query_ast, const ContextPtr context_ptr) mutable
|
||||
auto status_info_to_query_log
|
||||
= [](QueryLogElement & element, const QueryStatusInfo & info, const ASTPtr query_ast, const ContextPtr context_ptr) mutable
|
||||
{
|
||||
UInt64 query_time = static_cast<UInt64>(info.elapsed_seconds * 1000000);
|
||||
ProfileEvents::increment(ProfileEvents::QueryTimeMicroseconds, query_time);
|
||||
const auto time_now = std::chrono::system_clock::now();
|
||||
UInt64 elapsed_microseconds = info.elapsed_microseconds;
|
||||
element.event_time = timeInSeconds(time_now);
|
||||
element.event_time_microseconds = timeInMicroseconds(time_now);
|
||||
element.query_duration_ms = elapsed_microseconds / 1000;
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::QueryTimeMicroseconds, elapsed_microseconds);
|
||||
if (query_ast->as<ASTSelectQuery>() || query_ast->as<ASTSelectWithUnionQuery>())
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::SelectQueryTimeMicroseconds, query_time);
|
||||
ProfileEvents::increment(ProfileEvents::SelectQueryTimeMicroseconds, elapsed_microseconds);
|
||||
}
|
||||
else if (query_ast->as<ASTInsertQuery>())
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::InsertQueryTimeMicroseconds, query_time);
|
||||
ProfileEvents::increment(ProfileEvents::InsertQueryTimeMicroseconds, elapsed_microseconds);
|
||||
}
|
||||
else
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::OtherQueryTimeMicroseconds, query_time);
|
||||
ProfileEvents::increment(ProfileEvents::OtherQueryTimeMicroseconds, elapsed_microseconds);
|
||||
}
|
||||
|
||||
element.query_duration_ms = static_cast<UInt64>(info.elapsed_seconds * 1000);
|
||||
|
||||
element.read_rows = info.read_rows;
|
||||
element.read_bytes = info.read_bytes;
|
||||
|
||||
@ -844,16 +867,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
CurrentThread::finalizePerformanceCounters();
|
||||
|
||||
QueryStatusInfo info = process_list_elem->getInfo(true, context->getSettingsRef().log_profile_events);
|
||||
|
||||
double elapsed_seconds = info.elapsed_seconds;
|
||||
|
||||
elem.type = QueryLogElementType::QUERY_FINISH;
|
||||
|
||||
// construct event_time and event_time_microseconds using the same time point
|
||||
// so that the two times will always be equal up to a precision of a second.
|
||||
const auto finish_time = std::chrono::system_clock::now();
|
||||
elem.event_time = timeInSeconds(finish_time);
|
||||
elem.event_time_microseconds = timeInMicroseconds(finish_time);
|
||||
status_info_to_query_log(elem, info, ast, context);
|
||||
|
||||
if (pulling_pipeline)
|
||||
@ -877,9 +892,15 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
|
||||
if (elem.read_rows != 0)
|
||||
{
|
||||
LOG_INFO(&Poco::Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.",
|
||||
elem.read_rows, ReadableSize(elem.read_bytes), elapsed_seconds,
|
||||
static_cast<size_t>(elem.read_rows / elapsed_seconds),
|
||||
double elapsed_seconds = static_cast<double>(info.elapsed_microseconds) / 1000000.0;
|
||||
double rows_per_second = static_cast<double>(elem.read_rows) / elapsed_seconds;
|
||||
LOG_INFO(
|
||||
&Poco::Logger::get("executeQuery"),
|
||||
"Read {} rows, {} in {} sec., {} rows/sec., {}/sec.",
|
||||
elem.read_rows,
|
||||
ReadableSize(elem.read_bytes),
|
||||
elapsed_seconds,
|
||||
rows_per_second,
|
||||
ReadableSize(elem.read_bytes / elapsed_seconds));
|
||||
}
|
||||
|
||||
@ -893,8 +914,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
if (auto processors_profile_log = context->getProcessorsProfileLog())
|
||||
{
|
||||
ProcessorProfileLogElement processor_elem;
|
||||
processor_elem.event_time = timeInSeconds(finish_time);
|
||||
processor_elem.event_time_microseconds = timeInMicroseconds(finish_time);
|
||||
processor_elem.event_time = elem.event_time;
|
||||
processor_elem.event_time_microseconds = elem.event_time_microseconds;
|
||||
processor_elem.query_id = elem.client_info.current_query_id;
|
||||
|
||||
auto get_proc_id = [](const IProcessor & proc) -> UInt64
|
||||
@ -969,7 +990,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
}
|
||||
};
|
||||
|
||||
auto exception_callback = [elem,
|
||||
auto exception_callback = [start_watch,
|
||||
elem,
|
||||
context,
|
||||
ast,
|
||||
log_queries,
|
||||
@ -992,14 +1014,6 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
quota->used(QuotaType::ERRORS, 1, /* check_exceeded = */ false);
|
||||
|
||||
elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING;
|
||||
|
||||
// event_time and event_time_microseconds are being constructed from the same time point
|
||||
// to ensure that both the times will be equal up to the precision of a second.
|
||||
const auto time_now = std::chrono::system_clock::now();
|
||||
|
||||
elem.event_time = timeInSeconds(time_now);
|
||||
elem.event_time_microseconds = timeInMicroseconds(time_now);
|
||||
elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time);
|
||||
elem.exception_code = getCurrentExceptionCode();
|
||||
elem.exception = getCurrentExceptionMessage(false);
|
||||
|
||||
@ -1008,12 +1022,19 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
|
||||
/// Update performance counters before logging to query_log
|
||||
CurrentThread::finalizePerformanceCounters();
|
||||
const auto time_now = std::chrono::system_clock::now();
|
||||
elem.event_time = timeInSeconds(time_now);
|
||||
elem.event_time_microseconds = timeInMicroseconds(time_now);
|
||||
|
||||
if (process_list_elem)
|
||||
{
|
||||
QueryStatusInfo info = process_list_elem->getInfo(true, current_settings.log_profile_events, false);
|
||||
status_info_to_query_log(elem, info, ast, context);
|
||||
}
|
||||
else
|
||||
{
|
||||
elem.query_duration_ms = start_watch.elapsedMilliseconds();
|
||||
}
|
||||
|
||||
if (current_settings.calculate_text_stack_trace)
|
||||
setExceptionStackTrace(elem);
|
||||
@ -1063,7 +1084,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
}
|
||||
|
||||
if (!internal)
|
||||
onExceptionBeforeStart(query_for_logging, context, timeInMicroseconds(current_time), ast, query_span);
|
||||
onExceptionBeforeStart(query_for_logging, context, ast, query_span, start_watch.elapsedMilliseconds());
|
||||
|
||||
throw;
|
||||
}
|
||||
|
@ -76,6 +76,15 @@ public:
|
||||
|
||||
void doNotWritePrefix() { need_write_prefix = false; }
|
||||
|
||||
/// Reset the statistics watch to a specific point in time
|
||||
/// If set to not running it will stop on the call (elapsed = now() - given start)
|
||||
void setStartTime(UInt64 start, bool is_running)
|
||||
{
|
||||
statistics.watch = Stopwatch(CLOCK_MONOTONIC, start, true);
|
||||
if (!is_running)
|
||||
statistics.watch.stop();
|
||||
}
|
||||
|
||||
protected:
|
||||
friend class ParallelFormattingOutputFormat;
|
||||
|
||||
@ -132,9 +141,6 @@ protected:
|
||||
Chunk extremes;
|
||||
};
|
||||
|
||||
void setOutsideStatistics(Statistics statistics_) { statistics = std::make_shared<Statistics>(std::move(statistics_)); }
|
||||
std::shared_ptr<Statistics> getOutsideStatistics() const { return statistics; }
|
||||
|
||||
/// In some formats the way we print extremes depends on
|
||||
/// were totals printed or not. In this case in parallel formatting
|
||||
/// we should notify underling format if totals were printed.
|
||||
@ -160,10 +166,10 @@ protected:
|
||||
bool need_write_suffix = true;
|
||||
|
||||
RowsBeforeLimitCounterPtr rows_before_limit_counter;
|
||||
Statistics statistics;
|
||||
|
||||
private:
|
||||
size_t rows_read_before = 0;
|
||||
std::shared_ptr<Statistics> statistics = nullptr;
|
||||
bool are_totals_written = false;
|
||||
|
||||
/// Counters for consumed chunks. Are used for QueryLog.
|
||||
|
@ -77,10 +77,6 @@ void JSONColumnsWithMetadataBlockOutputFormat::consumeTotals(Chunk chunk)
|
||||
|
||||
void JSONColumnsWithMetadataBlockOutputFormat::finalizeImpl()
|
||||
{
|
||||
auto outside_statistics = getOutsideStatistics();
|
||||
if (outside_statistics)
|
||||
statistics = std::move(*outside_statistics);
|
||||
|
||||
JSONUtils::writeAdditionalInfo(
|
||||
rows,
|
||||
statistics.rows_before_limit,
|
||||
|
@ -60,7 +60,6 @@ protected:
|
||||
void writeExtremesElement(const char * title, const Columns & columns, size_t row_num);
|
||||
|
||||
DataTypes types;
|
||||
Statistics statistics;
|
||||
size_t rows;
|
||||
};
|
||||
|
||||
|
@ -112,10 +112,6 @@ void JSONRowOutputFormat::writeAfterExtremes()
|
||||
|
||||
void JSONRowOutputFormat::finalizeImpl()
|
||||
{
|
||||
auto outside_statistics = getOutsideStatistics();
|
||||
if (outside_statistics)
|
||||
statistics = std::move(*outside_statistics);
|
||||
|
||||
JSONUtils::writeAdditionalInfo(
|
||||
row_count,
|
||||
statistics.rows_before_limit,
|
||||
|
@ -66,7 +66,6 @@ protected:
|
||||
size_t row_count = 0;
|
||||
Names names; /// The column names are pre-escaped to be put into JSON string literal.
|
||||
|
||||
Statistics statistics;
|
||||
FormatSettings settings;
|
||||
|
||||
bool yield_strings;
|
||||
|
@ -79,11 +79,14 @@ void MySQLOutputFormat::finalizeImpl()
|
||||
CurrentThread::finalizePerformanceCounters();
|
||||
QueryStatusInfo info = process_list_elem->getInfo();
|
||||
affected_rows = info.written_rows;
|
||||
double elapsed_seconds = static_cast<double>(info.elapsed_microseconds) / 1000000.0;
|
||||
human_readable_info = fmt::format(
|
||||
"Read {} rows, {} in {} sec., {} rows/sec., {}/sec.",
|
||||
info.read_rows, ReadableSize(info.read_bytes), info.elapsed_seconds,
|
||||
static_cast<size_t>(info.read_rows / info.elapsed_seconds),
|
||||
ReadableSize(info.read_bytes / info.elapsed_seconds));
|
||||
info.read_rows,
|
||||
ReadableSize(info.read_bytes),
|
||||
elapsed_seconds,
|
||||
static_cast<size_t>(info.read_rows / elapsed_seconds),
|
||||
ReadableSize(info.read_bytes / elapsed_seconds));
|
||||
}
|
||||
|
||||
const auto & header = getPort(PortKind::Main).getHeader();
|
||||
|
@ -216,7 +216,7 @@ namespace DB
|
||||
}
|
||||
case ProcessingUnitType::FINALIZE:
|
||||
{
|
||||
formatter->setOutsideStatistics(std::move(unit.statistics));
|
||||
formatter->statistics = std::move(unit.statistics);
|
||||
formatter->finalizeImpl();
|
||||
break;
|
||||
}
|
||||
|
@ -227,7 +227,6 @@ private:
|
||||
size_t rows_consumed = 0;
|
||||
std::atomic_bool are_totals_written = false;
|
||||
|
||||
Statistics statistics;
|
||||
/// We change statistics in onProgress() which can be called from different threads.
|
||||
std::mutex statistics_mutex;
|
||||
bool save_totals_and_extremes_in_statistics;
|
||||
|
@ -137,10 +137,6 @@ void TemplateBlockOutputFormat::finalizeImpl()
|
||||
return;
|
||||
|
||||
size_t parts = format.format_idx_to_column_idx.size();
|
||||
auto outside_statistics = getOutsideStatistics();
|
||||
if (outside_statistics)
|
||||
statistics = std::move(*outside_statistics);
|
||||
|
||||
for (size_t i = 0; i < parts; ++i)
|
||||
{
|
||||
auto type = std::make_shared<DataTypeUInt64>();
|
||||
|
@ -57,8 +57,6 @@ private:
|
||||
ParsedTemplateFormatString format;
|
||||
ParsedTemplateFormatString row_format;
|
||||
|
||||
Statistics statistics;
|
||||
|
||||
size_t row_count = 0;
|
||||
|
||||
std::string row_between_delimiter;
|
||||
|
@ -190,9 +190,6 @@ void XMLRowOutputFormat::finalizeImpl()
|
||||
writeIntText(row_count, *ostr);
|
||||
writeCString("</rows>\n", *ostr);
|
||||
|
||||
auto outside_statistics = getOutsideStatistics();
|
||||
if (outside_statistics)
|
||||
statistics = std::move(*outside_statistics);
|
||||
|
||||
writeRowsBeforeLimitAtLeast();
|
||||
|
||||
|
@ -61,7 +61,6 @@ private:
|
||||
NamesAndTypes fields;
|
||||
Names field_tag_names;
|
||||
|
||||
Statistics statistics;
|
||||
const FormatSettings format_settings;
|
||||
};
|
||||
|
||||
|
@ -119,7 +119,7 @@ void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr c
|
||||
res_columns[i++]->insert(process.client_info.quota_key);
|
||||
res_columns[i++]->insert(process.client_info.distributed_depth);
|
||||
|
||||
res_columns[i++]->insert(process.elapsed_seconds);
|
||||
res_columns[i++]->insert(static_cast<double>(process.elapsed_microseconds) / 100000.0);
|
||||
res_columns[i++]->insert(process.is_cancelled);
|
||||
res_columns[i++]->insert(process.is_all_data_sent);
|
||||
res_columns[i++]->insert(process.read_rows);
|
||||
|
14
tests/queries/0_stateless/02483_elapsed_time.reference
Normal file
14
tests/queries/0_stateless/02483_elapsed_time.reference
Normal file
@ -0,0 +1,14 @@
|
||||
1 1
|
||||
Greater (Ok)
|
||||
Greater (Ok)
|
||||
Row 1:
|
||||
──────
|
||||
type: QueryFinish
|
||||
elapsed_more_than_one_second: 1
|
||||
end_minus_start_more_than_a_second: 1
|
||||
|
||||
Row 2:
|
||||
──────
|
||||
type: QueryFinish
|
||||
elapsed_more_than_one_second: 1
|
||||
end_minus_start_more_than_a_second: 1
|
69
tests/queries/0_stateless/02483_elapsed_time.sh
Executable file
69
tests/queries/0_stateless/02483_elapsed_time.sh
Executable file
@ -0,0 +1,69 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
|
||||
# The following query fails during query interpretation so it throws an ExceptionBeforeStart
|
||||
EXCEPTION_BEFORE_START_QUERY="WITH
|
||||
(
|
||||
SELECT sleepEachRow(1)
|
||||
) AS sub
|
||||
SELECT *
|
||||
FROM
|
||||
(
|
||||
SELECT *
|
||||
FROM system.numbers
|
||||
WHERE number IN (sub)
|
||||
)
|
||||
SETTINGS enable_global_with_statement = 0"
|
||||
|
||||
|
||||
# For this query the system.query_log needs to show ExceptionBeforeStart and elapsed seconds >= 1.0
|
||||
QUERY_ID="${CLICKHOUSE_DATABASE}_$(date +%s)_02883_q1"
|
||||
${CLICKHOUSE_CLIENT} -m --query "$EXCEPTION_BEFORE_START_QUERY" --query_id="$QUERY_ID" >/dev/null 2>&1
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT type == 'ExceptionBeforeStart' as expected_type, query_duration_ms >= 1000 as elapsed_more_than_one_second FROM system.query_log WHERE query_id='$QUERY_ID'"
|
||||
|
||||
# Now we test with a query that will take 1+ seconds. The CLI should show that as part of the output format
|
||||
OK_QUERY_JSON="
|
||||
WITH (
|
||||
SELECT sleepEachRow(1.0)
|
||||
) AS sub
|
||||
SELECT *
|
||||
FROM
|
||||
(
|
||||
SELECT *
|
||||
FROM system.one
|
||||
)
|
||||
FORMAT JSON
|
||||
SETTINGS enable_global_with_statement = 1"
|
||||
QUERY_ID_2="${CLICKHOUSE_DATABASE}_$(date +%s)_02883_q2"
|
||||
${CLICKHOUSE_CLIENT} --query "$OK_QUERY_JSON" --query_id="${QUERY_ID_2}" | grep elapsed | awk '{ if($2 >= 1.0) { print "Greater (Ok)" } else { print "Smaller than expected: " $2 } }'
|
||||
|
||||
OK_QUERY_XML="
|
||||
WITH (
|
||||
SELECT sleepEachRow(1.0)
|
||||
) AS sub
|
||||
SELECT *
|
||||
FROM
|
||||
(
|
||||
SELECT *
|
||||
FROM system.one
|
||||
)
|
||||
FORMAT XML
|
||||
SETTINGS enable_global_with_statement = 1"
|
||||
QUERY_ID_3="${CLICKHOUSE_DATABASE}_$(date +%s)_02883_q3"
|
||||
${CLICKHOUSE_CLIENT} --query "$OK_QUERY_XML" --query_id="${QUERY_ID_3}" | grep elapsed | awk -F '[<>]' '{ if($3 >= 1.0) { print "Greater (Ok)" } else { print "Smaller than expected: " $3 } }'
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
|
||||
${CLICKHOUSE_CLIENT} --query "
|
||||
SELECT
|
||||
type,
|
||||
query_duration_ms >= 1000 as elapsed_more_than_one_second,
|
||||
(toDecimal64(event_time_microseconds, 6) - toDecimal64(query_start_time_microseconds, 6)) > 1.0 AS end_minus_start_more_than_a_second
|
||||
FROM system.query_log
|
||||
WHERE type='QueryFinish' AND (query_id='$QUERY_ID_2' OR query_id='${QUERY_ID_3}')
|
||||
FORMAT Vertical"
|
@ -25,4 +25,7 @@ fi
|
||||
|
||||
set -x
|
||||
touch ${TESTS_PATH}/${NEW_TEST_NO}_${FILENAME}.${FILEEXT}
|
||||
if [[ $FILEEXT == "sh" ]] ; then
|
||||
chmod +x ${TESTS_PATH}/${NEW_TEST_NO}_${FILENAME}.${FILEEXT}
|
||||
fi
|
||||
touch ${TESTS_PATH}/${NEW_TEST_NO}_${FILENAME}.reference
|
||||
|
Loading…
Reference in New Issue
Block a user