ClickHouse/dbms/src/DataStreams/IProfilingBlockInputStream.cpp

434 lines
12 KiB
C++
Raw Normal View History

2011-08-27 22:43:31 +00:00
#include <iomanip>
2012-02-27 07:54:16 +00:00
/*#include <Poco/Mutex.h>
#include <Poco/Ext/ThreadNumber.h>*/
#include <DB/Columns/ColumnConst.h>
2011-09-04 21:23:19 +00:00
#include <DB/DataStreams/IProfilingBlockInputStream.h>
2011-08-27 22:43:31 +00:00
namespace DB
{
2013-05-22 14:57:43 +00:00
void BlockStreamProfileInfo::read(ReadBuffer & in)
{
readVarUInt(rows, in);
readVarUInt(blocks, in);
readVarUInt(bytes, in);
readBinary(applied_limit, in);
readVarUInt(rows_before_limit, in);
readBinary(calculated_rows_before_limit, in);
2013-05-22 14:57:43 +00:00
}
void BlockStreamProfileInfo::write(WriteBuffer & out) const
{
writeVarUInt(rows, out);
writeVarUInt(blocks, out);
writeVarUInt(bytes, out);
writeBinary(hasAppliedLimit(), out);
writeVarUInt(getRowsBeforeLimit(), out);
writeBinary(calculated_rows_before_limit, out);
}
size_t BlockStreamProfileInfo::getRowsBeforeLimit() const
{
if (!calculated_rows_before_limit)
calculateRowsBeforeLimit();
return rows_before_limit;
}
bool BlockStreamProfileInfo::hasAppliedLimit() const
{
if (!calculated_rows_before_limit)
calculateRowsBeforeLimit();
return applied_limit;
}
2013-05-22 14:57:43 +00:00
2012-05-17 19:15:53 +00:00
void BlockStreamProfileInfo::update(Block & block)
2011-08-27 22:43:31 +00:00
{
++blocks;
2014-08-22 22:30:21 +00:00
rows += block.rowsInFirstColumn();
2012-05-17 19:15:53 +00:00
bytes += block.bytes();
2011-09-05 00:51:25 +00:00
if (column_names.empty())
column_names = block.dumpNames();
2011-08-27 22:43:31 +00:00
}
void BlockStreamProfileInfo::collectInfosForStreamsWithName(const String & name, BlockStreamProfileInfos & res) const
2013-05-20 12:21:51 +00:00
{
if (stream_name == name)
2013-05-22 14:57:43 +00:00
{
res.push_back(this);
return;
2013-05-22 14:57:43 +00:00
}
for (BlockStreamProfileInfos::const_iterator it = nested_infos.begin(); it != nested_infos.end(); ++it)
(*it)->collectInfosForStreamsWithName(name, res);
}
void BlockStreamProfileInfo::calculateRowsBeforeLimit() const
{
calculated_rows_before_limit = true;
/// есть ли Limit?
BlockStreamProfileInfos limits;
collectInfosForStreamsWithName("Limit", limits);
if (limits.empty())
return;
applied_limit = true;
/** Берём количество строчек, прочитанных ниже PartialSorting-а, если есть, или ниже Limit-а.
* Это нужно, потому что сортировка может вернуть только часть строк.
*/
BlockStreamProfileInfos partial_sortings;
collectInfosForStreamsWithName("PartialSorting", partial_sortings);
BlockStreamProfileInfos & limits_or_sortings = partial_sortings.empty() ? limits : partial_sortings;
for (BlockStreamProfileInfos::const_iterator it = limits_or_sortings.begin(); it != limits_or_sortings.end(); ++it)
for (BlockStreamProfileInfos::const_iterator jt = (*it)->nested_infos.begin(); jt != (*it)->nested_infos.end(); ++jt)
rows_before_limit += (*jt)->rows;
2013-05-20 12:21:51 +00:00
}
2011-09-04 21:23:19 +00:00
Block IProfilingBlockInputStream::read()
2011-08-27 22:43:31 +00:00
{
if (!info.started)
2011-09-26 01:50:32 +00:00
{
2011-08-27 22:43:31 +00:00
info.total_stopwatch.start();
2013-05-22 12:10:33 +00:00
info.stream_name = getShortName();
2011-09-26 01:50:32 +00:00
for (BlockInputStreams::const_iterator it = children.begin(); it != children.end(); ++it)
2012-05-09 08:16:09 +00:00
if (const IProfilingBlockInputStream * child = dynamic_cast<const IProfilingBlockInputStream *>(&**it))
info.nested_infos.push_back(&child->info);
2011-09-26 01:50:32 +00:00
info.started = true;
}
2012-05-09 08:16:09 +00:00
Block res;
2012-10-20 05:54:35 +00:00
if (is_cancelled)
return res;
2012-05-31 00:47:05 +00:00
2014-01-08 16:23:31 +00:00
res = readImpl();
2011-08-27 22:43:31 +00:00
2011-09-25 05:07:47 +00:00
/* if (res)
2011-09-25 03:37:09 +00:00
{
2012-02-27 07:54:16 +00:00
static Poco::FastMutex mutex;
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
2012-03-05 07:58:34 +00:00
2011-09-25 03:37:09 +00:00
std::cerr << std::endl;
2012-06-24 23:17:06 +00:00
std::cerr << "[ " << Poco::ThreadNumber::get() << " ]\t" << getShortName() << std::endl;
2012-03-05 07:58:34 +00:00
std::cerr << "[ " << Poco::ThreadNumber::get() << " ]\t";
for (size_t i = 0; i < res.columns(); ++i)
{
if (i != 0)
std::cerr << ", ";
std::cerr << res.getByPosition(i).name << " (" << res.getByPosition(i).column->size() << ")";
}
2012-03-05 07:58:34 +00:00
std::cerr << std::endl;
2011-09-25 05:07:47 +00:00
}*/
2012-05-09 15:15:45 +00:00
2012-03-05 07:58:34 +00:00
if (res)
{
2012-05-17 19:15:53 +00:00
info.update(res);
if (enabled_extremes)
updateExtremes(res);
if (!checkLimits())
{
res.clear();
return res;
}
2014-04-08 07:31:51 +00:00
if (quota != nullptr)
checkQuota(res);
}
else
{
/** Если поток закончился, то ещё попросим всех детей прервать выполнение.
* Это имеет смысл при выполнении запроса с LIMIT-ом:
* - бывает ситуация, когда все необходимые данные уже прочитали,
* но источники-дети ещё продолжают работать,
* при чём они могут работать в отдельных потоках или даже удалённо.
*/
cancel();
}
2011-09-25 03:37:09 +00:00
progress(Progress(res.rowsInFirstColumn(), res.bytes()));
return res;
}
2013-09-13 20:33:09 +00:00
void IProfilingBlockInputStream::readSuffix()
{
for (BlockInputStreams::iterator it = children.begin(); it != children.end(); ++it)
(*it)->readSuffix();
readSuffixImpl();
}
void IProfilingBlockInputStream::updateExtremes(Block & block)
{
size_t columns = block.columns();
if (!extremes)
{
extremes = block.cloneEmpty();
for (size_t i = 0; i < columns; ++i)
{
Field min_value;
Field max_value;
block.getByPosition(i).column->getExtremes(min_value, max_value);
ColumnPtr & column = extremes.getByPosition(i).column;
if (column->isConst())
column = dynamic_cast<const IColumnConst &>(*column).convertToFullColumn();
column->insert(min_value);
column->insert(max_value);
}
}
else
{
for (size_t i = 0; i < columns; ++i)
{
ColumnPtr & column = extremes.getByPosition(i).column;
Field min_value = (*column)[0];
Field max_value = (*column)[1];
Field cur_min_value;
Field cur_max_value;
block.getByPosition(i).column->getExtremes(cur_min_value, cur_max_value);
if (cur_min_value < min_value)
min_value = cur_min_value;
if (cur_max_value > max_value)
max_value = cur_max_value;
column = column->cloneEmpty();
column->insert(min_value);
column->insert(max_value);
}
}
}
bool IProfilingBlockInputStream::checkLimits()
{
/// Проверка ограничений.
if ((limits.max_rows_to_read && info.rows > limits.max_rows_to_read)
|| (limits.max_bytes_to_read && info.bytes > limits.max_bytes_to_read))
{
if (limits.read_overflow_mode == OverflowMode::THROW)
throw Exception(std::string("Limit for ")
+ (limits.mode == LIMITS_CURRENT ? "result rows" : "rows to read")
+ " exceeded: read " + toString(info.rows)
2013-06-21 20:34:19 +00:00
+ " rows, maximum: " + toString(limits.max_rows_to_read),
ErrorCodes::TOO_MUCH_ROWS);
if (limits.read_overflow_mode == OverflowMode::BREAK)
return false;
2013-05-17 08:02:34 +00:00
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
}
if (limits.max_execution_time != 0
&& info.total_stopwatch.elapsed() > static_cast<UInt64>(limits.max_execution_time.totalMicroseconds()) * 1000)
{
if (limits.timeout_overflow_mode == OverflowMode::THROW)
2013-06-21 20:34:19 +00:00
throw Exception("Timeout exceeded: elapsed " + toString(info.total_stopwatch.elapsedSeconds())
+ " seconds, maximum: " + toString(limits.max_execution_time.totalMicroseconds() / 1000000.0),
ErrorCodes::TIMEOUT_EXCEEDED);
if (limits.timeout_overflow_mode == OverflowMode::BREAK)
return false;
2013-05-17 08:02:34 +00:00
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
}
return true;
}
void IProfilingBlockInputStream::checkQuota(Block & block)
{
switch (limits.mode)
{
case LIMITS_TOTAL:
/// Проверяется в методе progress.
break;
case LIMITS_CURRENT:
{
time_t current_time = time(0);
double total_elapsed = info.total_stopwatch.elapsedSeconds();
2014-08-22 22:30:21 +00:00
quota->checkAndAddResultRowsBytes(current_time, block.rowsInFirstColumn(), block.bytes());
quota->checkAndAddExecutionTime(current_time, Poco::Timespan((total_elapsed - prev_elapsed) * 1000000.0));
prev_elapsed = total_elapsed;
break;
}
default:
throw Exception("Logical error: unknown limits mode.", ErrorCodes::LOGICAL_ERROR);
}
2011-08-27 22:43:31 +00:00
}
2012-05-17 19:15:53 +00:00
void IProfilingBlockInputStream::progressImpl(const Progress & value)
2012-05-17 19:15:53 +00:00
{
/// Данные для прогресса берутся из листовых источников.
if (children.empty())
{
if (progress_callback)
progress_callback(value);
if (process_list_elem)
{
if (!process_list_elem->update(value))
cancel();
/// Общее количество данных, обработанных или предполагаемых к обработке во всех листовых источниках, возможно, на удалённых серверах.
size_t rows_processed = process_list_elem->progress.rows;
size_t bytes_processed = process_list_elem->progress.bytes;
size_t total_rows_estimate = std::max(process_list_elem->progress.rows, process_list_elem->progress.total_rows);
/** Проверяем ограничения на объём данных для чтения, скорость выполнения запроса, квоту на объём данных для чтения.
* NOTE: Может быть, имеет смысл сделать, чтобы они проверялись прямо в ProcessList?
*/
if (limits.mode == LIMITS_TOTAL
&& ((limits.max_rows_to_read && total_rows_estimate > limits.max_rows_to_read)
|| (limits.max_bytes_to_read && bytes_processed > limits.max_bytes_to_read)))
{
if (limits.read_overflow_mode == OverflowMode::THROW)
{
if (limits.max_rows_to_read && total_rows_estimate > limits.max_rows_to_read)
throw Exception("Limit for rows to read exceeded: " + toString(total_rows_estimate)
+ " rows read (or to read), maximum: " + toString(limits.max_rows_to_read),
ErrorCodes::TOO_MUCH_ROWS);
else
throw Exception("Limit for (uncompressed) bytes to read exceeded: " + toString(bytes_processed)
+ " bytes read, maximum: " + toString(limits.max_bytes_to_read),
ErrorCodes::TOO_MUCH_ROWS);
}
else if (limits.read_overflow_mode == OverflowMode::BREAK)
cancel();
else
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
}
if (limits.min_execution_speed)
{
double total_elapsed = info.total_stopwatch.elapsedSeconds();
if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0
&& rows_processed / total_elapsed < limits.min_execution_speed)
{
throw Exception("Query is executing too slow: " + toString(rows_processed / total_elapsed)
+ " rows/sec., minimum: " + toString(limits.min_execution_speed),
ErrorCodes::TOO_SLOW);
}
}
2014-04-08 07:31:51 +00:00
if (quota != nullptr && limits.mode == LIMITS_TOTAL)
{
quota->checkAndAddReadRowsBytes(time(0), value.rows, value.bytes);
}
}
}
2012-05-17 19:15:53 +00:00
}
2011-08-27 22:43:31 +00:00
void IProfilingBlockInputStream::cancel()
{
2012-11-10 05:13:46 +00:00
if (!__sync_bool_compare_and_swap(&is_cancelled, false, true))
return;
for (BlockInputStreams::iterator it = children.begin(); it != children.end(); ++it)
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&**it))
child->cancel();
}
2012-05-09 15:15:45 +00:00
void IProfilingBlockInputStream::setProgressCallback(ProgressCallback callback)
{
2012-05-17 19:15:53 +00:00
progress_callback = callback;
2012-05-17 19:15:53 +00:00
for (BlockInputStreams::iterator it = children.begin(); it != children.end(); ++it)
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&**it))
child->setProgressCallback(callback);
2012-05-09 08:16:09 +00:00
}
void IProfilingBlockInputStream::setProcessListElement(ProcessList::Element * elem)
{
process_list_elem = elem;
for (BlockInputStreams::iterator it = children.begin(); it != children.end(); ++it)
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&**it))
child->setProcessListElement(elem);
}
const Block & IProfilingBlockInputStream::getTotals()
{
if (totals)
return totals;
for (BlockInputStreams::iterator it = children.begin(); it != children.end(); ++it)
{
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&**it))
{
const Block & res = child->getTotals();
if (res)
return res;
}
}
return totals;
}
const Block & IProfilingBlockInputStream::getExtremes() const
{
if (extremes)
return extremes;
for (BlockInputStreams::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (const IProfilingBlockInputStream * child = dynamic_cast<const IProfilingBlockInputStream *>(&**it))
{
const Block & res = child->getExtremes();
if (res)
return res;
}
}
return extremes;
}
2011-08-27 22:43:31 +00:00
}