ClickHouse/dbms/src/DataStreams/copyData.cpp

53 lines
1.1 KiB
C++
Raw Normal View History

#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/copyData.h>
2010-03-04 19:20:28 +00:00
namespace DB
{
namespace
{
bool isAtomicSet(std::atomic<bool> * val)
{
return ((val != nullptr) && val->load(std::memory_order_seq_cst));
}
}
void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic<bool> * is_cancelled)
2010-03-04 19:20:28 +00:00
{
from.readPrefix();
to.writePrefix();
2011-10-31 06:37:12 +00:00
while (Block block = from.read())
{
if (isAtomicSet(is_cancelled))
break;
to.write(block);
}
if (isAtomicSet(is_cancelled))
return;
2011-10-31 06:37:12 +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
to.setTotals(input->getTotals());
to.setExtremes(input->getExtremes());
}
if (isAtomicSet(is_cancelled))
return;
from.readSuffix();
to.writeSuffix();
2010-03-04 19:20:28 +00:00
}
}