Test plan and pipeline for filtering step for join

This commit is contained in:
vdimir 2022-08-09 12:39:39 +00:00
parent 2deb28d56e
commit afb6b7d9cf
No known key found for this signature in database
GPG Key ID: 6EE4CE2BEDC51862
5 changed files with 70 additions and 8 deletions

View File

@ -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) \
\

View File

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

View File

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

View File

@ -0,0 +1,10 @@
Ok
Ok
Ok
Ok
Ok
Ok
Ok
Ok
Ok
Ok

View 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