ClickHouse/dbms/src/Storages/StorageTinyLog.cpp

398 lines
14 KiB
C++
Raw Normal View History

2012-06-25 00:17:19 +00:00
#include <map>
#include <Poco/Path.h>
2012-06-25 00:17:19 +00:00
#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>
2012-08-29 18:49:54 +00:00
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeNested.h>
2012-08-29 18:49:54 +00:00
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnNested.h>
2012-08-29 18:49:54 +00:00
2012-06-25 00:17:19 +00:00
#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_), finished(false)
2012-06-25 00:17:19 +00:00
{
}
String TinyLogBlockInputStream::getID() const
{
std::stringstream res;
res << "TinyLog(" << storage.getTableName() << ", " << &storage;
for (size_t i = 0; i < column_names.size(); ++i)
res << ", " << column_names[i];
res << ")";
return res.str();
}
2012-06-25 00:17:19 +00:00
Block TinyLogBlockInputStream::readImpl()
{
Block res;
2012-06-25 01:22:30 +00:00
if (finished || (!streams.empty() && streams.begin()->second->compressed.eof()))
{
/** Закрываем файлы (ещё до уничтожения объекта).
* Чтобы при создании многих источников, но одновременном чтении только из нескольких,
* буферы не висели в памяти.
*/
finished = true;
streams.clear();
return res;
}
2012-06-25 00:17:19 +00:00
/// Если файлы не открыты, то открываем их.
if (streams.empty())
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
2012-08-29 18:49:54 +00:00
addStream(*it, *storage.getDataTypeByName(*it));
/// Указатели на столбцы смещений, общие для столбцов из вложенных структур данных
typedef std::map<std::string, ColumnPtr> OffsetColumns;
OffsetColumns offset_columns;
2012-06-25 00:17:19 +00:00
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
{
ColumnWithNameAndType column;
column.name = *it;
column.type = storage.getDataTypeByName(*it);
bool read_offsets = true;
/// Для вложенных структур запоминаем указатели на столбцы со смещениями
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&*column.type))
{
String name = DataTypeNested::extractNestedTableName(column.name);
if (offset_columns.count(name) == 0)
offset_columns[name] = new ColumnArray::ColumnOffsets_t;
else
read_offsets = false; /// на предыдущих итерациях смещения уже считали вызовом readData
column.column = new ColumnArray(type_arr->getNestedType()->createColumn(), offset_columns[name]);
}
else
column.column = column.type->createColumn();
readData(*it, *column.type, *column.column, block_size, 0, read_offsets);
2012-06-25 00:17:19 +00:00
if (column.column->size())
res.insert(column);
}
2012-06-25 00:59:06 +00:00
if (!res || streams.begin()->second->compressed.eof())
2012-06-25 00:17:19 +00:00
{
2012-06-25 01:22:30 +00:00
finished = true;
2012-06-25 00:17:19 +00:00
streams.clear();
}
return res;
}
2012-08-29 18:49:54 +00:00
void TinyLogBlockInputStream::addStream(const String & name, const IDataType & type, size_t level)
{
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
2012-08-29 18:49:54 +00:00
{
String size_name = DataTypeNested::extractNestedTableName(name) + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
if (!streams.count(size_name))
2014-04-22 22:43:55 +00:00
streams.emplace(size_name, std::unique_ptr<Stream>(new Stream(storage.files[size_name].data_file.path())));
2012-08-29 18:49:54 +00:00
addStream(name, *type_arr->getNestedType(), level + 1);
}
else if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
2014-04-22 22:43:55 +00:00
streams[size_name].reset(new Stream(storage.files[size_name].data_file.path()));
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
addStream(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
}
2012-08-29 18:49:54 +00:00
else
2014-04-22 22:43:55 +00:00
streams[name].reset(new Stream(storage.files[name].data_file.path()));
2012-08-29 18:49:54 +00:00
}
void TinyLogBlockInputStream::readData(const String & name, const IDataType & type, IColumn & column, size_t limit, size_t level, bool read_offsets)
2012-08-29 18:49:54 +00:00
{
/// Для массивов требуется сначала десериализовать размеры, а потом значения.
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
2012-08-29 18:49:54 +00:00
{
if (read_offsets)
{
type_arr->deserializeOffsets(
column,
streams[DataTypeNested::extractNestedTableName(name) + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level)]->compressed,
limit);
}
2012-08-29 18:49:54 +00:00
2012-08-30 20:35:02 +00:00
if (column.size())
{
IColumn & nested_column = typeid_cast<ColumnArray &>(column).getData();
size_t nested_limit = typeid_cast<ColumnArray &>(column).getOffsets()[column.size() - 1];
readData(name, *type_arr->getNestedType(), nested_column, nested_limit, level + 1);
2012-08-30 20:35:02 +00:00
if (nested_column.size() != nested_limit)
2012-08-30 20:35:02 +00:00
throw Exception("Cannot read array data for all offsets", ErrorCodes::CANNOT_READ_ALL_DATA);
}
2012-08-29 18:49:54 +00:00
}
else if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&type))
{
type_nested->deserializeOffsets(
column,
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level)]->compressed,
limit);
if (column.size())
{
ColumnNested & column_nested = typeid_cast<ColumnNested &>(column);
NamesAndTypesList::const_iterator it = type_nested->getNestedTypesList()->begin();
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
{
readData(
DataTypeNested::concatenateNestedName(name, it->name),
*it->type,
*column_nested.getData()[i],
column_nested.getOffsets()[column.size() - 1],
level + 1);
}
}
}
2012-08-29 18:49:54 +00:00
else
type.deserializeBinary(column, streams[name]->compressed, limit);
2012-08-29 18:49:54 +00:00
}
TinyLogBlockOutputStream::TinyLogBlockOutputStream(StorageTinyLog & storage_)
: storage(storage_)
2012-06-25 00:17:19 +00:00
{
for (NamesAndTypesList::const_iterator it = storage.columns->begin(); it != storage.columns->end(); ++it)
addStream(it->name, *it->type);
2012-08-29 18:49:54 +00:00
}
void TinyLogBlockOutputStream::addStream(const String & name, const IDataType & type, size_t level)
{
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
2012-08-29 18:49:54 +00:00
{
String size_name = DataTypeNested::extractNestedTableName(name) + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
2014-01-20 16:41:24 +00:00
if (!streams.count(size_name))
2014-04-22 22:43:55 +00:00
streams.emplace(size_name, std::unique_ptr<Stream>(new Stream(storage.files[size_name].data_file.path(), storage.max_compress_block_size)));
2012-08-29 18:49:54 +00:00
addStream(name, *type_arr->getNestedType(), level + 1);
}
else if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
2014-04-22 22:43:55 +00:00
streams[size_name].reset(new Stream(storage.files[size_name].data_file.path(), storage.max_compress_block_size));
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
addStream(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
}
2012-08-29 18:49:54 +00:00
else
2014-04-22 22:43:55 +00:00
streams[name].reset(new Stream(storage.files[name].data_file.path(), storage.max_compress_block_size));
2012-08-29 18:49:54 +00:00
}
void TinyLogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column,
OffsetColumns & offset_columns, size_t level)
2012-08-29 18:49:54 +00:00
{
/// Для массивов требуется сначала сериализовать размеры, а потом значения.
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
2012-08-29 18:49:54 +00:00
{
String size_name = DataTypeNested::extractNestedTableName(name) + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
if (offset_columns.count(size_name) == 0)
{
offset_columns.insert(size_name);
type_arr->serializeOffsets(
column,
streams[size_name]->compressed);
}
writeData(name, *type_arr->getNestedType(), typeid_cast<const ColumnArray &>(column).getData(), offset_columns, level + 1);
2012-08-29 18:49:54 +00:00
}
else if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&type))
{
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
type_nested->serializeOffsets(column, streams[size_name]->compressed);
const ColumnNested & column_nested = typeid_cast<const ColumnNested &>(column);
NamesAndTypesList::const_iterator it = type_nested->getNestedTypesList()->begin();
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
{
writeData(
DataTypeNested::concatenateNestedName(name, it->name),
*it->type,
*column_nested.getData()[i],
offset_columns,
level + 1);
}
}
2012-08-29 18:49:54 +00:00
else
type.serializeBinary(column, streams[name]->compressed);
2012-06-25 00:17:19 +00:00
}
2013-09-15 01:40:29 +00:00
void TinyLogBlockOutputStream::writeSuffix()
{
/// Заканчиваем запись.
for (FileStreams::iterator it = streams.begin(); it != streams.end(); ++it)
2013-09-26 19:16:43 +00:00
it->second->finalize();
2013-09-15 01:40:29 +00:00
streams.clear();
}
2012-06-25 00:17:19 +00:00
void TinyLogBlockOutputStream::write(const Block & block)
{
2013-02-25 10:23:31 +00:00
storage.check(block, true);
/// Множество записанных столбцов со смещениями, чтобы не писать общие для вложенных структур столбцы несколько раз
OffsetColumns offset_columns;
2012-06-25 00:17:19 +00:00
for (size_t i = 0; i < block.columns(); ++i)
{
const ColumnWithNameAndType & column = block.getByPosition(i);
writeData(column.name, *column.type, *column.column, offset_columns);
2012-06-25 00:17:19 +00:00
}
}
2014-03-28 14:36:24 +00:00
StorageTinyLog::StorageTinyLog(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_, bool attach, size_t max_compress_block_size_)
: path(path_), name(name_), columns(columns_), max_compress_block_size(max_compress_block_size_)
2012-06-25 00:17:19 +00:00
{
if (columns->empty())
throw Exception("Empty list of columns passed to StorageTinyLog constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
if (!attach)
{
/// создаём файлы, если их нет
String full_path = path + escapeForFileName(name) + '/';
if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
throwFromErrno("Cannot create directory " + full_path, ErrorCodes::CANNOT_CREATE_DIRECTORY);
}
2012-06-25 00:17:19 +00:00
for (NamesAndTypesList::const_iterator it = columns->begin(); it != columns->end(); ++it)
addFile(it->name, *it->type);
2012-08-29 18:49:54 +00:00
}
2014-03-28 14:36:24 +00:00
StoragePtr StorageTinyLog::create(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_, bool attach, size_t max_compress_block_size_)
{
2014-03-28 14:36:24 +00:00
return (new StorageTinyLog(path_, name_, columns_, attach, max_compress_block_size_))->thisPtr();
}
2012-08-29 18:49:54 +00:00
void StorageTinyLog::addFile(const String & column_name, const IDataType & type, size_t level)
{
if (files.end() != files.find(column_name))
throw Exception("Duplicate column with name " + column_name + " in constructor of StorageTinyLog.",
ErrorCodes::DUPLICATE_COLUMN);
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
2012-06-25 00:17:19 +00:00
{
2013-06-21 20:34:19 +00:00
String size_column_suffix = ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
String size_name = DataTypeNested::extractNestedTableName(column_name) + size_column_suffix;
2012-06-25 00:17:19 +00:00
if (files.end() == files.find(size_name))
{
ColumnData column_data;
files.insert(std::make_pair(size_name, column_data));
files[size_name].data_file = Poco::File(
path + escapeForFileName(name) + '/' + escapeForFileName(DataTypeNested::extractNestedTableName(column_name)) + size_column_suffix + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
}
2012-08-29 18:49:54 +00:00
addFile(column_name, *type_arr->getNestedType(), level + 1);
}
else if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&type))
{
String size_column_suffix = ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
ColumnData column_data;
files.insert(std::make_pair(column_name + size_column_suffix, column_data));
files[column_name + size_column_suffix].data_file = Poco::File(
path + escapeForFileName(name) + '/' + escapeForFileName(column_name) + size_column_suffix + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
addFile(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
}
2012-08-29 18:49:54 +00:00
else
{
ColumnData column_data;
files.insert(std::make_pair(column_name, column_data));
files[column_name].data_file = Poco::File(
path + escapeForFileName(name) + '/' + escapeForFileName(column_name) + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
2012-06-25 00:17:19 +00:00
}
}
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));
2012-06-25 00:17:19 +00:00
path = new_path_to_db;
name = new_name;
2012-08-29 18:49:54 +00:00
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
it->second.data_file = Poco::File(path + escapeForFileName(name) + '/' + Poco::Path(it->second.data_file.path()).getFileName());
2012-06-25 00:17:19 +00:00
}
BlockInputStreams StorageTinyLog::read(
const Names & column_names,
ASTPtr query,
const Settings & settings,
2012-06-25 00:17:19 +00:00
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));
2012-06-25 00:17:19 +00:00
}
2012-06-25 00:17:19 +00:00
BlockOutputStreamPtr StorageTinyLog::write(
ASTPtr query)
{
return new TinyLogBlockOutputStream(*this);
2012-06-25 00:17:19 +00:00
}
2014-03-20 13:28:49 +00:00
void StorageTinyLog::drop()
2012-06-25 00:17:19 +00:00
{
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
if (it->second.data_file.exists())
it->second.data_file.remove();
}
}