2016-10-25 06:49:24 +00:00
|
|
|
#include <sys/stat.h>
|
|
|
|
#include <sys/types.h>
|
2018-05-09 04:22:30 +00:00
|
|
|
#include <errno.h>
|
2016-10-25 06:49:24 +00:00
|
|
|
|
2012-06-25 00:17:19 +00:00
|
|
|
#include <map>
|
2020-09-18 19:25:56 +00:00
|
|
|
#include <cassert>
|
2015-01-25 05:07:51 +00:00
|
|
|
|
|
|
|
#include <Poco/Util/XMLConfiguration.h>
|
2012-06-25 00:17:19 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/escapeForFileName.h>
|
|
|
|
#include <Common/Exception.h>
|
2019-12-23 16:57:16 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2012-06-25 00:17:19 +00:00
|
|
|
|
2020-02-14 14:28:33 +00:00
|
|
|
#include <IO/ReadBufferFromFileBase.h>
|
2020-02-20 16:39:32 +00:00
|
|
|
#include <IO/WriteBufferFromFileBase.h>
|
2019-12-23 16:57:16 +00:00
|
|
|
#include <Compression/CompressionFactory.h>
|
2020-10-26 17:24:15 +00:00
|
|
|
#include <Compression/CompressedReadBuffer.h>
|
2018-12-28 18:15:26 +00:00
|
|
|
#include <Compression/CompressedWriteBuffer.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2012-06-25 00:17:19 +00:00
|
|
|
|
2017-12-25 18:58:39 +00:00
|
|
|
#include <DataTypes/NestedUtils.h>
|
2012-08-29 18:49:54 +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 18:49:54 +00:00
|
|
|
|
2017-05-24 21:06:29 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2017-01-21 04:24:28 +00:00
|
|
|
|
2020-02-14 14:28:33 +00:00
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
2019-07-03 13:17:19 +00:00
|
|
|
#include <Storages/CheckResults.h>
|
2020-02-14 14:28:33 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
|
|
|
#include <Storages/StorageTinyLog.h>
|
2020-02-18 14:41:30 +00:00
|
|
|
#include "StorageLogSettings.h"
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2020-02-17 14:45:10 +00:00
|
|
|
#include <Processors/Sources/SourceWithProgress.h>
|
|
|
|
#include <Processors/Pipe.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2017-07-27 23:23:13 +00:00
|
|
|
#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin"
|
2012-06-25 00:17:19 +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 DUPLICATE_COLUMN;
|
2017-11-03 19:53:10 +00:00
|
|
|
extern const int INCORRECT_FILE_NAME;
|
2017-12-30 00:36:06 +00:00
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2012-06-25 00:17:19 +00:00
|
|
|
|
2020-02-17 14:45:10 +00:00
|
|
|
class TinyLogSource final : public SourceWithProgress
|
2012-06-25 00:17:19 +00:00
|
|
|
{
|
2015-01-18 08:25:56 +00:00
|
|
|
public:
|
|
|
|
|
2020-02-17 14:45:10 +00:00
|
|
|
static Block getHeader(const NamesAndTypesList & columns)
|
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 });
|
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
return res;
|
2018-08-10 04:02:56 +00:00
|
|
|
}
|
2018-01-06 18:10:44 +00:00
|
|
|
|
2020-02-17 14:45:10 +00:00
|
|
|
TinyLogSource(size_t block_size_, const NamesAndTypesList & columns_, StorageTinyLog & storage_, size_t max_read_buffer_size_)
|
|
|
|
: SourceWithProgress(getHeader(columns_))
|
|
|
|
, block_size(block_size_), columns(columns_), storage(storage_), lock(storage_.rwlock)
|
|
|
|
, max_read_buffer_size(max_read_buffer_size_) {}
|
|
|
|
|
|
|
|
String getName() const override { return "TinyLog"; }
|
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
protected:
|
2020-02-17 14:45:10 +00:00
|
|
|
Chunk generate() override;
|
2020-02-18 07:53:16 +00:00
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
private:
|
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
|
|
|
StorageTinyLog & storage;
|
2019-07-30 17:38:22 +00:00
|
|
|
std::shared_lock<std::shared_mutex> lock;
|
2020-02-18 07:53:16 +00:00
|
|
|
bool is_finished = false;
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t max_read_buffer_size;
|
|
|
|
|
|
|
|
struct Stream
|
|
|
|
{
|
2019-12-23 16:57:16 +00:00
|
|
|
Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_)
|
2020-10-26 17:24:15 +00:00
|
|
|
: plain(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path))))
|
|
|
|
, compressed(*plain)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2020-10-29 14:14:23 +00:00
|
|
|
std::unique_ptr<ReadBuffer> plain;
|
2020-10-26 17:24:15 +00:00
|
|
|
CompressedReadBuffer compressed;
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
|
|
|
|
2019-12-12 08:57:25 +00:00
|
|
|
using FileStreams = std::map<String, std::unique_ptr<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;
|
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, UInt64 limit, IDataType::SubstreamsCache & cache);
|
2015-01-18 08:25:56 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2017-09-08 04:58:57 +00:00
|
|
|
class TinyLogBlockOutputStream final : public IBlockOutputStream
|
2015-01-18 08:25:56 +00:00
|
|
|
{
|
|
|
|
public:
|
2020-06-16 15:51:29 +00:00
|
|
|
explicit TinyLogBlockOutputStream(StorageTinyLog & storage_, const StorageMetadataPtr & metadata_snapshot_)
|
|
|
|
: storage(storage_), metadata_snapshot(metadata_snapshot_), lock(storage_.rwlock)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
~TinyLogBlockOutputStream() override
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2020-07-12 02:31:58 +00:00
|
|
|
if (!done)
|
|
|
|
{
|
|
|
|
/// Rollback partial writes.
|
|
|
|
streams.clear();
|
|
|
|
storage.file_checker.repair();
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-06-16 15:51:29 +00:00
|
|
|
Block getHeader() const override { return metadata_snapshot->getSampleBlock(); }
|
2018-02-19 00:45:32 +00:00
|
|
|
|
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
|
|
|
StorageTinyLog & storage;
|
2020-06-16 15:51:29 +00:00
|
|
|
StorageMetadataPtr metadata_snapshot;
|
2019-07-30 17:38:22 +00:00
|
|
|
std::unique_lock<std::shared_mutex> lock;
|
2017-04-01 07:20:54 +00:00
|
|
|
bool done = false;
|
|
|
|
|
|
|
|
struct Stream
|
|
|
|
{
|
2019-12-12 08:57:25 +00:00
|
|
|
Stream(const DiskPtr & disk, const String & data_path, CompressionCodecPtr codec, size_t max_compress_block_size) :
|
2019-12-26 14:28:22 +00:00
|
|
|
plain(disk->writeFile(data_path, max_compress_block_size, WriteMode::Append)),
|
2019-12-12 08:57:25 +00:00
|
|
|
compressed(*plain, std::move(codec), max_compress_block_size)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2019-12-12 08:57:25 +00:00
|
|
|
std::unique_ptr<WriteBuffer> plain;
|
2017-04-01 07:20:54 +00:00
|
|
|
CompressedWriteBuffer compressed;
|
|
|
|
|
|
|
|
void finalize()
|
|
|
|
{
|
|
|
|
compressed.next();
|
2020-01-10 21:42:26 +00:00
|
|
|
plain->finalize();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2019-12-12 08:57:25 +00:00
|
|
|
using FileStreams = std::map<String, std::unique_ptr<Stream>>;
|
2017-04-01 07:20:54 +00:00
|
|
|
FileStreams streams;
|
|
|
|
|
2018-06-07 18:14:37 +00:00
|
|
|
using SerializeState = IDataType::SerializeBinaryBulkStatePtr;
|
|
|
|
using SerializeStates = std::map<String, SerializeState>;
|
|
|
|
SerializeStates serialize_states;
|
|
|
|
|
2019-12-12 08:57:25 +00:00
|
|
|
using WrittenStreams = std::set<String>;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-09-14 11:22:17 +00:00
|
|
|
IDataType::OutputStreamGetter createStreamGetter(const NameAndTypePair & column, WrittenStreams & written_streams);
|
|
|
|
void writeData(const NameAndTypePair & name_and_type, const IColumn & column, WrittenStreams & written_streams);
|
2015-01-18 08:25:56 +00:00
|
|
|
};
|
2012-06-25 00:17:19 +00:00
|
|
|
|
|
|
|
|
2020-02-17 14:45:10 +00:00
|
|
|
Chunk TinyLogSource::generate()
|
2012-06-25 00:17:19 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
Block res;
|
|
|
|
|
2020-02-18 07:53:16 +00:00
|
|
|
if (is_finished || (!streams.empty() && streams.begin()->second->compressed.eof()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/** 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.
|
|
|
|
*/
|
2020-02-18 07:53:16 +00:00
|
|
|
is_finished = true;
|
2017-04-01 07:20:54 +00:00
|
|
|
streams.clear();
|
2020-02-17 14:45:10 +00:00
|
|
|
return {};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2019-12-31 00:41:03 +00:00
|
|
|
/// if there are no files in the folder, it means that the table is empty
|
2019-12-12 08:57:25 +00:00
|
|
|
if (storage.disk->isDirectoryEmpty(storage.table_path))
|
2020-02-17 14:45:10 +00:00
|
|
|
return {};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
std::unordered_map<String, IDataType::SubstreamsCache> caches;
|
2018-01-02 06:13:22 +00:00
|
|
|
for (const auto & name_type : columns)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-11-10 17:32:00 +00:00
|
|
|
ColumnPtr column;
|
|
|
|
try
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-11-10 17:32:00 +00:00
|
|
|
column = name_type.type->createColumn();
|
2020-12-22 15:03:48 +00:00
|
|
|
readData(name_type, column, block_size, caches[name_type.getNameInStorage()]);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-11-10 17:32:00 +00:00
|
|
|
catch (Exception & e)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-11-10 17:32:00 +00:00
|
|
|
e.addMessage("while reading column " + name_type.name + " at " + fullPath(storage.disk, storage.table_path));
|
|
|
|
throw;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-02-17 14:45:10 +00:00
|
|
|
if (!column->empty())
|
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 || streams.begin()->second->compressed.eof())
|
|
|
|
{
|
2020-02-18 07:53:16 +00:00
|
|
|
is_finished = true;
|
2017-04-01 07:20:54 +00:00
|
|
|
streams.clear();
|
|
|
|
}
|
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
return Chunk(res.getColumns(), res.rows());
|
2012-06-25 00:17:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
void TinyLogSource::readData(const NameAndTypePair & name_and_type,
|
|
|
|
ColumnPtr & column, UInt64 limit, IDataType::SubstreamsCache & cache)
|
2012-08-29 18:49:54 +00:00
|
|
|
{
|
2018-06-07 18:14:37 +00:00
|
|
|
IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint.
|
2020-09-14 11:22:17 +00:00
|
|
|
const auto & [name, type] = name_and_type;
|
2018-06-07 18:14:37 +00:00
|
|
|
settings.getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer *
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-11-24 14:14:47 +00:00
|
|
|
if (cache.count(IDataType::getSubcolumnNameForStream(path)))
|
|
|
|
return nullptr;
|
|
|
|
|
2020-09-14 11:22:17 +00:00
|
|
|
String stream_name = IDataType::getFileNameForStream(name_and_type, path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-10-21 23:02:20 +00:00
|
|
|
auto & stream = streams[stream_name];
|
|
|
|
if (!stream)
|
|
|
|
stream = std::make_unique<Stream>(storage.disk, storage.files[stream_name].data_file_path, max_read_buffer_size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-10-21 23:02:20 +00:00
|
|
|
return &stream->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)
|
2020-09-14 11:22:17 +00:00
|
|
|
type->deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]);
|
2018-06-07 18:14:37 +00:00
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
type->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name], &cache);
|
2012-08-29 18:49:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-06-17 16:39:58 +00:00
|
|
|
IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter(
|
2020-09-14 11:22:17 +00:00
|
|
|
const NameAndTypePair & column,
|
2020-06-17 16:39:58 +00:00
|
|
|
WrittenStreams & written_streams)
|
2012-08-29 18:49:54 +00:00
|
|
|
{
|
2018-06-07 18:14:37 +00:00
|
|
|
return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer *
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-09-14 11:22:17 +00:00
|
|
|
String stream_name = IDataType::getFileNameForStream(column, path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-01-02 09:11:13 +00:00
|
|
|
if (!written_streams.insert(stream_name).second)
|
|
|
|
return nullptr;
|
|
|
|
|
2020-06-17 16:39:58 +00:00
|
|
|
const auto & columns = metadata_snapshot->getColumns();
|
2017-08-07 07:31:16 +00:00
|
|
|
if (!streams.count(stream_name))
|
2020-06-17 16:39:58 +00:00
|
|
|
streams[stream_name] = std::make_unique<Stream>(
|
|
|
|
storage.disk,
|
|
|
|
storage.files[stream_name].data_file_path,
|
2020-09-14 11:22:17 +00:00
|
|
|
columns.getCodecOrDefault(column.name),
|
2020-06-17 16:39:58 +00:00
|
|
|
storage.max_compress_block_size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-08-07 07:31:16 +00:00
|
|
|
return &streams[stream_name]->compressed;
|
|
|
|
};
|
2018-06-07 18:14:37 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-09-14 11:22:17 +00:00
|
|
|
void TinyLogBlockOutputStream::writeData(const NameAndTypePair & name_and_type, const IColumn & column, WrittenStreams & written_streams)
|
2018-06-07 18:14:37 +00:00
|
|
|
{
|
|
|
|
IDataType::SerializeBinaryBulkSettings settings;
|
2020-09-14 11:22:17 +00:00
|
|
|
const auto & [name, type] = name_and_type;
|
|
|
|
settings.getter = createStreamGetter(name_and_type, written_streams);
|
2018-06-07 18:14:37 +00:00
|
|
|
|
|
|
|
if (serialize_states.count(name) == 0)
|
2020-09-14 11:22:17 +00:00
|
|
|
type->serializeBinaryBulkStatePrefix(settings, serialize_states[name]);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-09-14 11:22:17 +00:00
|
|
|
type->serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]);
|
2012-06-25 00:17:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-09-15 01:40:29 +00:00
|
|
|
void TinyLogBlockOutputStream::writeSuffix()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (done)
|
|
|
|
return;
|
2015-04-02 23:58:26 +00:00
|
|
|
|
2018-07-07 19:02:10 +00:00
|
|
|
/// If nothing was written - leave the table in initial state.
|
|
|
|
if (streams.empty())
|
2020-07-12 02:31:58 +00:00
|
|
|
{
|
|
|
|
done = true;
|
2018-07-07 19:02:10 +00:00
|
|
|
return;
|
2020-07-12 02:31:58 +00:00
|
|
|
}
|
2018-07-07 19:02:10 +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())
|
|
|
|
{
|
2020-09-14 11:22:17 +00:00
|
|
|
settings.getter = createStreamGetter(NameAndTypePair(column.name, column.type), written_streams);
|
2018-06-07 18:14:37 +00:00
|
|
|
column.type->serializeBinaryBulkStateSuffix(settings, it->second);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Finish write.
|
2017-12-11 18:19:35 +00:00
|
|
|
for (auto & stream : streams)
|
|
|
|
stream.second->finalize();
|
2014-07-31 19:19:56 +00:00
|
|
|
|
2019-12-12 08:57:25 +00:00
|
|
|
Strings column_files;
|
2017-04-01 07:20:54 +00:00
|
|
|
for (auto & pair : streams)
|
2019-12-25 08:24:13 +00:00
|
|
|
column_files.push_back(storage.files[pair.first].data_file_path);
|
2014-08-01 13:19:27 +00:00
|
|
|
|
2020-07-12 02:31:58 +00:00
|
|
|
for (const auto & file : column_files)
|
|
|
|
storage.file_checker.update(file);
|
|
|
|
storage.file_checker.save();
|
2013-09-15 01:40:29 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
streams.clear();
|
2020-07-12 02:31:58 +00:00
|
|
|
done = true;
|
2013-09-15 01:40:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2012-06-25 00:17:19 +00:00
|
|
|
void TinyLogBlockOutputStream::write(const Block & block)
|
|
|
|
{
|
2020-06-17 14:32:25 +00:00
|
|
|
metadata_snapshot->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 columns for nested structures multiple times
|
|
|
|
WrittenStreams written_streams;
|
2012-06-25 00:17:19 +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);
|
2020-09-14 11:22:17 +00:00
|
|
|
writeData(NameAndTypePair(column.name, column.type), *column.column, written_streams);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2012-06-25 00:17:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
StorageTinyLog::StorageTinyLog(
|
2019-12-12 08:57:25 +00:00
|
|
|
DiskPtr disk_,
|
2019-12-26 14:03:32 +00:00
|
|
|
const String & relative_path_,
|
2019-12-04 16:06:55 +00:00
|
|
|
const StorageID & table_id_,
|
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
|
|
|
bool attach,
|
|
|
|
size_t max_compress_block_size_)
|
2019-12-04 16:06:55 +00:00
|
|
|
: IStorage(table_id_)
|
2020-01-13 11:41:42 +00:00
|
|
|
, disk(std::move(disk_))
|
|
|
|
, table_path(relative_path_)
|
2019-12-04 16:06:55 +00:00
|
|
|
, max_compress_block_size(max_compress_block_size_)
|
2020-01-13 11:41:42 +00:00
|
|
|
, file_checker(disk, table_path + "sizes.json")
|
2020-05-30 21:57:37 +00:00
|
|
|
, log(&Poco::Logger::get("StorageTinyLog"))
|
2012-06-25 00:17:19 +00:00
|
|
|
{
|
2020-06-19 15:39:41 +00:00
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(columns_);
|
|
|
|
storage_metadata.setConstraints(constraints_);
|
|
|
|
setInMemoryMetadata(storage_metadata);
|
2019-08-24 21:20:20 +00:00
|
|
|
|
2019-10-25 19:07:47 +00:00
|
|
|
if (relative_path_.empty())
|
2017-11-03 19:53:10 +00:00
|
|
|
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (!attach)
|
|
|
|
{
|
2019-12-12 08:57:25 +00:00
|
|
|
/// create directories if they do not exist
|
|
|
|
disk->createDirectories(table_path);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-07-12 02:31:58 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
file_checker.repair();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-06-19 15:39:41 +00:00
|
|
|
for (const auto & col : storage_metadata.getColumns().getAllPhysical())
|
2020-09-14 11:22:17 +00:00
|
|
|
addFiles(col);
|
2020-07-12 02:31:58 +00:00
|
|
|
|
|
|
|
if (!attach)
|
|
|
|
for (const auto & file : files)
|
|
|
|
file_checker.setEmpty(file.second.data_file_path);
|
2012-08-29 18:49:54 +00:00
|
|
|
}
|
|
|
|
|
2016-08-10 19:12:29 +00:00
|
|
|
|
2020-09-14 11:22:17 +00:00
|
|
|
void StorageTinyLog::addFiles(const NameAndTypePair & column)
|
2012-08-29 18:49:54 +00:00
|
|
|
{
|
2020-09-14 11:22:17 +00:00
|
|
|
const auto & [name, type] = column;
|
|
|
|
if (files.end() != files.find(name))
|
|
|
|
throw Exception("Duplicate column with name " + name + " in constructor of StorageTinyLog.",
|
2017-04-01 07:20:54 +00:00
|
|
|
ErrorCodes::DUPLICATE_COLUMN);
|
|
|
|
|
2020-09-18 11:37:58 +00:00
|
|
|
IDataType::StreamCallback stream_callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-09-14 11:22:17 +00:00
|
|
|
String stream_name = IDataType::getFileNameForStream(column, substream_path);
|
2017-08-07 07:31:16 +00:00
|
|
|
if (!files.count(stream_name))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
ColumnData column_data;
|
2017-08-07 07:31:16 +00:00
|
|
|
files.insert(std::make_pair(stream_name, column_data));
|
2019-12-25 08:24:13 +00:00
|
|
|
files[stream_name].data_file_path = table_path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION;
|
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;
|
2020-09-14 11:22:17 +00:00
|
|
|
type->enumerateStreams(stream_callback, substream_path);
|
2012-06-25 00:17:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-04-07 14:05:51 +00:00
|
|
|
void StorageTinyLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
2012-06-25 00:17:19 +00:00
|
|
|
{
|
2020-09-18 19:25:56 +00:00
|
|
|
assert(table_path != new_path_to_table_data);
|
2020-09-17 19:50:43 +00:00
|
|
|
{
|
|
|
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
2019-07-30 17:38:22 +00:00
|
|
|
|
2020-09-17 19:50:43 +00:00
|
|
|
disk->moveDirectory(table_path, new_path_to_table_data);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2020-09-17 19:50:43 +00:00
|
|
|
table_path = new_path_to_table_data;
|
|
|
|
file_checker.setPath(table_path + "sizes.json");
|
2012-06-25 00:17:19 +00:00
|
|
|
|
2020-09-17 19:50:43 +00:00
|
|
|
for (auto & file : files)
|
|
|
|
file.second.data_file_path = table_path + fileName(file.second.data_file_path);
|
|
|
|
}
|
2020-04-07 14:05:51 +00:00
|
|
|
renameInMemory(new_table_id);
|
2012-06-25 00:17:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
Pipe StorageTinyLog::read(
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & column_names,
|
2020-06-16 15:51:29 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
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
|
|
|
const size_t max_block_size,
|
2017-12-01 21:13:25 +00:00
|
|
|
const unsigned /*num_streams*/)
|
2012-06-25 00:17:19 +00:00
|
|
|
{
|
2020-06-19 17:17:13 +00:00
|
|
|
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
|
2020-02-17 14:45:10 +00:00
|
|
|
|
2020-11-24 14:14:47 +00:00
|
|
|
auto all_columns = metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names);
|
|
|
|
|
2020-04-05 23:57:24 +00:00
|
|
|
// When reading, we lock the entire storage, because we only have one file
|
|
|
|
// per column and can't modify it concurrently.
|
2020-08-03 13:54:14 +00:00
|
|
|
return Pipe(std::make_shared<TinyLogSource>(
|
2020-12-07 19:02:26 +00:00
|
|
|
max_block_size, Nested::convertToSubcolumns(all_columns),
|
2020-10-21 23:02:20 +00:00
|
|
|
*this, context.getSettingsRef().max_read_buffer_size));
|
2012-06-25 00:17:19 +00:00
|
|
|
}
|
|
|
|
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2020-06-16 15:51:29 +00:00
|
|
|
BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/)
|
2012-06-25 00:17:19 +00:00
|
|
|
{
|
2020-06-16 15:51:29 +00:00
|
|
|
return std::make_shared<TinyLogBlockOutputStream>(*this, metadata_snapshot);
|
2012-06-25 00:17:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-07-03 13:17:19 +00:00
|
|
|
CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context & /* context */)
|
2014-07-31 13:39:23 +00:00
|
|
|
{
|
2019-07-30 17:38:22 +00:00
|
|
|
std::shared_lock<std::shared_mutex> lock(rwlock);
|
2017-04-01 07:20:54 +00:00
|
|
|
return file_checker.check();
|
2014-07-31 13:39:23 +00:00
|
|
|
}
|
|
|
|
|
2020-06-18 10:29:13 +00:00
|
|
|
void StorageTinyLog::truncate(
|
2020-06-18 16:10:47 +00:00
|
|
|
const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2019-07-30 17:38:22 +00:00
|
|
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
|
|
|
|
2019-12-12 08:57:25 +00:00
|
|
|
disk->clearDirectory(table_path);
|
2018-04-21 00:35:20 +00:00
|
|
|
|
2018-06-09 15:48:22 +00:00
|
|
|
files.clear();
|
2019-12-12 08:57:25 +00:00
|
|
|
file_checker = FileChecker{disk, table_path + "sizes.json"};
|
2018-04-21 00:35:20 +00:00
|
|
|
|
2020-06-17 16:39:58 +00:00
|
|
|
for (const auto & column : metadata_snapshot->getColumns().getAllPhysical())
|
2020-09-14 11:22:17 +00:00
|
|
|
addFiles(column);
|
2018-04-21 00:35:20 +00:00
|
|
|
}
|
|
|
|
|
2020-01-22 11:30:11 +00:00
|
|
|
void StorageTinyLog::drop()
|
2019-12-17 13:45:53 +00:00
|
|
|
{
|
|
|
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
2020-01-22 11:30:11 +00:00
|
|
|
if (disk->exists(table_path))
|
|
|
|
disk->removeRecursive(table_path);
|
2019-12-17 13:45:53 +00:00
|
|
|
files.clear();
|
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2020-04-06 13:18:07 +00:00
|
|
|
|
2017-12-30 03:49:02 +00:00
|
|
|
void registerStorageTinyLog(StorageFactory & factory)
|
2017-12-30 00:36:06 +00:00
|
|
|
{
|
2020-02-18 14:41:30 +00:00
|
|
|
StorageFactory::StorageFeatures features{
|
|
|
|
.supports_settings = true
|
|
|
|
};
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
factory.registerStorage("TinyLog", [](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);
|
|
|
|
|
2020-02-18 14:41:30 +00:00
|
|
|
String disk_name = getDiskName(*args.storage_def);
|
|
|
|
DiskPtr disk = args.context.getDisk(disk_name);
|
2020-02-14 14:28:33 +00:00
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
return StorageTinyLog::create(
|
2020-02-14 14:28:33 +00:00
|
|
|
disk, args.relative_data_path, args.table_id, args.columns, args.constraints,
|
2017-12-30 03:49:02 +00:00
|
|
|
args.attach, args.context.getSettings().max_compress_block_size);
|
2020-02-18 14:41:30 +00:00
|
|
|
}, features);
|
2017-12-30 00:36:06 +00:00
|
|
|
}
|
|
|
|
|
2012-06-25 00:17:19 +00:00
|
|
|
}
|