Merge branch 'master' into fix-trash-in-aws-s3

This commit is contained in:
Alexey Milovidov 2024-08-12 06:01:07 +02:00
commit 741e1da2bd
8 changed files with 133 additions and 43 deletions

View File

@ -75,7 +75,7 @@ Data are received by this protocol and written to a [TimeSeries](/en/engines/tab
<my_rule_1>
<url>/write</url>
<handler>
<type>remote_write</type
<type>remote_write</type>
<database>db_name</database>
<table>time_series_table</table>
</handler>
@ -105,7 +105,7 @@ Data are read from a [TimeSeries](/en/engines/table-engines/special/time_series)
<my_rule_1>
<url>/read</url>
<handler>
<type>remote_read</type
<type>remote_read</type>
<database>db_name</database>
<table>time_series_table</table>
</handler>
@ -144,14 +144,14 @@ Multiple protocols can be specified together in one place:
<my_rule_2>
<url>/write</url>
<handler>
<type>remote_write</type
<type>remote_write</type>
<table>db_name.time_series_table</table>
</handler>
</my_rule_2>
<my_rule_3>
<url>/read</url>
<handler>
<type>remote_read</type
<type>remote_read</type>
<table>db_name.time_series_table</table>
</handler>
</my_rule_3>

View File

@ -143,7 +143,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
if (fs::exists(config_path))
{
ConfigProcessor config_processor(config_path, false, true);
ConfigProcessor config_processor(config_path);
ConfigProcessor::setConfigPath(fs::path(config_path).parent_path());
auto loaded_config = config_processor.loadConfig();
getClientConfiguration().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);

View File

@ -8,6 +8,7 @@
#include <Processors/QueryPlan/JoinStep.h>
#include <Processors/QueryPlan/LimitByStep.h>
#include <Processors/QueryPlan/LimitStep.h>
#include <Processors/QueryPlan/OffsetStep.h>
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/QueryPlanVisitor.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
@ -59,9 +60,10 @@ public:
if (typeid_cast<LimitStep *>(current_step)
|| typeid_cast<LimitByStep *>(current_step) /// (1) if there are LIMITs on top of ORDER BY, the ORDER BY is non-removable
|| typeid_cast<FillingStep *>(current_step) /// (2) if ORDER BY is with FILL WITH, it is non-removable
|| typeid_cast<SortingStep *>(current_step) /// (3) ORDER BY will change order of previous sorting
|| typeid_cast<AggregatingStep *>(current_step)) /// (4) aggregation change order
|| typeid_cast<OffsetStep *>(current_step) /// (2) OFFSET on top of ORDER BY, the ORDER BY is non-removable
|| typeid_cast<FillingStep *>(current_step) /// (3) if ORDER BY is with FILL WITH, it is non-removable
|| typeid_cast<SortingStep *>(current_step) /// (4) ORDER BY will change order of previous sorting
|| typeid_cast<AggregatingStep *>(current_step)) /// (5) aggregation change order
{
logStep("nodes_affect_order/push", current_node);
nodes_affect_order.push_back(current_node);

View File

@ -1,4 +1,6 @@
Code: 159
0
Code: 159
query_duration 1
0
query_duration 1
Code: 159
0

View File

@ -1,27 +1,23 @@
#!/usr/bin/env bash
# Tags: no-debug
# no-debug: Query is canceled by timeout after max_execution_time,
# but sending an exception to the client may hang
# for more than MAX_PROCESS_WAIT seconds in a slow debug build,
# and test will fail.
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
MAX_PROCESS_WAIT=5
IS_SANITIZER=$($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.warnings WHERE message like '%built with sanitizer%'")
if [ "$IS_SANITIZER" -gt 0 ]; then
# Query may hang for more than 5 seconds, especially in tsan build
MAX_PROCESS_WAIT=15
TIMEOUT=5
IS_SANITIZER_OR_DEBUG=$($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.warnings WHERE message like '%built with sanitizer%' or message like '%built in debug mode%'")
if [ "$IS_SANITIZER_OR_DEBUG" -gt 0 ]; then
# Increase the timeout due to in debug/sanitizers build:
# - client is slow
# - stacktrace resolving is slow
TIMEOUT=15
fi
# TCP CLIENT: As of today (02/12/21) uses PullingAsyncPipelineExecutor
### Should be cancelled after 1 second and return a 159 exception (timeout)
timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT --max_execution_time 1 -q \
"SELECT * FROM
query_id=$(random_str 12)
$CLICKHOUSE_CLIENT --query_id "$query_id" --max_execution_time 1 -q "
SELECT * FROM
(
SELECT a.name as n
FROM
@ -34,28 +30,35 @@ timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT --max_execution_time 1 -q \
GROUP BY n
)
LIMIT 20
FORMAT Null" 2>&1 | grep -o "Code: 159" | sort | uniq
FORMAT Null
" 2>&1 | grep -m1 -o "Code: 159"
$CLICKHOUSE_CLIENT -q "system flush logs"
${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'"
### Should stop pulling data and return what has been generated already (return code 0)
timeout -s KILL $MAX_PROCESS_WAIT $CLICKHOUSE_CLIENT -q \
"SELECT a.name as n
FROM
(
SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000
) AS a,
(
SELECT 'Name' as name2, number FROM system.numbers LIMIT 2000000
) as b
FORMAT Null
SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break'
"
query_id=$(random_str 12)
$CLICKHOUSE_CLIENT --query_id "$query_id" -q "
SELECT a.name as n
FROM
(
SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000
) AS a,
(
SELECT 'Name' as name2, number FROM system.numbers LIMIT 2000000
) as b
FORMAT Null
SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break'
"
echo $?
$CLICKHOUSE_CLIENT -q "system flush logs"
${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'"
# HTTP CLIENT: As of today (02/12/21) uses PullingPipelineExecutor
### Should be cancelled after 1 second and return a 159 exception (timeout)
${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL&max_execution_time=1" -d \
"SELECT * FROM
${CLICKHOUSE_CURL} -q --max-time $TIMEOUT -sS "$CLICKHOUSE_URL&max_execution_time=1" -d "
SELECT * FROM
(
SELECT a.name as n
FROM
@ -68,12 +71,13 @@ ${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL&max_exec
GROUP BY n
)
LIMIT 20
FORMAT Null" 2>&1 | grep -o "Code: 159" | sort | uniq
FORMAT Null
" 2>&1 | grep -o "Code: 159" | sort | uniq
### Should stop pulling data and return what has been generated already (return code 0)
${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL" -d \
"SELECT a.name as n
${CLICKHOUSE_CURL} -q --max-time $TIMEOUT -sS "$CLICKHOUSE_URL" -d "
SELECT a.name as n
FROM
(
SELECT 'Name' as name, number FROM system.numbers LIMIT 2000000
@ -83,5 +87,5 @@ ${CLICKHOUSE_CURL} -q --max-time $MAX_PROCESS_WAIT -sS "$CLICKHOUSE_URL" -d \
) as b
FORMAT Null
SETTINGS max_execution_time = 1, timeout_overflow_mode = 'break'
"
"
echo $?

View File

@ -465,6 +465,37 @@ Expression ((Projection + Before ORDER BY))
ReadFromStorage (SystemOne)
-- execute
Float64 9007199254740994
-- presence of an inner OFFSET retains the ORDER BY
-- query
WITH
t1 AS (
SELECT a, b
FROM
VALUES (
'b UInt32, a Int32',
(1, 1),
(2, 0)
)
)
SELECT
SUM(a)
FROM (
SELECT a, b
FROM t1
ORDER BY 1 DESC, 2
OFFSET 1
) t2
-- explain
Expression ((Projection + Before ORDER BY))
Aggregating
Expression (Before GROUP BY)
Offset
Expression (Projection)
Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + Before ORDER BY)))
ReadFromStorage (Values)
-- execute
0
-- disable common optimization to avoid functions to be lifted up (liftUpFunctions optimization), needed for testing with stateful function
-- neighbor() as stateful function prevents removing inner ORDER BY since its result depends on order
-- query

View File

@ -302,6 +302,27 @@ FROM
)"
run_query "$query"
echo "-- presence of an inner OFFSET retains the ORDER BY"
query="WITH
t1 AS (
SELECT a, b
FROM
VALUES (
'b UInt32, a Int32',
(1, 1),
(2, 0)
)
)
SELECT
SUM(a)
FROM (
SELECT a, b
FROM t1
ORDER BY 1 DESC, 2
OFFSET 1
) t2"
run_query "$query"
echo "-- disable common optimization to avoid functions to be lifted up (liftUpFunctions optimization), needed for testing with stateful function"
ENABLE_OPTIMIZATION="SET query_plan_enable_optimizations=0;$ENABLE_OPTIMIZATION"
echo "-- neighbor() as stateful function prevents removing inner ORDER BY since its result depends on order"

View File

@ -464,6 +464,36 @@ Expression ((Project names + Projection))
ReadFromStorage (SystemOne)
-- execute
Float64 9007199254740994
-- presence of an inner OFFSET retains the ORDER BY
-- query
WITH
t1 AS (
SELECT a, b
FROM
VALUES (
'b UInt32, a Int32',
(1, 1),
(2, 0)
)
)
SELECT
SUM(a)
FROM (
SELECT a, b
FROM t1
ORDER BY 1 DESC, 2
OFFSET 1
) t2
-- explain
Expression ((Project names + Projection))
Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + Project names)))
Offset
Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))))
ReadFromStorage (Values)
-- execute
0
-- disable common optimization to avoid functions to be lifted up (liftUpFunctions optimization), needed for testing with stateful function
-- neighbor() as stateful function prevents removing inner ORDER BY since its result depends on order
-- query