2017-04-01 09:19:00 +00:00
|
|
|
#include <DataStreams/RemoteBlockInputStream.h>
|
|
|
|
#include <DataStreams/OneBlockInputStream.h>
|
|
|
|
#include <Common/NetException.h>
|
2018-06-18 16:30:26 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2018-02-15 18:54:12 +00:00
|
|
|
#include <Interpreters/castColumn.h>
|
2018-06-18 16:30:26 +00:00
|
|
|
#include <Interpreters/InternalTextLogsQueue.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/IStorage.h>
|
2017-01-21 04:24:28 +00:00
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int UNKNOWN_PACKET_FROM_SERVER;
|
|
|
|
extern const int LOGICAL_ERROR;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-07-28 19:34:25 +00:00
|
|
|
RemoteBlockInputStream::RemoteBlockInputStream(
|
|
|
|
Connection & connection,
|
2018-02-15 18:54:12 +00:00
|
|
|
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
|
2017-07-28 19:34:25 +00:00
|
|
|
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
2018-02-15 18:54:12 +00:00
|
|
|
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
|
2015-11-06 17:44:01 +00:00
|
|
|
{
|
2017-07-31 15:03:22 +00:00
|
|
|
if (settings)
|
|
|
|
context.setSettings(*settings);
|
2017-07-28 19:34:25 +00:00
|
|
|
|
|
|
|
create_multiplexed_connections = [this, &connection, throttler]()
|
|
|
|
{
|
|
|
|
return std::make_unique<MultiplexedConnections>(connection, context.getSettingsRef(), throttler);
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
|
|
|
RemoteBlockInputStream::RemoteBlockInputStream(
|
|
|
|
std::vector<IConnectionPool::Entry> && connections,
|
2018-02-15 18:54:12 +00:00
|
|
|
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
|
2017-07-28 19:34:25 +00:00
|
|
|
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
2018-02-15 18:54:12 +00:00
|
|
|
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
|
2017-07-28 19:34:25 +00:00
|
|
|
{
|
|
|
|
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
|
|
|
}
|
|
|
|
|
2017-07-28 19:34:25 +00:00
|
|
|
RemoteBlockInputStream::RemoteBlockInputStream(
|
|
|
|
const ConnectionPoolWithFailoverPtr & pool,
|
2018-02-15 18:54:12 +00:00
|
|
|
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
|
2017-07-28 19:34:25 +00:00
|
|
|
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
2018-02-15 18:54:12 +00:00
|
|
|
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
|
2015-11-06 17:44:01 +00:00
|
|
|
{
|
2017-07-31 15:03:22 +00:00
|
|
|
if (settings)
|
|
|
|
context.setSettings(*settings);
|
2017-07-28 19:34:25 +00:00
|
|
|
|
|
|
|
create_multiplexed_connections = [this, pool, throttler]()
|
|
|
|
{
|
2018-08-10 04:02:56 +00:00
|
|
|
const Settings & current_settings = context.getSettingsRef();
|
2017-08-02 13:05:01 +00:00
|
|
|
|
|
|
|
std::vector<IConnectionPool::Entry> connections;
|
|
|
|
if (main_table)
|
|
|
|
{
|
2018-08-10 04:02:56 +00:00
|
|
|
auto try_results = pool->getManyChecked(¤t_settings, pool_mode, *main_table);
|
2017-08-02 13:05:01 +00:00
|
|
|
connections.reserve(try_results.size());
|
|
|
|
for (auto & try_result : try_results)
|
|
|
|
connections.emplace_back(std::move(try_result.entry));
|
|
|
|
}
|
|
|
|
else
|
2018-08-10 04:02:56 +00:00
|
|
|
connections = pool->getMany(¤t_settings, pool_mode);
|
2017-08-02 13:05:01 +00:00
|
|
|
|
2017-07-28 19:34:25 +00:00
|
|
|
return std::make_unique<MultiplexedConnections>(
|
2018-08-10 04:02:56 +00:00
|
|
|
std::move(connections), current_settings, throttler, append_extra_info);
|
2017-07-28 19:34:25 +00:00
|
|
|
};
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
RemoteBlockInputStream::~RemoteBlockInputStream()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/** 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()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
append_extra_info = true;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void RemoteBlockInputStream::readPrefix()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!sent_query)
|
|
|
|
sendQuery();
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
2018-03-05 21:09:39 +00:00
|
|
|
void RemoteBlockInputStream::cancel(bool kill)
|
2015-11-06 17:44:01 +00:00
|
|
|
{
|
2018-03-05 21:09:39 +00:00
|
|
|
if (kill)
|
|
|
|
is_killed = true;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
std::lock_guard<std::mutex> lock(external_tables_mutex);
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +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()))
|
2018-03-05 21:09:39 +00:00
|
|
|
stream->cancel(kill);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!isQueryPending() || hasThrownException())
|
|
|
|
return;
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
tryCancel("Cancelling query");
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void RemoteBlockInputStream::sendExternalTables()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
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;
|
2018-06-25 12:08:21 +00:00
|
|
|
QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage(context);
|
2018-03-13 15:00:28 +00:00
|
|
|
BlockInputStreams input = cur->read(cur->getColumns().getNamesOfPhysical(), {}, context,
|
2018-06-03 22:11:50 +00:00
|
|
|
read_from_table_stage, DEFAULT_BLOCK_SIZE, 1);
|
2017-04-01 07:20:54 +00:00
|
|
|
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
|
|
|
}
|
|
|
|
|
2018-01-06 18:10:44 +00:00
|
|
|
|
2018-02-15 18:54:12 +00:00
|
|
|
/** If we receive a block with slightly different column types, or with excessive columns,
|
|
|
|
* we will adapt it to expected structure.
|
|
|
|
*/
|
|
|
|
static Block adaptBlockStructure(const Block & block, const Block & header, const Context & context)
|
|
|
|
{
|
|
|
|
/// Special case when reader doesn't care about result structure. Deprecated and used only in Benchmark, PerformanceTest.
|
|
|
|
if (!header)
|
|
|
|
return block;
|
|
|
|
|
|
|
|
Block res;
|
2018-02-16 20:53:47 +00:00
|
|
|
res.info = block.info;
|
|
|
|
|
2018-02-15 18:54:12 +00:00
|
|
|
for (const auto & elem : header)
|
|
|
|
res.insert({ castColumn(block.getByName(elem.name), elem.type, context), elem.type, elem.name });
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
Block RemoteBlockInputStream::readImpl()
|
2015-11-06 17:44:01 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!sent_query)
|
|
|
|
{
|
|
|
|
sendQuery();
|
|
|
|
|
2017-05-24 21:06:29 +00:00
|
|
|
if (context.getSettingsRef().skip_unavailable_shards && (0 == multiplexed_connections->size()))
|
2017-04-01 07:20:54 +00:00
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
|
|
|
while (true)
|
|
|
|
{
|
2018-03-05 21:09:39 +00:00
|
|
|
if (isCancelledOrThrowIfKilled())
|
2018-02-15 18:54:12 +00:00
|
|
|
return Block();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Connection::Packet packet = multiplexed_connections->receivePacket();
|
|
|
|
|
|
|
|
switch (packet.type)
|
|
|
|
{
|
|
|
|
case Protocol::Server::Data:
|
2018-02-15 18:54:12 +00:00
|
|
|
/// If the block is not empty and is not a header block
|
|
|
|
if (packet.block && (packet.block.rows() > 0))
|
|
|
|
return adaptBlockStructure(packet.block, header, context);
|
|
|
|
break; /// If the block is empty - we will receive other packets before EndOfStream.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
case Protocol::Server::Exception:
|
|
|
|
got_exception_from_replica = true;
|
|
|
|
packet.exception->rethrow();
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::EndOfStream:
|
|
|
|
if (!multiplexed_connections->hasActiveConnections())
|
|
|
|
{
|
|
|
|
finished = true;
|
2018-02-15 18:54:12 +00:00
|
|
|
return Block();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
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;
|
|
|
|
|
2018-06-08 19:50:15 +00:00
|
|
|
case Protocol::Server::Log:
|
2018-06-18 16:30:26 +00:00
|
|
|
/// Pass logs from remote server to client
|
|
|
|
if (auto log_queue = CurrentThread::getInternalTextLogsQueue())
|
|
|
|
log_queue->pushBlock(std::move(packet.block));
|
2018-06-08 19:50:15 +00:00
|
|
|
break;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/** 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");
|
|
|
|
|
2017-09-10 02:06:16 +00:00
|
|
|
/// Get the remaining packets so that there is no out of sync in the connections to the replicas.
|
2017-04-01 07:20:54 +00:00
|
|
|
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()
|
|
|
|
{
|
2017-07-28 19:34:25 +00:00
|
|
|
multiplexed_connections = create_multiplexed_connections();
|
2016-03-01 17:47:53 +00:00
|
|
|
|
2017-05-24 21:06:29 +00:00
|
|
|
if (context.getSettingsRef().skip_unavailable_shards && 0 == multiplexed_connections->size())
|
2017-04-01 07:20:54 +00:00
|
|
|
return;
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
established = true;
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
multiplexed_connections->sendQuery(query, "", stage, &context.getClientInfo(), true);
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
established = false;
|
|
|
|
sent_query = true;
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
sendExternalTables();
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void RemoteBlockInputStream::tryCancel(const char * reason)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
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
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
LOG_TRACE(log, "(" << multiplexed_connections->dumpAddresses() << ") " << reason);
|
|
|
|
multiplexed_connections->sendCancel();
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool RemoteBlockInputStream::isQueryPending() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return sent_query && !finished;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool RemoteBlockInputStream::hasThrownException() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return got_exception_from_replica || got_unknown_packet_from_replica;
|
2015-11-06 17:44:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|