ClickHouse/dbms/src/DataStreams/copyData.cpp

80 lines
1.9 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));
}
}
template <typename TCancelCallback, typename TProgressCallback>
void copyDataImpl(IBlockInputStream & from, IBlockOutputStream & to, TCancelCallback && is_cancelled, TProgressCallback && progress)
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 (is_cancelled())
break;
to.write(block);
progress(block);
}
if (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 (is_cancelled())
return;
from.readSuffix();
to.writeSuffix();
2010-03-04 19:20:28 +00:00
}
inline void doNothing(const Block &) {}
void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic<bool> * is_cancelled)
{
auto is_cancelled_pred = [is_cancelled] ()
{
return isAtomicSet(is_cancelled);
};
copyDataImpl(from, to, is_cancelled_pred, doNothing);
}
void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function<bool()> & is_cancelled)
{
copyDataImpl(from, to, is_cancelled, doNothing);
}
void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function<bool()> & is_cancelled,
const std::function<void(const Block & block)> & progress)
{
copyDataImpl(from, to, is_cancelled, progress);
}
2010-03-04 19:20:28 +00:00
}