Skip to content

Commit

Permalink
Merge pull request ClickHouse#74180 from ClickHouse/expose-headers
Browse files Browse the repository at this point in the history
Expose X-ClickHouse HTTP headers to JavaScript in the browser
  • Loading branch information
alexey-milovidov authored Jan 6, 2025
2 parents a8bc412 + 80b4eca commit f689a2a
Show file tree
Hide file tree
Showing 19 changed files with 56 additions and 22 deletions.
5 changes: 1 addition & 4 deletions src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp
Original file line number Diff line number Diff line change
@@ -1,23 +1,20 @@
#include <Analyzer/Passes/RewriteAggregateFunctionWithIfPass.h>

#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeAggregateFunction.h>

#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/IAggregateFunction.h>

#include <Core/Settings.h>

#include <Functions/FunctionFactory.h>

#include <Interpreters/Context.h>

#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/Utils.h>


namespace DB
{
namespace Setting
Expand Down
14 changes: 14 additions & 0 deletions src/IO/Progress.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,20 @@ namespace
}
}


bool Progress::empty() const
{
return read_rows == 0
&& read_bytes == 0
&& written_rows == 0
&& written_bytes == 0
&& total_rows_to_read == 0
&& result_rows == 0
&& result_bytes == 0;
/// We deliberately don't include "elapsed_ns" as a volatile value.
}


