2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/StorageLog.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2011-11-05 23:31:19 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/Exception.h>
|
2018-01-15 19:07:47 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2010-03-18 19:32:14 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/ReadBufferFromFile.h>
|
|
|
|
#include <IO/WriteBufferFromFile.h>
|
2018-12-28 18:15:26 +00:00
|
|
|
#include <Compression/CompressedReadBuffer.h>
|
|
|
|
#include <Compression/CompressedWriteBuffer.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2012-01-09 19:20:48 +00:00
|
|
|
|
2017-12-25 18:58:39 +00:00
|
|
|
#include <DataTypes/NestedUtils.h>
|
2012-08-29 20:07:24 +00:00
|
|
|
|
2019-01-23 14:48:50 +00:00
|
|
|
#include <DataStreams/IBlockInputStream.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataStreams/IBlockOutputStream.h>
|
2015-01-18 08:25:56 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnArray.h>
|
2012-08-29 20:07:24 +00:00
|
|
|
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
|
2017-05-24 21:06:29 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2017-01-21 04:24:28 +00:00
|
|
|
|
2016-07-13 16:41:19 +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
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin"
|
|
|
|
#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
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
2017-08-07 07:31:16 +00:00
|
|
|
extern const int NO_SUCH_COLUMN_IN_TABLE;
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int DUPLICATE_COLUMN;
|
|
|
|
extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT;
|
2017-12-30 00:36:06 +00:00
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2017-11-03 19:53:10 +00:00
|
|
|
extern const int INCORRECT_FILE_NAME;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
|
2019-01-23 14:48:50 +00:00
|
|
|
class LogBlockInputStream final : public IBlockInputStream
|
2010-03-18 19:32:14 +00:00
|
|
|
{
|
2015-01-18 08:25:56 +00:00
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
LogBlockInputStream(
|
2018-01-02 06:13:22 +00:00
|
|
|
size_t block_size_, const NamesAndTypesList & columns_, StorageLog & storage_,
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t mark_number_, size_t rows_limit_, size_t max_read_buffer_size_)
|
|
|
|
: block_size(block_size_),
|
2018-01-02 06:13:22 +00:00
|
|
|
columns(columns_),
|
2017-04-01 07:20:54 +00:00
|
|
|
storage(storage_),
|
|
|
|
mark_number(mark_number_),
|
|
|
|
rows_limit(rows_limit_),
|
|
|
|
max_read_buffer_size(max_read_buffer_size_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
String getName() const override { return "Log"; }
|
|
|
|
|
2018-02-18 03:23:48 +00:00
|
|
|
Block getHeader() const override
|
2018-01-09 01:51:08 +00:00
|
|
|
{
|
|
|
|
Block res;
|
|
|
|
|
|
|
|
for (const auto & name_type : columns)
|
|
|
|
res.insert({ name_type.type->createColumn(), name_type.type, name_type.name });
|
|
|
|
|
2018-02-17 03:49:34 +00:00
|
|
|
return Nested::flatten(res);
|
2018-08-10 04:02:56 +00:00
|
|
|
}
|
2018-01-06 18:10:44 +00:00
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
protected:
|
2017-04-01 07:20:54 +00:00
|
|
|
Block readImpl() override;
|
2016-08-24 00:39:38 +00:00
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
private:
|
2018-06-07 18:14:37 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t block_size;
|
2018-01-02 06:13:22 +00:00
|
|
|
NamesAndTypesList columns;
|
2017-04-01 07:20:54 +00:00
|
|
|
StorageLog & storage;
|
|
|
|
size_t mark_number; /// from what mark to read data
|
|
|
|
size_t rows_limit; /// The maximum number of rows that can be read
|
|
|
|
size_t rows_read = 0;
|
|
|
|
size_t max_read_buffer_size;
|
|
|
|
|
|
|
|
struct Stream
|
|
|
|
{
|
2019-08-03 11:02:40 +00:00
|
|
|
Stream(const std::string & data_path, size_t offset, size_t max_read_buffer_size_)
|
|
|
|
: plain(data_path, std::min(static_cast<Poco::File::FileSize>(max_read_buffer_size_), Poco::File(data_path).getSize())),
|
2017-04-01 07:20:54 +00:00
|
|
|
compressed(plain)
|
|
|
|
{
|
|
|
|
if (offset)
|
|
|
|
plain.seek(offset);
|
|
|
|
}
|
|
|
|
|
|
|
|
ReadBufferFromFile plain;
|
|
|
|
CompressedReadBuffer compressed;
|
|
|
|
};
|
|
|
|
|
2017-11-26 19:22:33 +00:00
|
|
|
using FileStreams = std::map<std::string, Stream>;
|
2017-04-01 07:20:54 +00:00
|
|
|
FileStreams streams;
|
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
using DeserializeState = IDataType::DeserializeBinaryBulkStatePtr;
|
|
|
|
using DeserializeStates = std::map<String, DeserializeState>;
|
|
|
|
DeserializeStates deserialize_states;
|
|
|
|
|
2018-01-02 09:11:13 +00:00
|
|
|
void readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read);
|
2018-04-21 00:35:20 +00:00
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2017-09-08 04:58:57 +00:00
|
|
|
class LogBlockOutputStream final : public IBlockOutputStream
|
2014-03-19 10:45:13 +00:00
|
|
|
{
|
2015-01-18 08:25:56 +00:00
|
|
|
public:
|
2017-09-07 21:04:48 +00:00
|
|
|
explicit LogBlockOutputStream(StorageLog & storage_)
|
2017-04-01 07:20:54 +00:00
|
|
|
: storage(storage_),
|
|
|
|
lock(storage.rwlock),
|
2017-08-07 07:31:16 +00:00
|
|
|
marks_stream(storage.marks_file.path(), 4096, O_APPEND | O_CREAT | O_WRONLY)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
~LogBlockOutputStream() override
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
writeSuffix();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-02-19 00:45:32 +00:00
|
|
|
Block getHeader() const override { return storage.getSampleBlock(); }
|
2017-04-01 07:20:54 +00:00
|
|
|
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:
|
2017-04-01 07:20:54 +00:00
|
|
|
StorageLog & storage;
|
2017-07-28 17:34:02 +00:00
|
|
|
std::unique_lock<std::shared_mutex> lock;
|
2017-04-01 07:20:54 +00:00
|
|
|
bool done = false;
|
|
|
|
|
|
|
|
struct Stream
|
|
|
|
{
|
2019-01-21 14:00:06 +00:00
|
|
|
Stream(const std::string & data_path, CompressionCodecPtr codec, size_t max_compress_block_size) :
|
2017-04-01 07:20:54 +00:00
|
|
|
plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY),
|
2019-01-21 18:04:08 +00:00
|
|
|
compressed(plain, std::move(codec), max_compress_block_size)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
plain_offset = Poco::File(data_path).getSize();
|
|
|
|
}
|
|
|
|
|
|
|
|
WriteBufferFromFile plain;
|
|
|
|
CompressedWriteBuffer compressed;
|
|
|
|
|
|
|
|
size_t plain_offset; /// How many bytes were in the file at the time the LogBlockOutputStream was created.
|
|
|
|
|
|
|
|
void finalize()
|
|
|
|
{
|
|
|
|
compressed.next();
|
|
|
|
plain.next();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2017-11-26 19:22:33 +00:00
|
|
|
using Mark = StorageLog::Mark;
|
2017-08-30 18:13:32 +00:00
|
|
|
using MarksForColumns = std::vector<std::pair<size_t, Mark>>;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-11-26 19:22:33 +00:00
|
|
|
using FileStreams = std::map<std::string, Stream>;
|
2017-04-01 07:20:54 +00:00
|
|
|
FileStreams streams;
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
using WrittenStreams = std::set<std::string>;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
WriteBufferFromFile marks_stream; /// Declared below `lock` to make the file open when rwlock is captured.
|
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
using SerializeState = IDataType::SerializeBinaryBulkStatePtr;
|
|
|
|
using SerializeStates = std::map<String, SerializeState>;
|
|
|
|
SerializeStates serialize_states;
|
|
|
|
|
|
|
|
IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenStreams & written_streams);
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void writeData(const String & name, const IDataType & type, const IColumn & column,
|
2017-08-07 07:31:16 +00:00
|
|
|
MarksForColumns & out_marks,
|
|
|
|
WrittenStreams & written_streams);
|
|
|
|
|
2017-11-27 21:21:09 +00:00
|
|
|
void writeMarks(MarksForColumns && marks);
|
2015-01-18 08:25:56 +00:00
|
|
|
};
|
2014-03-19 10:45:13 +00:00
|
|
|
|
|
|
|
|
2011-09-04 21:23:19 +00:00
|
|
|
Block LogBlockInputStream::readImpl()
|
2010-03-18 19:32:14 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
Block res;
|
|
|
|
|
|
|
|
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;
|
|
|
|
|
|
|
|
/// How many rows to read for the next block.
|
|
|
|
size_t max_rows_to_read = std::min(block_size, rows_limit - rows_read);
|
|
|
|
|
2018-01-02 06:13:22 +00:00
|
|
|
for (const auto & name_type : columns)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-01-02 09:11:13 +00:00
|
|
|
MutableColumnPtr column = name_type.type->createColumn();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
try
|
|
|
|
{
|
2018-01-02 09:11:13 +00:00
|
|
|
readData(name_type.name, *name_type.type, *column, max_rows_to_read);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
catch (Exception & e)
|
|
|
|
{
|
2019-07-09 15:40:21 +00:00
|
|
|
e.addMessage("while reading column " + name_type.name + " at " + storage.path + escapeForFileName(storage.table_name));
|
2017-04-01 07:20:54 +00:00
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
2017-12-15 20:48:46 +00:00
|
|
|
if (column->size())
|
2018-01-02 06:13:22 +00:00
|
|
|
res.insert(ColumnWithTypeAndName(std::move(column), name_type.type, name_type.name));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (res)
|
|
|
|
rows_read += res.rows();
|
|
|
|
|
|
|
|
if (!res || rows_read == rows_limit)
|
|
|
|
{
|
|
|
|
/** Close the files (before destroying the object).
|
|
|
|
* When many sources are created, but simultaneously reading only a few of them,
|
|
|
|
* buffers don't waste memory.
|
|
|
|
*/
|
|
|
|
streams.clear();
|
|
|
|
}
|
|
|
|
|
2018-01-02 09:11:13 +00:00
|
|
|
return Nested::flatten(res);
|
2010-03-18 19:32:14 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-01-02 09:11:13 +00:00
|
|
|
void LogBlockInputStream::readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read)
|
2012-08-29 20:07:24 +00:00
|
|
|
{
|
2018-06-07 18:14:37 +00:00
|
|
|
IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint.
|
|
|
|
|
|
|
|
auto createStringGetter = [&](bool stream_for_prefix)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-09-02 02:27:56 +00:00
|
|
|
return [&, stream_for_prefix] (const IDataType::SubstreamPath & path) -> ReadBuffer *
|
2018-06-07 18:14:37 +00:00
|
|
|
{
|
|
|
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
const auto & file_it = storage.files.find(stream_name);
|
|
|
|
if (storage.files.end() == file_it)
|
|
|
|
throw Exception("Logical error: no information about file " + stream_name + " in StorageLog", ErrorCodes::LOGICAL_ERROR);
|
2017-11-28 02:13:46 +00:00
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
UInt64 offset = 0;
|
|
|
|
if (!stream_for_prefix && mark_number)
|
|
|
|
offset = file_it->second.marks[mark_number].offset;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
auto & data_file_path = file_it->second.data_file.path();
|
|
|
|
auto it = streams.try_emplace(stream_name, data_file_path, offset, max_read_buffer_size).first;
|
|
|
|
return &it->second.compressed;
|
|
|
|
};
|
2017-08-07 07:31:16 +00:00
|
|
|
};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
if (deserialize_states.count(name) == 0)
|
|
|
|
{
|
|
|
|
settings.getter = createStringGetter(true);
|
|
|
|
type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]);
|
|
|
|
}
|
|
|
|
|
|
|
|
settings.getter = createStringGetter(false);
|
|
|
|
type.deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_states[name]);
|
2012-08-29 20:07:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
void LogBlockOutputStream::write(const Block & block)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
storage.check(block, true);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
/// The set of written offset columns so that you do not write shared offsets of columns for nested structures multiple times
|
|
|
|
WrittenStreams written_streams;
|
2010-03-18 19:32:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
MarksForColumns marks;
|
|
|
|
marks.reserve(storage.file_count);
|
2016-07-13 10:35:00 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i < block.columns(); ++i)
|
|
|
|
{
|
|
|
|
const ColumnWithTypeAndName & column = block.safeGetByPosition(i);
|
2017-08-07 07:31:16 +00:00
|
|
|
writeData(column.name, *column.type, *column.column, marks, written_streams);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-08-10 19:12:29 +00:00
|
|
|
|
2017-11-27 21:21:09 +00:00
|
|
|
writeMarks(std::move(marks));
|
2012-08-29 20:07:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-09-15 01:40:29 +00:00
|
|
|
void LogBlockOutputStream::writeSuffix()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (done)
|
|
|
|
return;
|
|
|
|
done = true;
|
2015-04-02 23:58:26 +00:00
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
WrittenStreams written_streams;
|
|
|
|
IDataType::SerializeBinaryBulkSettings settings;
|
|
|
|
for (const auto & column : getHeader())
|
|
|
|
{
|
|
|
|
auto it = serialize_states.find(column.name);
|
|
|
|
if (it != serialize_states.end())
|
|
|
|
{
|
|
|
|
settings.getter = createStreamGetter(column.name, written_streams);
|
|
|
|
column.type->serializeBinaryBulkStateSuffix(settings, it->second);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Finish write.
|
|
|
|
marks_stream.next();
|
2013-09-15 01:40:29 +00:00
|
|
|
|
2017-11-26 19:22:33 +00:00
|
|
|
for (auto & name_stream : streams)
|
|
|
|
name_stream.second.finalize();
|
2013-09-15 01:40:29 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::vector<Poco::File> column_files;
|
2017-11-26 19:22:33 +00:00
|
|
|
for (const auto & name_stream : streams)
|
|
|
|
column_files.push_back(storage.files[name_stream.first].data_file);
|
2017-04-01 07:20:54 +00:00
|
|
|
column_files.push_back(storage.marks_file);
|
2014-08-04 06:36:24 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
storage.file_checker.update(column_files.begin(), column_files.end());
|
2014-08-04 06:36:24 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
streams.clear();
|
2013-09-15 01:40:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
IDataType::OutputStreamGetter LogBlockOutputStream::createStreamGetter(const String & name,
|
|
|
|
WrittenStreams & written_streams)
|
|
|
|
{
|
|
|
|
return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer *
|
|
|
|
{
|
|
|
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
|
|
|
if (written_streams.count(stream_name))
|
|
|
|
return nullptr;
|
|
|
|
|
|
|
|
auto it = streams.find(stream_name);
|
|
|
|
if (streams.end() == it)
|
|
|
|
throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream",
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
return &it->second.compressed;
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-07-13 10:35:00 +00:00
|
|
|
void LogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column,
|
2017-08-07 07:31:16 +00:00
|
|
|
MarksForColumns & out_marks,
|
|
|
|
WrittenStreams & written_streams)
|
2012-08-29 20:07:24 +00:00
|
|
|
{
|
2018-06-07 18:14:37 +00:00
|
|
|
IDataType::SerializeBinaryBulkSettings settings;
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
type.enumerateStreams([&] (const IDataType::SubstreamPath & path)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
2017-11-28 02:13:46 +00:00
|
|
|
if (written_streams.count(stream_name))
|
|
|
|
return;
|
2017-09-05 02:36:35 +00:00
|
|
|
|
2019-01-21 14:00:06 +00:00
|
|
|
const auto & columns = storage.getColumns();
|
|
|
|
streams.try_emplace(
|
|
|
|
stream_name,
|
|
|
|
storage.files[stream_name].data_file.path(),
|
|
|
|
columns.getCodecOrDefault(name),
|
|
|
|
storage.max_compress_block_size);
|
2018-06-07 18:14:37 +00:00
|
|
|
}, settings.path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
settings.getter = createStreamGetter(name, written_streams);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
if (serialize_states.count(name) == 0)
|
|
|
|
type.serializeBinaryBulkStatePrefix(settings, serialize_states[name]);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
type.enumerateStreams([&] (const IDataType::SubstreamPath & path)
|
2017-11-26 19:22:33 +00:00
|
|
|
{
|
|
|
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
2017-11-28 02:13:46 +00:00
|
|
|
if (written_streams.count(stream_name))
|
2018-06-07 18:14:37 +00:00
|
|
|
return;
|
2017-11-26 19:22:33 +00:00
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
const auto & file = storage.files[stream_name];
|
|
|
|
const auto stream_it = streams.find(stream_name);
|
|
|
|
|
|
|
|
Mark mark;
|
|
|
|
mark.rows = (file.marks.empty() ? 0 : file.marks.back().rows) + column.size();
|
|
|
|
mark.offset = stream_it->second.plain_offset + stream_it->second.plain.count();
|
|
|
|
|
|
|
|
out_marks.emplace_back(file.column_index, mark);
|
|
|
|
}, settings.path);
|
2017-11-26 19:22:33 +00:00
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
type.serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
type.enumerateStreams([&] (const IDataType::SubstreamPath & path)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
String stream_name = IDataType::getFileNameForStream(name, path);
|
2017-11-28 02:13:46 +00:00
|
|
|
if (!written_streams.emplace(stream_name).second)
|
|
|
|
return;
|
|
|
|
|
2017-11-26 19:22:33 +00:00
|
|
|
auto it = streams.find(stream_name);
|
|
|
|
if (streams.end() == it)
|
|
|
|
throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
it->second.compressed.next();
|
2018-06-07 18:14:37 +00:00
|
|
|
}, settings.path);
|
2010-03-18 19:32:14 +00:00
|
|
|
}
|
|
|
|
|
2013-02-26 13:06:01 +00:00
|
|
|
|
2017-11-27 21:21:09 +00:00
|
|
|
void LogBlockOutputStream::writeMarks(MarksForColumns && marks)
|
2016-07-12 18:08:16 +00:00
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
if (marks.size() != storage.file_count)
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Wrong number of marks generated from block. Makes no sense.", ErrorCodes::LOGICAL_ERROR);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
std::sort(marks.begin(), marks.end(), [](const auto & a, const auto & b) { return a.first < b.first; });
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-11-26 19:22:33 +00:00
|
|
|
for (const auto & mark : marks)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-11-26 19:22:33 +00:00
|
|
|
writeIntBinary(mark.second.rows, marks_stream);
|
|
|
|
writeIntBinary(mark.second.offset, marks_stream);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-11-26 19:22:33 +00:00
|
|
|
size_t column_index = mark.first;
|
2019-01-04 12:10:00 +00:00
|
|
|
storage.files[storage.column_names_by_idx[column_index]].marks.push_back(mark.second);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2013-02-26 13:06:01 +00:00
|
|
|
}
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
StorageLog::StorageLog(
|
2017-04-01 07:20:54 +00:00
|
|
|
const std::string & path_,
|
2019-07-09 15:40:21 +00:00
|
|
|
const std::string & database_name_,
|
|
|
|
const std::string & table_name_,
|
2018-03-06 20:18:34 +00:00
|
|
|
const ColumnsDescription & columns_,
|
2019-08-24 21:20:20 +00:00
|
|
|
const ConstraintsDescription & constraints_,
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t max_compress_block_size_)
|
2019-08-24 21:20:20 +00:00
|
|
|
: path(path_), table_name(table_name_), database_name(database_name_),
|
2017-11-27 23:47:14 +00:00
|
|
|
max_compress_block_size(max_compress_block_size_),
|
2019-07-09 15:40:21 +00:00
|
|
|
file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json")
|
2010-03-18 19:32:14 +00:00
|
|
|
{
|
2019-08-24 21:20:20 +00:00
|
|
|
setColumns(columns_);
|
|
|
|
setConstraints(constraints_);
|
|
|
|
|
2017-11-03 19:53:10 +00:00
|
|
|
if (path.empty())
|
|
|
|
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// create files if they do not exist
|
2019-07-09 15:40:21 +00:00
|
|
|
Poco::File(path + escapeForFileName(table_name) + '/').createDirectories();
|
2012-01-10 22:11:51 +00:00
|
|
|
|
2018-03-13 15:00:28 +00:00
|
|
|
for (const auto & column : getColumns().getAllPhysical())
|
2017-08-07 07:31:16 +00:00
|
|
|
addFiles(column.name, *column.type);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2019-07-09 15:40:21 +00:00
|
|
|
marks_file = Poco::File(path + escapeForFileName(table_name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
|
2012-08-29 20:07:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
void StorageLog::addFiles(const String & column_name, const IDataType & type)
|
2012-08-29 20:07:24 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (files.end() != files.find(column_name))
|
|
|
|
throw Exception("Duplicate column with name " + column_name + " in constructor of StorageLog.",
|
|
|
|
ErrorCodes::DUPLICATE_COLUMN);
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
IDataType::StreamCallback stream_callback = [&] (const IDataType::SubstreamPath & substream_path)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-09-05 02:36:35 +00:00
|
|
|
String stream_name = IDataType::getFileNameForStream(column_name, substream_path);
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
if (!files.count(stream_name))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
ColumnData & column_data = files[stream_name];
|
2017-09-05 02:36:35 +00:00
|
|
|
column_data.column_index = file_count;
|
2017-04-01 07:20:54 +00:00
|
|
|
column_data.data_file = Poco::File{
|
2019-07-09 15:40:21 +00:00
|
|
|
path + escapeForFileName(table_name) + '/' + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-01-04 12:10:00 +00:00
|
|
|
column_names_by_idx.push_back(stream_name);
|
2017-09-05 02:36:35 +00:00
|
|
|
++file_count;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-08-07 07:31:16 +00:00
|
|
|
};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-01-04 12:10:00 +00:00
|
|
|
IDataType::SubstreamPath substream_path;
|
|
|
|
type.enumerateStreams(stream_callback, substream_path);
|
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()
|
|
|
|
{
|
2017-07-28 17:34:02 +00:00
|
|
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (loaded_marks)
|
|
|
|
return;
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using FilesByIndex = std::vector<Files_t::iterator>;
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
FilesByIndex files_by_index(file_count);
|
2017-04-01 07:20:54 +00:00
|
|
|
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
|
|
|
|
files_by_index[it->second.column_index] = it;
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
if (marks_file.exists())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-08-07 07:31:16 +00:00
|
|
|
size_t file_size = marks_file.getSize();
|
|
|
|
if (file_size % (file_count * sizeof(Mark)) != 0)
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Size of marks file is inconsistent", ErrorCodes::SIZES_OF_MARKS_FILES_ARE_INCONSISTENT);
|
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
size_t marks_count = file_size / (file_count * sizeof(Mark));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-11-26 19:22:33 +00:00
|
|
|
for (auto & file : files_by_index)
|
|
|
|
file->second.marks.reserve(marks_count);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
ReadBufferFromFile marks_rb(marks_file.path(), 32768);
|
2017-04-01 07:20:54 +00:00
|
|
|
while (!marks_rb.eof())
|
|
|
|
{
|
|
|
|
for (size_t i = 0; i < files_by_index.size(); ++i)
|
|
|
|
{
|
|
|
|
Mark mark;
|
|
|
|
readIntBinary(mark.rows, marks_rb);
|
|
|
|
readIntBinary(mark.offset, marks_rb);
|
|
|
|
files_by_index[i]->second.marks.push_back(mark);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2017-08-07 07:31:16 +00:00
|
|
|
|
|
|
|
loaded_marks = true;
|
2010-03-18 19:32:14 +00:00
|
|
|
}
|
|
|
|
|
2013-02-07 13:03:19 +00:00
|
|
|
|
2019-08-27 20:43:08 +00:00
|
|
|
void StorageLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
2012-06-18 06:19:13 +00:00
|
|
|
{
|
2017-07-28 17:34:02 +00:00
|
|
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Rename directory with data.
|
2019-07-09 15:40:21 +00:00
|
|
|
Poco::File(path + escapeForFileName(table_name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
path = new_path_to_db;
|
2019-07-09 15:40:21 +00:00
|
|
|
table_name = new_table_name;
|
|
|
|
database_name = new_database_name;
|
|
|
|
file_checker.setPath(path + escapeForFileName(table_name) + '/' + "sizes.json");
|
2012-06-18 06:19:13 +00:00
|
|
|
|
2017-11-26 19:22:33 +00:00
|
|
|
for (auto & file : files)
|
2019-07-09 15:40:21 +00:00
|
|
|
file.second.data_file = Poco::File(path + escapeForFileName(table_name) + '/' + Poco::Path(file.second.data_file.path()).getFileName());
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2019-07-09 15:40:21 +00:00
|
|
|
marks_file = Poco::File(path + escapeForFileName(table_name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
|
2012-06-18 06:19:13 +00:00
|
|
|
}
|
|
|
|
|
2019-08-27 20:43:08 +00:00
|
|
|
void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
|
|
|
std::shared_lock<std::shared_mutex> lock(rwlock);
|
|
|
|
|
2019-07-09 15:40:21 +00:00
|
|
|
String table_dir = path + escapeForFileName(table_name);
|
2018-04-21 00:35:20 +00:00
|
|
|
|
2018-06-09 15:48:22 +00:00
|
|
|
files.clear();
|
|
|
|
file_count = 0;
|
|
|
|
loaded_marks = false;
|
2018-04-21 00:35:20 +00:00
|
|
|
|
|
|
|
std::vector<Poco::File> data_files;
|
|
|
|
Poco::File(table_dir).list(data_files);
|
|
|
|
|
|
|
|
for (auto & file : data_files)
|
|
|
|
file.remove(false);
|
|
|
|
|
2018-11-26 00:56:50 +00:00
|
|
|
for (const auto & column : getColumns().getAllPhysical())
|
2018-04-21 00:35:20 +00:00
|
|
|
addFiles(column.name, *column.type);
|
|
|
|
|
2018-06-09 15:48:22 +00:00
|
|
|
file_checker = FileChecker{table_dir + "/" + "sizes.json"};
|
|
|
|
marks_file = Poco::File(table_dir + "/" + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
|
2018-04-21 00:35:20 +00:00
|
|
|
}
|
|
|
|
|
2012-06-18 06:19:13 +00:00
|
|
|
|
2017-11-26 19:22:33 +00:00
|
|
|
const StorageLog::Marks & StorageLog::getMarksWithRealRowCount() const
|
2016-07-12 18:08:16 +00:00
|
|
|
{
|
2019-03-14 15:20:51 +00:00
|
|
|
const String & column_name = getColumns().begin()->name;
|
|
|
|
const IDataType & column_type = *getColumns().begin()->type;
|
2017-04-01 07:20:54 +00:00
|
|
|
String filename;
|
|
|
|
|
|
|
|
/** We take marks from first column.
|
2017-12-02 22:12:27 +00:00
|
|
|
* If this is a data type with multiple stream, get the first stream, that we assume have real row count.
|
|
|
|
* (Example: for Array data type, first stream is array sizes; and number of array sizes is the number of arrays).
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2019-01-04 12:10:00 +00:00
|
|
|
IDataType::SubstreamPath substream_root_path;
|
2017-12-02 22:12:27 +00:00
|
|
|
column_type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path)
|
|
|
|
{
|
|
|
|
if (filename.empty())
|
|
|
|
filename = IDataType::getFileNameForStream(column_name, substream_path);
|
2019-01-04 12:10:00 +00:00
|
|
|
}, substream_root_path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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;
|
2013-12-12 22:55:47 +00:00
|
|
|
}
|
|
|
|
|
2012-01-09 19:20:48 +00:00
|
|
|
BlockInputStreams StorageLog::read(
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & column_names,
|
2017-12-01 21:13:25 +00:00
|
|
|
const SelectQueryInfo & /*query_info*/,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context,
|
2018-09-08 11:29:23 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
2019-02-18 23:38:44 +00:00
|
|
|
size_t max_block_size,
|
2017-06-02 15:54:39 +00:00
|
|
|
unsigned num_streams)
|
2010-03-18 19:32:14 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
check(column_names);
|
|
|
|
loadMarks();
|
|
|
|
|
2018-03-13 15:00:28 +00:00
|
|
|
NamesAndTypesList all_columns = Nested::collect(getColumns().getAllPhysical().addTypes(column_names));
|
2018-01-02 06:13:22 +00:00
|
|
|
|
2017-07-28 17:34:02 +00:00
|
|
|
std::shared_lock<std::shared_mutex> lock(rwlock);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
BlockInputStreams res;
|
|
|
|
|
|
|
|
const Marks & marks = getMarksWithRealRowCount();
|
|
|
|
size_t marks_size = marks.size();
|
|
|
|
|
2017-06-02 15:54:39 +00:00
|
|
|
if (num_streams > marks_size)
|
|
|
|
num_streams = marks_size;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-05-24 21:06:29 +00:00
|
|
|
size_t max_read_buffer_size = context.getSettingsRef().max_read_buffer_size;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
for (size_t stream = 0; stream < num_streams; ++stream)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-11-27 21:21:09 +00:00
|
|
|
size_t mark_begin = stream * marks_size / num_streams;
|
|
|
|
size_t mark_end = (stream + 1) * marks_size / num_streams;
|
|
|
|
|
|
|
|
size_t rows_begin = mark_begin ? marks[mark_begin - 1].rows : 0;
|
|
|
|
size_t rows_end = mark_end ? marks[mark_end - 1].rows : 0;
|
|
|
|
|
|
|
|
res.emplace_back(std::make_shared<LogBlockInputStream>(
|
2017-08-07 07:31:16 +00:00
|
|
|
max_block_size,
|
2018-03-06 20:18:34 +00:00
|
|
|
all_columns,
|
2017-08-07 07:31:16 +00:00
|
|
|
*this,
|
2017-11-27 21:21:09 +00:00
|
|
|
mark_begin,
|
|
|
|
rows_end - rows_begin,
|
2017-08-07 07:31:16 +00:00
|
|
|
max_read_buffer_size));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return res;
|
2010-03-18 19:32:14 +00:00
|
|
|
}
|
|
|
|
|
2011-08-28 02:22:23 +00:00
|
|
|
BlockOutputStreamPtr StorageLog::write(
|
2019-02-27 18:26:24 +00:00
|
|
|
const ASTPtr & /*query*/, const Context & /*context*/)
|
2010-03-18 19:32:14 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
loadMarks();
|
|
|
|
return std::make_shared<LogBlockOutputStream>(*this);
|
2010-03-18 19:32:14 +00:00
|
|
|
}
|
|
|
|
|
2019-07-03 13:17:19 +00:00
|
|
|
CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & /* context */)
|
2014-08-04 06:36:24 +00:00
|
|
|
{
|
2017-07-28 17:34:02 +00:00
|
|
|
std::shared_lock<std::shared_mutex> lock(rwlock);
|
2017-04-01 07:20:54 +00:00
|
|
|
return file_checker.check();
|
2014-08-04 06:36:24 +00:00
|
|
|
}
|
2011-11-05 23:31:19 +00:00
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
|
|
|
|
void registerStorageLog(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("Log", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
if (!args.engine_args.empty())
|
|
|
|
throw Exception(
|
|
|
|
"Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
return StorageLog::create(
|
2019-08-24 21:20:20 +00:00
|
|
|
args.data_path, args.database_name, args.table_name, args.columns, args.constraints,
|
2017-12-30 00:36:06 +00:00
|
|
|
args.context.getSettings().max_compress_block_size);
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2010-03-18 19:32:14 +00:00
|
|
|
}
|