dbms: development [#CONV-2944].

This commit is contained in:
Alexey Milovidov 2012-06-25 00:17:19 +00:00
parent 9dea897645
commit 28b0950770
4 changed files with 290 additions and 7 deletions

View File

@ -48,20 +48,26 @@ public:
if (threads_data.empty())
return res;
if (pool.pending() + pool.active() < pool.size())
ssize_t max_threads_to_start = static_cast<ssize_t>(pool.size()) - (pool.pending() + pool.active());
if (max_threads_to_start > 0)
{
/// Запустим вычисления для как можно большего количества источников, которые ещё ни разу не брались
std::cerr << "Starting initial threads" << std::endl;
/// Запустим вычисления для как можно большего количества источников, которые ещё ни разу не брались
size_t started_threads = 0;
size_t max_threads_to_start = pool.size() - pool.pending() + pool.active();
for (ThreadsData::iterator it = threads_data.begin(); it != threads_data.end() && 0 == it->count; ++it)
ssize_t started_threads = 0;
ThreadsData::iterator it = threads_data.begin();
while (it != threads_data.end() && 0 == it->count)
{
std::cerr << "Scheduling initial " << it->i << std::endl;
++it->count;
++started_threads;
pool.schedule(boost::bind(&UnionBlockInputStream::calculate, this, boost::ref(*it)));
/// Переносим этот источник в конец списка
threads_data.push_back(*it);
threads_data.erase(it++);
pool.schedule(boost::bind(&UnionBlockInputStream::calculate, this, boost::ref(threads_data.back())));
if (started_threads == max_threads_to_start)
break;

View File

@ -0,0 +1,123 @@
#pragma once
#include <Poco/File.h>
#include <DB/Core/NamesAndTypes.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/WriteBufferFromFile.h>
#include <DB/IO/CompressedReadBuffer.h>
#include <DB/IO/CompressedWriteBuffer.h>
#include <DB/Storages/IStorage.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
namespace DB
{
class StorageTinyLog;
class TinyLogBlockInputStream : public IProfilingBlockInputStream
{
public:
TinyLogBlockInputStream(size_t block_size_, const Names & column_names_, StorageTinyLog & storage_);
Block readImpl();
String getName() const { return "TinyLogBlockInputStream"; }
BlockInputStreamPtr clone() { return new TinyLogBlockInputStream(block_size, column_names, storage); }
private:
size_t block_size;
Names column_names;
StorageTinyLog & storage;
struct Stream
{
Stream(const std::string & data_path)
: plain(data_path, std::min(static_cast<size_t>(DBMS_DEFAULT_BUFFER_SIZE), Poco::File(data_path).getSize())),
compressed(plain)
{
}
ReadBufferFromFile plain;
CompressedReadBuffer compressed;
};
typedef std::map<std::string, SharedPtr<Stream> > FileStreams;
FileStreams streams;
};
class TinyLogBlockOutputStream : public IBlockOutputStream
{
public:
TinyLogBlockOutputStream(StorageTinyLog & storage_);
void write(const Block & block);
BlockOutputStreamPtr clone() { return new TinyLogBlockOutputStream(storage); }
private:
StorageTinyLog & storage;
struct Stream
{
Stream(const std::string & data_path) :
plain(data_path, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY),
compressed(plain)
{
}
WriteBufferFromFile plain;
CompressedWriteBuffer compressed;
};
typedef std::map<std::string, SharedPtr<Stream> > FileStreams;
FileStreams streams;
};
/** Реализует хранилище, подходящее для маленьких кусочков лога.
* Отличается от StorageLog отсутствием файлов с засечками.
*/
class StorageTinyLog : public IStorage
{
friend class TinyLogBlockInputStream;
friend class TinyLogBlockOutputStream;
public:
/** Подцепить таблицу с соответствующим именем, по соответствующему пути (с / на конце),
* (корректность имён и путей не проверяется)
* состоящую из указанных столбцов; создать файлы, если их нет.
*/
StorageTinyLog(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_);
std::string getName() const { return "TinyLog"; }
std::string getTableName() const { return name; }
const NamesAndTypesList & getColumnsList() const { return *columns; }
BlockInputStreams read(
const Names & column_names,
ASTPtr query,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1);
BlockOutputStreamPtr write(
ASTPtr query);
void drop();
void rename(const String & new_path_to_db, const String & new_name);
private:
String path;
String name;
NamesAndTypesListPtr columns;
/// Данные столбца
struct ColumnData
{
Poco::File data_file;
};
typedef std::map<String, ColumnData> Files_t;
Files_t files;
};
}

View File

@ -5,6 +5,7 @@
#include <DB/Interpreters/Context.h>
#include <DB/Storages/StorageLog.h>
#include <DB/Storages/StorageTinyLog.h>
#include <DB/Storages/StorageMemory.h>
#include <DB/Storages/StorageMerge.h>
#include <DB/Storages/StorageDistributed.h>
@ -29,6 +30,10 @@ StoragePtr StorageFactory::get(
{
return new StorageLog(data_path, table_name, columns);
}
else if (name == "Log")
{
return new StorageTinyLog(data_path, table_name, columns);
}
else if (name == "Memory")
{
return new StorageMemory(table_name, columns);

View File

@ -0,0 +1,149 @@
#include <map>
#include <DB/Common/escapeForFileName.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/Storages/StorageTinyLog.h>
#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin"
namespace DB
{
using Poco::SharedPtr;
TinyLogBlockInputStream::TinyLogBlockInputStream(size_t block_size_, const Names & column_names_, StorageTinyLog & storage_)
: block_size(block_size_), column_names(column_names_), storage(storage_)
{
}
Block TinyLogBlockInputStream::readImpl()
{
Block res;
/// Если файлы не открыты, то открываем их.
if (streams.empty())
{
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
streams.insert(std::make_pair(*it, new Stream(storage.files[*it].data_file.path())));
}
else if (streams[0]->compressed.eof())
return res;
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
{
ColumnWithNameAndType column;
column.name = *it;
column.type = storage.getDataTypeByName(*it);
column.column = column.type->createColumn();
column.type->deserializeBinary(*column.column, streams[column.name]->compressed, block_size);
if (column.column->size())
res.insert(column);
}
if (!res || streams[0]->compressed.eof())
{
/** Закрываем файлы (ещё до уничтожения объекта).
* Чтобы при создании многих источников, но одновременном чтении только из нескольких,
* буферы не висели в памяти.
*/
streams.clear();
}
return res;
}
TinyLogBlockOutputStream::TinyLogBlockOutputStream(StorageTinyLog & storage_)
: storage(storage_)
{
for (NamesAndTypesList::const_iterator it = storage.columns->begin(); it != storage.columns->end(); ++it)
streams.insert(std::make_pair(it->first, new Stream(storage.files[it->first].data_file.path())));
}
void TinyLogBlockOutputStream::write(const Block & block)
{
storage.check(block);
for (size_t i = 0; i < block.columns(); ++i)
{
const ColumnWithNameAndType & column = block.getByPosition(i);
column.type->serializeBinary(*column.column, streams[column.name]->compressed);
}
}
StorageTinyLog::StorageTinyLog(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_)
: path(path_), name(name_), columns(columns_)
{
if (columns->empty())
throw Exception("Empty list of columns passed to StorageTinyLog constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
/// создаём файлы, если их нет
Poco::File(path + escapeForFileName(name) + '/').createDirectories();
for (NamesAndTypesList::const_iterator it = columns->begin(); it != columns->end(); ++it)
{
if (files.end() != files.find(it->first))
throw Exception("Duplicate column with name " + it->first + " in constructor of StorageTinyLog.",
ErrorCodes::DUPLICATE_COLUMN);
ColumnData column_data;
files.insert(std::make_pair(it->first, column_data));
files[it->first].data_file = Poco::File(path + escapeForFileName(name) + '/' + escapeForFileName(it->first) + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
}
}
void StorageTinyLog::rename(const String & new_path_to_db, const String & new_name)
{
/// Переименовываем директорию с данными.
Poco::File(path + escapeForFileName(name)).renameTo(new_path_to_db + escapeForFileName(new_name));
path = new_path_to_db;
name = new_name;
for (NamesAndTypesList::const_iterator it = columns->begin(); it != columns->end(); ++it)
files[it->first].data_file = Poco::File(path + escapeForFileName(name) + '/' + escapeForFileName(it->first) + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
}
BlockInputStreams StorageTinyLog::read(
const Names & column_names,
ASTPtr query,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size,
unsigned threads)
{
check(column_names);
processed_stage = QueryProcessingStage::FetchColumns;
return BlockInputStreams(1, new TinyLogBlockInputStream(max_block_size, column_names, *this));
}
BlockOutputStreamPtr StorageTinyLog::write(
ASTPtr query)
{
return new TinyLogBlockOutputStream(*this);
}
void StorageTinyLog::drop()
{
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
if (it->second.data_file.exists())
it->second.data_file.remove();
}
}