mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
ReplicatedMergeTree: Max streams to send data [#CLICKHOUSE-2878] (#656)
* ReplicatedMergeTree: Max streams to send data [#CLICKHOUSE-2878] * fix * better messages on client * change code to 429 TOO_MANY_REQUESTS * wip * better message * Update InterserverIOHandler.h * Update InterserverIOHTTPHandler.cpp * Update StorageReplicatedMergeTree.cpp
This commit is contained in:
parent
13e9b5c949
commit
1bc311eefd
@ -366,6 +366,7 @@ namespace ErrorCodes
|
||||
extern const int SEEK_POSITION_OUT_OF_BOUND = 361;
|
||||
extern const int CURRENT_WRITE_BUFFER_IS_EXHAUSTED = 362;
|
||||
extern const int CANNOT_CREATE_IO_BUFFER = 363;
|
||||
extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS = 364;
|
||||
|
||||
extern const int KEEPER_EXCEPTION = 999;
|
||||
extern const int POCO_EXCEPTION = 1000;
|
||||
|
@ -18,9 +18,9 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
|
||||
extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
|
||||
}
|
||||
|
||||
|
||||
static Poco::Net::IPAddress resolveHostImpl(const String & host)
|
||||
{
|
||||
return Poco::Net::DNS::resolveOne(host);
|
||||
@ -91,9 +91,9 @@ ReadBufferFromHTTP::ReadBufferFromHTTP(
|
||||
{
|
||||
std::stringstream error_message;
|
||||
error_message << "Received error from remote server " << uri.str() << ". HTTP status code: "
|
||||
<< status << ", body: " << istr->rdbuf();
|
||||
<< status << " " << response.getReason() << ", body: " << istr->rdbuf();
|
||||
|
||||
throw Exception(error_message.str(), ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER);
|
||||
throw Exception(error_message.str(), status == HTTP_TOO_MANY_REQUESTS ? ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS : ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER);
|
||||
}
|
||||
|
||||
impl = std::make_unique<ReadBufferFromIStream>(*istr, buffer_size_);
|
||||
|
@ -14,6 +14,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
const int HTTP_TOO_MANY_REQUESTS = 429;
|
||||
|
||||
/** Perform HTTP-request and provide response to read.
|
||||
*/
|
||||
class ReadBufferFromHTTP : public ReadBuffer
|
||||
|
@ -21,6 +21,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
|
||||
extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
|
||||
}
|
||||
|
||||
static Poco::Net::IPAddress resolveHostImpl(const String & host)
|
||||
@ -86,9 +87,9 @@ ReadWriteBufferFromHTTP::ReadWriteBufferFromHTTP(
|
||||
{
|
||||
std::stringstream error_message;
|
||||
error_message << "Received error from remote server " << uri.toString() << ". HTTP status code: "
|
||||
<< status << ", body: " << istr->rdbuf();
|
||||
<< status << " " << response.getReason() << ", body: " << istr->rdbuf();
|
||||
|
||||
throw Exception(error_message.str(), ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER);
|
||||
throw Exception(error_message.str(), status == HTTP_TOO_MANY_REQUESTS ? ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS : ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER);
|
||||
}
|
||||
|
||||
impl = std::make_unique<ReadBufferFromIStream>(*istr, buffer_size_);
|
||||
|
@ -44,7 +44,8 @@ void WriteBufferFromHTTPServerResponse::finishSendHeaders()
|
||||
|
||||
#if POCO_CLICKHOUSE_PATCH
|
||||
/// Send end of headers delimiter.
|
||||
*response_header_ostr << "\r\n" << std::flush;
|
||||
if (response_header_ostr)
|
||||
*response_header_ostr << "\r\n" << std::flush;
|
||||
#else
|
||||
/// Newline autosent by response.send()
|
||||
/// if nothing to send in body:
|
||||
|
@ -11,6 +11,8 @@
|
||||
#include <atomic>
|
||||
#include <Poco/Net/HTMLForm.h>
|
||||
|
||||
namespace Poco { namespace Net { class HTTPServerResponse; } }
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -64,7 +66,7 @@ class InterserverIOEndpoint
|
||||
{
|
||||
public:
|
||||
virtual std::string getId(const std::string & path) const = 0;
|
||||
virtual void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out) = 0;
|
||||
virtual void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response) = 0;
|
||||
virtual ~InterserverIOEndpoint() {}
|
||||
|
||||
void cancel() { is_cancelled = true; }
|
||||
|
@ -14,9 +14,9 @@ namespace ErrorCodes
|
||||
extern const int POCO_EXCEPTION;
|
||||
extern const int STD_EXCEPTION;
|
||||
extern const int UNKNOWN_EXCEPTION;
|
||||
extern const int TOO_MUCH_SIMULTANEOUS_QUERIES;
|
||||
}
|
||||
|
||||
|
||||
void InterserverIOHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
HTMLForm params(request);
|
||||
@ -37,11 +37,11 @@ void InterserverIOHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & reque
|
||||
if (compress)
|
||||
{
|
||||
CompressedWriteBuffer compressed_out(out);
|
||||
endpoint->processQuery(params, body, compressed_out);
|
||||
endpoint->processQuery(params, body, compressed_out, response);
|
||||
}
|
||||
else
|
||||
{
|
||||
endpoint->processQuery(params, body, out);
|
||||
endpoint->processQuery(params, body, out, response);
|
||||
}
|
||||
|
||||
out.finalize();
|
||||
@ -61,6 +61,14 @@ void InterserverIOHTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & requ
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
|
||||
if (e.code() == ErrorCodes::TOO_MUCH_SIMULTANEOUS_QUERIES)
|
||||
{
|
||||
if (!response.sent())
|
||||
response.send();
|
||||
return;
|
||||
}
|
||||
|
||||
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR);
|
||||
|
||||
/// Sending to remote server was cancelled due to server shutdown or drop table.
|
||||
|
@ -4,6 +4,8 @@
|
||||
#include <Common/NetException.h>
|
||||
#include <IO/ReadBufferFromHTTP.h>
|
||||
#include <Poco/File.h>
|
||||
#include <ext/scope_guard.hpp>
|
||||
#include <Poco/Net/HTTPServerResponse.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
@ -19,6 +21,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int ABORTED;
|
||||
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
|
||||
extern const int TOO_MUCH_SIMULTANEOUS_QUERIES;
|
||||
}
|
||||
|
||||
namespace DataPartsExchange
|
||||
@ -39,7 +42,7 @@ std::string Service::getId(const std::string & node_id) const
|
||||
return getEndpointId(node_id);
|
||||
}
|
||||
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out)
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
if (is_cancelled)
|
||||
throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED);
|
||||
@ -49,6 +52,24 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body
|
||||
|
||||
bool send_sharded_part = !shard_str.empty();
|
||||
|
||||
static std::atomic_uint total_sends {0};
|
||||
|
||||
if (total_sends >= data.settings.replicated_max_parallel_sends
|
||||
|| data.current_table_sends >= data.settings.replicated_max_parallel_sends_for_table)
|
||||
{
|
||||
response.setStatus(std::to_string(HTTP_TOO_MANY_REQUESTS));
|
||||
response.setReason("Too many concurrent fetches, try again later");
|
||||
response.set("Retry-After", "10");
|
||||
response.setChunkedTransferEncoding(false);
|
||||
return;
|
||||
}
|
||||
++total_sends;
|
||||
SCOPE_EXIT({--total_sends;});
|
||||
|
||||
++data.current_table_sends;
|
||||
SCOPE_EXIT({--data.current_table_sends;});
|
||||
|
||||
|
||||
LOG_TRACE(log, "Sending part " << part_name);
|
||||
|
||||
try
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
Service & operator=(const Service &) = delete;
|
||||
|
||||
std::string getId(const std::string & node_id) const override;
|
||||
void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out) override;
|
||||
void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response) override;
|
||||
|
||||
private:
|
||||
MergeTreeData::DataPartPtr findPart(const String & name);
|
||||
|
@ -472,6 +472,9 @@ public:
|
||||
Block primary_key_sample;
|
||||
DataTypes primary_key_data_types;
|
||||
|
||||
/// Limiting parallel sends per one table, used in DataPartsExchange
|
||||
std::atomic_uint current_table_sends {0};
|
||||
|
||||
private:
|
||||
friend struct MergeTreeDataPart;
|
||||
friend class StorageMergeTree;
|
||||
|
@ -78,6 +78,9 @@ struct MergeTreeSettings
|
||||
/// Limit parallel fetches
|
||||
size_t replicated_max_parallel_fetches = 4;
|
||||
size_t replicated_max_parallel_fetches_for_table = 2;
|
||||
/// Limit parallel sends
|
||||
size_t replicated_max_parallel_sends = 4;
|
||||
size_t replicated_max_parallel_sends_for_table = 2;
|
||||
|
||||
/// If ration of wrong parts to total number of parts is less than this - allow to start anyway.
|
||||
double replicated_max_ratio_of_wrong_parts = 0.05;
|
||||
@ -138,6 +141,8 @@ struct MergeTreeSettings
|
||||
SET_SIZE_T(replicated_max_missing_active_parts);
|
||||
SET_SIZE_T(replicated_max_parallel_fetches);
|
||||
SET_SIZE_T(replicated_max_parallel_fetches_for_table);
|
||||
SET_SIZE_T(replicated_max_parallel_sends);
|
||||
SET_SIZE_T(replicated_max_parallel_sends_for_table);
|
||||
SET_DOUBLE(replicated_max_ratio_of_wrong_parts);
|
||||
SET_SIZE_T(zookeeper_session_expiration_check_period);
|
||||
SET_SIZE_T(check_delay_period);
|
||||
|
@ -36,7 +36,7 @@ std::string Service::getId(const std::string & node_id) const
|
||||
return getEndpointId(node_id);
|
||||
}
|
||||
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out)
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
if (is_cancelled)
|
||||
throw Exception{"RemoteDiskSpaceMonitor service terminated", ErrorCodes::ABORTED};
|
||||
|
@ -21,7 +21,7 @@ public:
|
||||
Service(const Service &) = delete;
|
||||
Service & operator=(const Service &) = delete;
|
||||
std::string getId(const std::string & node_id) const override;
|
||||
void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out) override;
|
||||
void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response) override;
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
|
@ -33,7 +33,7 @@ std::string Service::getId(const std::string & node_id) const
|
||||
return getEndpointId(node_id);
|
||||
}
|
||||
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out)
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
auto part_name = params.get("part");
|
||||
auto hash = params.get("hash");
|
||||
|
@ -27,7 +27,7 @@ public:
|
||||
Service(const Service &) = delete;
|
||||
Service & operator=(const Service &) = delete;
|
||||
std::string getId(const std::string & node_id) const override;
|
||||
void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out) override;
|
||||
void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response) override;
|
||||
|
||||
private:
|
||||
StoragePtr owned_storage;
|
||||
|
@ -35,7 +35,7 @@ std::string Service::getId(const std::string & node_id) const
|
||||
return getEndpointId(node_id);
|
||||
}
|
||||
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out)
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
if (is_cancelled)
|
||||
throw Exception{"RemoteQueryExecutor service terminated", ErrorCodes::ABORTED};
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
Service(const Service &) = delete;
|
||||
Service & operator=(const Service &) = delete;
|
||||
std::string getId(const std::string & node_id) const override;
|
||||
void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out) override;
|
||||
void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response) override;
|
||||
|
||||
private:
|
||||
Context & context;
|
||||
|
@ -45,7 +45,7 @@ std::string Service::getId(const std::string & node_id) const
|
||||
return getEndpointId(node_id);
|
||||
}
|
||||
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out)
|
||||
void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
std::string part_name = params.get("path");
|
||||
std::string replica_path = params.get("endpoint");
|
||||
|
@ -23,7 +23,7 @@ public:
|
||||
Service(const Service &) = delete;
|
||||
Service & operator=(const Service &) = delete;
|
||||
std::string getId(const std::string & node_id) const override;
|
||||
void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out) override;
|
||||
void processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body, WriteBuffer & out, Poco::Net::HTTPServerResponse & response) override;
|
||||
|
||||
private:
|
||||
StoragePtr owned_storage;
|
||||
|
@ -103,6 +103,7 @@ namespace ErrorCodes
|
||||
extern const int UNFINISHED;
|
||||
extern const int METADATA_MISMATCH;
|
||||
extern const int RESHARDING_NULLABLE_SHARDING_KEY;
|
||||
extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
|
||||
}
|
||||
|
||||
|
||||
@ -1417,8 +1418,21 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry)
|
||||
}
|
||||
}
|
||||
|
||||
if (!fetchPart(covering_part, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
|
||||
return false;
|
||||
try
|
||||
{
|
||||
if (!fetchPart(covering_part, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
|
||||
return false;
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// No stacktrace, just log message
|
||||
if (e.code() == ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS)
|
||||
{
|
||||
LOG_INFO(log, "Too busy replica. Will try later. " << e.message());
|
||||
return false;
|
||||
}
|
||||
throw;
|
||||
}
|
||||
|
||||
if (entry.type == LogEntry::MERGE_PARTS)
|
||||
ProfileEvents::increment(ProfileEvents::ReplicatedPartFetchesOfMerged);
|
||||
@ -3444,7 +3458,7 @@ void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const S
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
if (e.code() != ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER)
|
||||
if (e.code() != ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER && e.code() != ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS)
|
||||
throw;
|
||||
|
||||
LOG_INFO(log, e.displayText());
|
||||
|
Loading…
Reference in New Issue
Block a user