Merge pull request #42322 from ClickHouse/additional-filters-and-distributed

Fix additional_table_filters for Distributed.
This commit is contained in:
Nikolai Kochetov 2022-10-19 17:09:51 +02:00 committed by GitHub
commit d85bfe6ea3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 111 additions and 5 deletions

View File

@ -7,6 +7,7 @@
#include <Interpreters/ProcessList.h>
#include <Interpreters/OptimizeShardingKeyRewriteInVisitor.h>
#include <QueryPipeline/Pipe.h>
#include <Parsers/queryToString.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromRemote.h>
#include <Processors/QueryPlan/UnionStep.h>
@ -26,7 +27,7 @@ namespace ErrorCodes
namespace ClusterProxy
{
ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log)
ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table, const SelectQueryInfo * query_info, Poco::Logger * log)
{
Settings new_settings = settings;
new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time);
@ -96,6 +97,20 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr c
new_settings.limit.changed = false;
}
/// Setting additional_table_filters may be applied to Distributed table.
/// In case if query is executed up to WithMergableState on remote shard, it is impossible to filter on initiator.
/// We need to propagate the setting, but change the table name from distributed to source.
///
/// Here we don't try to analyze setting again. In case if query_info->additional_filter_ast is not empty, some filter was applied.
/// It's just easier to add this filter for a source table.
if (query_info && query_info->additional_filter_ast)
{
Tuple tuple;
tuple.push_back(main_table.getShortName());
tuple.push_back(queryToString(query_info->additional_filter_ast));
new_settings.additional_table_filters.value.push_back(std::move(tuple));
}
auto new_context = Context::createCopy(context);
new_context->setSettings(new_settings);
return new_context;
@ -121,7 +136,7 @@ void executeQuery(
std::vector<QueryPlanPtr> plans;
SelectStreamFactory::Shards remote_shards;
auto new_context = updateSettingsForCluster(*query_info.getCluster(), context, settings, log);
auto new_context = updateSettingsForCluster(*query_info.getCluster(), context, settings, main_table, &query_info, log);
new_context->getClientInfo().distributed_depth += 1;

View File

@ -35,7 +35,7 @@ class SelectStreamFactory;
///
/// @return new Context with adjusted settings
ContextMutablePtr updateSettingsForCluster(
const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log = nullptr);
const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table, const SelectQueryInfo * query_info = nullptr, Poco::Logger * log = nullptr);
/// Execute a distributed query, creating a query plan, from which the query pipeline can be built.
/// `stream_factory` object encapsulates the logic of creating plans for a different type of query

View File

@ -58,7 +58,7 @@ ColumnsDescription getStructureOfRemoteTableInShard(
}
ColumnsDescription res;
auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef());
auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef(), table_id);
/// Expect only needed columns from the result of DESC TABLE. NOTE 'comment' column is ignored for compatibility reasons.
Block sample_block
@ -169,7 +169,7 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables(
const auto & shards_info = cluster.getShardsInfo();
auto query = "DESC TABLE " + remote_table_id.getFullTableName();
auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef());
auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef(), remote_table_id);
new_context->setSetting("describe_extend_object_types", true);
/// Expect only needed columns from the result of DESC TABLE.

View File

@ -60,6 +60,14 @@ select * from remote('127.0.0.{1,2}', system.one) settings additional_table_filt
0
0
select * from remote('127.0.0.{1,2}', system.one) settings additional_table_filters={'system.one' : 'dummy != 0'};
select * from distr_table settings additional_table_filters={'distr_table' : 'x = 2'};
2 bb
2 bb
select * from distr_table settings additional_table_filters={'distr_table' : 'x != 2 and x != 3'};
1 a
4 dddd
1 a
4 dddd
select * from system.numbers limit 5;
0
1

View File

