ClickHouse/dbms/src/DataStreams/copyData.cpp

82 lines
1.4 KiB
C++
Raw Normal View History

2010-03-04 19:20:28 +00:00
#include <DB/DataStreams/RowInputStreamFromBlockInputStream.h>
2010-05-24 17:55:57 +00:00
#include <DB/DataStreams/BlockInputStreamFromRowInputStream.h>
2010-03-04 19:20:28 +00:00
#include <DB/DataStreams/copyData.h>
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
{
2011-10-31 06:37:12 +00:00
from.readPrefix();
to.writePrefix();
2010-03-04 19:20:28 +00:00
while (Block block = from.read())
{
if (isAtomicSet(is_cancelled))
break;
2010-03-04 19:20:28 +00:00
to.write(block);
}
if (isAtomicSet(is_cancelled))
return;
2011-10-31 06:37:12 +00:00
/// Для вывода дополнительной информации в некоторых форматах.
if (IProfilingBlockInputStream * input = dynamic_cast<IProfilingBlockInputStream *>(&from))
{
2013-05-31 17:34:30 +00:00
if (input->getInfo().hasAppliedLimit())
to.setRowsBeforeLimit(input->getInfo().getRowsBeforeLimit());
to.setTotals(input->getTotals());
to.setExtremes(input->getExtremes());
}
if (isAtomicSet(is_cancelled))
return;
2011-10-31 06:37:12 +00:00
from.readSuffix();
to.writeSuffix();
2010-03-04 19:20:28 +00:00
}
void copyData(IRowInputStream & from, IRowOutputStream & to)
{
2011-10-31 06:37:12 +00:00
from.readPrefix();
to.writePrefix();
bool first = true;
2010-03-04 19:20:28 +00:00
while (1)
{
2011-10-31 06:37:12 +00:00
if (first)
first = false;
else
{
from.readRowBetweenDelimiter();
to.writeRowBetweenDelimiter();
}
Row row;
bool has_rows = from.read(row);
if (!has_rows)
2010-03-04 19:20:28 +00:00
break;
to.write(row);
}
2011-10-31 06:37:12 +00:00
from.readSuffix();
to.writeSuffix();
2010-03-04 19:20:28 +00:00
}
}