2011-10-31 17:55:06 +00:00
|
|
|
#include <map>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/Exception.h>
|
2011-10-31 17:55:06 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataStreams/IProfilingBlockInputStream.h>
|
2015-01-18 08:25:56 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/StorageMemory.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2011-10-31 17:55:06 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
class MemoryBlockInputStream : public IProfilingBlockInputStream
|
2011-10-31 17:55:06 +00:00
|
|
|
{
|
2015-01-18 08:25:56 +00:00
|
|
|
public:
|
2018-01-06 18:10:44 +00:00
|
|
|
MemoryBlockInputStream(const Names & column_names_, BlocksList::iterator begin_, BlocksList::iterator end_, const StorageMemory & storage_)
|
|
|
|
: column_names(column_names_), begin(begin_), end(end_), it(begin), storage(storage_) {}
|
2011-10-31 17:55:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getName() const override { return "Memory"; }
|
2011-10-31 17:55:06 +00:00
|
|
|
|
2018-02-18 03:23:48 +00:00
|
|
|
Block getHeader() const override { return storage.getSampleBlockForColumns(column_names); }
|
2018-01-06 18:10:44 +00:00
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
protected:
|
2017-04-01 07:20:54 +00:00
|
|
|
Block readImpl() override
|
|
|
|
{
|
|
|
|
if (it == end)
|
|
|
|
{
|
|
|
|
return Block();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
Block src = *it;
|
|
|
|
Block res;
|
|
|
|
|
|
|
|
/// Add only required columns to `res`.
|
|
|
|
for (size_t i = 0, size = column_names.size(); i < size; ++i)
|
|
|
|
res.insert(src.getByName(column_names[i]));
|
|
|
|
|
|
|
|
++it;
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
}
|
2015-01-18 08:25:56 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
Names column_names;
|
|
|
|
BlocksList::iterator begin;
|
|
|
|
BlocksList::iterator end;
|
|
|
|
BlocksList::iterator it;
|
2018-01-06 18:10:44 +00:00
|
|
|
const StorageMemory & storage;
|
2015-01-18 08:25:56 +00:00
|
|
|
};
|
2011-10-31 17:55:06 +00:00
|
|
|
|
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
class MemoryBlockOutputStream : public IBlockOutputStream
|
2011-10-31 17:55:06 +00:00
|
|
|
{
|
2015-01-18 08:25:56 +00:00
|
|
|
public:
|
2017-09-07 21:04:48 +00:00
|
|
|
explicit MemoryBlockOutputStream(StorageMemory & storage_) : storage(storage_) {}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-19 00:45:32 +00:00
|
|
|
Block getHeader() const override { return storage.getSampleBlock(); }
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void write(const Block & block) override
|
|
|
|
{
|
|
|
|
storage.check(block, true);
|
|
|
|
std::lock_guard<std::mutex> lock(storage.mutex);
|
|
|
|
storage.data.push_back(block);
|
|
|
|
}
|
2015-01-18 08:25:56 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
StorageMemory & storage;
|
2015-01-18 08:25:56 +00:00
|
|
|
};
|
2011-10-31 17:55:06 +00:00
|
|
|
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
StorageMemory::StorageMemory(
|
2017-04-01 07:20:54 +00:00
|
|
|
const std::string & name_,
|
2017-12-25 21:57:29 +00:00
|
|
|
const NamesAndTypesList & columns_,
|
|
|
|
const NamesAndTypesList & materialized_columns_,
|
|
|
|
const NamesAndTypesList & alias_columns_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const ColumnDefaults & column_defaults_)
|
2018-01-25 14:42:39 +00:00
|
|
|
: IStorage{columns_, materialized_columns_, alias_columns_, column_defaults_},
|
|
|
|
name(name_)
|
2014-09-30 03:08:47 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2012-01-09 19:20:48 +00:00
|
|
|
BlockInputStreams StorageMemory::read(
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & column_names,
|
2017-12-01 21:13:25 +00:00
|
|
|
const SelectQueryInfo & /*query_info*/,
|
|
|
|
const Context & /*context*/,
|
2017-04-01 07:20:54 +00:00
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
2017-12-01 21:13:25 +00:00
|
|
|
size_t /*max_block_size*/,
|
2017-06-02 15:54:39 +00:00
|
|
|
unsigned num_streams)
|
2011-10-31 17:55:06 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
check(column_names);
|
|
|
|
processed_stage = QueryProcessingStage::FetchColumns;
|
2012-05-30 04:45:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
2012-11-30 04:28:13 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t size = data.size();
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2017-06-02 15:54:39 +00:00
|
|
|
if (num_streams > size)
|
|
|
|
num_streams = size;
|
2012-05-30 04:45:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
BlockInputStreams res;
|
2012-05-30 04:45:49 +00:00
|
|
|
|
2017-06-02 15:54:39 +00:00
|
|
|
for (size_t stream = 0; stream < num_streams; ++stream)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
BlocksList::iterator begin = data.begin();
|
|
|
|
BlocksList::iterator end = data.begin();
|
2012-11-30 04:28:13 +00:00
|
|
|
|
2017-06-02 15:54:39 +00:00
|
|
|
std::advance(begin, stream * size / num_streams);
|
|
|
|
std::advance(end, (stream + 1) * size / num_streams);
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2018-01-06 18:10:44 +00:00
|
|
|
res.push_back(std::make_shared<MemoryBlockInputStream>(column_names, begin, end, *this));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return res;
|
2011-10-31 17:55:06 +00:00
|
|
|
}
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2011-10-31 17:55:06 +00:00
|
|
|
BlockOutputStreamPtr StorageMemory::write(
|
2017-12-01 21:13:25 +00:00
|
|
|
const ASTPtr & /*query*/, const Settings & /*settings*/)
|
2011-10-31 17:55:06 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return std::make_shared<MemoryBlockOutputStream>(*this);
|
2011-10-31 17:55:06 +00:00
|
|
|
}
|
|
|
|
|
2011-11-05 23:31:19 +00:00
|
|
|
|
2014-03-20 13:28:49 +00:00
|
|
|
void StorageMemory::drop()
|
2011-11-05 23:31:19 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
std::lock_guard<std::mutex> lock(mutex);
|
|
|
|
data.clear();
|
2011-11-05 23:31:19 +00:00
|
|
|
}
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
|
|
|
|
void registerStorageMemory(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("Memory", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
if (!args.engine_args.empty())
|
|
|
|
throw Exception(
|
|
|
|
"Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
return StorageMemory::create(args.table_name, args.columns, args.materialized_columns, args.alias_columns, args.column_defaults);
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2011-10-31 17:55:06 +00:00
|
|
|
}
|