2012-06-25 00:17:19 +00:00
|
|
|
|
#include <map>
|
2013-02-26 11:28:00 +00:00
|
|
|
|
#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>
|
2013-07-12 13:35:05 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypeNested.h>
|
2012-08-29 18:49:54 +00:00
|
|
|
|
|
|
|
|
|
#include <DB/Columns/ColumnArray.h>
|
2013-07-12 13:35:05 +00:00
|
|
|
|
#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>
|
2014-08-29 18:44:44 +00:00
|
|
|
|
#include <Poco/DirectoryIterator.h>
|
2012-06-25 00:17:19 +00:00
|
|
|
|
|
|
|
|
|
#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin"
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
using Poco::SharedPtr;
|
|
|
|
|
|
|
|
|
|
|
2014-03-19 10:45:13 +00:00
|
|
|
|
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
|
|
|
|
{
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-03-19 10:45:13 +00:00
|
|
|
|
String TinyLogBlockInputStream::getID() const
|
|
|
|
|
{
|
|
|
|
|
std::stringstream res;
|
|
|
|
|
res << "TinyLog(" << storage.getTableName() << ", " << &storage;
|
|
|
|
|
|
2014-10-26 00:01:36 +00:00
|
|
|
|
for (const auto & name : column_names)
|
|
|
|
|
res << ", " << name;
|
2014-03-19 10:45:13 +00:00
|
|
|
|
|
|
|
|
|
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;
|
|
|
|
|
}
|
|
|
|
|
|
2014-08-29 18:44:44 +00:00
|
|
|
|
{
|
|
|
|
|
/// если в папке нет файлов, то это значит, что таблица пока пуста
|
|
|
|
|
if (Poco::DirectoryIterator(storage.full_path()) == Poco::DirectoryIterator())
|
|
|
|
|
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));
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
/// Указатели на столбцы смещений, общие для столбцов из вложенных структур данных
|
|
|
|
|
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);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
bool read_offsets = true;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
/// Для вложенных структур запоминаем указатели на столбцы со смещениями
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&*column.type))
|
2013-07-16 14:55:01 +00:00
|
|
|
|
{
|
|
|
|
|
String name = DataTypeNested::extractNestedTableName(column.name);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
if (offset_columns.count(name) == 0)
|
|
|
|
|
offset_columns[name] = new ColumnArray::ColumnOffsets_t;
|
|
|
|
|
else
|
|
|
|
|
read_offsets = false; /// на предыдущих итерациях смещения уже считали вызовом readData
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
column.column = new ColumnArray(type_arr->getNestedType()->createColumn(), offset_columns[name]);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
column.column = column.type->createColumn();
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
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)
|
|
|
|
|
{
|
|
|
|
|
/// Для массивов используются отдельные потоки для размеров.
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
|
2012-08-29 18:49:54 +00:00
|
|
|
|
{
|
2013-07-16 14:55:01 +00:00
|
|
|
|
String size_name = DataTypeNested::extractNestedTableName(name) + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
|
2014-01-15 16:12:48 +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())));
|
2012-08-29 18:49:54 +00:00
|
|
|
|
|
|
|
|
|
addStream(name, *type_arr->getNestedType(), level + 1);
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&type))
|
2013-07-12 13:35:05 +00:00
|
|
|
|
{
|
|
|
|
|
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()));
|
2013-07-12 13:35:05 +00:00
|
|
|
|
|
|
|
|
|
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
|
|
|
|
|
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
|
2014-07-09 11:45:51 +00:00
|
|
|
|
addStream(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
|
2013-07-12 13:35:05 +00:00
|
|
|
|
}
|
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
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2013-07-16 14:55:01 +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
|
|
|
|
{
|
|
|
|
|
/// Для массивов требуется сначала десериализовать размеры, а потом значения.
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
|
2012-08-29 18:49:54 +00:00
|
|
|
|
{
|
2013-07-16 14:55:01 +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())
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
IColumn & nested_column = typeid_cast<ColumnArray &>(column).getData();
|
|
|
|
|
size_t nested_limit = typeid_cast<ColumnArray &>(column).getOffsets()[column.size() - 1];
|
2013-03-07 17:40:24 +00:00
|
|
|
|
readData(name, *type_arr->getNestedType(), nested_column, nested_limit, level + 1);
|
2012-08-30 20:35:02 +00:00
|
|
|
|
|
2013-03-07 17:40:24 +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
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&type))
|
2013-07-12 13:35:05 +00:00
|
|
|
|
{
|
|
|
|
|
type_nested->deserializeOffsets(
|
|
|
|
|
column,
|
|
|
|
|
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level)]->compressed,
|
|
|
|
|
limit);
|
|
|
|
|
|
|
|
|
|
if (column.size())
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
ColumnNested & column_nested = typeid_cast<ColumnNested &>(column);
|
|
|
|
|
|
2013-07-12 13:35:05 +00:00
|
|
|
|
NamesAndTypesList::const_iterator it = type_nested->getNestedTypesList()->begin();
|
|
|
|
|
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
|
|
|
|
|
{
|
|
|
|
|
readData(
|
2014-07-09 11:45:51 +00:00
|
|
|
|
DataTypeNested::concatenateNestedName(name, it->name),
|
|
|
|
|
*it->type,
|
2013-07-12 13:35:05 +00:00
|
|
|
|
*column_nested.getData()[i],
|
|
|
|
|
column_nested.getOffsets()[column.size() - 1],
|
|
|
|
|
level + 1);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-08-29 18:49:54 +00:00
|
|
|
|
else
|
2013-03-07 17:40:24 +00:00
|
|
|
|
type.deserializeBinary(column, streams[name]->compressed, limit);
|
2012-08-29 18:49:54 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-03-19 10:45:13 +00:00
|
|
|
|
TinyLogBlockOutputStream::TinyLogBlockOutputStream(StorageTinyLog & storage_)
|
|
|
|
|
: storage(storage_)
|
2012-06-25 00:17:19 +00:00
|
|
|
|
{
|
2014-10-10 15:45:43 +00:00
|
|
|
|
for (const auto & col : storage.getColumnsList())
|
|
|
|
|
addStream(col.name, *col.type);
|
2012-08-29 18:49:54 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void TinyLogBlockOutputStream::addStream(const String & name, const IDataType & type, size_t level)
|
|
|
|
|
{
|
|
|
|
|
/// Для массивов используются отдельные потоки для размеров.
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
|
2012-08-29 18:49:54 +00:00
|
|
|
|
{
|
2013-07-16 14:55:01 +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)));
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-08-29 18:49:54 +00:00
|
|
|
|
addStream(name, *type_arr->getNestedType(), level + 1);
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&type))
|
2013-07-12 13:35:05 +00:00
|
|
|
|
{
|
|
|
|
|
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));
|
2013-07-12 13:35:05 +00:00
|
|
|
|
|
|
|
|
|
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
|
|
|
|
|
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
|
2014-07-09 11:45:51 +00:00
|
|
|
|
addStream(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
|
2013-07-12 13:35:05 +00:00
|
|
|
|
}
|
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
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2013-07-16 14:55:01 +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
|
|
|
|
{
|
|
|
|
|
/// Для массивов требуется сначала сериализовать размеры, а потом значения.
|
2014-06-26 00:58:14 +00:00
|
|
|
|
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
|
2012-08-29 18:49:54 +00:00
|
|
|
|
{
|
2013-07-16 14:55:01 +00:00
|
|
|
|
String size_name = DataTypeNested::extractNestedTableName(name) + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
if (offset_columns.count(size_name) == 0)
|
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
offset_columns.insert(size_name);
|
2013-07-16 14:55:01 +00:00
|
|
|
|
type_arr->serializeOffsets(
|
|
|
|
|
column,
|
|
|
|
|
streams[size_name]->compressed);
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
|
|
|
|
writeData(name, *type_arr->getNestedType(), typeid_cast<const ColumnArray &>(column).getData(), offset_columns, level + 1);
|
2012-08-29 18:49:54 +00:00
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&type))
|
2013-07-12 13:35:05 +00:00
|
|
|
|
{
|
|
|
|
|
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
|
|
|
|
|
|
|
|
|
|
type_nested->serializeOffsets(column, streams[size_name]->compressed);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
|
|
|
|
const ColumnNested & column_nested = typeid_cast<const ColumnNested &>(column);
|
|
|
|
|
|
2013-07-12 13:35:05 +00:00
|
|
|
|
NamesAndTypesList::const_iterator it = type_nested->getNestedTypesList()->begin();
|
|
|
|
|
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
|
|
|
|
|
{
|
|
|
|
|
writeData(
|
2014-07-09 11:45:51 +00:00
|
|
|
|
DataTypeNested::concatenateNestedName(name, it->name),
|
|
|
|
|
*it->type,
|
2013-07-12 13:35:05 +00:00
|
|
|
|
*column_nested.getData()[i],
|
2013-07-16 14:55:01 +00:00
|
|
|
|
offset_columns,
|
2013-07-12 13:35:05 +00:00
|
|
|
|
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();
|
2014-07-31 19:19:56 +00:00
|
|
|
|
|
2014-08-04 06:36:24 +00:00
|
|
|
|
std::vector<Poco::File> column_files;
|
2014-08-01 13:19:27 +00:00
|
|
|
|
for (auto & pair : streams)
|
2014-08-04 06:36:24 +00:00
|
|
|
|
column_files.push_back(storage.files[pair.first].data_file);
|
2014-08-01 13:19:27 +00:00
|
|
|
|
|
2014-08-04 06:36:24 +00:00
|
|
|
|
storage.file_checker.update(column_files.begin(), column_files.end());
|
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);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
/// Множество записанных столбцов со смещениями, чтобы не писать общие для вложенных структур столбцы несколько раз
|
|
|
|
|
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);
|
2013-07-16 14:55:01 +00:00
|
|
|
|
writeData(column.name, *column.type, *column.column, offset_columns);
|
2012-06-25 00:17:19 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
|
StorageTinyLog::StorageTinyLog(
|
|
|
|
|
const std::string & path_,
|
|
|
|
|
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_,
|
|
|
|
|
bool attach,
|
|
|
|
|
size_t max_compress_block_size_)
|
2014-10-03 15:30:10 +00:00
|
|
|
|
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
|
|
|
|
|
path(path_), name(name_), columns(columns_),
|
2014-09-30 03:08:47 +00:00
|
|
|
|
max_compress_block_size(max_compress_block_size_),
|
|
|
|
|
file_checker(path + escapeForFileName(name) + '/' + "sizes.json", *this),
|
|
|
|
|
log(&Logger::get("StorageTinyLog"))
|
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);
|
2013-01-17 20:21:03 +00:00
|
|
|
|
|
2014-07-31 19:19:56 +00:00
|
|
|
|
String full_path = path + escapeForFileName(name) + '/';
|
2013-01-17 20:21:03 +00:00
|
|
|
|
if (!attach)
|
|
|
|
|
{
|
|
|
|
|
/// создаём файлы, если их нет
|
|
|
|
|
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
|
|
|
|
|
2014-10-10 15:45:43 +00:00
|
|
|
|
for (const auto & col : getColumnsList())
|
|
|
|
|
addFile(col.name, *col.type);
|
2012-08-29 18:49:54 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
|
StoragePtr StorageTinyLog::create(
|
|
|
|
|
const std::string & path_,
|
|
|
|
|
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_,
|
|
|
|
|
bool attach,
|
|
|
|
|
size_t max_compress_block_size_)
|
2013-02-06 11:26:35 +00:00
|
|
|
|
{
|
2014-09-30 03:08:47 +00:00
|
|
|
|
return (new StorageTinyLog{
|
2014-10-03 15:30:10 +00:00
|
|
|
|
path_, name_, columns_,
|
|
|
|
|
materialized_columns_, alias_columns_, column_defaults_,
|
2014-09-30 03:08:47 +00:00
|
|
|
|
attach, max_compress_block_size_
|
|
|
|
|
})->thisPtr();
|
2013-02-06 11:26:35 +00:00
|
|
|
|
}
|
|
|
|
|
|
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);
|
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
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);
|
2013-07-16 14:55:01 +00:00
|
|
|
|
String size_name = DataTypeNested::extractNestedTableName(column_name) + size_column_suffix;
|
2012-06-25 00:17:19 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +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);
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-08-29 18:49:54 +00:00
|
|
|
|
addFile(column_name, *type_arr->getNestedType(), level + 1);
|
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
else if (const DataTypeNested * type_nested = typeid_cast<const DataTypeNested *>(&type))
|
2013-07-12 13:35:05 +00:00
|
|
|
|
{
|
|
|
|
|
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)
|
2014-07-09 11:45:51 +00:00
|
|
|
|
addFile(DataTypeNested::concatenateNestedName(name, it->name), *it->type, level + 1);
|
2013-07-12 13:35:05 +00:00
|
|
|
|
}
|
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
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-07-28 14:33:05 +00:00
|
|
|
|
void StorageTinyLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
|
2012-06-25 00:17:19 +00:00
|
|
|
|
{
|
|
|
|
|
/// Переименовываем директорию с данными.
|
2014-07-28 14:33:05 +00:00
|
|
|
|
Poco::File(path + escapeForFileName(name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-06-25 00:17:19 +00:00
|
|
|
|
path = new_path_to_db;
|
2014-07-28 14:33:05 +00:00
|
|
|
|
name = new_table_name;
|
2014-08-06 13:22:52 +00:00
|
|
|
|
file_checker.setPath(path + escapeForFileName(name) + "/" + "sizes.json");
|
2012-06-25 00:17:19 +00:00
|
|
|
|
|
2012-08-29 18:49:54 +00:00
|
|
|
|
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
|
2013-02-26 11:37:49 +00:00
|
|
|
|
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,
|
2013-02-01 19:02:04 +00:00
|
|
|
|
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;
|
2014-03-19 10:45:13 +00:00
|
|
|
|
return BlockInputStreams(1, new TinyLogBlockInputStream(max_block_size, column_names, *this));
|
2012-06-25 00:17:19 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
|
2012-06-25 00:17:19 +00:00
|
|
|
|
BlockOutputStreamPtr StorageTinyLog::write(
|
|
|
|
|
ASTPtr query)
|
|
|
|
|
{
|
2014-03-19 10:45:13 +00:00
|
|
|
|
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();
|
|
|
|
|
}
|
|
|
|
|
|
2014-07-31 13:39:23 +00:00
|
|
|
|
bool StorageTinyLog::checkData() const
|
|
|
|
|
{
|
2014-08-05 12:50:20 +00:00
|
|
|
|
return file_checker.check();
|
2014-07-31 13:39:23 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-08-01 13:19:27 +00:00
|
|
|
|
StorageTinyLog::Files_t & StorageTinyLog::getFiles()
|
2014-07-31 13:39:23 +00:00
|
|
|
|
{
|
2014-08-01 13:19:27 +00:00
|
|
|
|
return files;
|
2014-07-31 13:39:23 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-07-31 19:19:56 +00:00
|
|
|
|
TinyLogBlockOutputStream::~TinyLogBlockOutputStream()
|
|
|
|
|
{
|
|
|
|
|
writeSuffix();
|
|
|
|
|
}
|
2014-07-31 13:39:23 +00:00
|
|
|
|
|
2012-06-25 00:17:19 +00:00
|
|
|
|
}
|