mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #46364 from CurtizJ/add-query-id-in-response
Add `X-ClickHouse-Query-Id` header in all queries
This commit is contained in:
commit
37570f10ea
@ -1276,6 +1276,12 @@ void executeQuery(
|
|||||||
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr);
|
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr);
|
||||||
auto & pipeline = streams.pipeline;
|
auto & pipeline = streams.pipeline;
|
||||||
|
|
||||||
|
QueryResultDetails result_details
|
||||||
|
{
|
||||||
|
.query_id = context->getClientInfo().current_query_id,
|
||||||
|
.timezone = DateLUT::instance().getTimeZone(),
|
||||||
|
};
|
||||||
|
|
||||||
std::unique_ptr<WriteBuffer> compressed_buffer;
|
std::unique_ptr<WriteBuffer> compressed_buffer;
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
@ -1334,9 +1340,8 @@ void executeQuery(
|
|||||||
out->onProgress(progress);
|
out->onProgress(progress);
|
||||||
});
|
});
|
||||||
|
|
||||||
if (set_result_details)
|
result_details.content_type = out->getContentType();
|
||||||
set_result_details(
|
result_details.format = format_name;
|
||||||
context->getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone());
|
|
||||||
|
|
||||||
pipeline.complete(std::move(out));
|
pipeline.complete(std::move(out));
|
||||||
}
|
}
|
||||||
@ -1345,6 +1350,9 @@ void executeQuery(
|
|||||||
pipeline.setProgressCallback(context->getProgressCallback());
|
pipeline.setProgressCallback(context->getProgressCallback());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (set_result_details)
|
||||||
|
set_result_details(result_details);
|
||||||
|
|
||||||
if (pipeline.initialized())
|
if (pipeline.initialized())
|
||||||
{
|
{
|
||||||
CompletedPipelineExecutor executor(pipeline);
|
CompletedPipelineExecutor executor(pipeline);
|
||||||
|
@ -11,7 +11,15 @@ namespace DB
|
|||||||
class ReadBuffer;
|
class ReadBuffer;
|
||||||
class WriteBuffer;
|
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.
|
/// Parse and execute a query.
|
||||||
void executeQuery(
|
void executeQuery(
|
||||||
|
@ -831,12 +831,20 @@ void HTTPHandler::processQuery(
|
|||||||
customizeContext(request, context);
|
customizeContext(request, context);
|
||||||
|
|
||||||
executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, 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", details.query_id);
|
||||||
response.add("X-ClickHouse-Query-Id", current_query_id);
|
|
||||||
response.add("X-ClickHouse-Format", format);
|
if (content_type_override)
|
||||||
response.add("X-ClickHouse-Timezone", timezone);
|
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);
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -352,11 +352,15 @@ void MySQLHandler::comQuery(ReadBuffer & payload)
|
|||||||
format_settings.mysql_wire.max_packet_size = max_packet_size;
|
format_settings.mysql_wire.max_packet_size = max_packet_size;
|
||||||
format_settings.mysql_wire.sequence_id = &sequence_id;
|
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)
|
||||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL protocol does not support custom output formats");
|
{
|
||||||
with_output = true;
|
if (*details.format != "MySQLWire")
|
||||||
|
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL protocol does not support custom output formats");
|
||||||
|
|
||||||
|
with_output = true;
|
||||||
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
executeQuery(should_replace ? replacement : payload, *out, false, query_context, set_result_details, format_settings);
|
executeQuery(should_replace ? replacement : payload, *out, false, query_context, set_result_details, format_settings);
|
||||||
|
22
tests/queries/0_stateless/02564_query_id_header.reference
Normal file
22
tests/queries/0_stateless/02564_query_id_header.reference
Normal file
@ -0,0 +1,22 @@
|
|||||||
|
CREATE TABLE t_query_id_header (a UInt64) ENGINE = Memory
|
||||||
|
< Content-Type: text/plain; charset=UTF-8
|
||||||
|
< X-ClickHouse-Query-Id: query_id
|
||||||
|
< X-ClickHouse-Timezone: timezone
|
||||||
|
INSERT INTO t_query_id_header VALUES (1)
|
||||||
|
< Content-Type: text/plain; charset=UTF-8
|
||||||
|
< X-ClickHouse-Query-Id: query_id
|
||||||
|
< X-ClickHouse-Timezone: timezone
|
||||||
|
EXISTS TABLE t_query_id_header
|
||||||
|
< Content-Type: text/tab-separated-values; charset=UTF-8
|
||||||
|
< X-ClickHouse-Format: TabSeparated
|
||||||
|
< X-ClickHouse-Query-Id: query_id
|
||||||
|
< X-ClickHouse-Timezone: timezone
|
||||||
|
SELECT * FROM t_query_id_header
|
||||||
|
< Content-Type: text/tab-separated-values; charset=UTF-8
|
||||||
|
< X-ClickHouse-Format: TabSeparated
|
||||||
|
< X-ClickHouse-Query-Id: query_id
|
||||||
|
< X-ClickHouse-Timezone: timezone
|
||||||
|
DROP TABLE t_query_id_header
|
||||||
|
< Content-Type: text/plain; charset=UTF-8
|
||||||
|
< X-ClickHouse-Query-Id: query_id
|
||||||
|
< X-ClickHouse-Timezone: timezone
|
30
tests/queries/0_stateless/02564_query_id_header.sh
Executable file
30
tests/queries/0_stateless/02564_query_id_header.sh
Executable file
@ -0,0 +1,30 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
# shellcheck source=../shell_config.sh
|
||||||
|
. "$CURDIR"/../shell_config.sh
|
||||||
|
|
||||||
|
CLICKHOUSE_TIMEZONE_ESCAPED=$($CLICKHOUSE_CLIENT --query="SELECT timezone()" | sed 's/[]\/$*.^+:()[]/\\&/g')
|
||||||
|
|
||||||
|
function run_and_check_headers()
|
||||||
|
{
|
||||||
|
query=$1
|
||||||
|
query_id="${CLICKHOUSE_DATABASE}_${RANDOM}"
|
||||||
|
|
||||||
|
echo "$query"
|
||||||
|
|
||||||
|
${CLICKHOUSE_CURL} -sS -v "${CLICKHOUSE_URL}&query_id=$query_id" -d "$1" 2>&1 \
|
||||||
|
| grep -e "< X-ClickHouse-Query-Id" -e "< X-ClickHouse-Timezone" -e "< X-ClickHouse-Format" -e "< Content-Type" \
|
||||||
|
| sed "s/$CLICKHOUSE_TIMEZONE_ESCAPED/timezone/" \
|
||||||
|
| sed "s/$query_id/query_id/" \
|
||||||
|
| sed "s/\r$//" \
|
||||||
|
| sort
|
||||||
|
}
|
||||||
|
|
||||||
|
${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_query_id_header"
|
||||||
|
|
||||||
|
run_and_check_headers "CREATE TABLE t_query_id_header (a UInt64) ENGINE = Memory"
|
||||||
|
run_and_check_headers "INSERT INTO t_query_id_header VALUES (1)"
|
||||||
|
run_and_check_headers "EXISTS TABLE t_query_id_header"
|
||||||
|
run_and_check_headers "SELECT * FROM t_query_id_header"
|
||||||
|
run_and_check_headers "DROP TABLE t_query_id_header"
|
Loading…
Reference in New Issue
Block a user