mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge branch 'master' into refactor-object-storages
This commit is contained in:
commit
9005256bd0
@ -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>
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -1,4 +1,5 @@
|
||||
-- Tags: long
|
||||
-- Tags: long, no-parallel
|
||||
-- set no-parallel tag is to prevent timeout of this test
|
||||
|
||||
drop table if exists t;
|
||||
|
||||
|
@ -1,4 +1,6 @@
|
||||
Code: 159
|
||||
0
|
||||
Code: 159
|
||||
query_duration 1
|
||||
0
|
||||
query_duration 1
|
||||
Code: 159
|
||||
0
|
||||
|
@ -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 $?
|
||||
|
@ -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
|
||||
|
@ -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"
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user