mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-13 09:52:38 +00:00
Support parallel replicas with the analyzer
This commit is contained in:
parent
eb5985e5fc
commit
2f08b6738f
@ -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(
|
||||
|
@ -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`
|
@ -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;
|
Loading…
Reference in New Issue
Block a user