2011-10-31 17:55:06 +00:00
|
|
|
#include <map>
|
|
|
|
|
|
|
|
#include <DB/Core/Exception.h>
|
|
|
|
#include <DB/Core/ErrorCodes.h>
|
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
|
|
|
|
2011-10-31 17:55:06 +00:00
|
|
|
#include <DB/Storages/StorageMemory.h>
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
using Poco::SharedPtr;
|
|
|
|
|
|
|
|
|
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:
|
|
|
|
MemoryBlockInputStream(const Names & column_names_, BlocksList::iterator begin_, BlocksList::iterator end_)
|
|
|
|
: column_names(column_names_), begin(begin_), end(end_), it(begin) {}
|
2011-10-31 17:55:06 +00:00
|
|
|
|
2015-06-08 20:22:02 +00:00
|
|
|
String getName() const { return "Memory"; }
|
2011-10-31 17:55:06 +00:00
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
String getID() const
|
2013-08-22 16:14:44 +00:00
|
|
|
{
|
2015-01-18 08:25:56 +00:00
|
|
|
std::stringstream res;
|
|
|
|
res << "Memory(" << &*begin << ", " << &*end;
|
2013-08-22 16:14:44 +00:00
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
for (const auto & name : column_names)
|
|
|
|
res << ", " << name;
|
2013-08-22 16:14:44 +00:00
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
res << ")";
|
|
|
|
return res.str();
|
2013-08-22 16:14:44 +00:00
|
|
|
}
|
2011-10-31 17:55:06 +00:00
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
protected:
|
|
|
|
Block readImpl()
|
|
|
|
{
|
|
|
|
if (it == end)
|
|
|
|
{
|
|
|
|
return Block();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
Block src = *it;
|
|
|
|
Block res;
|
|
|
|
|
|
|
|
/// Добавляем только нужные столбцы в res.
|
|
|
|
for (size_t i = 0, size = column_names.size(); i < size; ++i)
|
|
|
|
res.insert(src.getByName(column_names[i]));
|
|
|
|
|
|
|
|
++it;
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
private:
|
|
|
|
Names column_names;
|
|
|
|
BlocksList::iterator begin;
|
|
|
|
BlocksList::iterator end;
|
|
|
|
BlocksList::iterator it;
|
|
|
|
};
|
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:
|
|
|
|
MemoryBlockOutputStream(StorageMemory & storage_) : storage(storage_) {}
|
|
|
|
|
|
|
|
void write(const Block & block)
|
|
|
|
{
|
|
|
|
storage.check(block, true);
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(storage.mutex);
|
|
|
|
storage.data.push_back(block);
|
|
|
|
}
|
|
|
|
private:
|
|
|
|
StorageMemory & storage;
|
|
|
|
};
|
2011-10-31 17:55:06 +00:00
|
|
|
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
StorageMemory::StorageMemory(
|
|
|
|
const std::string & name_,
|
|
|
|
NamesAndTypesListPtr columns_)
|
2011-10-31 17:55:06 +00:00
|
|
|
: name(name_), columns(columns_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
|
|
|
|
StorageMemory::StorageMemory(
|
|
|
|
const std::string & name_,
|
|
|
|
NamesAndTypesListPtr columns_,
|
2014-10-03 15:30:10 +00:00
|
|
|
const NamesAndTypesList & materialized_columns_,
|
2014-09-30 03:08:47 +00:00
|
|
|
const NamesAndTypesList & alias_columns_,
|
|
|
|
const ColumnDefaults & column_defaults_)
|
2014-10-03 15:30:10 +00:00
|
|
|
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
|
2014-09-30 03:08:47 +00:00
|
|
|
name(name_), columns(columns_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
StoragePtr StorageMemory::create(
|
|
|
|
const std::string & name_,
|
|
|
|
NamesAndTypesListPtr columns_)
|
|
|
|
{
|
|
|
|
return (new StorageMemory{
|
|
|
|
name_, columns_
|
|
|
|
})->thisPtr();
|
|
|
|
}
|
|
|
|
|
|
|
|
StoragePtr StorageMemory::create(
|
|
|
|
const std::string & name_,
|
|
|
|
NamesAndTypesListPtr columns_,
|
2014-10-03 15:30:10 +00:00
|
|
|
const NamesAndTypesList & materialized_columns_,
|
2014-09-30 03:08:47 +00:00
|
|
|
const NamesAndTypesList & alias_columns_,
|
|
|
|
const ColumnDefaults & column_defaults_)
|
2013-02-06 11:26:35 +00:00
|
|
|
{
|
2014-09-30 03:08:47 +00:00
|
|
|
return (new StorageMemory{
|
|
|
|
name_, columns_,
|
2014-10-03 15:30:10 +00:00
|
|
|
materialized_columns_, alias_columns_, column_defaults_
|
2014-09-30 03:08:47 +00:00
|
|
|
})->thisPtr();
|
2013-02-06 11:26:35 +00:00
|
|
|
}
|
|
|
|
|
2011-10-31 17:55:06 +00:00
|
|
|
|
2012-01-09 19:20:48 +00:00
|
|
|
BlockInputStreams StorageMemory::read(
|
2011-10-31 17:55:06 +00:00
|
|
|
const Names & column_names,
|
|
|
|
ASTPtr query,
|
2014-12-17 11:53:17 +00:00
|
|
|
const Context & context,
|
2013-02-01 19:02:04 +00:00
|
|
|
const Settings & settings,
|
2012-05-22 18:32:45 +00:00
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
2012-01-09 19:20:48 +00:00
|
|
|
size_t max_block_size,
|
2012-05-30 04:45:49 +00:00
|
|
|
unsigned threads)
|
2011-10-31 17:55:06 +00:00
|
|
|
{
|
|
|
|
check(column_names);
|
2012-05-22 18:32:45 +00:00
|
|
|
processed_stage = QueryProcessingStage::FetchColumns;
|
2012-05-30 04:45:49 +00:00
|
|
|
|
2012-11-30 04:28:13 +00:00
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
|
|
|
|
|
|
|
size_t size = data.size();
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2012-11-30 04:28:13 +00:00
|
|
|
if (threads > size)
|
|
|
|
threads = size;
|
2012-05-30 04:45:49 +00:00
|
|
|
|
|
|
|
BlockInputStreams res;
|
|
|
|
|
|
|
|
for (size_t thread = 0; thread < threads; ++thread)
|
2012-11-30 04:28:13 +00:00
|
|
|
{
|
|
|
|
BlocksList::iterator begin = data.begin();
|
|
|
|
BlocksList::iterator end = data.begin();
|
|
|
|
|
|
|
|
std::advance(begin, thread * size / threads);
|
|
|
|
std::advance(end, (thread + 1) * size / threads);
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2014-03-19 10:45:13 +00:00
|
|
|
res.push_back(new MemoryBlockInputStream(column_names, begin, end));
|
2012-11-30 04:28:13 +00:00
|
|
|
}
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2012-05-30 04:45:49 +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(
|
|
|
|
ASTPtr query)
|
|
|
|
{
|
2014-03-19 10:45:13 +00:00
|
|
|
return new 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
|
|
|
{
|
2012-11-30 04:28:13 +00:00
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
2011-11-05 23:31:19 +00:00
|
|
|
data.clear();
|
|
|
|
}
|
|
|
|
|
2011-10-31 17:55:06 +00:00
|
|
|
}
|