ClickHouse/dbms/src/Storages/StorageLog.cpp

953 lines
30 KiB
C++
Raw Normal View History

#include <DB/Storages/StorageLog.h>
2011-11-05 23:31:19 +00:00
2015-10-05 01:35:28 +00:00
#include <DB/Common/Exception.h>
#include <DB/Common/StringUtils.h>
2010-03-18 19:32:14 +00:00
2015-01-25 05:07:51 +00:00
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/WriteBufferFromFile.h>
#include <DB/IO/CompressedReadBuffer.h>
#include <DB/IO/CompressedWriteBuffer.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>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeNested.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
2012-08-29 20:07:24 +00:00
2015-01-18 08:25:56 +00:00
#include <DB/DataStreams/IProfilingBlockInputStream.h>
2015-01-25 05:07:51 +00:00
#include <DB/DataStreams/IBlockOutputStream.h>
2015-01-18 08:25:56 +00:00
2012-08-29 20:07:24 +00:00
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnNullable.h>
2012-08-29 20:07:24 +00:00
#include <Poco/Path.h>
#include <Poco/DirectoryIterator.h>
2014-01-17 15:19:20 +00:00
2010-03-18 19:32:14 +00:00
2012-01-09 19:20:48 +00:00
#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin"
#define DBMS_STORAGE_LOG_MARKS_FILE_EXTENSION ".mrk"
#define DBMS_STORAGE_LOG_MARKS_FILE_NAME "__marks.mrk"
#define DBMS_STORAGE_LOG_NULL_MARKS_FILE_NAME "__null_marks.mrk"
#define DBMS_STORAGE_LOG_DATA_BINARY_NULL_MAP_EXTENSION ".null"
2012-01-09 19:20:48 +00:00
2010-03-18 19:32:14 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
extern const int DUPLICATE_COLUMN;
extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT;
}
2010-03-18 19:32:14 +00:00
2015-01-18 08:25:56 +00:00
class LogBlockInputStream : public IProfilingBlockInputStream
2010-03-18 19:32:14 +00:00
{
2015-01-18 08:25:56 +00:00
public:
LogBlockInputStream(
size_t block_size_, const Names & column_names_, StorageLog & storage_,
size_t mark_number_, size_t rows_limit_, size_t max_read_buffer_size_)
2016-08-05 15:44:19 +00:00
: block_size(block_size_),
column_names(column_names_),
column_types(column_names.size()),
storage(storage_),
mark_number(mark_number_),
null_mark_number(0),
2016-08-05 15:44:19 +00:00
rows_limit(rows_limit_),
2016-09-11 10:06:34 +00:00
max_read_buffer_size(max_read_buffer_size_)
{
}
LogBlockInputStream(
size_t block_size_, const Names & column_names_, StorageLog & storage_,
size_t mark_number_, size_t null_mark_number_, size_t rows_limit_, size_t max_read_buffer_size_)
2016-08-05 15:44:19 +00:00
: block_size(block_size_),
column_names(column_names_),
column_types(column_names.size()),
storage(storage_),
mark_number(mark_number_),
null_mark_number(null_mark_number_),
rows_limit(rows_limit_),
2016-09-11 10:06:34 +00:00
max_read_buffer_size(max_read_buffer_size_)
{
}
2015-01-18 08:25:56 +00:00
String getName() const override { return "Log"; }
2015-01-18 08:25:56 +00:00
String getID() const override
2015-01-18 08:25:56 +00:00
{
std::stringstream res;
res << "Log(" << storage.getTableName() << ", " << &storage << ", " << mark_number << ", " << rows_limit;
2010-03-18 19:32:14 +00:00
2015-01-18 08:25:56 +00:00
for (const auto & name : column_names)
res << ", " << name;
2010-03-18 19:32:14 +00:00
2015-01-18 08:25:56 +00:00
res << ")";
return res.str();
}
protected:
Block readImpl() override;
2016-08-24 00:39:38 +00:00
2015-01-18 08:25:56 +00:00
private:
size_t block_size;
Names column_names;
2016-08-05 02:28:34 +00:00
DataTypes column_types;
2015-01-18 08:25:56 +00:00
StorageLog & storage;
size_t mark_number; /// С какой засечки читать данные
size_t null_mark_number;
2015-01-18 08:25:56 +00:00
size_t rows_limit; /// Максимальное количество строк, которых можно прочитать
size_t rows_read = 0;
size_t max_read_buffer_size;
2015-01-18 08:25:56 +00:00
struct Stream
{
Stream(const std::string & data_path, size_t offset, size_t max_read_buffer_size)
: plain(data_path, std::min(max_read_buffer_size, Poco::File(data_path).getSize())),
2015-01-18 08:25:56 +00:00
compressed(plain)
{
if (offset)
plain.seek(offset);
}
ReadBufferFromFile plain;
CompressedReadBuffer compressed;
};
2016-08-05 02:40:45 +00:00
using FileStreams = std::map<std::string, std::unique_ptr<Stream>>;
2015-01-18 08:25:56 +00:00
FileStreams streams;
void addStream(const String & name, const IDataType & type, size_t level = 0);
void readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read, size_t level = 0, bool read_offsets = true);
};
class LogBlockOutputStream : public IBlockOutputStream
{
2015-01-18 08:25:56 +00:00
public:
LogBlockOutputStream(StorageLog & storage_)
: storage(storage_),
lock(storage.rwlock),
marks_stream(storage.marks_file.path(), 4096, O_APPEND | O_CREAT | O_WRONLY),
null_marks_stream(storage.has_nullable_columns ?
std::make_unique<WriteBufferFromFile>(storage.null_marks_file.path(), 4096, O_APPEND | O_CREAT | O_WRONLY) : nullptr)
2015-01-18 08:25:56 +00:00
{
for (const auto & column : storage.getColumnsList())
addStream(column.name, *column.type);
}
~LogBlockOutputStream() override
2015-04-02 23:58:26 +00:00
{
try
{
writeSuffix();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void write(const Block & block) override;
void writeSuffix() override;
2015-04-02 23:58:26 +00:00
2015-01-18 08:25:56 +00:00
private:
StorageLog & storage;
Poco::ScopedWriteRWLock lock;
2015-04-02 23:58:26 +00:00
bool done = false;
2015-01-18 08:25:56 +00:00
struct Stream
{
Stream(const std::string & data_path, size_t max_compress_block_size) :
plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY),
compressed(plain, CompressionMethod::LZ4, max_compress_block_size)
2015-01-18 08:25:56 +00:00
{
plain_offset = Poco::File(data_path).getSize();
}
WriteBufferFromFile plain;
CompressedWriteBuffer compressed;
size_t plain_offset; /// Сколько байт было в файле на момент создания LogBlockOutputStream.
void finalize()
{
compressed.next();
plain.next();
}
};
using MarksForColumns = std::vector<std::pair<size_t, Mark> >;
2015-01-18 08:25:56 +00:00
2016-08-05 02:40:45 +00:00
using FileStreams = std::map<std::string, std::unique_ptr<Stream>>;
2015-01-18 08:25:56 +00:00
FileStreams streams;
using OffsetColumns = std::set<std::string>;
2015-01-18 08:25:56 +00:00
WriteBufferFromFile marks_stream; /// Объявлен ниже lock, чтобы файл открывался при захваченном rwlock.
std::unique_ptr<WriteBufferFromFile> null_marks_stream;
2015-01-18 08:25:56 +00:00
void addStream(const String & name, const IDataType & type, size_t level = 0);
void addNullStream(const String & name);
void writeData(const String & name, const IDataType & type, const IColumn & column,
MarksForColumns & out_marks, MarksForColumns & out_null_marks,
OffsetColumns & offset_columns, size_t level = 0);
2016-08-24 00:39:38 +00:00
void writeMarks(MarksForColumns marks, bool write_null_marks);
2015-01-18 08:25:56 +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;
/// If there are no files in the folder, the table is empty.
if (Poco::DirectoryIterator(storage.getFullPath()) == Poco::DirectoryIterator())
return res;
/// Если файлы не открыты, то открываем их.
if (streams.empty())
{
Poco::ScopedReadRWLock lock(storage.rwlock);
2016-08-05 02:28:34 +00:00
for (size_t i = 0, size = column_names.size(); i < size; ++i)
{
const auto & name = column_names[i];
column_types[i] = storage.getDataTypeByName(name);
2016-12-08 02:49:04 +00:00
addStream(name, *column_types[i]);
2016-08-05 02:28:34 +00:00
}
}
2012-06-22 16:54:51 +00:00
/// Сколько строк читать для следующего блока.
size_t max_rows_to_read = std::min(block_size, rows_limit - rows_read);
/// Указатели на столбцы смещений, общие для столбцов из вложенных структур данных
using OffsetColumns = std::map<std::string, ColumnPtr>;
OffsetColumns offset_columns;
2016-08-05 02:28:34 +00:00
for (size_t i = 0, size = column_names.size(); i < size; ++i)
2010-03-18 19:32:14 +00:00
{
2016-08-05 02:28:34 +00:00
const auto & name = column_names[i];
ColumnWithTypeAndName column;
2016-08-05 02:28:34 +00:00
column.name = name;
column.type = column_types[i];
bool read_offsets = true;
2016-07-14 12:58:30 +00:00
const IDataType * observed_type;
2016-08-24 00:39:38 +00:00
bool is_nullable;
if (column.type->isNullable())
2016-07-14 12:58:30 +00:00
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*column.type);
2016-07-14 12:58:30 +00:00
observed_type = nullable_type.getNestedType().get();
2016-08-24 00:39:38 +00:00
is_nullable = true;
2016-07-14 12:58:30 +00:00
}
else
2016-08-24 00:39:38 +00:00
{
2016-07-14 12:58:30 +00:00
observed_type = column.type.get();
2016-08-24 00:39:38 +00:00
is_nullable = false;
}
2016-07-14 12:58:30 +00:00
/// Для вложенных структур запоминаем указатели на столбцы со смещениями
2016-07-14 12:58:30 +00:00
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(observed_type))
{
String name = DataTypeNested::extractNestedTableName(column.name);
if (offset_columns.count(name) == 0)
offset_columns[name] = std::make_shared<ColumnArray::ColumnOffsets_t>();
else
read_offsets = false; /// на предыдущих итерациях смещения уже считали вызовом readData
column.column = std::make_shared<ColumnArray>(type_arr->getNestedType()->createColumn(), offset_columns[name]);
2016-08-24 00:39:38 +00:00
if (is_nullable)
column.column = std::make_shared<ColumnNullable>(column.column, std::make_shared<ColumnUInt8>());
}
else
column.column = column.type->createColumn();
try
{
2016-08-05 02:28:34 +00:00
readData(name, *column.type, *column.column, max_rows_to_read, 0, read_offsets);
}
catch (Exception & e)
{
2016-08-05 02:28:34 +00:00
e.addMessage("while reading column " + name + " at " + storage.path + escapeForFileName(storage.name));
throw;
}
2010-03-18 19:32:14 +00:00
2010-05-24 18:58:14 +00:00
if (column.column->size())
res.insert(std::move(column));
2010-03-18 19:32:14 +00:00
}
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)
{
/** Закрываем файлы (ещё до уничтожения объекта).
* Чтобы при создании многих источников, но одновременном чтении только из нескольких,
* буферы не висели в памяти.
*/
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 (type.isNullable())
2012-08-29 20:07:24 +00:00
{
/// First create the stream that handles the null map of the given column.
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(type);
const IDataType & nested_type = *nullable_type.getNestedType();
std::string filename = name + DBMS_STORAGE_LOG_DATA_BINARY_NULL_MAP_EXTENSION;
streams.emplace(filename, std::make_unique<Stream>(
storage.files[filename].data_file.path(),
null_mark_number
? storage.files[filename].marks[null_mark_number].offset
: 0,
max_read_buffer_size));
/// Then create the stream that handles the data of the given column.
addStream(name, nested_type, level);
}
else if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
/// Для массивов используются отдельные потоки для размеров.
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(),
mark_number
? storage.files[size_name].marks[mark_number].offset
: 0,
max_read_buffer_size)));
2012-08-29 20:07:24 +00:00
addStream(name, *type_arr->getNestedType(), level + 1);
}
else
2016-09-23 20:33:06 +00:00
streams[name] = std::make_unique<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
: 0,
2016-09-23 20:33:06 +00:00
max_read_buffer_size);
2012-08-29 20:07:24 +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 (type.isNullable())
2012-08-29 20:07:24 +00:00
{
/// First read from the null map.
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(type);
const IDataType & nested_type = *nullable_type.getNestedType();
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(column);
IColumn & nested_col = *nullable_col.getNestedColumn();
DataTypeUInt8{}.deserializeBinaryBulk(nullable_col.getNullMapConcreteColumn(),
streams[name + DBMS_STORAGE_LOG_DATA_BINARY_NULL_MAP_EXTENSION]->compressed, max_rows_to_read, 0);
/// Then read data.
readData(name, nested_type, nested_col, max_rows_to_read, level, read_offsets);
}
else if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
/// Для массивов требуется сначала десериализовать размеры, а потом значения.
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(),
typeid_cast<ColumnArray &>(column).getData(),
typeid_cast<const ColumnArray &>(column).getOffsets()[column.size() - 1],
2012-08-30 20:35:02 +00:00
level + 1);
2012-08-29 20:07:24 +00:00
}
else
type.deserializeBinaryBulk(column, streams[name]->compressed, max_rows_to_read, 0); /// TODO Использовать avg_value_size_hint.
2012-08-29 20:07:24 +00:00
}
2010-03-18 19:32:14 +00:00
void LogBlockOutputStream::write(const Block & block)
{
storage.check(block, true);
/// Множество записанных столбцов со смещениями, чтобы не писать общие для вложенных структур столбцы несколько раз
OffsetColumns offset_columns;
2010-03-18 19:32:14 +00:00
MarksForColumns marks;
marks.reserve(storage.file_count);
MarksForColumns null_marks;
if (null_marks_stream)
null_marks.reserve(storage.null_file_count);
2010-03-18 19:32:14 +00:00
for (size_t i = 0; i < block.columns(); ++i)
{
const ColumnWithTypeAndName & column = block.safeGetByPosition(i);
writeData(column.name, *column.type, *column.column, marks, null_marks, offset_columns);
2012-08-29 20:07:24 +00:00
}
2016-08-24 00:39:38 +00:00
writeMarks(marks, false);
if (null_marks_stream)
2016-08-24 00:39:38 +00:00
writeMarks(null_marks, true);
2012-08-29 20:07:24 +00:00
}
2013-09-15 01:40:29 +00:00
void LogBlockOutputStream::writeSuffix()
{
2015-04-02 23:58:26 +00:00
if (done)
return;
done = true;
2013-09-15 01:40:29 +00:00
/// Заканчиваем запись.
2013-09-26 19:16:43 +00:00
marks_stream.next();
if (null_marks_stream)
null_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
std::vector<Poco::File> column_files;
for (auto & pair : streams)
column_files.push_back(storage.files[pair.first].data_file);
column_files.push_back(storage.marks_file);
storage.file_checker.update(column_files.begin(), column_files.end());
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 (type.isNullable())
2012-08-29 20:07:24 +00:00
{
/// First create the stream that handles the null map of the given column.
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(type);
const IDataType & nested_type = *nullable_type.getNestedType();
std::string filename = name + DBMS_STORAGE_LOG_DATA_BINARY_NULL_MAP_EXTENSION;
streams.emplace(filename, std::make_unique<Stream>(storage.files[filename].data_file.path(),
storage.max_compress_block_size));
/// Then create the stream that handles the data of the given column.
addStream(name, nested_type, level);
}
else if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
/// Для массивов используются отдельные потоки для размеров.
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(
2014-03-28 14:36:24 +00:00
storage.files[size_name].data_file.path(), storage.max_compress_block_size)));
2012-08-29 20:07:24 +00:00
addStream(name, *type_arr->getNestedType(), level + 1);
}
else
2016-09-23 20:33:06 +00:00
streams[name] = std::make_unique<Stream>(storage.files[name].data_file.path(), storage.max_compress_block_size);
2012-08-29 20:07:24 +00:00
}
void LogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column,
MarksForColumns & out_marks, MarksForColumns & out_null_marks,
OffsetColumns & offset_columns, size_t level)
2012-08-29 20:07:24 +00:00
{
if (type.isNullable())
2012-08-29 20:07:24 +00:00
{
/// First write to the null map.
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(type);
const IDataType & nested_type = *nullable_type.getNestedType();
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(column);
const IColumn & nested_col = *nullable_col.getNestedColumn();
std::string filename = name + DBMS_STORAGE_LOG_DATA_BINARY_NULL_MAP_EXTENSION;
Mark mark;
mark.rows = (storage.files[filename].marks.empty() ? 0 : storage.files[filename].marks.back().rows) + column.size();
mark.offset = streams[filename]->plain_offset + streams[filename]->plain.count();
out_null_marks.emplace_back(storage.files[filename].column_index, mark);
DataTypeUInt8{}.serializeBinaryBulk(nullable_col.getNullMapConcreteColumn(), streams[filename]->compressed, 0, 0);
streams[filename]->compressed.next();
/// Then write data.
writeData(name, nested_type, nested_col, out_marks, out_null_marks, offset_columns, level);
}
else if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
/// Для массивов требуется сначала сериализовать размеры, а потом значения.
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);
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, 0, 0);
streams[size_name]->compressed.next();
}
2012-08-29 20:07:24 +00:00
writeData(name, *type_arr->getNestedType(), typeid_cast<const ColumnArray &>(column).getData(),
out_marks, out_null_marks, offset_columns, 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
out_marks.push_back(std::make_pair(storage.files[name].column_index, mark));
2012-08-29 20:07:24 +00:00
type.serializeBinaryBulk(column, streams[name]->compressed, 0, 0);
2012-08-29 20:07:24 +00:00
streams[name]->compressed.next();
2010-03-18 19:32:14 +00:00
}
}
static bool ColumnIndexLess(const std::pair<size_t, Mark> & a, const std::pair<size_t, Mark> & b)
{
return a.first < b.first;
}
2016-08-24 00:39:38 +00:00
void LogBlockOutputStream::writeMarks(MarksForColumns marks, bool write_null_marks)
{
2016-08-24 00:39:38 +00:00
size_t count = write_null_marks ? storage.null_file_count : storage.file_count;
WriteBufferFromFile & stream = write_null_marks ? *null_marks_stream : marks_stream;
const Names & names = write_null_marks ? storage.null_map_filenames : storage.column_names;
if (marks.size() != count)
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)
{
Mark mark = marks[i].second;
writeIntBinary(mark.rows, stream);
writeIntBinary(mark.offset, stream);
2016-08-24 00:39:38 +00:00
size_t column_index = marks[i].first;
storage.files[names[column_index]].marks.push_back(mark);
}
}
StorageLog::StorageLog(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
size_t max_compress_block_size_)
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
path(path_), name(name_), columns(columns_),
loaded_marks(false), max_compress_block_size(max_compress_block_size_),
file_checker(path + escapeForFileName(name) + '/' + "sizes.json")
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
for (const auto & column : getColumnsList())
addFile(column.name, *column.type);
marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
if (has_nullable_columns)
null_marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_NULL_MARKS_FILE_NAME);
2012-08-29 20:07:24 +00:00
}
StoragePtr StorageLog::create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
size_t max_compress_block_size_)
{
return make_shared(
path_, name_, columns_,
materialized_columns_, alias_columns_, column_defaults_,
max_compress_block_size_
);
}
StoragePtr StorageLog::create(
const std::string & path_,
const std::string & name_,
NamesAndTypesListPtr columns_,
size_t max_compress_block_size_)
{
return make_shared(
path_, name_, columns_,
NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{},
max_compress_block_size_
);
}
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))
throw Exception("Duplicate column with name " + column_name + " in constructor of StorageLog.",
2012-08-29 20:07:24 +00:00
ErrorCodes::DUPLICATE_COLUMN);
if (type.isNullable())
{
/// First add the file describing the null map of the column.
has_nullable_columns = true;
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(type);
const IDataType & actual_type = *nullable_type.getNestedType();
std::string filename = column_name + DBMS_STORAGE_LOG_DATA_BINARY_NULL_MAP_EXTENSION;
ColumnData & column_data = files.emplace(filename, ColumnData{}).first->second;
++null_file_count;
2016-08-24 00:39:38 +00:00
column_data.column_index = null_map_filenames.size();
column_data.data_file = Poco::File{
path + escapeForFileName(name) + '/' + escapeForFileName(column_name)
+ DBMS_STORAGE_LOG_DATA_BINARY_NULL_MAP_EXTENSION};
2016-08-24 00:39:38 +00:00
null_map_filenames.push_back(filename);
/// Then add the file describing the column data.
addFile(column_name, actual_type, level);
}
else if (const DataTypeArray * type_arr = typeid_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);
String size_name = DataTypeNested::extractNestedTableName(column_name) + size_column_suffix;
2010-03-18 19:32:14 +00:00
if (files.end() == files.find(size_name))
{
ColumnData & column_data = files.insert(std::make_pair(size_name, ColumnData())).first->second;
++file_count;
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);
}
else
{
ColumnData & column_data = files.insert(std::make_pair(column_name, ColumnData())).first->second;
++file_count;
column_data.column_index = column_names.size();
column_data.data_file = Poco::File{
path + escapeForFileName(name) + '/'
+ escapeForFileName(column_name) + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION};
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()
{
Poco::ScopedWriteRWLock lock(rwlock);
2012-06-21 16:33:00 +00:00
if (loaded_marks)
return;
loadMarksImpl(false);
if (has_nullable_columns)
loadMarksImpl(true);
loaded_marks = true;
}
void StorageLog::loadMarksImpl(bool load_null_marks)
{
using FilesByIndex = std::vector<Files_t::iterator>;
size_t count = load_null_marks ? null_file_count : file_count;
Poco::File & marks_file_handle = load_null_marks ? null_marks_file : marks_file;
FilesByIndex files_by_index(count);
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
2012-06-21 16:33:00 +00:00
{
bool has_null_extension = endsWith(it->first, DBMS_STORAGE_LOG_DATA_BINARY_NULL_MAP_EXTENSION);
if (!load_null_marks && has_null_extension)
continue;
if (load_null_marks && !has_null_extension)
continue;
files_by_index[it->second.column_index] = it;
}
if (marks_file_handle.exists())
{
size_t file_size = marks_file_handle.getSize();
if (file_size % (count * sizeof(Mark)) != 0)
throw Exception("Size of marks file is inconsistent", ErrorCodes::SIZES_OF_MARKS_FILES_ARE_INCONSISTENT);
int marks_count = file_size / (count * sizeof(Mark));
2012-06-21 16:33:00 +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_handle.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);
files_by_index[i]->second.marks.push_back(mark);
}
}
}
2010-03-18 19:32:14 +00:00
}
2016-12-10 04:58:40 +00:00
size_t StorageLog::marksCount()
{
return files.begin()->second.marks.size();
}
void StorageLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
2012-06-18 06:19: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_table_name));
2012-06-18 06:19:13 +00:00
path = new_path_to_db;
name = new_table_name;
file_checker.setPath(path + escapeForFileName(name) + '/' + "sizes.json");
2012-06-18 06:19:13 +00:00
2012-08-29 20:07:24 +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());
marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
2016-07-13 17:00:10 +00:00
if (has_nullable_columns)
null_marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_NULL_MARKS_FILE_NAME);
2012-06-18 06:19:13 +00:00
}
const Marks & StorageLog::getMarksWithRealRowCount() const
{
auto init_column_type = [&]()
{
const IDataType * type = columns->front().type.get();
if (type->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*type);
type = nullable_type.getNestedType().get();
}
return type;
};
const String & column_name = columns->front().name;
const IDataType & column_type = *init_column_type();
String filename;
/** Засечки достаём из первого столбца.
* Если это - массив, то берём засечки, соответствующие размерам, а не внутренностям массивов.
*/
if (typeid_cast<const DataTypeArray *>(&column_type))
filename = DataTypeNested::extractNestedTableName(column_name) + ARRAY_SIZES_COLUMN_NAME_SUFFIX "0";
else
filename = column_name;
Files_t::const_iterator it = files.find(filename);
if (files.end() == it)
throw Exception("Cannot find file " + filename, ErrorCodes::LOGICAL_ERROR);
return it->second.marks;
}
2012-01-09 19:20:48 +00:00
BlockInputStreams StorageLog::read(
size_t from_mark,
size_t to_mark,
size_t from_null_mark,
2011-08-09 15:57:33 +00:00
const Names & column_names,
2011-08-15 01:12:57 +00:00
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)
2010-03-18 19:32:14 +00:00
{
2013-06-15 08:38:30 +00:00
/** Если читаем все данные в один поток, то засечки не требуются.
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
* Отсутствие необходимости загружать засечки позволяет уменьшить потребление памяти.
2013-06-15 08:38:30 +00:00
*/
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();
2016-12-08 02:49:04 +00:00
check(column_names);
2014-01-17 15:19:20 +00:00
2012-05-22 18:32:45 +00:00
processed_stage = QueryProcessingStage::FetchColumns;
2012-01-10 22:11:51 +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(std::make_shared<LogBlockInputStream>(
2012-01-10 22:11:51 +00:00
max_block_size,
column_names,
*this,
0, marksCount() ? std::numeric_limits<size_t>::max() : 0,
settings.max_read_buffer_size));
2012-01-10 22:11:51 +00:00
}
else
{
const Marks & marks = getMarksWithRealRowCount();
size_t marks_size = marks.size();
/// Given a thread, return the start of the area from which
/// it can read data, i.e. a mark number.
auto mark_from_thread = [&](size_t thread)
{
/// The computation below reflects the fact that marks
/// are uniformly distributed among threads.
return from_mark + thread * (to_mark - from_mark) / threads;
};
/// Given a thread, get the parameters that specify the area
/// from which it can read data, i.e. a mark number and a
/// maximum number of rows.
auto get_reader_parameters = [&](size_t thread)
{
size_t mark_number = mark_from_thread(thread);
2016-08-24 00:39:38 +00:00
size_t cur_total_row_count = ((thread == 0 && from_mark == 0)
? 0
: marks[mark_number - 1].rows);
size_t next_total_row_count = marks[mark_from_thread(thread + 1) - 1].rows;
2016-08-24 00:39:38 +00:00
size_t rows_limit = next_total_row_count - cur_total_row_count;
return std::make_pair(mark_number, rows_limit);
};
2013-06-15 08:38:30 +00:00
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;
if (has_nullable_columns)
2013-06-15 08:38:30 +00:00
{
for (size_t thread = 0; thread < threads; ++thread)
{
size_t mark_number;
size_t rows_limit;
std::tie(mark_number, rows_limit) = get_reader_parameters(thread);
/// This works since we have the same number of marks and null marks.
size_t null_mark_number = from_null_mark + (mark_number - from_mark);
res.push_back(std::make_shared<LogBlockInputStream>(
max_block_size,
column_names,
*this,
mark_number,
null_mark_number,
rows_limit,
settings.max_read_buffer_size));
}
}
else
{
for (size_t thread = 0; thread < threads; ++thread)
{
size_t mark_number;
size_t rows_limit;
std::tie(mark_number, rows_limit) = get_reader_parameters(thread);
res.push_back(std::make_shared<LogBlockInputStream>(
max_block_size,
column_names,
*this,
mark_number,
rows_limit,
settings.max_read_buffer_size));
}
2013-06-15 08:38:30 +00:00
}
}
2012-01-10 22:11:51 +00:00
return res;
2010-03-18 19:32:14 +00:00
}
BlockInputStreams StorageLog::read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,
const unsigned threads)
{
return read(
0, std::numeric_limits<size_t>::max(),
0,
column_names,
query, context, settings, processed_stage,
max_block_size, threads);
}
2011-08-28 02:22:23 +00:00
BlockOutputStreamPtr StorageLog::write(
ASTPtr query, const Settings & settings)
2010-03-18 19:32:14 +00:00
{
2012-06-21 16:33:00 +00:00
loadMarks();
return std::make_shared<LogBlockOutputStream>(*this);
2010-03-18 19:32:14 +00:00
}
bool StorageLog::checkData() const
{
Poco::ScopedReadRWLock lock(const_cast<Poco::RWLock &>(rwlock));
return file_checker.check();
}
2011-11-05 23:31:19 +00:00
2010-03-18 19:32:14 +00:00
}