ClickHouse/src/DataStreams/RemoteBlockInputStream.h

81 lines
3.0 KiB
C++
Raw Normal View History

2012-05-17 19:15:53 +00:00
#pragma once
#include <optional>
2015-09-29 19:19:54 +00:00
#include <common/logger_useful.h>
#include <DataStreams/IBlockInputStream.h>
#include <Common/Throttler.h>
#include <Client/ConnectionPool.h>
#include <Client/MultiplexedConnections.h>
#include <Interpreters/Cluster.h>
2020-06-02 16:27:05 +00:00
#include <DataStreams/RemoteQueryExecutor.h>
2012-05-17 19:15:53 +00:00
namespace DB
{
class Context;
/** This class allows one to launch queries on remote replicas of one shard and get results
2012-05-17 19:15:53 +00:00
*/
class RemoteBlockInputStream : public IBlockInputStream
2012-05-17 19:15:53 +00:00
{
public:
/// Takes already set connection.
RemoteBlockInputStream(
Connection & connection,
const String & query_, const Block & header_, const Context & context_,
2019-10-19 20:36:35 +00:00
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
/// Accepts several connections already taken from pool.
RemoteBlockInputStream(
std::vector<IConnectionPool::Entry> && connections,
const String & query_, const Block & header_, const Context & context_,
2019-10-19 20:36:35 +00:00
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
/// Takes a pool and gets one or several connections from it.
RemoteBlockInputStream(
const ConnectionPoolWithFailoverPtr & pool,
const String & query_, const Block & header_, const Context & context_,
2019-10-19 20:36:35 +00:00
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
/// Set the query_id. For now, used by performance test to later find the query
2020-06-02 16:27:05 +00:00
/// in the server query_log. Must be called before sending the query to the server.
void setQueryId(const std::string & query_id) { query_executor.setQueryId(query_id); }
/// Specify how we allocate connections on a shard.
2020-06-02 16:27:05 +00:00
void setPoolMode(PoolMode pool_mode) { query_executor.setPoolMode(pool_mode); }
2020-06-02 16:27:05 +00:00
void setMainTable(StorageID main_table_) { query_executor.setMainTable(std::move(main_table_)); }
/// Sends query (initiates calculation) before read()
void readPrefix() override;
2020-06-02 16:27:05 +00:00
/// Prevent default progress notification because progress' callback is called by its own.
2017-12-01 18:36:55 +00:00
void progress(const Progress & /*value*/) override {}
void cancel(bool kill) override;
String getName() const override { return "Remote"; }
2020-06-02 16:27:05 +00:00
Block getHeader() const override { return query_executor.getHeader(); }
2020-06-02 16:37:30 +00:00
Block getTotals() override { return query_executor.getTotals(); }
Block getExtremes() override { return query_executor.getExtremes(); }
2012-10-20 02:10:47 +00:00
protected:
Block readImpl() override;
void readSuffixImpl() override;
2013-09-13 20:33:09 +00:00
private:
2020-06-02 16:27:05 +00:00
RemoteQueryExecutor query_executor;
2020-05-30 21:57:37 +00:00
Poco::Logger * log = &Poco::Logger::get("RemoteBlockInputStream");
2020-06-02 16:27:05 +00:00
void init();
2012-05-17 19:15:53 +00:00
};
}