ClickHouse/dbms/src/Storages/StorageMemory.cpp

174 lines
3.4 KiB
C++
Raw Normal View History

2011-10-31 17:55:06 +00:00
#include <map>
2015-10-05 01:35:28 +00:00
#include <DB/Common/Exception.h>
2011-10-31 17:55:06 +00:00
#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
String getName() const { return "Memory"; }
2011-10-31 17:55:06 +00:00
2015-01-18 08:25:56 +00:00
String getID() const
{
2015-01-18 08:25:56 +00:00
std::stringstream res;
res << "Memory(" << &*begin << ", " << &*end;
2015-01-18 08:25:56 +00:00
for (const auto & name : column_names)
res << ", " << name;
2015-01-18 08:25:56 +00:00
res << ")";
return res.str();
}
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
StorageMemory::StorageMemory(
const std::string & name_,
NamesAndTypesListPtr columns_)
2011-10-31 17:55:06 +00:00
: name(name_), columns(columns_)
{
}
StorageMemory::StorageMemory(
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_)
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
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_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_)
{
return (new StorageMemory{
name_, columns_,
materialized_columns_, alias_columns_, column_defaults_
})->thisPtr();
}
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,
const Context & context,
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
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
size_t size = data.size();
if (threads > size)
threads = size;
2012-05-30 04:45:49 +00:00
BlockInputStreams res;
for (size_t thread = 0; thread < threads; ++thread)
{
BlocksList::iterator begin = data.begin();
BlocksList::iterator end = data.begin();
std::advance(begin, thread * size / threads);
std::advance(end, (thread + 1) * size / threads);
res.push_back(new MemoryBlockInputStream(column_names, begin, end));
}
2012-05-30 04:45:49 +00:00
return res;
2011-10-31 17:55:06 +00:00
}
2011-10-31 17:55:06 +00:00
BlockOutputStreamPtr StorageMemory::write(
ASTPtr query, const Settings & settings)
2011-10-31 17:55:06 +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
{
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
2011-11-05 23:31:19 +00:00
data.clear();
}
2011-10-31 17:55:06 +00:00
}