mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
EXPLAIN tests
This commit is contained in:
parent
15bdeba074
commit
1439664df6
@ -1,4 +1,3 @@
|
||||
#include <algorithm>
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
@ -7,7 +6,6 @@
|
||||
#include <IO/Operators.h>
|
||||
#include <Interpreters/JoinSwitcher.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
#include "Core/SortDescription.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -235,8 +235,10 @@ void SortingStep::fullSort(QueryPipelineBuilder & pipeline, const SortDescriptio
|
||||
|
||||
void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
/// we consider that a caller has more information what sorting to apply (depends on what constructor was used)
|
||||
/// so we'll try to infer what sorting to use only in case of Full sorting
|
||||
/// We consider that a caller has more information what type of sorting to apply.
|
||||
/// The type depends on constructor used to create sorting step.
|
||||
/// So we'll try to infer sorting to use only in case of Full sorting
|
||||
|
||||
if (type == Type::MergingSorted)
|
||||
{
|
||||
mergingSorted(pipeline, result_description, limit);
|
||||
|
@ -0,0 +1,26 @@
|
||||
-- disable optimize_sorting_for_input_stream
|
||||
-- PIPELINE: sorting order is NOT propagated from subquery -> full sort
|
||||
MergeSortingTransform
|
||||
LimitsCheckingTransform
|
||||
PartialSortingTransform
|
||||
-- enable optimize_sorting_for_input_stream
|
||||
-- PIPELINE: sorting order is propagated from subquery -> merge sort
|
||||
MergeSortingTransform
|
||||
-- PLAN: ExpressionStep preserves sort mode
|
||||
Sort Mode: Stream: a ASC
|
||||
Sort Mode: Stream: a ASC
|
||||
Sort Mode: Port: a ASC
|
||||
Sort Mode: Port: a ASC
|
||||
-- PLAN: ExpressionStep breaks sort mode
|
||||
Sort Mode: Stream: plus(a, 1) ASC
|
||||
Sort Mode: Stream: plus(a, 1) ASC
|
||||
Sort Mode: None
|
||||
Sort Mode: Chunk: a ASC
|
||||
-- PLAN: FilterStep preserves sort mode
|
||||
Sort Mode: Chunk: a ASC
|
||||
Sort Mode: Chunk: a ASC
|
||||
Sort Mode: Chunk: a ASC
|
||||
-- PLAN: FilterStep breaks sort mode
|
||||
Sort Mode: None
|
||||
Sort Mode: None
|
||||
Sort Mode: Chunk: a ASC
|
39
tests/queries/0_stateless/02377_optimize_sorting_for_input_stream_explain.sh
Executable file
39
tests/queries/0_stateless/02377_optimize_sorting_for_input_stream_explain.sh
Executable file
@ -0,0 +1,39 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
DISABLE_OPTIMIZATION="set optimize_sorting_for_input_stream=0;set max_threads=1"
|
||||
ENABLE_OPTIMIZATION="set optimize_sorting_for_input_stream=1;set max_threads=1"
|
||||
GREP_SORTING="grep 'PartialSortingTransform\|LimitsCheckingTransform\|MergeSortingTransform\|MergingSortedTransform'"
|
||||
GREP_SORTMODE="grep 'Sort Mode'"
|
||||
TRIM_LEADING_SPACES="sed -e 's/^[ \t]*//'"
|
||||
FIND_SORTING="$GREP_SORTING | $TRIM_LEADING_SPACES"
|
||||
FIND_SORTMODE="$GREP_SORTMODE | $TRIM_LEADING_SPACES"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync"
|
||||
$CLICKHOUSE_CLIENT -q "create table optimize_sorting (a int, b int, c int) engine=MergeTree() order by (a, b)"
|
||||
$CLICKHOUSE_CLIENT -q "insert into optimize_sorting select number, number % 5, number % 2 from numbers(0,10)"
|
||||
$CLICKHOUSE_CLIENT -q "insert into optimize_sorting select number, number % 5, number % 2 from numbers(10,10)"
|
||||
$CLICKHOUSE_CLIENT -q "insert into optimize_sorting select number, number % 5, number % 2 from numbers(20,10)"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select '-- disable optimize_sorting_for_input_stream'"
|
||||
$CLICKHOUSE_CLIENT -q "select '-- PIPELINE: sorting order is NOT propagated from subquery -> full sort'"
|
||||
$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;explain pipeline select a from (select a from optimize_sorting) order by a" | eval $FIND_SORTING
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select '-- enable optimize_sorting_for_input_stream'"
|
||||
$CLICKHOUSE_CLIENT -q "select '-- PIPELINE: sorting order is propagated from subquery -> merge sort'"
|
||||
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select a from (select a from optimize_sorting) order by a" | eval $FIND_SORTING
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select '-- PLAN: ExpressionStep preserves sort mode'"
|
||||
$CLICKHOUSE_CLIENT -nq "EXPLAIN PLAN sortmode=1 SELECT a FROM optimize_sorting ORDER BY a" | eval $FIND_SORTMODE
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select '-- PLAN: ExpressionStep breaks sort mode'"
|
||||
$CLICKHOUSE_CLIENT -nq "EXPLAIN PLAN sortmode=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1" | eval $FIND_SORTMODE
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select '-- PLAN: FilterStep preserves sort mode'"
|
||||
$CLICKHOUSE_CLIENT -nq "EXPLAIN PLAN sortmode=1 SELECT a FROM optimize_sorting WHERE a > 0" | eval $FIND_SORTMODE
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "select '-- PLAN: FilterStep breaks sort mode'"
|
||||
$CLICKHOUSE_CLIENT -nq "EXPLAIN PLAN sortmode=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0" | eval $FIND_SORTMODE
|
Loading…
Reference in New Issue
Block a user