Support parallel replicas with the analyzer

This commit is contained in:
Raúl Marín 2023-06-01 18:34:00 +02:00
parent eb5985e5fc
commit 2f08b6738f
3 changed files with 75 additions and 4 deletions

View File

@ -77,16 +77,17 @@
#include <IO/Operators.h>
#include <IO/ConnectionTimeouts.h>
#include <Interpreters/ClusterProxy/executeQuery.h>
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
#include <Interpreters/ClusterProxy/executeQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLTask.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/InterserverCredentials.h>
#include <Interpreters/JoinedTables.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Interpreters/JoinedTables.h>
#include <Backups/BackupEntriesCollector.h>
@ -4707,8 +4708,14 @@ void StorageReplicatedMergeTree::read(
auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas);
Block header =
InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
Block header;
if (local_context->getSettingsRef().allow_experimental_analyzer)
header = InterpreterSelectQueryAnalyzer::getSampleBlock(
modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze());
else
header
= InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
ClusterProxy::SelectStreamFactory select_stream_factory =
ClusterProxy::SelectStreamFactory(

View File

@ -0,0 +1,12 @@
-8888150036649430454
-2788931093724180887
-75175454385331084
368066018677693974
821735343441964030
2804162938822577320
4357435422797280898
5935810273536892891
7885388429666205427
8124171311239967992
1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n allow_experimental_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1,\n use_hedged_requests = 0;
0 2 SELECT `default`.`join_inner_table__fuzz_146_replicated`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated`

View File

@ -0,0 +1,52 @@
-- Tags: zookeeper
CREATE TABLE join_inner_table__fuzz_146_replicated
(
`id` UUID,
`key` String,
`number` Int64,
`value1` String,
`value2` String,
`time` Nullable(Int64)
)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/join_inner_table__fuzz_146_replicated', '{replica}')
ORDER BY (id, number, key)
SETTINGS index_granularity = 8192;
INSERT INTO join_inner_table__fuzz_146_replicated
SELECT CAST('833c9e22-c245-4eb5-8745-117a9a1f26b1', 'UUID') AS id, CAST(rowNumberInAllBlocks(), 'String') AS key, *
FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 10;
-- Simple query with analyzer and pure parallel replicas
SELECT number
FROM join_inner_table__fuzz_146_replicated
SETTINGS
allow_experimental_analyzer = 1,
max_parallel_replicas = 2,
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost',
allow_experimental_parallel_reading_from_replicas = 1,
use_hedged_requests = 0;
SYSTEM FLUSH LOGS;
-- There should be 2 different queries
-- The initial query
-- The query sent to each replica (which should appear 2 times as we are setting max_parallel_replicas to 2)
SELECT
is_initial_query,
count() as c, query,
FROM system.query_log
WHERE
event_date >= yesterday()
AND type = 'QueryFinish'
AND initial_query_id =
(
SELECT query_id
FROM system.query_log
WHERE
current_database = currentDatabase()
AND event_date >= yesterday()
AND type = 'QueryFinish'
AND query LIKE '-- Simple query with analyzer and pure parallel replicas%'
)
GROUP BY is_initial_query, query
ORDER BY is_initial_query DESC, c, query;