2010-03-18 19:32:14 +00:00
|
|
|
|
#include <map>
|
2013-02-26 11:48:09 +00:00
|
|
|
|
#include <Poco/Path.h>
|
2010-03-18 19:32:14 +00:00
|
|
|
|
|
2011-11-05 23:31:19 +00:00
|
|
|
|
#include <DB/Common/escapeForFileName.h>
|
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
|
#include <DB/Core/Exception.h>
|
|
|
|
|
#include <DB/Core/ErrorCodes.h>
|
|
|
|
|
|
2012-01-10 22:11:51 +00:00
|
|
|
|
#include <DB/IO/ReadHelpers.h>
|
2012-01-09 19:20:48 +00:00
|
|
|
|
#include <DB/IO/WriteHelpers.h>
|
|
|
|
|
|
2012-08-29 20:07:24 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypeArray.h>
|
2013-07-12 13:35:05 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypeNested.h>
|
2012-08-29 20:07:24 +00:00
|
|
|
|
|
|
|
|
|
#include <DB/Columns/ColumnArray.h>
|
2013-07-12 13:35:05 +00:00
|
|
|
|
#include <DB/Columns/ColumnNested.h>
|
2012-08-29 20:07:24 +00:00
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
|
#include <DB/Storages/StorageLog.h>
|
|
|
|
|
|
2014-01-17 15:19:20 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypeString.h>
|
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
|
|
2012-01-09 19:20:48 +00:00
|
|
|
|
#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin"
|
2013-02-26 13:06:01 +00:00
|
|
|
|
#define DBMS_STORAGE_LOG_MARKS_FILE_EXTENSION ".mrk"
|
|
|
|
|
#define DBMS_STORAGE_LOG_MARKS_FILE_NAME "__marks.mrk"
|
2012-01-09 19:20:48 +00:00
|
|
|
|
|
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
using Poco::SharedPtr;
|
|
|
|
|
|
|
|
|
|
|
2013-01-23 17:38:03 +00:00
|
|
|
|
LogBlockInputStream::LogBlockInputStream(size_t block_size_, const Names & column_names_, StoragePtr owned_storage, size_t mark_number_, size_t rows_limit_)
|
2014-01-17 15:19:20 +00:00
|
|
|
|
: IProfilingBlockInputStream(owned_storage), block_size(block_size_), column_names(column_names_), storage(dynamic_cast<StorageLog &>(*owned_storage)), mark_number(mark_number_), rows_limit(rows_limit_), rows_read(0), current_mark(mark_number_)
|
2010-03-18 19:32:14 +00:00
|
|
|
|
{
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2011-09-04 21:23:19 +00:00
|
|
|
|
Block LogBlockInputStream::readImpl()
|
2010-03-18 19:32:14 +00:00
|
|
|
|
{
|
|
|
|
|
Block res;
|
|
|
|
|
|
2012-06-22 17:13:03 +00:00
|
|
|
|
if (rows_read == rows_limit)
|
|
|
|
|
return res;
|
|
|
|
|
|
2012-06-21 16:16:58 +00:00
|
|
|
|
/// Если файлы не открыты, то открываем их.
|
|
|
|
|
if (streams.empty())
|
2012-11-30 04:28:13 +00:00
|
|
|
|
{
|
|
|
|
|
Poco::ScopedReadRWLock lock(storage.rwlock);
|
|
|
|
|
|
2012-06-21 16:16:58 +00:00
|
|
|
|
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
2014-01-17 15:19:20 +00:00
|
|
|
|
if (*it != storage._table_column_name) /// Для виртуального столбца не надо ничего открывать
|
|
|
|
|
addStream(*it, *storage.getDataTypeByName(*it));
|
2012-11-30 04:28:13 +00:00
|
|
|
|
}
|
2012-06-21 16:16:58 +00:00
|
|
|
|
|
2014-01-17 15:19:20 +00:00
|
|
|
|
bool has_virtual_column_table = false;
|
|
|
|
|
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
|
|
|
|
if (*it == storage._table_column_name)
|
|
|
|
|
has_virtual_column_table = true;
|
|
|
|
|
|
2012-06-22 16:54:51 +00:00
|
|
|
|
/// Сколько строк читать для следующего блока.
|
|
|
|
|
size_t max_rows_to_read = std::min(block_size, rows_limit - rows_read);
|
2014-01-17 15:19:20 +00:00
|
|
|
|
const Marks & marks = storage.getMarksWithRealRowCount();
|
|
|
|
|
std::pair<String, size_t> current_table;
|
|
|
|
|
|
|
|
|
|
/// Отдельно обрабатываем виртуальный столбец
|
|
|
|
|
if (has_virtual_column_table)
|
|
|
|
|
{
|
|
|
|
|
size_t current_row = rows_read;
|
|
|
|
|
if (mark_number > 0)
|
|
|
|
|
current_row += marks[mark_number-1].rows;
|
|
|
|
|
while (current_mark < marks.size() && marks[current_mark].rows <= current_row)
|
|
|
|
|
current_mark ++;
|
|
|
|
|
|
|
|
|
|
current_table = storage.getTableFromMark(current_mark);
|
|
|
|
|
current_table.second = std::min(current_table.second, marks.size() - 1);
|
|
|
|
|
max_rows_to_read = std::min(max_rows_to_read, marks[current_table.second].rows - current_row);
|
|
|
|
|
}
|
2012-06-22 16:54:51 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
/// Указатели на столбцы смещений, общие для столбцов из вложенных структур данных
|
|
|
|
|
typedef std::map<std::string, ColumnPtr> OffsetColumns;
|
|
|
|
|
OffsetColumns offset_columns;
|
|
|
|
|
|
2011-08-09 15:57:33 +00:00
|
|
|
|
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
2010-03-18 19:32:14 +00:00
|
|
|
|
{
|
2014-01-17 15:19:20 +00:00
|
|
|
|
/// Виртуальный столбец не надо считывать с жесткого диска
|
|
|
|
|
if (*it == storage._table_column_name)
|
|
|
|
|
continue;
|
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
|
ColumnWithNameAndType column;
|
|
|
|
|
column.name = *it;
|
2011-11-01 17:12:11 +00:00
|
|
|
|
column.type = storage.getDataTypeByName(*it);
|
2013-07-16 14:55:01 +00:00
|
|
|
|
|
|
|
|
|
bool read_offsets = true;
|
|
|
|
|
|
|
|
|
|
/// Для вложенных структур запоминаем указатели на столбцы со смещениями
|
|
|
|
|
if (const DataTypeArray * type_arr = dynamic_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, max_rows_to_read, 0, read_offsets);
|
2010-03-18 19:32:14 +00:00
|
|
|
|
|
2010-05-24 18:58:14 +00:00
|
|
|
|
if (column.column->size())
|
|
|
|
|
res.insert(column);
|
2010-03-18 19:32:14 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-01-17 15:19:20 +00:00
|
|
|
|
/// Отдельно обрабатываем виртуальный столбец
|
|
|
|
|
if (has_virtual_column_table)
|
|
|
|
|
{
|
|
|
|
|
size_t rows = max_rows_to_read;
|
|
|
|
|
if (res.columns() > 0)
|
|
|
|
|
rows = res.rows();
|
|
|
|
|
if (rows > 0)
|
|
|
|
|
{
|
2014-01-22 12:50:19 +00:00
|
|
|
|
ColumnPtr column_ptr = ColumnConst<String> (rows, current_table.first, new DataTypeString).convertToFullColumn();
|
2014-01-17 15:19:20 +00:00
|
|
|
|
ColumnWithNameAndType column(column_ptr, new DataTypeString, storage._table_column_name);
|
|
|
|
|
res.insert(column);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-03-05 02:34:20 +00:00
|
|
|
|
if (res)
|
2012-06-22 17:00:59 +00:00
|
|
|
|
rows_read += res.rows();
|
2012-06-22 16:54:51 +00:00
|
|
|
|
|
2012-06-22 17:00:59 +00:00
|
|
|
|
if (!res || rows_read == rows_limit)
|
2012-06-21 16:16:58 +00:00
|
|
|
|
{
|
|
|
|
|
/** Закрываем файлы (ещё до уничтожения объекта).
|
|
|
|
|
* Чтобы при создании многих источников, но одновременном чтении только из нескольких,
|
|
|
|
|
* буферы не висели в памяти.
|
|
|
|
|
*/
|
|
|
|
|
streams.clear();
|
|
|
|
|
}
|
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
|
return res;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2012-08-29 20:07:24 +00:00
|
|
|
|
void LogBlockInputStream::addStream(const String & name, const IDataType & type, size_t level)
|
|
|
|
|
{
|
|
|
|
|
/// Для массивов используются отдельные потоки для размеров.
|
|
|
|
|
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
|
|
|
|
|
{
|
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))
|
|
|
|
|
streams.insert(std::make_pair(size_name, new Stream(
|
|
|
|
|
storage.files[size_name].data_file.path(),
|
|
|
|
|
mark_number
|
|
|
|
|
? storage.files[size_name].marks[mark_number].offset
|
|
|
|
|
: 0)));
|
2012-08-29 20:07:24 +00:00
|
|
|
|
|
|
|
|
|
addStream(name, *type_arr->getNestedType(), level + 1);
|
|
|
|
|
}
|
2013-07-12 13:35:05 +00:00
|
|
|
|
else if (const DataTypeNested * type_nested = dynamic_cast<const DataTypeNested *>(&type))
|
|
|
|
|
{
|
|
|
|
|
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
|
2014-01-15 16:12:48 +00:00
|
|
|
|
streams[size_name] = new Stream(
|
2013-07-12 13:35:05 +00:00
|
|
|
|
storage.files[size_name].data_file.path(),
|
|
|
|
|
mark_number
|
|
|
|
|
? storage.files[size_name].marks[mark_number].offset
|
2014-01-15 16:12:48 +00:00
|
|
|
|
: 0);
|
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)
|
2013-07-16 14:55:01 +00:00
|
|
|
|
addStream(DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1);
|
2013-07-12 13:35:05 +00:00
|
|
|
|
}
|
2012-08-29 20:07:24 +00:00
|
|
|
|
else
|
2014-01-15 16:12:48 +00:00
|
|
|
|
streams[name] = new Stream(
|
2012-08-29 20:07:24 +00:00
|
|
|
|
storage.files[name].data_file.path(),
|
2013-06-15 08:38:30 +00:00
|
|
|
|
mark_number
|
|
|
|
|
? storage.files[name].marks[mark_number].offset
|
2014-01-15 16:12:48 +00:00
|
|
|
|
: 0);
|
2012-08-29 20:07:24 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
void LogBlockInputStream::readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read,
|
|
|
|
|
size_t level, bool read_offsets)
|
2012-08-29 20:07:24 +00:00
|
|
|
|
{
|
|
|
|
|
/// Для массивов требуется сначала десериализовать размеры, а потом значения.
|
|
|
|
|
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
|
|
|
|
|
{
|
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,
|
|
|
|
|
max_rows_to_read);
|
|
|
|
|
}
|
2012-08-29 20:07:24 +00:00
|
|
|
|
|
2012-08-30 20:35:02 +00:00
|
|
|
|
if (column.size())
|
|
|
|
|
readData(
|
|
|
|
|
name,
|
|
|
|
|
*type_arr->getNestedType(),
|
|
|
|
|
dynamic_cast<ColumnArray &>(column).getData(),
|
|
|
|
|
dynamic_cast<const ColumnArray &>(column).getOffsets()[column.size() - 1],
|
|
|
|
|
level + 1);
|
2012-08-29 20:07:24 +00:00
|
|
|
|
}
|
2013-07-12 13:35:05 +00:00
|
|
|
|
else if (const DataTypeNested * type_nested = dynamic_cast<const DataTypeNested *>(&type))
|
|
|
|
|
{
|
|
|
|
|
type_nested->deserializeOffsets(
|
|
|
|
|
column,
|
|
|
|
|
streams[name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level)]->compressed,
|
|
|
|
|
max_rows_to_read);
|
|
|
|
|
|
|
|
|
|
if (column.size())
|
|
|
|
|
{
|
|
|
|
|
ColumnNested & column_nested = dynamic_cast<ColumnNested &>(column);
|
|
|
|
|
|
|
|
|
|
NamesAndTypesList::const_iterator it = type_nested->getNestedTypesList()->begin();
|
|
|
|
|
for (size_t i = 0; i < column_nested.getData().size(); ++i, ++it)
|
|
|
|
|
{
|
|
|
|
|
readData(
|
2013-07-16 14:55:01 +00:00
|
|
|
|
DataTypeNested::concatenateNestedName(name, it->first),
|
2013-07-12 13:35:05 +00:00
|
|
|
|
*it->second,
|
|
|
|
|
*column_nested.getData()[i],
|
|
|
|
|
column_nested.getOffsets()[column.size() - 1],
|
|
|
|
|
level + 1);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-08-29 20:07:24 +00:00
|
|
|
|
else
|
|
|
|
|
type.deserializeBinary(column, streams[name]->compressed, max_rows_to_read);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2013-01-23 17:38:03 +00:00
|
|
|
|
LogBlockOutputStream::LogBlockOutputStream(StoragePtr owned_storage)
|
2013-02-26 13:06:01 +00:00
|
|
|
|
: IBlockOutputStream(owned_storage), storage(dynamic_cast<StorageLog &>(*owned_storage)),
|
|
|
|
|
lock(storage.rwlock), marks_stream(storage.marks_file.path(), 4096, O_APPEND | O_CREAT | O_WRONLY)
|
2010-03-18 19:32:14 +00:00
|
|
|
|
{
|
2011-11-01 17:12:11 +00:00
|
|
|
|
for (NamesAndTypesList::const_iterator it = storage.columns->begin(); it != storage.columns->end(); ++it)
|
2012-08-29 20:07:24 +00:00
|
|
|
|
addStream(it->first, *it->second);
|
2010-03-18 19:32:14 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void LogBlockOutputStream::write(const Block & block)
|
|
|
|
|
{
|
2013-02-26 11:47:15 +00:00
|
|
|
|
storage.check(block, true);
|
2013-07-16 14:55:01 +00:00
|
|
|
|
|
|
|
|
|
/// Множество записанных столбцов со смещениями, чтобы не писать общие для вложенных структур столбцы несколько раз
|
|
|
|
|
OffsetColumns offset_columns;
|
2010-03-18 19:32:14 +00:00
|
|
|
|
|
2013-02-26 13:06:01 +00:00
|
|
|
|
MarksForColumns marks;
|
|
|
|
|
marks.reserve(storage.files.size());
|
2010-03-18 19:32:14 +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, marks, offset_columns);
|
2012-08-29 20:07:24 +00:00
|
|
|
|
}
|
2013-02-26 13:06:01 +00:00
|
|
|
|
writeMarks(marks);
|
2012-08-29 20:07:24 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2013-09-15 01:40:29 +00:00
|
|
|
|
void LogBlockOutputStream::writeSuffix()
|
|
|
|
|
{
|
|
|
|
|
/// Заканчиваем запись.
|
2013-09-26 19:16:43 +00:00
|
|
|
|
marks_stream.next();
|
2013-09-15 01:40:29 +00:00
|
|
|
|
|
|
|
|
|
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-08-29 20:07:24 +00:00
|
|
|
|
void LogBlockOutputStream::addStream(const String & name, const IDataType & type, size_t level)
|
|
|
|
|
{
|
|
|
|
|
/// Для массивов используются отдельные потоки для размеров.
|
|
|
|
|
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
|
|
|
|
|
{
|
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))
|
|
|
|
|
streams.insert(std::make_pair(size_name, new Stream(
|
|
|
|
|
storage.files[size_name].data_file.path())));
|
2012-08-29 20:07:24 +00:00
|
|
|
|
|
|
|
|
|
addStream(name, *type_arr->getNestedType(), level + 1);
|
|
|
|
|
}
|
2013-07-12 13:35:05 +00:00
|
|
|
|
else if (const DataTypeNested * type_nested = dynamic_cast<const DataTypeNested *>(&type))
|
|
|
|
|
{
|
|
|
|
|
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
|
2014-01-15 16:12:48 +00:00
|
|
|
|
streams[size_name] = 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)
|
2013-07-16 14:55:01 +00:00
|
|
|
|
addStream(DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1);
|
2013-07-12 13:35:05 +00:00
|
|
|
|
}
|
2012-08-29 20:07:24 +00:00
|
|
|
|
else
|
2014-01-15 16:12:48 +00:00
|
|
|
|
streams[name] = new Stream(storage.files[name].data_file.path());
|
2012-08-29 20:07:24 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
void LogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, MarksForColumns & out_marks,
|
|
|
|
|
OffsetColumns & offset_columns, size_t level)
|
2012-08-29 20:07:24 +00:00
|
|
|
|
{
|
|
|
|
|
/// Для массивов требуется сначала сериализовать размеры, а потом значения.
|
|
|
|
|
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
|
|
|
|
|
{
|
2013-07-16 14:55:01 +00:00
|
|
|
|
String size_name = DataTypeNested::extractNestedTableName(name) + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
|
2013-02-26 13:06:01 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
if (offset_columns.count(size_name) == 0)
|
|
|
|
|
{
|
|
|
|
|
offset_columns.insert(size_name);
|
|
|
|
|
|
|
|
|
|
Mark mark;
|
|
|
|
|
mark.rows = (storage.files[size_name].marks.empty() ? 0 : storage.files[size_name].marks.back().rows) + column.size();
|
|
|
|
|
mark.offset = streams[size_name]->plain_offset + streams[size_name]->plain.count();
|
|
|
|
|
|
|
|
|
|
out_marks.push_back(std::make_pair(storage.files[size_name].column_index, mark));
|
|
|
|
|
|
|
|
|
|
type_arr->serializeOffsets(column, streams[size_name]->compressed);
|
|
|
|
|
streams[size_name]->compressed.next();
|
|
|
|
|
}
|
2012-08-29 20:07:24 +00:00
|
|
|
|
|
2013-07-16 14:55:01 +00:00
|
|
|
|
writeData(name, *type_arr->getNestedType(), dynamic_cast<const ColumnArray &>(column).getData(), out_marks, offset_columns, level + 1);
|
2012-08-29 20:07:24 +00:00
|
|
|
|
}
|
2013-07-12 13:35:05 +00:00
|
|
|
|
else if (const DataTypeNested * type_nested = dynamic_cast<const DataTypeNested *>(&type))
|
|
|
|
|
{
|
|
|
|
|
String size_name = name + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
|
|
|
|
|
|
|
|
|
|
Mark mark;
|
|
|
|
|
mark.rows = (storage.files[size_name].marks.empty() ? 0 : storage.files[size_name].marks.back().rows) + column.size();
|
|
|
|
|
mark.offset = streams[size_name]->plain_offset + streams[size_name]->plain.count();
|
|
|
|
|
|
|
|
|
|
out_marks.push_back(std::make_pair(storage.files[size_name].column_index, mark));
|
|
|
|
|
|
|
|
|
|
type_nested->serializeOffsets(column, streams[size_name]->compressed);
|
|
|
|
|
streams[size_name]->compressed.next();
|
|
|
|
|
|
|
|
|
|
const ColumnNested & column_nested = dynamic_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(
|
2013-07-16 14:55:01 +00:00
|
|
|
|
DataTypeNested::concatenateNestedName(name, it->first),
|
2013-07-12 13:35:05 +00:00
|
|
|
|
*it->second,
|
|
|
|
|
*column_nested.getData()[i],
|
|
|
|
|
out_marks,
|
2013-07-16 14:55:01 +00:00
|
|
|
|
offset_columns,
|
2013-07-12 13:35:05 +00:00
|
|
|
|
level + 1);
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-08-29 20:07:24 +00:00
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
Mark mark;
|
|
|
|
|
mark.rows = (storage.files[name].marks.empty() ? 0 : storage.files[name].marks.back().rows) + column.size();
|
2012-09-19 18:45:01 +00:00
|
|
|
|
mark.offset = streams[name]->plain_offset + streams[name]->plain.count();
|
2012-08-29 20:07:24 +00:00
|
|
|
|
|
2013-02-26 13:06:01 +00:00
|
|
|
|
out_marks.push_back(std::make_pair(storage.files[name].column_index, mark));
|
2012-08-29 20:07:24 +00:00
|
|
|
|
|
|
|
|
|
type.serializeBinary(column, streams[name]->compressed);
|
|
|
|
|
streams[name]->compressed.next();
|
2010-03-18 19:32:14 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-02-26 13:06:01 +00:00
|
|
|
|
static bool ColumnIndexLess(const std::pair<size_t, Mark> & a, const std::pair<size_t, Mark> & b)
|
|
|
|
|
{
|
|
|
|
|
return a.first < b.first;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void LogBlockOutputStream::writeMarks(MarksForColumns marks)
|
|
|
|
|
{
|
|
|
|
|
if (marks.size() != storage.files.size())
|
|
|
|
|
throw Exception("Wrong number of marks generated from block. Makes no sense.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
|
|
sort(marks.begin(), marks.end(), ColumnIndexLess);
|
|
|
|
|
|
|
|
|
|
for (size_t i = 0; i < marks.size(); ++i)
|
|
|
|
|
{
|
|
|
|
|
if (marks[i].first != i)
|
|
|
|
|
throw Exception("Invalid marks generated from block. Makes no sense.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
|
|
Mark mark = marks[i].second;
|
|
|
|
|
|
|
|
|
|
writeIntBinary(mark.rows, marks_stream);
|
|
|
|
|
writeIntBinary(mark.offset, marks_stream);
|
|
|
|
|
|
|
|
|
|
storage.files[storage.column_names[i]].marks.push_back(mark);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
|
|
2012-01-09 19:20:48 +00:00
|
|
|
|
StorageLog::StorageLog(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_)
|
2012-06-21 16:33:00 +00:00
|
|
|
|
: path(path_), name(name_), columns(columns_), loaded_marks(false)
|
2010-03-18 19:32:14 +00:00
|
|
|
|
{
|
2012-01-10 22:11:51 +00:00
|
|
|
|
if (columns->empty())
|
|
|
|
|
throw Exception("Empty list of columns passed to StorageLog constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
|
/// создаём файлы, если их нет
|
2011-11-05 23:31:19 +00:00
|
|
|
|
Poco::File(path + escapeForFileName(name) + '/').createDirectories();
|
2012-01-10 22:11:51 +00:00
|
|
|
|
|
2011-11-01 17:12:11 +00:00
|
|
|
|
for (NamesAndTypesList::const_iterator it = columns->begin(); it != columns->end(); ++it)
|
2012-08-29 20:07:24 +00:00
|
|
|
|
addFile(it->first, *it->second);
|
2013-02-26 13:06:01 +00:00
|
|
|
|
|
|
|
|
|
marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
|
2012-08-29 20:07:24 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-02-06 11:26:35 +00:00
|
|
|
|
StoragePtr StorageLog::create(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_)
|
|
|
|
|
{
|
|
|
|
|
return (new StorageLog(path_, name_, columns_))->thisPtr();
|
|
|
|
|
}
|
|
|
|
|
|
2012-08-29 20:07:24 +00:00
|
|
|
|
|
|
|
|
|
void StorageLog::addFile(const String & column_name, const IDataType & type, size_t level)
|
|
|
|
|
{
|
|
|
|
|
if (files.end() != files.find(column_name))
|
2013-07-12 13:35:05 +00:00
|
|
|
|
throw Exception("Duplicate column with name " + column_name + " in constructor of StorageLog.",
|
2012-08-29 20:07:24 +00:00
|
|
|
|
ErrorCodes::DUPLICATE_COLUMN);
|
|
|
|
|
|
|
|
|
|
if (const DataTypeArray * type_arr = dynamic_cast<const DataTypeArray *>(&type))
|
2010-03-18 19:32:14 +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;
|
2010-03-18 19:32:14 +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, ColumnData())).first->second;
|
|
|
|
|
column_data.column_index = column_names.size();
|
|
|
|
|
column_data.data_file = Poco::File(
|
|
|
|
|
path + escapeForFileName(name) + '/' + escapeForFileName(DataTypeNested::extractNestedTableName(column_name)) + size_column_suffix + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
|
|
|
|
|
|
|
|
|
|
column_names.push_back(size_name);
|
|
|
|
|
}
|
2012-08-29 20:07:24 +00:00
|
|
|
|
|
|
|
|
|
addFile(column_name, *type_arr->getNestedType(), level + 1);
|
|
|
|
|
}
|
2013-07-12 13:35:05 +00:00
|
|
|
|
else if (const DataTypeNested * type_nested = dynamic_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, ColumnData())).first->second;
|
|
|
|
|
column_data.column_index = column_names.size();
|
|
|
|
|
column_data.data_file = Poco::File(
|
|
|
|
|
path + escapeForFileName(name) + '/' + escapeForFileName(column_name) + size_column_suffix + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
|
|
|
|
|
|
|
|
|
|
column_names.push_back(column_name + size_column_suffix);
|
|
|
|
|
|
|
|
|
|
const NamesAndTypesList & columns = *type_nested->getNestedTypesList();
|
|
|
|
|
for (NamesAndTypesList::const_iterator it = columns.begin(); it != columns.end(); ++it)
|
2013-07-16 14:55:01 +00:00
|
|
|
|
addFile(DataTypeNested::concatenateNestedName(name, it->first), *it->second, level + 1);
|
2013-07-12 13:35:05 +00:00
|
|
|
|
}
|
2012-08-29 20:07:24 +00:00
|
|
|
|
else
|
|
|
|
|
{
|
2013-02-26 13:06:01 +00:00
|
|
|
|
ColumnData & column_data = files.insert(std::make_pair(column_name, ColumnData())).first->second;
|
|
|
|
|
column_data.column_index = column_names.size();
|
|
|
|
|
column_data.data_file = Poco::File(
|
2012-08-29 20:07:24 +00:00
|
|
|
|
path + escapeForFileName(name) + '/' + escapeForFileName(column_name) + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
|
2013-02-26 13:06:01 +00:00
|
|
|
|
|
|
|
|
|
column_names.push_back(column_name);
|
2012-06-21 16:33:00 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2012-01-10 22:11:51 +00:00
|
|
|
|
|
2012-06-21 16:33:00 +00:00
|
|
|
|
|
|
|
|
|
void StorageLog::loadMarks()
|
|
|
|
|
{
|
2012-11-30 04:28:13 +00:00
|
|
|
|
Poco::ScopedWriteRWLock lock(rwlock);
|
|
|
|
|
|
2012-06-21 16:33:00 +00:00
|
|
|
|
if (loaded_marks)
|
|
|
|
|
return;
|
|
|
|
|
|
2013-02-26 13:06:01 +00:00
|
|
|
|
typedef std::vector<Files_t::iterator> FilesByIndex;
|
|
|
|
|
FilesByIndex files_by_index(files.size());
|
2012-08-29 20:07:24 +00:00
|
|
|
|
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
|
2012-06-21 16:33:00 +00:00
|
|
|
|
{
|
2013-02-26 13:06:01 +00:00
|
|
|
|
files_by_index[it->second.column_index] = it;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (marks_file.exists())
|
|
|
|
|
{
|
|
|
|
|
size_t file_size = marks_file.getSize();
|
|
|
|
|
if (file_size % (files.size() * sizeof(Mark)) != 0)
|
|
|
|
|
throw Exception("Size of marks file is inconsistent", ErrorCodes::SIZES_OF_MARKS_FILES_ARE_INCONSISTENT);
|
|
|
|
|
|
|
|
|
|
int marks_count = file_size / (files.size() * sizeof(Mark));
|
2012-06-21 16:33:00 +00:00
|
|
|
|
|
2013-02-26 13:06:01 +00:00
|
|
|
|
for (size_t i = 0; i < files_by_index.size(); ++i)
|
|
|
|
|
{
|
|
|
|
|
files_by_index[i]->second.marks.reserve(marks_count);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
ReadBufferFromFile marks_rb(marks_file.path(), 32768);
|
|
|
|
|
while (!marks_rb.eof())
|
|
|
|
|
{
|
|
|
|
|
for (size_t i = 0; i < files_by_index.size(); ++i)
|
2012-01-10 22:11:51 +00:00
|
|
|
|
{
|
|
|
|
|
Mark mark;
|
|
|
|
|
readIntBinary(mark.rows, marks_rb);
|
|
|
|
|
readIntBinary(mark.offset, marks_rb);
|
2013-02-26 13:06:01 +00:00
|
|
|
|
files_by_index[i]->second.marks.push_back(mark);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-06-21 16:33:00 +00:00
|
|
|
|
|
|
|
|
|
loaded_marks = true;
|
2010-03-18 19:32:14 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2013-02-07 13:03:19 +00:00
|
|
|
|
size_t StorageLog::marksCount()
|
|
|
|
|
{
|
|
|
|
|
return files.begin()->second.marks.size();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2012-06-18 06:19:13 +00:00
|
|
|
|
void StorageLog::rename(const String & new_path_to_db, const String & new_name)
|
|
|
|
|
{
|
2012-11-30 04:28:13 +00:00
|
|
|
|
Poco::ScopedWriteRWLock lock(rwlock);
|
|
|
|
|
|
2012-06-18 06:19:13 +00:00
|
|
|
|
/// Переименовываем директорию с данными.
|
|
|
|
|
Poco::File(path + escapeForFileName(name)).renameTo(new_path_to_db + escapeForFileName(new_name));
|
|
|
|
|
|
|
|
|
|
path = new_path_to_db;
|
|
|
|
|
name = new_name;
|
|
|
|
|
|
2012-08-29 20:07:24 +00:00
|
|
|
|
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
|
2012-06-18 06:19:13 +00:00
|
|
|
|
{
|
2013-02-26 11:47:15 +00:00
|
|
|
|
it->second.data_file = Poco::File(path + escapeForFileName(name) + '/' + Poco::Path(it->second.data_file.path()).getFileName());
|
2012-06-18 06:19:13 +00:00
|
|
|
|
}
|
2013-02-26 13:06:01 +00:00
|
|
|
|
|
|
|
|
|
marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
|
2012-06-18 06:19:13 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2013-12-12 22:55:47 +00:00
|
|
|
|
const Marks & StorageLog::getMarksWithRealRowCount() const
|
|
|
|
|
{
|
|
|
|
|
const String & column_name = columns->front().first;
|
|
|
|
|
const IDataType & column_type = *columns->front().second;
|
|
|
|
|
String file_name;
|
|
|
|
|
|
|
|
|
|
/** Засечки достаём из первого столбца.
|
|
|
|
|
* Если это - массив, то берём засечки, соответствующие размерам, а не внутренностям массивов.
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
if (dynamic_cast<const DataTypeArray *>(&column_type))
|
|
|
|
|
{
|
|
|
|
|
file_name = DataTypeNested::extractNestedTableName(column_name) + ARRAY_SIZES_COLUMN_NAME_SUFFIX "0";
|
|
|
|
|
}
|
|
|
|
|
else if (dynamic_cast<const DataTypeNested *>(&column_type))
|
|
|
|
|
{
|
|
|
|
|
file_name = column_name + ARRAY_SIZES_COLUMN_NAME_SUFFIX "0";
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
file_name = column_name;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
Files_t::const_iterator it = files.find(file_name);
|
|
|
|
|
if (files.end() == it)
|
|
|
|
|
throw Exception("Cannot find file " + file_name, ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
|
|
return it->second.marks;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2012-01-09 19:20:48 +00:00
|
|
|
|
BlockInputStreams StorageLog::read(
|
2013-02-07 13:03:19 +00:00
|
|
|
|
size_t from_mark,
|
|
|
|
|
size_t to_mark,
|
2011-08-09 15:57:33 +00:00
|
|
|
|
const Names & column_names,
|
2011-08-15 01:12:57 +00:00
|
|
|
|
ASTPtr query,
|
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)
|
2010-03-18 19:32:14 +00:00
|
|
|
|
{
|
2013-06-15 08:38:30 +00:00
|
|
|
|
/** Если читаем все данные в один поток, то засечки не требуются.
|
|
|
|
|
* Отсутствие необходимости загружать засечки позволяет уменьшить потребление памяти при использовании таблицы типа ChunkMerger.
|
|
|
|
|
*/
|
|
|
|
|
bool read_all_data_in_one_thread = (threads == 1 && from_mark == 0 && to_mark == std::numeric_limits<size_t>::max());
|
|
|
|
|
if (!read_all_data_in_one_thread)
|
|
|
|
|
loadMarks();
|
|
|
|
|
|
2014-01-17 15:19:20 +00:00
|
|
|
|
bool has_virtual_column = false;
|
|
|
|
|
Names real_column_names;
|
|
|
|
|
for (const auto & column : column_names)
|
|
|
|
|
if (column != _table_column_name)
|
|
|
|
|
real_column_names.push_back(column);
|
|
|
|
|
else
|
|
|
|
|
has_virtual_column = true;
|
|
|
|
|
|
|
|
|
|
/// Если есть виртуальный столбец и нет остальных, то ничего проверять не надо
|
|
|
|
|
if (!(has_virtual_column && real_column_names.size() == 0))
|
|
|
|
|
check(real_column_names);
|
|
|
|
|
|
2012-05-22 18:32:45 +00:00
|
|
|
|
processed_stage = QueryProcessingStage::FetchColumns;
|
2012-01-10 22:11:51 +00:00
|
|
|
|
|
2012-11-30 04:28:13 +00:00
|
|
|
|
Poco::ScopedReadRWLock lock(rwlock);
|
|
|
|
|
|
2012-01-10 22:11:51 +00:00
|
|
|
|
BlockInputStreams res;
|
2013-06-15 08:38:30 +00:00
|
|
|
|
|
|
|
|
|
if (read_all_data_in_one_thread)
|
2012-01-10 22:11:51 +00:00
|
|
|
|
{
|
|
|
|
|
res.push_back(new LogBlockInputStream(
|
|
|
|
|
max_block_size,
|
|
|
|
|
column_names,
|
2013-01-23 17:38:03 +00:00
|
|
|
|
thisPtr(),
|
2013-06-15 08:38:30 +00:00
|
|
|
|
0, std::numeric_limits<size_t>::max()));
|
2012-01-10 22:11:51 +00:00
|
|
|
|
}
|
2013-06-15 08:38:30 +00:00
|
|
|
|
else
|
|
|
|
|
{
|
2013-12-12 22:55:47 +00:00
|
|
|
|
const Marks & marks = getMarksWithRealRowCount();
|
2013-06-15 08:38:30 +00:00
|
|
|
|
size_t marks_size = marks.size();
|
|
|
|
|
|
|
|
|
|
if (to_mark == std::numeric_limits<size_t>::max())
|
|
|
|
|
to_mark = marks_size;
|
|
|
|
|
|
|
|
|
|
if (to_mark > marks_size || to_mark < from_mark)
|
|
|
|
|
throw Exception("Marks out of range in StorageLog::read", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
|
|
if (threads > to_mark - from_mark)
|
|
|
|
|
threads = to_mark - from_mark;
|
|
|
|
|
|
|
|
|
|
for (size_t thread = 0; thread < threads; ++thread)
|
|
|
|
|
{
|
|
|
|
|
res.push_back(new LogBlockInputStream(
|
|
|
|
|
max_block_size,
|
|
|
|
|
column_names,
|
|
|
|
|
thisPtr(),
|
|
|
|
|
from_mark + thread * (to_mark - from_mark) / threads,
|
|
|
|
|
marks[from_mark + (thread + 1) * (to_mark - from_mark) / threads - 1].rows -
|
|
|
|
|
((thread == 0 && from_mark == 0)
|
|
|
|
|
? 0
|
|
|
|
|
: marks[from_mark + thread * (to_mark - from_mark) / threads - 1].rows)));
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-01-10 22:11:51 +00:00
|
|
|
|
return res;
|
2010-03-18 19:32:14 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-02-07 13:03:19 +00:00
|
|
|
|
|
|
|
|
|
BlockInputStreams StorageLog::read(
|
|
|
|
|
const Names & column_names,
|
|
|
|
|
ASTPtr query,
|
|
|
|
|
const Settings & settings,
|
|
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
|
|
|
|
size_t max_block_size,
|
|
|
|
|
unsigned threads)
|
|
|
|
|
{
|
2013-06-15 08:38:30 +00:00
|
|
|
|
return read(0, std::numeric_limits<size_t>::max(), column_names, query, settings, processed_stage, max_block_size, threads);
|
2013-02-07 13:03:19 +00:00
|
|
|
|
}
|
|
|
|
|
|
2013-02-11 08:53:34 +00:00
|
|
|
|
|
2011-08-28 02:22:23 +00:00
|
|
|
|
BlockOutputStreamPtr StorageLog::write(
|
2011-08-15 01:12:57 +00:00
|
|
|
|
ASTPtr query)
|
2010-03-18 19:32:14 +00:00
|
|
|
|
{
|
2012-06-21 16:33:00 +00:00
|
|
|
|
loadMarks();
|
2013-01-23 17:38:03 +00:00
|
|
|
|
return new LogBlockOutputStream(thisPtr());
|
2010-03-18 19:32:14 +00:00
|
|
|
|
}
|
|
|
|
|
|
2011-11-05 23:31:19 +00:00
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
|
}
|