ClickHouse/dbms/src/Storages/StorageSystemNumbers.cpp

90 lines
2.3 KiB
C++
Raw Normal View History

2010-03-04 19:20:28 +00:00
#include <Poco/SharedPtr.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
2010-03-12 18:25:35 +00:00
#include <DB/Columns/ColumnsNumber.h>
2010-03-05 17:38:01 +00:00
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
2010-03-04 19:20:28 +00:00
#include <DB/Storages/StorageSystemNumbers.h>
namespace DB
{
using Poco::SharedPtr;
class NumbersBlockInputStream : public IProfilingBlockInputStream
2010-03-04 19:20:28 +00:00
{
public:
NumbersBlockInputStream(size_t block_size_, size_t offset_, size_t step_)
: block_size(block_size_), next(offset_), step(step_) {}
String getName() const { return "Numbers"; }
String getID() const { return "Numbers"; }
protected:
Block readImpl()
{
Block res;
ColumnWithTypeAndName column_with_name_and_type;
column_with_name_and_type.name = "number";
column_with_name_and_type.type = new DataTypeUInt64();
ColumnUInt64 * column = new ColumnUInt64(block_size);
ColumnUInt64::Container_t & vec = column->getData();
column_with_name_and_type.column = column;
size_t curr = next; /// Локальная переменная почему-то работает быстрее (>20%), чем член класса.
UInt64 * pos = &vec[0]; /// Это тоже ускоряет код.
UInt64 * end = &vec[block_size];
while (pos < end)
*pos++ = curr++;
res.insert(column_with_name_and_type);
next += step;
return res;
}
private:
size_t block_size;
UInt64 next;
UInt64 step;
};
StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_)
2015-03-26 23:32:16 +00:00
: name(name_), columns{{"number", new DataTypeUInt64}}, multithreaded(multithreaded_)
2010-03-04 19:20:28 +00:00
{
2011-08-15 02:24:44 +00:00
}
2010-03-04 19:20:28 +00:00
StoragePtr StorageSystemNumbers::create(const std::string & name_, bool multithreaded_)
{
return (new StorageSystemNumbers(name_, multithreaded_))->thisPtr();
}
2010-03-04 19:20:28 +00:00
2012-01-09 19:20:48 +00:00
BlockInputStreams StorageSystemNumbers::read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,
unsigned threads)
2011-08-15 02:24:44 +00:00
{
check(column_names);
2012-05-22 18:32:45 +00:00
processed_stage = QueryProcessingStage::FetchColumns;
if (!multithreaded)
threads = 1;
BlockInputStreams res(threads);
for (size_t i = 0; i < threads; ++i)
res[i] = new NumbersBlockInputStream(max_block_size, i * max_block_size, threads * max_block_size);
return res;
2010-03-04 19:20:28 +00:00
}
}