Merge pull request #56309 from yariks5s/preferred_optimize_projection_name

New setting preferred_projection_name
This commit is contained in:
Yarik Briukhovetskyi 2023-11-08 18:06:03 +01:00 committed by GitHub
commit b8ea31db43
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 157 additions and 2 deletions

View File

@ -3943,6 +3943,17 @@ Possible values:
Default value: `''`.
## preferred_optimize_projection_name {#preferred_optimize_projection_name}
If it is set to a non-empty string, ClickHouse will try to apply specified projection in query.
Possible values:
- string: name of preferred projection
Default value: `''`.
## alter_sync {#alter-sync}
Allows to set up waiting for actions to be executed on replicas by [ALTER](../../sql-reference/statements/alter/index.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries.

View File

@ -604,6 +604,7 @@ class IColumn;
M(Bool, optimize_use_implicit_projections, true, "Automatically choose implicit projections to perform SELECT query", 0) \
M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \
M(String, force_optimize_projection_name, "", "If it is set to a non-empty string, check that this projection is used in the query at least once.", 0) \
M(String, preferred_optimize_projection_name, "", "If it is set to a non-empty string, ClickHouse tries to apply specified projection", 0) \
M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \
M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \
M(Bool, insert_null_as_default, true, "Insert DEFAULT values instead of NULL in INSERT SELECT (UNION ALL)", 0) \

View File

@ -444,6 +444,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
const auto & projections = metadata->projections;
std::vector<const ProjectionDescription *> agg_projections;
for (const auto & projection : projections)
if (projection.type == ProjectionDescription::Type::Aggregate)
agg_projections.push_back(&projection);
@ -584,6 +585,9 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
auto ordinary_reading_select_result = reading->selectRangesToRead(parts, /* alter_conversions = */ {});
size_t ordinary_reading_marks = ordinary_reading_select_result->marks();
const auto & proj_name_from_settings = context->getSettings().preferred_optimize_projection_name.value;
bool found_best_candidate = false;
/// Selecting best candidate.
for (auto & candidate : candidates.real)
{
@ -602,8 +606,13 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes &
if (candidate.sum_marks > ordinary_reading_marks)
continue;
if (best_candidate == nullptr || best_candidate->sum_marks > candidate.sum_marks)
if ((best_candidate == nullptr || best_candidate->sum_marks > candidate.sum_marks) && !found_best_candidate)
best_candidate = &candidate;
if (!proj_name_from_settings.empty() && candidate.projection->name == proj_name_from_settings)
{
best_candidate = &candidate;
found_best_candidate = true;
}
}
if (!best_candidate)

View File

@ -7,6 +7,8 @@
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Processors/Sources/NullSource.h>
#include <Common/logger_useful.h>
#include <Storages/ProjectionsDescription.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <stack>
@ -131,6 +133,21 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
std::shared_ptr<PartitionIdToMaxBlock> max_added_blocks = getMaxAddedBlocks(reading);
// Here we iterate over the projections and check if we have the same projections as we specified in preferred_projection_name
bool is_projection_found = false;
const auto & proj_name_from_settings = context->getSettings().preferred_optimize_projection_name.value;
if (!proj_name_from_settings.empty())
{
for (const auto * projection : normal_projections)
{
if (projection->name == proj_name_from_settings)
{
is_projection_found = true;
break;
}
}
}
for (const auto * projection : normal_projections)
{
if (!hasAllRequiredColumns(projection, required_columns))
@ -153,7 +170,9 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes)
if (candidate.sum_marks >= ordinary_reading_marks)
continue;
if (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks)
if (!is_projection_found && (best_candidate == nullptr || candidate.sum_marks < best_candidate->sum_marks))
best_candidate = &candidate;
else if (is_projection_found && projection->name == proj_name_from_settings)
best_candidate = &candidate;
}

View File

@ -0,0 +1,8 @@
test
projection_test_by_string
Executing query with setting
test
projection_test_by_more
Executing query with wrong projection
test
projection_test_by_string

View File

@ -0,0 +1,107 @@
#!/usr/bin/env bash
# Tags: no-fasttest
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_opt_proj;"
$CLICKHOUSE_CLIENT -q "
CREATE TABLE test_opt_proj (
test_id UInt64,
test_name String,
test_count Nullable(Float64),
test_string String,
PROJECTION projection_test_by_string (
SELECT test_string,
sum(test_count)
GROUP BY test_id,
test_string,
test_name
),
PROJECTION projection_test_by_more (
SELECT test_string,
test_name,
sum(test_count)
GROUP BY test_id,
test_string,
test_name
)
) ENGINE = MergeTree
ORDER BY test_string;"
$CLICKHOUSE_CLIENT -q "
INSERT INTO test_opt_proj
SELECT number,
'test',
1.* (number / 2),
'test'
FROM numbers(100, 500);"
$CLICKHOUSE_CLIENT --query_id 02907_test_$CLICKHOUSE_DATABASE -q "
SELECT test_string
FROM test_opt_proj
WHERE (test_id > 50)
AND (test_id < 150)
GROUP BY test_string;"
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS;"
$CLICKHOUSE_CLIENT -q "
SELECT projections
FROM system.query_log
WHERE query_id = '02907_test_$CLICKHOUSE_DATABASE' AND current_database=currentDatabase()
LIMIT 1;" | grep -o "projection_test_by_string" || true
$CLICKHOUSE_CLIENT -q "
SELECT projections
FROM system.query_log
WHERE query_id = '02907_test_$CLICKHOUSE_DATABASE' AND current_database=currentDatabase()
LIMIT 1;" | grep -o "projection_test_by_more" || true
echo "Executing query with setting"
$CLICKHOUSE_CLIENT --query_id 02907_test_1_$CLICKHOUSE_DATABASE --preferred_optimize_projection_name 'projection_test_by_more' -q "
SELECT test_string
FROM test_opt_proj
WHERE (test_id > 50)
AND (test_id < 150)
GROUP BY test_string;"
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS;"
$CLICKHOUSE_CLIENT -q "
SELECT projections
FROM system.query_log
WHERE query_id = '02907_test_1_$CLICKHOUSE_DATABASE' AND current_database=currentDatabase()
LIMIT 1;" | grep -o "projection_test_by_more" || true
$CLICKHOUSE_CLIENT -q "
SELECT projections
FROM system.query_log
WHERE query_id = '02907_test_1_$CLICKHOUSE_DATABASE' AND current_database=currentDatabase()
LIMIT 1" | grep -o "projection_test_by_string" || true
echo "Executing query with wrong projection"
$CLICKHOUSE_CLIENT --query_id 02907_test_2_$CLICKHOUSE_DATABASE --preferred_optimize_projection_name 'non_existing_projection' -q "
SELECT test_string
FROM test_opt_proj
WHERE (test_id > 50)
AND (test_id < 150)
GROUP BY test_string;"
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS;"
$CLICKHOUSE_CLIENT -q "
SELECT projections
FROM system.query_log
WHERE query_id = '02907_test_2_$CLICKHOUSE_DATABASE' AND current_database=currentDatabase()
LIMIT 1;" | grep -o "projection_test_by_string" || true
$CLICKHOUSE_CLIENT -q "
SELECT projections
FROM system.query_log
WHERE query_id = '02907_test_2_$CLICKHOUSE_DATABASE' AND current_database=currentDatabase()
LIMIT 1;" | grep -o "projection_test_by_more" || true