void ProgressValues::read(ReadBuffer & in, UInt64 server_revision)
{
readVarUInt(read_rows, in);
Expand Down
2 changes: 2 additions & 0 deletions src/IO/Progress.h
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,8 @@ struct Progress

void incrementElapsedNs(UInt64 elapsed_ns_);

bool empty() const;

void reset();

ProgressValues getValues() const;
Expand Down
11 changes: 6 additions & 5 deletions src/Interpreters/executeQuery.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1670,8 +1670,8 @@ void executeQuery(

/// Set the result details in case of any exception raised during query execution
SCOPE_EXIT({
if (set_result_details == nullptr)
/// Either the result_details have been set in the flow below or the caller of this function does not provide this callback
/// Either the result_details have been set in the flow below or the caller of this function does not provide this callback
if (!set_result_details)
return;

try
Expand Down Expand Up @@ -1706,17 +1706,18 @@ void executeQuery(
result_details.content_type = output_format->getContentType();
result_details.format = format_name;

fiu_do_on(FailPoints::execute_query_calling_empty_set_result_func_on_exception, {
fiu_do_on(FailPoints::execute_query_calling_empty_set_result_func_on_exception,
{
// it will throw std::bad_function_call
set_result_details = nullptr;
set_result_details = {};
set_result_details(result_details);
});

if (set_result_details)
{
/// reset set_result_details func to avoid calling in SCOPE_EXIT()
auto set_result_details_copy = set_result_details;
set_result_details = nullptr;
set_result_details = {};
set_result_details_copy(result_details);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ void JSONEachRowWithProgressRowOutputFormat::writeRowEndDelimiter()

void JSONEachRowWithProgressRowOutputFormat::writeProgress(const Progress & value)
{
if (value.empty())
return;
writeCString("{\"progress\":", *ostr);
value.writeJSON(*ostr);
writeCString("}\n", *ostr);
Expand Down
7 changes: 1 addition & 6 deletions src/Server/HTTPHandler.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
#include <Formats/FormatFactory.h>

#include <base/getFQDNOrHostName.h>
#include <base/scope_guard.h>
#include <base/isSharedPtrUnique.h>
#include <Server/HTTP/HTTPResponse.h>
#include <Server/HTTP/authenticateUserByHTTP.h>
Expand All @@ -41,14 +40,9 @@

#include <Poco/Net/HTTPMessage.h>

#include "config.h"

#include <algorithm>
#include <chrono>
#include <iterator>
#include <memory>
#include <optional>
#include <sstream>
#include <string_view>
#include <unordered_map>
#include <utility>
Expand Down Expand Up @@ -723,6 +717,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
thread_trace_context->root_span.addAttribute("http.method", request.getMethod());

response.setContentType("text/plain; charset=UTF-8");
response.add("Access-Control-Expose-Headers", "X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone");
response.set("X-ClickHouse-Server-Display-Name", server_display_name);

if (!request.get("Origin", "").empty())
Expand Down
2 changes: 1 addition & 1 deletion src/Server/HTTPHandler.h
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ class HTTPHandler : public HTTPRequestHandler
/* Raw data
* ↓
* CascadeWriteBuffer out_maybe_delayed_and_compressed (optional)
* ↓ (forwards data if an overflow is occur or explicitly via pushDelayedResults)
* ↓ (forwards data if an overflow occurs or explicitly via pushDelayedResults)
* CompressedWriteBuffer out_maybe_compressed (optional)
* ↓
* WriteBufferFromHTTPServerResponse out
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,4 +24,5 @@
7
8
9
< Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
< X-ClickHouse-Summary: {"read_rows":"10","read_bytes":"80","written_rows":"10","written_bytes":"40","total_rows_to_read":"10","result_rows":"10","result_bytes":"40"}
2 changes: 2 additions & 0 deletions tests/queries/0_stateless/00501_http_head.reference
Original file line number Diff line number Diff line change
@@ -1,12 +1,14 @@
HTTP/1.1 200 OK
Connection: Keep-Alive
Content-Type: text/tab-separated-values; charset=UTF-8
Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
Transfer-Encoding: chunked
Keep-Alive: timeout=10, max=?

HTTP/1.1 200 OK
Connection: Keep-Alive
Content-Type: text/tab-separated-values; charset=UTF-8
Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
Transfer-Encoding: chunked
Keep-Alive: timeout=10, max=?

Original file line number Diff line number Diff line change
@@ -1 +1,2 @@
X-ClickHouse-Query-Id
X-ClickHouse-Query-Id
1 change: 1 addition & 0 deletions tests/queries/0_stateless/02136_scalar_progress.reference
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
< Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
< X-ClickHouse-Progress: {"total_rows_to_read":"100000"}
< X-ClickHouse-Progress: {"read_rows":"65505","read_bytes":"524040","total_rows_to_read":"100000"}
< X-ClickHouse-Progress: {"read_rows":"100000","read_bytes":"800000","total_rows_to_read":"100000"}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ CURL_OUTPUT=$(echo 'SELECT 1 + sleepEachRow(0.00002) FROM numbers(100000)' | \
${CLICKHOUSE_CURL_COMMAND} --max-time 10 -vsS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=0&max_execution_time=1" --data-binary @- 2>&1)

READ_ROWS=$(echo "${CURL_OUTPUT}" | \
grep 'X-ClickHouse-Summary' | \
grep 'X-ClickHouse-Summary' | grep -v 'Access-Control-Expose-Headers' | \
awk '{print $3}' | \
sed -E 's/.*"read_rows":"?([^,"]*)"?.*/\1/'
)
Expand Down
Original file line number Diff line number Diff line change
@@ -1 +1,2 @@
< Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
< X-ClickHouse-Summary: {"read_rows":"100","read_bytes":"800","total_rows_to_read":"100","result_rows":"100","result_bytes":"227"}
Original file line number Diff line number Diff line change
@@ -1,8 +1,14 @@
No materialized views
< Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
< X-ClickHouse-Summary: {"read_rows":"1","read_bytes":"8","written_rows":"1","written_bytes":"8","result_rows":"1","result_bytes":"8"}
< Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
< X-ClickHouse-Summary: {"read_rows":"10","read_bytes":"80","written_rows":"10","written_bytes":"80","result_rows":"10","result_bytes":"80"}
< Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
< X-ClickHouse-Summary: {"read_rows":"10","read_bytes":"80","written_rows":"10","written_bytes":"80","result_rows":"10","result_bytes":"80"}
With materialized views
< Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
< X-ClickHouse-Summary: {"read_rows":"5","read_bytes":"40","written_rows":"4","written_bytes":"32","total_rows_to_read":"2","result_rows":"4","result_bytes":"32"}
< Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
< X-ClickHouse-Summary: {"read_rows":"32","read_bytes":"256","written_rows":"40","written_bytes":"320","total_rows_to_read":"2","result_rows":"40","result_bytes":"320"}
< Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
< X-ClickHouse-Summary: {"read_rows":"32","read_bytes":"256","written_rows":"40","written_bytes":"320","total_rows_to_read":"2","result_rows":"40","result_bytes":"320"}
4 changes: 2 additions & 2 deletions tests/queries/0_stateless/02841_parallel_replicas_summary.sh
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ echo "
parallel_replicas_local_plan=0
"\
| ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_0" --data-binary @- -vvv 2>&1 \
| grep "Summary" | grep -cv '"read_rows":"0"'
| grep "Summary" | grep -v 'Access-Control-Expose-Headers' | grep -cv '"read_rows":"0"'

echo "
SELECT *
Expand All @@ -56,7 +56,7 @@ echo "
parallel_replicas_local_plan=0
"\
| ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_high" --data-binary @- -vvv 2>&1 \
| grep "Summary" | grep -cv '"read_rows":"0"'
| grep "Summary" | grep -v 'Access-Control-Expose-Headers' | grep -cv '"read_rows":"0"'

$CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS"
involved_parallel_replicas "${query_id_base}"
4 changes: 2 additions & 2 deletions tests/queries/0_stateless/02869_http_headers_elapsed_ns.sh
Original file line number Diff line number Diff line change
Expand Up @@ -9,13 +9,13 @@ CURL_OUTPUT=$(echo 'SELECT number FROM numbers(10)' | \
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1)

ELAPSED_NS_PROGRESS="$(echo "${CURL_OUTPUT}" | \
grep 'X-ClickHouse-Progress' | \
grep 'X-ClickHouse-Progress' | grep -v 'Access-Control-Expose-Headers' | \
awk '{print $3}' | \
jq -cM '.elapsed_ns | tonumber'
)"

ELAPSED_NS_SUMMARY="$(echo "${CURL_OUTPUT}" | \
grep 'X-ClickHouse-Summary' | \
grep 'X-ClickHouse-Summary' | grep -v 'Access-Control-Expose-Headers' | \
awk '{print $3}' | \
jq -cM '.elapsed_ns | tonumber'
)"
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,3 @@
{"progress":{}}
{"progress":{"read_rows":"100000","read_bytes":"800000"}}
{"row":{"number":"1"}}
{"progress":{"read_rows":"200000","read_bytes":"1600000"}}
Expand All @@ -8,3 +7,4 @@
{"progress":{"read_rows":"600000","read_bytes":"4800000"}}
{"progress":{"read_rows":"700000","read_bytes":"5600000"}}
{"progress":{"read_rows":"800000","read_bytes":"6400000"}}
{"progress":{"read_rows":"900000","read_bytes":"7200000"}}
2 changes: 2 additions & 0 deletions tests/queries/0_stateless/03306_expose_headers.reference
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
< Access-Control-Expose-Headers: X-ClickHouse-Query-Id,X-ClickHouse-Summary,X-ClickHouse-Server-Display-Name,X-ClickHouse-Format,X-ClickHouse-Timezone
< X-ClickHouse-Format: JSONEachRowWithProgress
9 changes: 9 additions & 0 deletions tests/queries/0_stateless/03306_expose_headers.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,9 @@
#!/usr/bin/env bash

set -e

CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh

${CLICKHOUSE_CURL} -v "${CLICKHOUSE_URL}&default_format=JSONEachRowWithProgress" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -P 'X-ClickHouse-Format'

0 comments on commit f689a2a

Please sign in to comment.