ClickHouse/src/Processors/QueryPlan/ReadFromRemote.h

115 lines
3.2 KiB
C++
Raw Normal View History

2021-07-15 16:15:16 +00:00
#pragma once
#include <Processors/QueryPlan/ISourceStep.h>
#include <Core/QueryProcessingStage.h>
#include <Client/IConnections.h>
2021-07-15 16:15:16 +00:00
#include <Storages/IStorage_fwd.h>
#include <Interpreters/StorageID.h>
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
2021-07-15 16:15:16 +00:00
namespace DB
{
class ConnectionPoolWithFailover;
using ConnectionPoolWithFailoverPtr = std::shared_ptr<ConnectionPoolWithFailover>;
class Throttler;
using ThrottlerPtr = std::shared_ptr<Throttler>;
2021-07-16 10:56:13 +00:00
/// Reading step from remote servers.
/// Unite query results from several shards.
2021-07-15 16:15:16 +00:00
class ReadFromRemote final : public ISourceStep
{
public:
ReadFromRemote(
ClusterProxy::SelectStreamFactory::Shards shards_,
2021-07-15 16:15:16 +00:00
Block header_,
QueryProcessingStage::Enum stage_,
StorageID main_table_,
ASTPtr table_func_ptr_,
ContextPtr context_,
ThrottlerPtr throttler_,
Scalars scalars_,
Tables external_tables_,
Poco::Logger * log_,
2022-05-31 14:43:38 +00:00
UInt32 shard_count_,
std::shared_ptr<const StorageLimitsList> storage_limits_);
2021-07-15 16:15:16 +00:00
String getName() const override { return "ReadFromRemote"; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
2021-07-15 16:15:16 +00:00
private:
enum class Mode
{
PerReplica,
PerShard
};
ClusterProxy::SelectStreamFactory::Shards shards;
2021-07-15 16:15:16 +00:00
QueryProcessingStage::Enum stage;
StorageID main_table;
ASTPtr table_func_ptr;
ContextPtr context;
ThrottlerPtr throttler;
Scalars scalars;
Tables external_tables;
2022-05-31 14:43:38 +00:00
std::shared_ptr<const StorageLimitsList> storage_limits;
2021-07-15 16:15:16 +00:00
Poco::Logger * log;
UInt32 shard_count;
void addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard);
void addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard);
};
class ReadFromParallelRemoteReplicasStep : public ISourceStep
{
public:
ReadFromParallelRemoteReplicasStep(
ParallelReplicasReadingCoordinatorPtr coordinator_,
ClusterProxy::SelectStreamFactory::Shard shard,
Block header_,
QueryProcessingStage::Enum stage_,
StorageID main_table_,
ASTPtr table_func_ptr_,
ContextPtr context_,
ThrottlerPtr throttler_,
Scalars scalars_,
Tables external_tables_,
Poco::Logger * log_,
2022-06-02 19:38:47 +00:00
std::shared_ptr<const StorageLimitsList> storage_limits_);
String getName() const override { return "ReadFromRemoteParallelReplicas"; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
private:
void addPipeForSingeReplica(Pipes & pipes, std::shared_ptr<ConnectionPoolWithFailover> pool, IConnections::ReplicaInfo replica_info);
ParallelReplicasReadingCoordinatorPtr coordinator;
ClusterProxy::SelectStreamFactory::Shard shard;
QueryProcessingStage::Enum stage;
StorageID main_table;
ASTPtr table_func_ptr;
ContextPtr context;
ThrottlerPtr throttler;
Scalars scalars;
Tables external_tables;
2022-06-02 19:38:47 +00:00
std::shared_ptr<const StorageLimitsList> storage_limits;
Poco::Logger * log;
2021-07-15 16:15:16 +00:00
};
}