add query_id header in all queries

This commit is contained in:
Anton Popov 2023-02-13 16:44:54 +00:00
parent 2190a2a6c4
commit e9e6d735ef
6 changed files with 53 additions and 11 deletions

View File

@ -1249,6 +1249,12 @@ void executeQuery(
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr);
auto & pipeline = streams.pipeline;
QueryResultDetails result_details
{
.query_id = context->getClientInfo().current_query_id,
.timezone = DateLUT::instance().getTimeZone(),
};
std::unique_ptr<WriteBuffer> compressed_buffer;
try
{
@ -1307,9 +1313,8 @@ void executeQuery(
out->onProgress(progress);
});
if (set_result_details)
set_result_details(
context->getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone());
result_details.content_type = out->getContentType();
result_details.format = format_name;
pipeline.complete(std::move(out));
}
@ -1318,6 +1323,9 @@ void executeQuery(
pipeline.setProgressCallback(context->getProgressCallback());
}
if (set_result_details)
set_result_details(result_details);
if (pipeline.initialized())
{
CompletedPipelineExecutor executor(pipeline);

View File

@ -11,7 +11,15 @@ namespace DB
class ReadBuffer;
class WriteBuffer;
using SetResultDetailsFunc = std::function<void(const String &, const String &, const String &, const String &)>;
struct QueryResultDetails
{
String query_id;
std::optional<String> content_type;
std::optional<String> format;
std::optional<String> timezone;
};
using SetResultDetailsFunc = std::function<void(const QueryResultDetails &)>;
/// Parse and execute a query.
void executeQuery(

View File

@ -819,12 +819,20 @@ void HTTPHandler::processQuery(
customizeContext(request, context);
executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context,
[&response, this] (const String & current_query_id, const String & content_type, const String & format, const String & timezone)
[&response, this] (const QueryResultDetails & details)
{
response.setContentType(content_type_override.value_or(content_type));
response.add("X-ClickHouse-Query-Id", current_query_id);
response.add("X-ClickHouse-Format", format);
response.add("X-ClickHouse-Timezone", timezone);
response.add("X-ClickHouse-Query-Id", details.query_id);
if (content_type_override)
response.setContentType(*content_type_override);
else if (details.content_type)
response.setContentType(*details.content_type);
if (details.format)
response.add("X-ClickHouse-Format", *details.format);
if (details.timezone)
response.add("X-ClickHouse-Timezone", *details.timezone);
}
);

View File

@ -352,9 +352,9 @@ void MySQLHandler::comQuery(ReadBuffer & payload)
format_settings.mysql_wire.max_packet_size = max_packet_size;
format_settings.mysql_wire.sequence_id = &sequence_id;
auto set_result_details = [&with_output](const String &, const String &, const String &format, const String &)
auto set_result_details = [&with_output](const QueryResultDetails & details)
{
if (format != "MySQLWire")
if (details.format && *details.format != "MySQLWire")
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL protocol does not support custom output formats");
with_output = true;
};

View File

@ -0,0 +1,5 @@
X-ClickHouse-Query-Id
X-ClickHouse-Query-Id
X-ClickHouse-Query-Id
X-ClickHouse-Query-Id
X-ClickHouse-Query-Id

View File

@ -0,0 +1,13 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_query_id_header"
${CLICKHOUSE_CURL} -sS -v "${CLICKHOUSE_URL}" -d "CREATE TABLE t_query_id_header (a UInt64) ENGINE = Memory" 2>&1 | grep -o "X-ClickHouse-Query-Id"
${CLICKHOUSE_CURL} -sS -v "${CLICKHOUSE_URL}" -d "INSERT INTO t_query_id_header VALUES (1)" 2>&1 | grep -o "X-ClickHouse-Query-Id"
${CLICKHOUSE_CURL} -sS -v "${CLICKHOUSE_URL}" -d "EXISTS TABLE t_query_id_header" 2>&1 | grep -o "X-ClickHouse-Query-Id"
${CLICKHOUSE_CURL} -sS -v "${CLICKHOUSE_URL}" -d "SELECT * FROM t_query_id_header" 2>&1 | grep -o "X-ClickHouse-Query-Id"
${CLICKHOUSE_CURL} -sS -v "${CLICKHOUSE_URL}" -d "DROP TABLE t_query_id_header" 2>&1 | grep -o "X-ClickHouse-Query-Id"