Merge branch 'fix-test-replace-current-query' into tighten-limits-functional-tests

This commit is contained in:
Alexey Milovidov 2024-08-12 07:43:55 +02:00
commit b1d4a261f8
6 changed files with 107 additions and 7 deletions

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

@ -13,7 +13,18 @@ ${CLICKHOUSE_CLIENT} -q "grant select on system.numbers to u_00600${TEST_PREFIX}
function wait_for_query_to_start()
{
while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done
while [[ 0 -eq $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE query_id = '$1'") ]]
do
sleep 0.1
done
}
function wait_for_queries_to_finish()
{
while [[ 0 -ne $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT count() FROM system.processes WHERE current_database = '${CLICKHOUSE_DATABASE}' AND query NOT LIKE '%this query%'") ]]
do
sleep 0.1
done
}
@ -25,8 +36,9 @@ $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=hello&replace_running_query=1" -d
# Wait for it to be replaced
wait
wait_for_queries_to_finish
${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id=42 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' &
${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id=42 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'QUERY_WAS_CANCELLED' &
wait_for_query_to_start '42'
# Trying to run another query with the same query_id
@ -37,10 +49,13 @@ $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=42&replace_running_query=1" -d 'S
$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = '42' SYNC" > /dev/null
wait
wait_for_queries_to_finish
${CLICKHOUSE_CLIENT} --query_id=42 --max_rows_to_read=0 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' &
${CLICKHOUSE_CLIENT} --query_id=42 --max_rows_to_read=0 --query='SELECT 3, count() FROM system.numbers' 2>&1 | grep -cF 'QUERY_WAS_CANCELLED' &
wait_for_query_to_start '42'
${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --replace_running_query_max_wait_ms=500 --query='SELECT 43' 2>&1 | grep -F "can't be stopped" > /dev/null
wait
wait_for_queries_to_finish
${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --query='SELECT 44'
${CLICKHOUSE_CLIENT} -q "drop user u_00600${TEST_PREFIX}"

View File

@ -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;

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