@ -1,3 +1,4 @@
-- Tags: distributed
drop table if exists table_1;
drop table if exists table_2;
drop table if exists v_numbers;
@ -6,6 +7,8 @@ drop table if exists mv_table;
create table table_1 (x UInt32, y String) engine = MergeTree order by x;
insert into table_1 values (1, 'a'), (2, 'bb'), (3, 'ccc'), (4, 'dddd');
CREATE TABLE distr_table (x UInt32, y String) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), 'table_1');
-- { echoOn }
select * from table_1;
@ -29,6 +32,9 @@ select x from table_1 prewhere x != 2 where x != 2 settings additional_table_fil
select * from remote('127.0.0.{1,2}', system.one) settings additional_table_filters={'system.one' : 'dummy = 0'};
select * from remote('127.0.0.{1,2}', system.one) settings additional_table_filters={'system.one' : 'dummy != 0'};
select * from distr_table settings additional_table_filters={'distr_table' : 'x = 2'};
select * from distr_table settings additional_table_filters={'distr_table' : 'x != 2 and x != 3'};
select * from system.numbers limit 5;
select * from system.numbers as t limit 5 settings additional_table_filters={'t' : 'number % 2 != 0'};
select * from system.numbers limit 5 settings additional_table_filters={'system.numbers' : 'number != 3'};

View File

@ -0,0 +1,3 @@
4 dddd
5 a
6 bb

View File

@ -0,0 +1,20 @@
-- Tags: no-parallel, distributed
create database if not exists shard_0;
create database if not exists shard_1;
drop table if exists dist_02346;
drop table if exists shard_0.data_02346;
drop table if exists shard_1.data_02346;
create table shard_0.data_02346 (x UInt32, y String) engine = MergeTree order by x settings index_granularity = 2;
insert into shard_0.data_02346 values (1, 'a'), (2, 'bb'), (3, 'ccc'), (4, 'dddd');
create table shard_1.data_02346 (x UInt32, y String) engine = MergeTree order by x settings index_granularity = 2;
insert into shard_1.data_02346 values (5, 'a'), (6, 'bb'), (7, 'ccc'), (8, 'dddd');
create table dist_02346 (x UInt32, y String) engine=Distributed('test_cluster_two_shards_different_databases', /* default_database= */ '', data_02346);
set max_rows_to_read=4;
select * from dist_02346 order by x settings additional_table_filters={'dist_02346' : 'x > 3 and x < 7'};

View File

@ -0,0 +1,30 @@
-- { echoOn }
set max_rows_to_read = 2;
select * from table_1 order by x settings additional_table_filters={'table_1' : 'x > 3'};
4 dddd
select * from table_1 order by x settings additional_table_filters={'table_1' : 'x < 3'};
1 a
2 bb
select * from table_1 order by x settings additional_table_filters={'table_1' : 'length(y) >= 3'};
3 ccc
4 dddd
select * from table_1 order by x settings additional_table_filters={'table_1' : 'length(y) < 3'};
1 a
2 bb
set max_rows_to_read = 4;
select * from distr_table order by x settings additional_table_filters={'distr_table' : 'x > 3'};
4 dddd
4 dddd
select * from distr_table order by x settings additional_table_filters={'distr_table' : 'x < 3'};
1 a
1 a
2 bb
2 bb
select * from distr_table order by x settings additional_table_filters={'distr_table' : 'length(y) > 3'};
4 dddd
4 dddd
select * from distr_table order by x settings additional_table_filters={'distr_table' : 'length(y) < 3'};
1 a
1 a
2 bb
2 bb

View File

@ -0,0 +1,24 @@
-- Tags: distributed
create table table_1 (x UInt32, y String, INDEX a (length(y)) TYPE minmax GRANULARITY 1) engine = MergeTree order by x settings index_granularity = 2;
insert into table_1 values (1, 'a'), (2, 'bb'), (3, 'ccc'), (4, 'dddd');
CREATE TABLE distr_table (x UInt32, y String) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), 'table_1');
-- { echoOn }
set max_rows_to_read = 2;
select * from table_1 order by x settings additional_table_filters={'table_1' : 'x > 3'};
select * from table_1 order by x settings additional_table_filters={'table_1' : 'x < 3'};
select * from table_1 order by x settings additional_table_filters={'table_1' : 'length(y) >= 3'};
select * from table_1 order by x settings additional_table_filters={'table_1' : 'length(y) < 3'};
set max_rows_to_read = 4;
select * from distr_table order by x settings additional_table_filters={'distr_table' : 'x > 3'};
select * from distr_table order by x settings additional_table_filters={'distr_table' : 'x < 3'};
select * from distr_table order by x settings additional_table_filters={'distr_table' : 'length(y) > 3'};
select * from distr_table order by x settings additional_table_filters={'distr_table' : 'length(y) < 3'};