2016-01-28 01:00:27 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/DataPartsExchange.h>
|
2014-07-22 13:49:52 +00:00
|
|
|
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
2016-01-21 01:47:28 +00:00
|
|
|
|
#include <DB/Common/CurrentMetrics.h>
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
|
namespace ErrorCodes
|
|
|
|
|
{
|
|
|
|
|
extern const int ABORTED;
|
2016-01-28 01:00:27 +00:00
|
|
|
|
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
|
2016-01-11 21:46:36 +00:00
|
|
|
|
}
|
|
|
|
|
|
2016-01-28 01:00:27 +00:00
|
|
|
|
namespace DataPartsExchange
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
namespace
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
std::string getEndpointId(const std::string & node_id)
|
|
|
|
|
{
|
|
|
|
|
return "DataPartsExchange:" + node_id;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
std::string Service::getId(const std::string & node_id) const
|
|
|
|
|
{
|
|
|
|
|
return getEndpointId(node_id);
|
|
|
|
|
}
|
2016-01-11 21:46:36 +00:00
|
|
|
|
|
2016-03-01 17:47:53 +00:00
|
|
|
|
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out)
|
2014-07-22 13:49:52 +00:00
|
|
|
|
{
|
2015-12-24 21:28:18 +00:00
|
|
|
|
if (is_cancelled)
|
|
|
|
|
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
|
|
|
|
|
|
2014-07-22 13:49:52 +00:00
|
|
|
|
String part_name = params.get("part");
|
2016-01-28 01:00:27 +00:00
|
|
|
|
String shard_str = params.get("shard");
|
|
|
|
|
|
|
|
|
|
bool send_sharded_part = !shard_str.empty();
|
|
|
|
|
|
2014-07-22 13:49:52 +00:00
|
|
|
|
LOG_TRACE(log, "Sending part " << part_name);
|
|
|
|
|
|
|
|
|
|
try
|
|
|
|
|
{
|
2016-03-03 04:30:36 +00:00
|
|
|
|
auto storage_lock = owned_storage->lockStructure(false);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
2016-01-28 01:00:27 +00:00
|
|
|
|
MergeTreeData::DataPartPtr part;
|
|
|
|
|
|
|
|
|
|
if (send_sharded_part)
|
|
|
|
|
{
|
|
|
|
|
size_t shard_no = std::stoul(shard_str);
|
|
|
|
|
part = findShardedPart(part_name, shard_no);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
part = findPart(part_name);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
|
|
|
|
Poco::ScopedReadRWLock part_lock(part->columns_lock);
|
|
|
|
|
|
2016-01-21 01:47:28 +00:00
|
|
|
|
CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedSend};
|
|
|
|
|
|
2014-07-22 13:49:52 +00:00
|
|
|
|
/// Список файлов возьмем из списка контрольных сумм.
|
|
|
|
|
MergeTreeData::DataPart::Checksums checksums = part->checksums;
|
|
|
|
|
/// Добавим файлы, которых нет в списке контрольных сумм.
|
|
|
|
|
checksums.files["checksums.txt"];
|
|
|
|
|
checksums.files["columns.txt"];
|
|
|
|
|
|
|
|
|
|
MergeTreeData::DataPart::Checksums data_checksums;
|
|
|
|
|
|
|
|
|
|
writeBinary(checksums.files.size(), out);
|
|
|
|
|
for (const auto & it : checksums.files)
|
|
|
|
|
{
|
|
|
|
|
String file_name = it.first;
|
|
|
|
|
|
2016-01-28 01:00:27 +00:00
|
|
|
|
String path;
|
|
|
|
|
|
|
|
|
|
if (send_sharded_part)
|
|
|
|
|
path = data.getFullPath() + "reshard/" + shard_str + "/" + part_name + "/" + file_name;
|
|
|
|
|
else
|
|
|
|
|
path = data.getFullPath() + part_name + "/" + file_name;
|
|
|
|
|
|
2014-07-22 13:49:52 +00:00
|
|
|
|
UInt64 size = Poco::File(path).getSize();
|
|
|
|
|
|
|
|
|
|
writeStringBinary(it.first, out);
|
|
|
|
|
writeBinary(size, out);
|
|
|
|
|
|
|
|
|
|
ReadBufferFromFile file_in(path);
|
|
|
|
|
HashingWriteBuffer hashing_out(out);
|
2015-12-24 21:28:18 +00:00
|
|
|
|
copyData(file_in, hashing_out, is_cancelled);
|
|
|
|
|
|
|
|
|
|
if (is_cancelled)
|
|
|
|
|
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
|
|
|
|
if (hashing_out.count() != size)
|
|
|
|
|
throw Exception("Unexpected size of file " + path, ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART);
|
|
|
|
|
|
|
|
|
|
writeBinary(hashing_out.getHash(), out);
|
|
|
|
|
|
|
|
|
|
if (file_name != "checksums.txt" &&
|
|
|
|
|
file_name != "columns.txt")
|
2014-07-23 09:50:29 +00:00
|
|
|
|
data_checksums.addFile(file_name, hashing_out.count(), hashing_out.getHash());
|
2014-07-22 13:49:52 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-07-23 09:50:29 +00:00
|
|
|
|
part->checksums.checkEqual(data_checksums, false);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
}
|
2016-03-03 03:54:37 +00:00
|
|
|
|
catch (const Exception & e)
|
|
|
|
|
{
|
|
|
|
|
if (e.code() != ErrorCodes::ABORTED)
|
2016-03-03 04:30:36 +00:00
|
|
|
|
typeid_cast<StorageReplicatedMergeTree &>(*owned_storage).enqueuePartForCheck(part_name);
|
|
|
|
|
throw;
|
2016-03-03 03:54:37 +00:00
|
|
|
|
}
|
2014-07-22 13:49:52 +00:00
|
|
|
|
catch (...)
|
|
|
|
|
{
|
2016-03-03 04:30:36 +00:00
|
|
|
|
typeid_cast<StorageReplicatedMergeTree &>(*owned_storage).enqueuePartForCheck(part_name);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
throw;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-01-28 01:00:27 +00:00
|
|
|
|
MergeTreeData::DataPartPtr Service::findPart(const String & name)
|
|
|
|
|
{
|
|
|
|
|
MergeTreeData::DataPartPtr part = data.getPartIfExists(name);
|
|
|
|
|
if (part)
|
|
|
|
|
return part;
|
|
|
|
|
throw Exception("No part " + name + " in table");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
MergeTreeData::DataPartPtr Service::findShardedPart(const String & name, size_t shard_no)
|
|
|
|
|
{
|
|
|
|
|
MergeTreeData::DataPartPtr part = data.getShardedPartIfExists(name, shard_no);
|
|
|
|
|
if (part)
|
|
|
|
|
return part;
|
|
|
|
|
throw Exception("No part " + name + " in table");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
|
2014-07-22 13:49:52 +00:00
|
|
|
|
const String & part_name,
|
|
|
|
|
const String & replica_path,
|
|
|
|
|
const String & host,
|
2014-10-13 17:28:59 +00:00
|
|
|
|
int port,
|
|
|
|
|
bool to_detached)
|
2016-01-28 01:00:27 +00:00
|
|
|
|
{
|
|
|
|
|
return fetchPartImpl(part_name, replica_path, host, port, "", to_detached);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
MergeTreeData::MutableDataPartPtr Fetcher::fetchShardedPart(
|
|
|
|
|
const InterserverIOEndpointLocation & location,
|
|
|
|
|
const String & part_name,
|
|
|
|
|
size_t shard_no)
|
|
|
|
|
{
|
|
|
|
|
return fetchPartImpl(part_name, location.name, location.host, location.port, toString(shard_no), true);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
MergeTreeData::MutableDataPartPtr Fetcher::fetchPartImpl(
|
|
|
|
|
const String & part_name,
|
|
|
|
|
const String & replica_path,
|
|
|
|
|
const String & host,
|
|
|
|
|
int port,
|
|
|
|
|
const String & shard_no,
|
|
|
|
|
bool to_detached)
|
2014-07-22 13:49:52 +00:00
|
|
|
|
{
|
2014-11-30 07:00:05 +00:00
|
|
|
|
ReadBufferFromHTTP::Params params =
|
|
|
|
|
{
|
2016-01-28 01:00:27 +00:00
|
|
|
|
{"endpoint", getEndpointId(replica_path)},
|
2014-11-30 07:00:05 +00:00
|
|
|
|
{"part", part_name},
|
2016-01-28 01:00:27 +00:00
|
|
|
|
{"shard", shard_no},
|
2014-11-30 07:00:05 +00:00
|
|
|
|
{"compress", "false"}
|
|
|
|
|
};
|
|
|
|
|
|
2014-07-22 13:49:52 +00:00
|
|
|
|
ReadBufferFromHTTP in(host, port, params);
|
|
|
|
|
|
2014-10-13 18:01:58 +00:00
|
|
|
|
String full_part_name = String(to_detached ? "detached/" : "") + "tmp_" + part_name;
|
|
|
|
|
String part_path = data.getFullPath() + full_part_name + "/";
|
2014-08-14 01:37:46 +00:00
|
|
|
|
Poco::File part_file(part_path);
|
|
|
|
|
|
|
|
|
|
if (part_file.exists())
|
|
|
|
|
{
|
|
|
|
|
LOG_ERROR(log, "Directory " + part_path + " already exists. Removing.");
|
|
|
|
|
part_file.remove(true);
|
|
|
|
|
}
|
|
|
|
|
|
2016-01-21 01:47:28 +00:00
|
|
|
|
CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedFetch};
|
|
|
|
|
|
2014-08-14 01:37:46 +00:00
|
|
|
|
part_file.createDirectory();
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
|
|
|
|
MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared<MergeTreeData::DataPart>(data);
|
2014-10-13 18:01:58 +00:00
|
|
|
|
new_data_part->name = full_part_name;
|
2014-07-22 13:49:52 +00:00
|
|
|
|
new_data_part->is_temp = true;
|
|
|
|
|
|
|
|
|
|
size_t files;
|
|
|
|
|
readBinary(files, in);
|
|
|
|
|
MergeTreeData::DataPart::Checksums checksums;
|
|
|
|
|
for (size_t i = 0; i < files; ++i)
|
|
|
|
|
{
|
|
|
|
|
String file_name;
|
|
|
|
|
UInt64 file_size;
|
|
|
|
|
|
|
|
|
|
readStringBinary(file_name, in);
|
|
|
|
|
readBinary(file_size, in);
|
|
|
|
|
|
|
|
|
|
WriteBufferFromFile file_out(part_path + file_name);
|
|
|
|
|
HashingWriteBuffer hashing_out(file_out);
|
2015-12-24 21:28:18 +00:00
|
|
|
|
copyData(in, hashing_out, file_size, is_cancelled);
|
|
|
|
|
|
|
|
|
|
if (is_cancelled)
|
|
|
|
|
{
|
|
|
|
|
/// NOTE Флаг is_cancelled также имеет смысл проверять при каждом чтении по сети, осуществляя poll с не очень большим таймаутом.
|
|
|
|
|
/// А сейчас мы проверяем его только между прочитанными кусками (в функции copyData).
|
|
|
|
|
part_file.remove(true);
|
|
|
|
|
throw Exception("Fetching of part was cancelled", ErrorCodes::ABORTED);
|
|
|
|
|
}
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
|
|
|
|
uint128 expected_hash;
|
|
|
|
|
readBinary(expected_hash, in);
|
|
|
|
|
|
|
|
|
|
if (expected_hash != hashing_out.getHash())
|
|
|
|
|
throw Exception("Checksum mismatch for file " + part_path + file_name + " transferred from " + replica_path);
|
|
|
|
|
|
|
|
|
|
if (file_name != "checksums.txt" &&
|
|
|
|
|
file_name != "columns.txt")
|
|
|
|
|
checksums.addFile(file_name, file_size, expected_hash);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
assertEOF(in);
|
|
|
|
|
|
|
|
|
|
ActiveDataPartSet::parsePartName(part_name, *new_data_part);
|
|
|
|
|
new_data_part->modification_time = time(0);
|
2014-08-08 08:28:13 +00:00
|
|
|
|
new_data_part->loadColumns(true);
|
|
|
|
|
new_data_part->loadChecksums(true);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
new_data_part->loadIndex();
|
2016-01-28 01:00:27 +00:00
|
|
|
|
new_data_part->is_sharded = false;
|
2014-07-22 13:49:52 +00:00
|
|
|
|
new_data_part->checksums.checkEqual(checksums, false);
|
|
|
|
|
|
|
|
|
|
return new_data_part;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
2016-01-28 01:00:27 +00:00
|
|
|
|
|
|
|
|
|
}
|