2016-02-16 16:39:39 +00:00
|
|
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
|
|
|
#include <DB/DataStreams/IBlockOutputStream.h>
|
2010-03-04 19:20:28 +00:00
|
|
|
#include <DB/DataStreams/copyData.h>
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2015-03-20 16:20:47 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
bool isAtomicSet(std::atomic<bool> * val)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return ((val != nullptr) && val->load(std::memory_order_seq_cst));
|
2015-03-20 16:20:47 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic<bool> * is_cancelled)
|
2010-03-04 19:20:28 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
from.readPrefix();
|
|
|
|
to.writePrefix();
|
2011-10-31 06:37:12 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
while (Block block = from.read())
|
|
|
|
{
|
|
|
|
if (isAtomicSet(is_cancelled))
|
|
|
|
break;
|
2015-01-07 18:07:00 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
to.write(block);
|
|
|
|
}
|
2015-01-07 18:07:00 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (isAtomicSet(is_cancelled))
|
|
|
|
return;
|
2011-10-31 06:37:12 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// For outputting additional information in some formats.
|
|
|
|
if (IProfilingBlockInputStream * input = dynamic_cast<IProfilingBlockInputStream *>(&from))
|
|
|
|
{
|
|
|
|
if (input->getProfileInfo().hasAppliedLimit())
|
|
|
|
to.setRowsBeforeLimit(input->getProfileInfo().getRowsBeforeLimit());
|
2013-05-31 17:34:30 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
to.setTotals(input->getTotals());
|
|
|
|
to.setExtremes(input->getExtremes());
|
|
|
|
}
|
2013-09-01 04:55:41 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (isAtomicSet(is_cancelled))
|
|
|
|
return;
|
2015-01-07 18:07:00 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
from.readSuffix();
|
|
|
|
to.writeSuffix();
|
2010-03-04 19:20:28 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|