mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Revert "Add QueryTimeMicroseconds, SelectQueryTimeMicroseconds and InsertQuer…"
This commit is contained in:
parent
5cc886b5ba
commit
65c755190e
@ -11,9 +11,6 @@
|
|||||||
M(FailedQuery, "Number of failed queries.") \
|
M(FailedQuery, "Number of failed queries.") \
|
||||||
M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.") \
|
M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.") \
|
||||||
M(FailedInsertQuery, "Same as FailedQuery, but only for INSERT queries.") \
|
M(FailedInsertQuery, "Same as FailedQuery, but only for INSERT queries.") \
|
||||||
M(QueryTimeMicroseconds, "Total time of all queries.") \
|
|
||||||
M(SelectQueryTimeMicroseconds, "Total time of SELECT queries.") \
|
|
||||||
M(InsertQueryTimeMicroseconds, "Total time of INSERT queries.") \
|
|
||||||
M(FileOpen, "Number of files opened.") \
|
M(FileOpen, "Number of files opened.") \
|
||||||
M(Seek, "Number of times the 'lseek' function was called.") \
|
M(Seek, "Number of times the 'lseek' function was called.") \
|
||||||
M(ReadBufferFromFileDescriptorRead, "Number of reads (read/pread) from a file descriptor. Does not include sockets.") \
|
M(ReadBufferFromFileDescriptorRead, "Number of reads (read/pread) from a file descriptor. Does not include sockets.") \
|
||||||
|
@ -51,9 +51,6 @@ namespace ProfileEvents
|
|||||||
extern const Event FailedQuery;
|
extern const Event FailedQuery;
|
||||||
extern const Event FailedInsertQuery;
|
extern const Event FailedInsertQuery;
|
||||||
extern const Event FailedSelectQuery;
|
extern const Event FailedSelectQuery;
|
||||||
extern const Event QueryTimeMicroseconds;
|
|
||||||
extern const Event SelectQueryTimeMicroseconds;
|
|
||||||
extern const Event InsertQueryTimeMicroseconds;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -483,34 +480,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
query_log->add(elem);
|
query_log->add(elem);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Coomon code for finish and exception callbacks
|
|
||||||
auto status_info_to_query_log = [ast](QueryLogElement &element, const QueryStatusInfo &info) mutable
|
|
||||||
{
|
|
||||||
DB::UInt64 query_time = info.elapsed_seconds * 1000000;
|
|
||||||
ProfileEvents::increment(ProfileEvents::QueryTimeMicroseconds, query_time);
|
|
||||||
if (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>())
|
|
||||||
{
|
|
||||||
ProfileEvents::increment(ProfileEvents::SelectQueryTimeMicroseconds, query_time);
|
|
||||||
}
|
|
||||||
else if (ast->as<ASTInsertQuery>())
|
|
||||||
{
|
|
||||||
ProfileEvents::increment(ProfileEvents::InsertQueryTimeMicroseconds, query_time);
|
|
||||||
}
|
|
||||||
|
|
||||||
element.query_duration_ms = info.elapsed_seconds * 1000;
|
|
||||||
|
|
||||||
element.read_rows = info.read_rows;
|
|
||||||
element.read_bytes = info.read_bytes;
|
|
||||||
|
|
||||||
element.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0;
|
|
||||||
|
|
||||||
element.thread_ids = std::move(info.thread_ids);
|
|
||||||
element.profile_counters = std::move(info.profile_counters);
|
|
||||||
};
|
|
||||||
|
|
||||||
/// Also make possible for caller to log successful query finish and exception during execution.
|
/// Also make possible for caller to log successful query finish and exception during execution.
|
||||||
auto finish_callback = [elem, &context, ast, log_queries, log_queries_min_type = settings.log_queries_min_type,
|
auto finish_callback = [elem, &context, log_queries, log_queries_min_type = settings.log_queries_min_type]
|
||||||
status_info_to_query_log]
|
|
||||||
(IBlockInputStream * stream_in, IBlockOutputStream * stream_out, QueryPipeline * query_pipeline) mutable
|
(IBlockInputStream * stream_in, IBlockOutputStream * stream_out, QueryPipeline * query_pipeline) mutable
|
||||||
{
|
{
|
||||||
QueryStatus * process_list_elem = context.getProcessListElement();
|
QueryStatus * process_list_elem = context.getProcessListElement();
|
||||||
@ -528,14 +499,21 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
elem.type = QueryLogElementType::QUERY_FINISH;
|
elem.type = QueryLogElementType::QUERY_FINISH;
|
||||||
|
|
||||||
elem.event_time = time(nullptr);
|
elem.event_time = time(nullptr);
|
||||||
|
elem.query_duration_ms = elapsed_seconds * 1000;
|
||||||
|
|
||||||
status_info_to_query_log(elem, info);
|
elem.read_rows = info.read_rows;
|
||||||
|
elem.read_bytes = info.read_bytes;
|
||||||
|
|
||||||
|
elem.written_rows = info.written_rows;
|
||||||
|
elem.written_bytes = info.written_bytes;
|
||||||
|
|
||||||
auto progress_callback = context.getProgressCallback();
|
auto progress_callback = context.getProgressCallback();
|
||||||
|
|
||||||
if (progress_callback)
|
if (progress_callback)
|
||||||
progress_callback(Progress(WriteProgress(info.written_rows, info.written_bytes)));
|
progress_callback(Progress(WriteProgress(info.written_rows, info.written_bytes)));
|
||||||
|
|
||||||
|
elem.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0;
|
||||||
|
|
||||||
if (stream_in)
|
if (stream_in)
|
||||||
{
|
{
|
||||||
const BlockStreamProfileInfo & stream_in_info = stream_in->getProfileInfo();
|
const BlockStreamProfileInfo & stream_in_info = stream_in->getProfileInfo();
|
||||||
@ -580,8 +558,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
auto exception_callback = [elem, &context, ast, log_queries, log_queries_min_type = settings.log_queries_min_type, quota(quota),
|
auto exception_callback = [elem, &context, ast, log_queries, log_queries_min_type = settings.log_queries_min_type, quota(quota)] () mutable
|
||||||
status_info_to_query_log] () mutable
|
|
||||||
{
|
{
|
||||||
if (quota)
|
if (quota)
|
||||||
quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false);
|
quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false);
|
||||||
@ -602,7 +579,16 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
if (process_list_elem)
|
if (process_list_elem)
|
||||||
{
|
{
|
||||||
QueryStatusInfo info = process_list_elem->getInfo(true, current_settings.log_profile_events, false);
|
QueryStatusInfo info = process_list_elem->getInfo(true, current_settings.log_profile_events, false);
|
||||||
status_info_to_query_log(elem, info);
|
|
||||||
|
elem.query_duration_ms = info.elapsed_seconds * 1000;
|
||||||
|
|
||||||
|
elem.read_rows = info.read_rows;
|
||||||
|
elem.read_bytes = info.read_bytes;
|
||||||
|
|
||||||
|
elem.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0;
|
||||||
|
|
||||||
|
elem.thread_ids = std::move(info.thread_ids);
|
||||||
|
elem.profile_counters = std::move(info.profile_counters);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (current_settings.calculate_text_stack_trace)
|
if (current_settings.calculate_text_stack_trace)
|
||||||
|
@ -1,3 +0,0 @@
|
|||||||
QueryTimeMicroseconds: Ok
|
|
||||||
SelectQueryTimeMicroseconds: Ok
|
|
||||||
InsertQueryTimeMicroseconds: Ok
|
|
@ -1,51 +0,0 @@
|
|||||||
#!/usr/bin/env bash
|
|
||||||
|
|
||||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|
||||||
. $CURDIR/../shell_config.sh
|
|
||||||
|
|
||||||
DATA_BEFORE=`${CLICKHOUSE_CLIENT} --query="SELECT event,value FROM system.events WHERE event IN ('QueryTimeMicroseconds','SelectQueryTimeMicroseconds','InsertQueryTimeMicroseconds') FORMAT CSV"`
|
|
||||||
|
|
||||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test"
|
|
||||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test (k UInt32) ENGINE=MergeTree ORDER BY k"
|
|
||||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO test (k) SELECT sleep(1)"
|
|
||||||
${CLICKHOUSE_CLIENT} --query="SELECT sleep(1)" > /dev/null
|
|
||||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test"
|
|
||||||
|
|
||||||
DATA_AFTER=`${CLICKHOUSE_CLIENT} --query="SELECT event,value FROM system.events WHERE event IN ('QueryTimeMicroseconds','SelectQueryTimeMicroseconds','InsertQueryTimeMicroseconds') FORMAT CSV"`
|
|
||||||
|
|
||||||
declare -A VALUES_BEFORE
|
|
||||||
VALUES_BEFORE=(["\"QueryTimeMicroseconds\""]="0" ["\"SelectQueryTimeMicroseconds\""]="0" ["\"InsertQueryTimeMicroseconds\""]="0")
|
|
||||||
declare -A VALUES_AFTER
|
|
||||||
VALUES_AFTER=(["\"QueryTimeMicroseconds\""]="0" ["\"SelectQueryTimeMicroseconds\""]="0" ["\"InsertQueryTimeMicroseconds\""]="0")
|
|
||||||
|
|
||||||
for RES in ${DATA_BEFORE}
|
|
||||||
do
|
|
||||||
IFS=',' read -ra FIELDS <<< ${RES}
|
|
||||||
VALUES_BEFORE[${FIELDS[0]}]=${FIELDS[1]}
|
|
||||||
done
|
|
||||||
|
|
||||||
for RES in ${DATA_AFTER}
|
|
||||||
do
|
|
||||||
IFS=',' read -ra FIELDS <<< ${RES}
|
|
||||||
VALUES_AFTER[${FIELDS[0]}]=${FIELDS[1]}
|
|
||||||
done
|
|
||||||
|
|
||||||
let QUERY_TIME=${VALUES_AFTER[\"QueryTimeMicroseconds\"]}-${VALUES_BEFORE[\"QueryTimeMicroseconds\"]}
|
|
||||||
let SELECT_QUERY_TIME=${VALUES_AFTER[\"SelectQueryTimeMicroseconds\"]}-${VALUES_BEFORE[\"SelectQueryTimeMicroseconds\"]}
|
|
||||||
let INSERT_QUERY_TIME=${VALUES_AFTER[\"InsertQueryTimeMicroseconds\"]}-${VALUES_BEFORE[\"InsertQueryTimeMicroseconds\"]}
|
|
||||||
if [[ "${QUERY_TIME}" -lt "2000000" ]]; then
|
|
||||||
echo "QueryTimeMicroseconds: Fail (${QUERY_TIME})"
|
|
||||||
else
|
|
||||||
echo "QueryTimeMicroseconds: Ok"
|
|
||||||
fi
|
|
||||||
if [[ "${SELECT_QUERY_TIME}" -lt "1000000" ]]; then
|
|
||||||
echo "SelectQueryTimeMicroseconds: Fail (${SELECT_QUERY_TIME})"
|
|
||||||
else
|
|
||||||
echo "SelectQueryTimeMicroseconds: Ok"
|
|
||||||
fi
|
|
||||||
if [[ "${INSERT_QUERY_TIME}" -lt "1000000" ]]; then
|
|
||||||
echo "InsertQueryTimeMicroseconds: Fail (${INSERT_QUERY_TIME})"
|
|
||||||
else
|
|
||||||
echo "InsertQueryTimeMicroseconds: Ok"
|
|
||||||
fi
|
|
||||||
|
|
Loading…
Reference in New Issue
Block a user