ClickHouse/src/Storages/MergeTree/DataPartsExchange.h

139 lines
4.3 KiB
C++
Raw Normal View History

2016-01-28 01:00:27 +00:00
#pragma once
#include <Interpreters/InterserverIOHandler.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/IStorage_fwd.h>
#include <IO/HashingWriteBuffer.h>
#include <IO/copyData.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/ReadWriteBufferFromHTTP.h>
2021-05-26 20:37:44 +00:00
#include <Common/Throttler.h>
2016-01-28 01:00:27 +00:00
namespace zkutil
{
class ZooKeeper;
using ZooKeeperPtr = std::shared_ptr<ZooKeeper>;
}
2016-01-28 01:00:27 +00:00
namespace DB
{
2021-05-26 20:37:44 +00:00
class StorageReplicatedMergeTree;
2016-01-28 01:00:27 +00:00
namespace DataPartsExchange
{
2021-05-26 20:37:44 +00:00
/** Service for sending parts from the table *ReplicatedMergeTree.
2016-01-28 01:00:27 +00:00
*/
class Service final : public InterserverIOEndpoint
{
public:
2021-05-26 20:37:44 +00:00
explicit Service(StorageReplicatedMergeTree & data_);
2016-01-28 01:00:27 +00:00
Service(const Service &) = delete;
Service & operator=(const Service &) = delete;
2016-01-28 01:00:27 +00:00
std::string getId(const std::string & node_id) const override;
void processQuery(const HTMLForm & params, ReadBuffer & body, WriteBuffer & out, HTTPServerResponse & response) override;
2016-01-28 01:00:27 +00:00
private:
MergeTreeData::DataPartPtr findPart(const String & name);
void sendPartFromMemory(
const MergeTreeData::DataPartPtr & part,
WriteBuffer & out,
const std::map<String, std::shared_ptr<IMergeTreeDataPart>> & projections = {});
MergeTreeData::DataPart::Checksums sendPartFromDisk(
const MergeTreeData::DataPartPtr & part,
WriteBuffer & out,
int client_protocol_version,
const std::map<String, std::shared_ptr<IMergeTreeDataPart>> & projections = {});
2021-06-24 08:25:05 +00:00
void sendPartFromDiskRemoteMeta(const MergeTreeData::DataPartPtr & part, WriteBuffer & out);
2016-01-28 01:00:27 +00:00
2020-01-14 14:27:48 +00:00
/// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish,
/// so Service will never access dangling reference to storage
2021-05-26 20:37:44 +00:00
StorageReplicatedMergeTree & data;
2020-05-30 21:57:37 +00:00
Poco::Logger * log;
2016-01-28 01:00:27 +00:00
};
2017-04-16 15:00:33 +00:00
/** Client for getting the parts from the table *MergeTree.
2016-01-28 01:00:27 +00:00
*/
class Fetcher final : private boost::noncopyable
2016-01-28 01:00:27 +00:00
{
public:
explicit Fetcher(MergeTreeData & data_) : data(data_), log(&Poco::Logger::get("Fetcher")) {}
2016-01-28 01:00:27 +00:00
2017-04-16 15:00:33 +00:00
/// Downloads a part to tmp_directory. If to_detached - downloads to the `detached` directory.
MergeTreeData::MutableDataPartPtr fetchPart(
2020-06-26 11:30:23 +00:00
const StorageMetadataPtr & metadata_snapshot,
ContextPtr context,
const String & part_name,
const String & replica_path,
const String & host,
int port,
const ConnectionTimeouts & timeouts,
const String & user,
const String & password,
const String & interserver_scheme,
2021-05-26 20:37:44 +00:00
ThrottlerPtr throttler,
bool to_detached = false,
const String & tmp_prefix_ = "",
std::optional<CurrentlySubmergingEmergingTagger> * tagger_ptr = nullptr,
2021-06-24 08:25:05 +00:00
bool try_zero_copy = true,
2021-07-05 03:32:56 +00:00
DiskPtr dest_disk = nullptr);
2016-01-28 01:00:27 +00:00
/// You need to stop the data transfer.
ActionBlocker blocker;
2016-01-28 01:00:27 +00:00
private:
void downloadBaseOrProjectionPartToDisk(
const String & replica_path,
const String & part_download_path,
bool sync,
DiskPtr disk,
PooledReadWriteBufferFromHTTP & in,
2021-05-26 20:37:44 +00:00
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler) const;
2020-04-29 17:14:49 +00:00
MergeTreeData::MutableDataPartPtr downloadPartToDisk(
const String & part_name,
const String & replica_path,
bool to_detached,
const String & tmp_prefix_,
bool sync,
DiskPtr disk,
PooledReadWriteBufferFromHTTP & in,
size_t projections,
2021-05-26 20:37:44 +00:00
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler);
2020-04-29 17:14:49 +00:00
MergeTreeData::MutableDataPartPtr downloadPartToMemory(
const String & part_name,
2020-10-29 16:18:25 +00:00
const UUID & part_uuid,
2020-06-26 11:30:23 +00:00
const StorageMetadataPtr & metadata_snapshot,
ContextPtr context,
2021-07-05 03:32:56 +00:00
DiskPtr disk,
PooledReadWriteBufferFromHTTP & in,
2021-05-26 20:37:44 +00:00
size_t projections,
ThrottlerPtr throttler);
2020-04-29 17:14:49 +00:00
2021-06-24 08:25:05 +00:00
MergeTreeData::MutableDataPartPtr downloadPartToDiskRemoteMeta(
const String & part_name,
const String & replica_path,
bool to_detached,
const String & tmp_prefix_,
2021-07-05 03:32:56 +00:00
DiskPtr disk,
2021-05-26 20:37:44 +00:00
PooledReadWriteBufferFromHTTP & in,
ThrottlerPtr throttler);
MergeTreeData & data;
2020-05-30 21:57:37 +00:00
Poco::Logger * log;
2016-01-28 01:00:27 +00:00
};
}
}