mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Test plan and pipeline for filtering step for join
This commit is contained in:
parent
2deb28d56e
commit
afb6b7d9cf
@ -366,7 +366,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \
|
||||
M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \
|
||||
M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \
|
||||
M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining (-1 to disable).", 0) \
|
||||
M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining.", 0) \
|
||||
\
|
||||
M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \
|
||||
\
|
||||
|
@ -1449,11 +1449,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
auto crosswise_connection = CreateSetAndFilterOnTheFlyStep::createCrossConnection();
|
||||
auto add_create_set = [&settings, crosswise_connection](QueryPlan & plan, const Names & key_names, JoinTableSide join_pos)
|
||||
{
|
||||
/// Small number of rows is unreasonable
|
||||
size_t max_rows = std::max<size_t>(100, settings.max_rows_in_set_to_optimize_join);
|
||||
|
||||
auto creating_set_step = std::make_unique<CreateSetAndFilterOnTheFlyStep>(
|
||||
plan.getCurrentDataStream(), key_names, max_rows, crosswise_connection, join_pos);
|
||||
plan.getCurrentDataStream(), key_names, settings.max_rows_in_set_to_optimize_join, crosswise_connection, join_pos);
|
||||
creating_set_step->setStepDescription(fmt::format("Create set and filter {} joined stream", join_pos));
|
||||
|
||||
auto * step_raw_ptr = creating_set_step.get();
|
||||
@ -1468,7 +1465,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
|
||||
auto join_kind = table_join.kind();
|
||||
bool kind_allows_filtering = isInner(join_kind) || isLeft(join_kind) || isRight(join_kind);
|
||||
if (settings.max_rows_in_set_to_optimize_join >= 0 && kind_allows_filtering)
|
||||
if (settings.max_rows_in_set_to_optimize_join > 0 && kind_allows_filtering)
|
||||
{
|
||||
auto * left_set = add_create_set(query_plan, join_clause.key_names_left, JoinTableSide::Left);
|
||||
auto * right_set = add_create_set(*joined_plan, join_clause.key_names_right, JoinTableSide::Right);
|
||||
|
@ -39,8 +39,6 @@ public:
|
||||
|
||||
PingPongProcessor(const Block & header, size_t num_ports, Order order_);
|
||||
|
||||
String getName() const override { return "PingPongProcessor"; }
|
||||
|
||||
Status prepare() override;
|
||||
|
||||
std::pair<InputPort *, OutputPort *> getAuxPorts();
|
||||
@ -90,6 +88,8 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override { return "ReadHeadBalancedProcessor"; }
|
||||
|
||||
bool isReady(const Chunk & chunk) override
|
||||
{
|
||||
data_consumed += chunk.getNumRows();
|
||||
|
@ -0,0 +1,10 @@
|
||||
Ok
|
||||
Ok
|
||||
Ok
|
||||
Ok
|
||||
Ok
|
||||
Ok
|
||||
Ok
|
||||
Ok
|
||||
Ok
|
||||
Ok
|
55
tests/queries/0_stateless/02383_join_and_filtering_set.sh
Executable file
55
tests/queries/0_stateless/02383_join_and_filtering_set.sh
Executable file
@ -0,0 +1,55 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-asan,no-msan,no-tsan,no-ubsan
|
||||
#
|
||||
# Test doesn't run complex queries, just test the logic of setting, so no need to run with different builds.
|
||||
# Also, we run similar queries in 02382_join_and_filtering_set.sql which is enabled for these builds.
|
||||
#
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -mn -q """
|
||||
CREATE TABLE t1 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y
|
||||
AS SELECT sipHash64(number, 't1_x') % 100 AS x, sipHash64(number, 't1_y') % 100 AS y FROM numbers(100);
|
||||
|
||||
CREATE TABLE t2 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y
|
||||
AS SELECT sipHash64(number, 't2_x') % 100 AS x, sipHash64(number, 't2_y') % 100 AS y FROM numbers(100);
|
||||
"""
|
||||
|
||||
# Arguments:
|
||||
# - value of max_rows_in_set_to_optimize_join
|
||||
# - join kind
|
||||
# - expected number of steps in plan
|
||||
# - expected number of steps in pipeline
|
||||
function test() {
|
||||
|
||||
PARAM_VALUE=$1
|
||||
JOIN_KIND=${2:-}
|
||||
|
||||
EXPECTED_PLAN_STEPS=$3
|
||||
RES=$(
|
||||
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join=${PARAM_VALUE} --join_algorithm='full_sorting_merge' \
|
||||
-q "EXPLAIN PLAN SELECT count() FROM t1 ${JOIN_KIND} JOIN t2 ON t1.x = t2.x" | grep -o 'CreateSetAndFilterOnTheFlyStep' | wc -l
|
||||
)
|
||||
[ "$RES" -eq "$EXPECTED_PLAN_STEPS" ] && echo "Ok" || echo "Fail: $RES != $EXPECTED_PLAN_STEPS"
|
||||
|
||||
EXPECTED_PIPELINE_STEPS=$4
|
||||
RES=$(
|
||||
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join=${PARAM_VALUE} --join_algorithm='full_sorting_merge' \
|
||||
-q "EXPLAIN PIPELINE SELECT count() FROM t1 ${JOIN_KIND} JOIN t2 ON t1.x = t2.x" \
|
||||
| grep -o -e ReadHeadBalancedProcessor -e FilterBySetOnTheFlyTransform -e CreatingSetsOnTheFlyTransform | wc -l
|
||||
)
|
||||
[ "$RES" -eq "$EXPECTED_PIPELINE_STEPS" ] && echo "Ok" || echo "Fail: $RES != $EXPECTED_PIPELINE_STEPS"
|
||||
|
||||
}
|
||||
|
||||
test 1000 '' 2 6
|
||||
|
||||
# no filtering for left/right side
|
||||
test 1000 'LEFT' 2 5
|
||||
test 1000 'RIGHT' 2 5
|
||||
|
||||
# when disabled no extra steps should be created
|
||||
test 1000 'FULL' 0 0
|
||||
test 0 '' 0 0
|
Loading…
Reference in New Issue
Block a user