ClickHouse/dbms/src/DataStreams/RemoteBlockInputStream.cpp

284 lines
9.2 KiB
C++
Raw Normal View History

#include <DataStreams/RemoteBlockInputStream.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Common/VirtualColumnUtils.h>
#include <Common/NetException.h>
#include <Interpreters/Context.h>
#include <Storages/IStorage.h>
2015-11-06 17:44:01 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_PACKET_FROM_SERVER;
extern const int LOGICAL_ERROR;
}
RemoteBlockInputStream::RemoteBlockInputStream(
Connection & connection,
const String & query_, const Context & context_, const Settings * settings,
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: query(query_), context(context_), external_tables(external_tables_), stage(stage_)
2015-11-06 17:44:01 +00:00
{
if (settings)
context.setSettings(*settings);
create_multiplexed_connections = [this, &connection, throttler]()
{
return std::make_unique<MultiplexedConnections>(connection, context.getSettingsRef(), throttler);
};
}
RemoteBlockInputStream::RemoteBlockInputStream(
std::vector<IConnectionPool::Entry> && connections,
const String & query_, const Context & context_, const Settings * settings,
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: query(query_), context(context_), external_tables(external_tables_), stage(stage_)
{
if (settings)
context.setSettings(*settings);
create_multiplexed_connections = [this, connections, throttler]() mutable
{
return std::make_unique<MultiplexedConnections>(
std::move(connections), context.getSettingsRef(), throttler, append_extra_info);
};
2015-11-06 17:44:01 +00:00
}
RemoteBlockInputStream::RemoteBlockInputStream(
const ConnectionPoolWithFailoverPtr & pool,
const String & query_, const Context & context_, const Settings * settings,
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: query(query_), context(context_), external_tables(external_tables_), stage(stage_)
2015-11-06 17:44:01 +00:00
{
if (settings)
context.setSettings(*settings);
create_multiplexed_connections = [this, pool, throttler]()
{
const QualifiedTableName * main_table_ptr = main_table ? &main_table.value() : nullptr;
return std::make_unique<MultiplexedConnections>(
*pool, context.getSettingsRef(), throttler, append_extra_info, pool_mode, main_table_ptr);
};
2015-11-06 17:44:01 +00:00
}
RemoteBlockInputStream::~RemoteBlockInputStream()
{
/** If interrupted in the middle of the loop of communication with replicas, then interrupt
* all connections, then read and skip the remaining packets to make sure
* these connections did not remain hanging in the out-of-sync state.
*/
if (established || isQueryPending())
multiplexed_connections->disconnect();
2015-11-06 17:44:01 +00:00
}
void RemoteBlockInputStream::appendExtraInfo()
{
append_extra_info = true;
2015-11-06 17:44:01 +00:00
}
void RemoteBlockInputStream::readPrefix()
{
if (!sent_query)
sendQuery();
2015-11-06 17:44:01 +00:00
}
void RemoteBlockInputStream::cancel()
{
bool old_val = false;
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
2015-11-06 17:44:01 +00:00
{
std::lock_guard<std::mutex> lock(external_tables_mutex);
2015-11-06 17:44:01 +00:00
/// Stop sending external data.
for (auto & vec : external_tables_data)
for (auto & elem : vec)
if (IProfilingBlockInputStream * stream = dynamic_cast<IProfilingBlockInputStream *>(elem.first.get()))
stream->cancel();
}
2015-11-06 17:44:01 +00:00
if (!isQueryPending() || hasThrownException())
return;
2015-11-06 17:44:01 +00:00
tryCancel("Cancelling query");
2015-11-06 17:44:01 +00:00
}
void RemoteBlockInputStream::sendExternalTables()
{
size_t count = multiplexed_connections->size();
{
std::lock_guard<std::mutex> lock(external_tables_mutex);
external_tables_data.reserve(count);
for (size_t i = 0; i < count; ++i)
{
ExternalTablesData res;
for (const auto & table : external_tables)
{
StoragePtr cur = table.second;
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
BlockInputStreams input = cur->read(cur->getColumnNamesList(), {}, context,
stage, DEFAULT_BLOCK_SIZE, 1);
if (input.size() == 0)
res.push_back(std::make_pair(std::make_shared<OneBlockInputStream>(cur->getSampleBlock()), table.first));
else
res.push_back(std::make_pair(input[0], table.first));
}
external_tables_data.push_back(std::move(res));
}
}
multiplexed_connections->sendExternalTablesData(external_tables_data);
2015-11-06 17:44:01 +00:00
}
Block RemoteBlockInputStream::readImpl()
{
if (!sent_query)
{
sendQuery();
if (context.getSettingsRef().skip_unavailable_shards && (0 == multiplexed_connections->size()))
return {};
}
while (true)
{
if (isCancelled())
return Block();
Connection::Packet packet = multiplexed_connections->receivePacket();
switch (packet.type)
{
case Protocol::Server::Data:
/// If the block is not empty and is not a header block
if (packet.block && (packet.block.rows() > 0))
return packet.block;
break; /// If the block is empty - we will receive other packets before EndOfStream.
case Protocol::Server::Exception:
got_exception_from_replica = true;
packet.exception->rethrow();
break;
case Protocol::Server::EndOfStream:
if (!multiplexed_connections->hasActiveConnections())
{
finished = true;
return Block();
}
break;
case Protocol::Server::Progress:
/** We use the progress from a remote server.
* We also include in ProcessList,
* and we use it to check
* constraints (for example, the minimum speed of query execution)
* and quotas (for example, the number of lines to read).
*/
progressImpl(packet.progress);
break;
case Protocol::Server::ProfileInfo:
/// Use own (client-side) info about read bytes, it is more correct info than server-side one.
info.setFrom(packet.profile_info, true);
break;
case Protocol::Server::Totals:
totals = packet.block;
break;
case Protocol::Server::Extremes:
extremes = packet.block;
break;
default:
got_unknown_packet_from_replica = true;
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
}
}
2015-11-06 17:44:01 +00:00
}
void RemoteBlockInputStream::readSuffixImpl()
{
/** If one of:
* - nothing started to do;
* - received all packets before EndOfStream;
* - received exception from one replica;
* - received an unknown packet from one replica;
* then you do not need to read anything.
*/
if (!isQueryPending() || hasThrownException())
return;
/** If you have not read all the data yet, but they are no longer needed.
* This may be due to the fact that the data is sufficient (for example, when using LIMIT).
*/
/// Send the request to abort the execution of the request, if not already sent.
tryCancel("Cancelling query because enough data has been read");
/// Get the remaining packages so that there is no out of sync in the connections to the replicas.
Connection::Packet packet = multiplexed_connections->drain();
switch (packet.type)
{
case Protocol::Server::EndOfStream:
finished = true;
break;
case Protocol::Server::Exception:
got_exception_from_replica = true;
packet.exception->rethrow();
break;
default:
got_unknown_packet_from_replica = true;
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
}
2015-11-06 17:44:01 +00:00
}
void RemoteBlockInputStream::sendQuery()
{
multiplexed_connections = create_multiplexed_connections();
2016-03-01 17:47:53 +00:00
if (context.getSettingsRef().skip_unavailable_shards && 0 == multiplexed_connections->size())
return;
2015-11-06 17:44:01 +00:00
established = true;
2015-11-06 17:44:01 +00:00
multiplexed_connections->sendQuery(query, "", stage, &context.getClientInfo(), true);
2015-11-06 17:44:01 +00:00
established = false;
sent_query = true;
2015-11-06 17:44:01 +00:00
sendExternalTables();
2015-11-06 17:44:01 +00:00
}
void RemoteBlockInputStream::tryCancel(const char * reason)
{
bool old_val = false;
if (!was_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
2015-11-06 17:44:01 +00:00
LOG_TRACE(log, "(" << multiplexed_connections->dumpAddresses() << ") " << reason);
multiplexed_connections->sendCancel();
2015-11-06 17:44:01 +00:00
}
bool RemoteBlockInputStream::isQueryPending() const
{
return sent_query && !finished;
2015-11-06 17:44:01 +00:00
}
bool RemoteBlockInputStream::hasThrownException() const
{
return got_exception_from_replica || got_unknown_packet_from_replica;
2015-11-06 17:44:01 +00:00
}
}