2017-04-01 09:19:00 +00:00
|
|
|
#include <Client/MultiplexedConnections.h>
|
2015-11-06 17:44:01 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-12 02:21:15 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int MISMATCH_REPLICAS_DATA_SOURCES;
|
|
|
|
extern const int NO_AVAILABLE_REPLICA;
|
|
|
|
extern const int TIMEOUT_EXCEEDED;
|
2016-01-12 02:21:15 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
MultiplexedConnections::MultiplexedConnections(Connection & connection, const Settings & settings_, const ThrottlerPtr & throttler)
|
|
|
|
: settings(settings_)
|
2015-11-06 17:44:01 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
connection.setThrottler(throttler);
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ReplicaState replica_state;
|
2017-08-02 13:05:01 +00:00
|
|
|
replica_state.connection = &connection;
|
|
|
|
replica_states.push_back(replica_state);
|
2017-09-28 19:43:31 +00:00
|
|
|
fd_to_replica_state_idx.emplace(connection.socket->impl()->sockfd(), 0);
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
active_connection_count = 1;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
2017-04-17 16:16:04 +00:00
|
|
|
MultiplexedConnections::MultiplexedConnections(
|
2017-07-28 19:34:25 +00:00
|
|
|
std::vector<IConnectionPool::Entry> && connections,
|
2017-08-02 13:05:01 +00:00
|
|
|
const Settings & settings_, const ThrottlerPtr & throttler, bool append_extra_info)
|
|
|
|
: settings(settings_)
|
2015-11-06 17:44:01 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
/// If we didn't get any connections from pool and getMany() did not throw exceptions, this means that
|
|
|
|
/// `skip_unavailable_shards` was set. Then just return.
|
|
|
|
if (connections.empty())
|
|
|
|
return;
|
2017-07-28 19:34:25 +00:00
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
replica_states.reserve(connections.size());
|
|
|
|
fd_to_replica_state_idx.reserve(connections.size());
|
|
|
|
for (size_t i = 0; i < connections.size(); ++i)
|
|
|
|
{
|
|
|
|
Connection * connection = &(*connections[i]);
|
|
|
|
connection->setThrottler(throttler);
|
2017-07-28 19:34:25 +00:00
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
ReplicaState replica_state;
|
|
|
|
replica_state.pool_entry = std::move(connections[i]);
|
|
|
|
replica_state.connection = connection;
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
replica_states.push_back(std::move(replica_state));
|
2017-09-28 19:43:31 +00:00
|
|
|
fd_to_replica_state_idx.emplace(connection->socket->impl()->sockfd(), i);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
active_connection_count = connections.size();
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (append_extra_info)
|
|
|
|
block_extra_info = std::make_unique<BlockExtraInfo>();
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void MultiplexedConnections::sendExternalTablesData(std::vector<ExternalTablesData> & data)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::lock_guard<std::mutex> lock(cancel_mutex);
|
|
|
|
|
|
|
|
if (!sent_query)
|
|
|
|
throw Exception("Cannot send external tables data: query not yet sent.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
if (data.size() != active_connection_count)
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Mismatch between replicas and data sources", ErrorCodes::MISMATCH_REPLICAS_DATA_SOURCES);
|
|
|
|
|
|
|
|
auto it = data.begin();
|
2017-08-02 13:05:01 +00:00
|
|
|
for (ReplicaState & state : replica_states)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
Connection * connection = state.connection;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (connection != nullptr)
|
2017-08-02 13:05:01 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
connection->sendExternalTablesData(*it);
|
2017-08-02 13:05:01 +00:00
|
|
|
++it;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
2016-10-24 21:40:39 +00:00
|
|
|
void MultiplexedConnections::sendQuery(
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & query,
|
|
|
|
const String & query_id,
|
|
|
|
UInt64 stage,
|
|
|
|
const ClientInfo * client_info,
|
|
|
|
bool with_pending_data)
|
2015-11-06 17:44:01 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::lock_guard<std::mutex> lock(cancel_mutex);
|
|
|
|
|
|
|
|
if (sent_query)
|
|
|
|
throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
if (replica_states.size() > 1)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
Settings query_settings = settings;
|
|
|
|
query_settings.parallel_replicas_count = replica_states.size();
|
|
|
|
|
|
|
|
for (size_t i = 0; i < replica_states.size(); ++i)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
Connection * connection = replica_states[i].connection;
|
|
|
|
if (connection == nullptr)
|
|
|
|
throw Exception("MultiplexedConnections: Internal error", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
query_settings.parallel_replica_offset = i;
|
|
|
|
connection->sendQuery(query, query_id, stage, &query_settings, client_info, with_pending_data);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
Connection * connection = replica_states[0].connection;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (connection == nullptr)
|
|
|
|
throw Exception("MultiplexedConnections: Internal error", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2017-07-31 15:03:22 +00:00
|
|
|
connection->sendQuery(query, query_id, stage, &settings, client_info, with_pending_data);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
sent_query = true;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
Connection::Packet MultiplexedConnections::receivePacket()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::lock_guard<std::mutex> lock(cancel_mutex);
|
|
|
|
Connection::Packet packet = receivePacketUnlocked();
|
|
|
|
if (block_extra_info)
|
|
|
|
{
|
|
|
|
if (packet.type == Protocol::Server::Data)
|
|
|
|
current_connection->fillBlockExtraInfo(*block_extra_info);
|
|
|
|
else
|
|
|
|
block_extra_info->is_valid = false;
|
|
|
|
}
|
|
|
|
return packet;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
BlockExtraInfo MultiplexedConnections::getBlockExtraInfo() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!block_extra_info)
|
|
|
|
throw Exception("MultiplexedConnections object not configured for block extra info support",
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
return *block_extra_info;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void MultiplexedConnections::disconnect()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::lock_guard<std::mutex> lock(cancel_mutex);
|
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
for (ReplicaState & state : replica_states)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
Connection * connection = state.connection;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (connection != nullptr)
|
|
|
|
{
|
|
|
|
connection->disconnect();
|
2017-08-02 13:05:01 +00:00
|
|
|
invalidateReplica(state);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void MultiplexedConnections::sendCancel()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::lock_guard<std::mutex> lock(cancel_mutex);
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!sent_query || cancelled)
|
|
|
|
throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR);
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
for (ReplicaState & state : replica_states)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
Connection * connection = state.connection;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (connection != nullptr)
|
|
|
|
connection->sendCancel();
|
|
|
|
}
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
cancelled = true;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
Connection::Packet MultiplexedConnections::drain()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::lock_guard<std::mutex> lock(cancel_mutex);
|
|
|
|
|
|
|
|
if (!cancelled)
|
|
|
|
throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
Connection::Packet res;
|
|
|
|
res.type = Protocol::Server::EndOfStream;
|
|
|
|
|
|
|
|
while (hasActiveConnections())
|
|
|
|
{
|
|
|
|
Connection::Packet packet = receivePacketUnlocked();
|
|
|
|
|
|
|
|
switch (packet.type)
|
|
|
|
{
|
|
|
|
case Protocol::Server::Data:
|
|
|
|
case Protocol::Server::Progress:
|
|
|
|
case Protocol::Server::ProfileInfo:
|
|
|
|
case Protocol::Server::Totals:
|
|
|
|
case Protocol::Server::Extremes:
|
|
|
|
case Protocol::Server::EndOfStream:
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::Exception:
|
|
|
|
default:
|
2017-07-28 19:34:25 +00:00
|
|
|
/// If we receive an exception or an unknown packet, we save it.
|
2017-04-01 07:20:54 +00:00
|
|
|
res = std::move(packet);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return res;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
std::string MultiplexedConnections::dumpAddresses() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::lock_guard<std::mutex> lock(cancel_mutex);
|
|
|
|
return dumpAddressesUnlocked();
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
std::string MultiplexedConnections::dumpAddressesUnlocked() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
bool is_first = true;
|
|
|
|
std::ostringstream os;
|
2017-08-02 13:05:01 +00:00
|
|
|
for (const ReplicaState & state : replica_states)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
const Connection * connection = state.connection;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (connection != nullptr)
|
|
|
|
{
|
|
|
|
os << (is_first ? "" : "; ") << connection->getDescription();
|
|
|
|
is_first = false;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return os.str();
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
Connection::Packet MultiplexedConnections::receivePacketUnlocked()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!sent_query)
|
|
|
|
throw Exception("Cannot receive packets: no query sent.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
if (!hasActiveConnections())
|
|
|
|
throw Exception("No more packets are available.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
ReplicaState & state = getReplicaForReading();
|
|
|
|
current_connection = state.connection;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (current_connection == nullptr)
|
2017-08-02 13:05:01 +00:00
|
|
|
throw Exception("Logical error: no available replica", ErrorCodes::NO_AVAILABLE_REPLICA);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Connection::Packet packet = current_connection->receivePacket();
|
|
|
|
|
|
|
|
switch (packet.type)
|
|
|
|
{
|
|
|
|
case Protocol::Server::Data:
|
|
|
|
case Protocol::Server::Progress:
|
|
|
|
case Protocol::Server::ProfileInfo:
|
|
|
|
case Protocol::Server::Totals:
|
|
|
|
case Protocol::Server::Extremes:
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::EndOfStream:
|
2017-08-02 13:05:01 +00:00
|
|
|
invalidateReplica(state);
|
2017-04-01 07:20:54 +00:00
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::Exception:
|
|
|
|
default:
|
|
|
|
current_connection->disconnect();
|
2017-08-02 13:05:01 +00:00
|
|
|
invalidateReplica(state);
|
2017-04-01 07:20:54 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
return packet;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForReading()
|
2015-11-06 17:44:01 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
if (replica_states.size() == 1)
|
|
|
|
return replica_states[0];
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Poco::Net::Socket::SocketList read_list;
|
2017-08-02 13:05:01 +00:00
|
|
|
read_list.reserve(active_connection_count);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// First, we check if there are data already in the buffer
|
|
|
|
/// of at least one connection.
|
2017-08-02 13:05:01 +00:00
|
|
|
for (const ReplicaState & state : replica_states)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
Connection * connection = state.connection;
|
2017-04-01 07:20:54 +00:00
|
|
|
if ((connection != nullptr) && connection->hasReadBufferPendingData())
|
2017-09-28 19:43:31 +00:00
|
|
|
read_list.push_back(*connection->socket);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// If no data was found, then we check if there are any connections
|
|
|
|
/// ready for reading.
|
|
|
|
if (read_list.empty())
|
|
|
|
{
|
|
|
|
Poco::Net::Socket::SocketList write_list;
|
|
|
|
Poco::Net::Socket::SocketList except_list;
|
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
for (const ReplicaState & state : replica_states)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
Connection * connection = state.connection;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (connection != nullptr)
|
2017-09-28 19:43:31 +00:00
|
|
|
read_list.push_back(*connection->socket);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-07-31 15:03:22 +00:00
|
|
|
int n = Poco::Net::Socket::select(read_list, write_list, except_list, settings.receive_timeout);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (n == 0)
|
|
|
|
throw Exception("Timeout exceeded while reading from " + dumpAddressesUnlocked(), ErrorCodes::TIMEOUT_EXCEEDED);
|
|
|
|
}
|
|
|
|
|
|
|
|
auto & socket = read_list[rand() % read_list.size()];
|
2017-08-02 13:05:01 +00:00
|
|
|
return replica_states[fd_to_replica_state_idx.at(socket.impl()->sockfd())];
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
2017-08-02 13:05:01 +00:00
|
|
|
void MultiplexedConnections::invalidateReplica(ReplicaState & state)
|
2015-11-06 17:44:01 +00:00
|
|
|
{
|
2017-08-02 13:05:01 +00:00
|
|
|
state.connection = nullptr;
|
|
|
|
state.pool_entry = IConnectionPool::Entry();
|
|
|
|
--active_connection_count;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|