ClickHouse/src/Storages/MergeTree/DataPartsExchange.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

146 lines
4.4 KiB
C++
Raw Normal View History

2016-01-28 01:00:27 +00:00
#pragma once
#include <Storages/MergeTree/MergeTreePartInfo.h>
#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>
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;
class PooledReadWriteBufferFromHTTP;
2021-05-26 20:37:44 +00:00
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;
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,
2022-10-29 14:26:34 +00:00
bool send_projections);
MergeTreeData::DataPart::Checksums sendPartFromDisk(
const MergeTreeData::DataPartPtr & part,
WriteBuffer & out,
int client_protocol_version,
bool from_remote_disk,
2022-10-29 14:26:34 +00:00
bool send_projections);
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(StorageReplicatedMergeTree & data_);
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.
2022-09-08 14:18:21 +00:00
MergeTreeData::MutableDataPartPtr fetchSelectedPart(
2020-06-26 11:30:23 +00:00
const StorageMetadataPtr & metadata_snapshot,
ContextPtr context,
2016-01-28 01:00:27 +00:00
const String & part_name,
Add zookeeper name in endpoint id (#49780) * Add zookeeper name in endpoint id When we migrate a replicated table from one zookeeper cluster to another (the reason why we migration is that zookeeper's load is too high), we will create a new table with the same zpath, but it will fail and the old table will be in trouble. Here is some infomation: 1.old table: CREATE TABLE a1 (`id` UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/default/a1/{shard}', '{replica}') ORDER BY (id); 2.new table: CREATE TABLE a2 (`id` UInt64) ENGINE = ReplicatedMergeTree('aux1:/clickhouse/tables/default/a1/{shard}', '{replica}') ORDER BY (id); 3.error info: <Error> executeQuery: Code: 220. DB::Exception: Duplicate interserver IO endpoint: DataPartsExchange:/clickhouse/tables/default/a1/01/replicas/02. (DUPLICATE_INTERSERVER_IO_ENDPOINT) <Error> InterserverIOHTTPHandler: Code: 221. DB::Exception: No interserver IO endpoint named DataPartsExchange:/clickhouse/tables/default/a1/01/replicas/02. (NO_SUCH_INTERSERVER_IO_ENDPOINT) * Revert "Add zookeeper name in endpoint id" This reverts commit 9deb75b249619b7abdd38e3949ca8b3a76c9df8e. * Add zookeeper name in endpoint id When we migrate a replicated table from one zookeeper cluster to another (the reason why we migration is that zookeeper's load is too high), we will create a new table with the same zpath, but it will fail and the old table will be in trouble. * Fix incompatible with a new setting * add a test, fix other issues * Update 02442_auxiliary_zookeeper_endpoint_id.sql * Update 02735_system_zookeeper_connection.reference * Update 02735_system_zookeeper_connection.sql * Update run.sh * Remove the 'no-fasttest' tag * Update 02442_auxiliary_zookeeper_endpoint_id.sql --------- Co-authored-by: Alexander Tokmakov <tavplubix@clickhouse.com> Co-authored-by: Alexander Tokmakov <tavplubix@gmail.com>
2023-05-25 09:50:14 +00:00
const String & zookeeper_name,
2016-01-28 01:00:27 +00:00
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:
2023-01-25 20:13:44 +00:00
using OutputBufferGetter = std::function<std::unique_ptr<WriteBufferFromFileBase>(IDataPartStorage &, const String &, size_t)>;
void downloadBaseOrProjectionPartToDisk(
const String & replica_path,
2022-10-22 22:51:59 +00:00
const MutableDataPartStoragePtr & data_part_storage,
PooledReadWriteBufferFromHTTP & in,
OutputBufferGetter output_buffer_getter,
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler,
bool sync) const;
2021-05-26 20:37:44 +00:00
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_,
DiskPtr disk,
bool to_remote_disk,
PooledReadWriteBufferFromHTTP & in,
OutputBufferGetter output_buffer_getter,
size_t projections,
ThrottlerPtr throttler,
bool sync);
2020-04-29 17:14:49 +00:00
MergeTreeData::MutableDataPartPtr downloadPartToMemory(
2022-10-29 14:26:34 +00:00
MutableDataPartStoragePtr data_part_storage,
const String & part_name,
2022-10-29 14:26:34 +00:00
const MergeTreePartInfo & part_info,
const UUID & part_uuid,
const StorageMetadataPtr & metadata_snapshot,
ContextPtr context,
PooledReadWriteBufferFromHTTP & in,
size_t projections,
2022-10-29 14:26:34 +00:00
bool is_projection,
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_,
DiskPtr disk,
PooledReadWriteBufferFromHTTP & in,
size_t projections,
MergeTreeData::DataPart::Checksums & checksums,
ThrottlerPtr throttler);
2022-02-14 09:20:27 +00:00
StorageReplicatedMergeTree & data;
2020-05-30 21:57:37 +00:00
Poco::Logger * log;
2016-01-28 01:00:27 +00:00
};
}
}