Skip to content

Commit

Permalink
Unify query elapsed time measurements (ClickHouse#43455)
Browse files Browse the repository at this point in the history
* Unify query elapsed time reporting

* add-test: Make shell tests executable

* Add some tests around query elapsed time

* Style and ubsan
  • Loading branch information
Algunenano authored Dec 28, 2022
1 parent 4fce109 commit 5de1197
Show file tree
Hide file tree
Showing 22 changed files with 218 additions and 100 deletions.
6 changes: 6 additions & 0 deletions src/Client/ClientBase.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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();

Expand Down
6 changes: 6 additions & 0 deletions src/Common/Stopwatch.h
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,10 @@ class Stopwatch
* 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; }
Expand All @@ -51,6 +55,8 @@ class Stopwatch
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;
Expand Down
3 changes: 3 additions & 0 deletions src/Formats/FormatFactory.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
}

Expand Down
26 changes: 16 additions & 10 deletions src/Interpreters/ProcessList.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down Expand Up @@ -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);

Expand Down Expand Up @@ -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_)
Expand Down Expand Up @@ -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;
Expand Down
12 changes: 7 additions & 5 deletions src/Interpreters/ProcessList.h
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -142,15 +142,14 @@ class QueryStatus : public WithContext
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();

Expand Down Expand Up @@ -221,6 +220,9 @@ class QueryStatus : public WithContext
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>;
Expand Down Expand Up @@ -382,7 +384,7 @@ class ProcessList : public ProcessListBase
* 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(); }
Expand Down
Loading

0 comments on commit 5de1197

Please sign in to comment.