2016-10-25 06:49:24 +00:00
|
|
|
#include <sys/stat.h>
|
|
|
|
#include <sys/types.h>
|
|
|
|
|
2017-11-20 04:15:43 +00:00
|
|
|
#include <optional>
|
2015-08-16 07:01:41 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/escapeForFileName.h>
|
|
|
|
#include <Common/Exception.h>
|
2015-08-16 07:01:41 +00:00
|
|
|
|
2020-02-20 16:39:32 +00:00
|
|
|
#include <IO/WriteBufferFromFileBase.h>
|
2021-10-26 09:48:31 +00:00
|
|
|
#include <Compression/CompressedReadBuffer.h>
|
2018-12-28 18:15:26 +00:00
|
|
|
#include <Compression/CompressedReadBufferFromFile.h>
|
|
|
|
#include <Compression/CompressedWriteBuffer.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2021-10-26 09:48:31 +00:00
|
|
|
#include <IO/copyData.h>
|
2015-08-16 07:01:41 +00:00
|
|
|
|
2021-10-15 20:18:20 +00:00
|
|
|
#include <Formats/NativeReader.h>
|
|
|
|
#include <Formats/NativeWriter.h>
|
2015-08-16 07:01:41 +00:00
|
|
|
|
2018-02-18 03:23:48 +00:00
|
|
|
#include <DataTypes/DataTypeFactory.h>
|
|
|
|
|
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 <Parsers/ASTLiteral.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2020-02-14 14:28:33 +00:00
|
|
|
#include <Storages/StorageStripeLog.h>
|
2020-02-18 14:41:30 +00:00
|
|
|
#include "StorageLogSettings.h"
|
2022-05-20 19:49:31 +00:00
|
|
|
#include <Processors/ISource.h>
|
2020-02-14 10:57:09 +00:00
|
|
|
#include <Processors/Sources/NullSource.h>
|
2021-07-23 14:25:35 +00:00
|
|
|
#include <Processors/Sinks/SinkToStorage.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2015-08-16 07:01:41 +00:00
|
|
|
|
2022-05-29 19:53:56 +00:00
|
|
|
#include <Backups/BackupEntriesCollector.h>
|
2022-01-31 06:35:07 +00:00
|
|
|
#include <Backups/BackupEntryFromAppendOnlyFile.h>
|
2021-10-26 09:48:31 +00:00
|
|
|
#include <Backups/BackupEntryFromSmallFile.h>
|
|
|
|
#include <Backups/IBackup.h>
|
2022-05-31 09:33:23 +00:00
|
|
|
#include <Backups/RestorerFromBackup.h>
|
2021-10-26 09:48:31 +00:00
|
|
|
#include <Disks/TemporaryFileOnDisk.h>
|
|
|
|
|
|
|
|
#include <base/insertAtEnd.h>
|
2015-08-16 07:01:41 +00:00
|
|
|
|
2020-09-18 19:25:56 +00:00
|
|
|
#include <cassert>
|
|
|
|
|
2015-08-16 07:01:41 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
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;
|
2020-09-24 23:29:16 +00:00
|
|
|
extern const int TIMEOUT_EXCEEDED;
|
2022-06-29 12:42:23 +00:00
|
|
|
extern const int CANNOT_RESTORE_TABLE;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2015-08-16 07:01:41 +00:00
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
/// NOTE: The lock `StorageStripeLog::rwlock` is NOT kept locked while reading,
|
|
|
|
/// because we read ranges of data that do not change.
|
2022-05-20 19:49:31 +00:00
|
|
|
class StripeLogSource final : public ISource
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
|
|
|
public:
|
2020-02-14 10:57:09 +00:00
|
|
|
static Block getHeader(
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-02-14 10:57:09 +00:00
|
|
|
const Names & column_names,
|
|
|
|
IndexForNativeFormat::Blocks::const_iterator index_begin,
|
|
|
|
IndexForNativeFormat::Blocks::const_iterator index_end)
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
2020-02-14 10:57:09 +00:00
|
|
|
if (index_begin == index_end)
|
2021-07-09 03:15:41 +00:00
|
|
|
return storage_snapshot->getSampleBlockForColumns(column_names);
|
2020-02-14 10:57:09 +00:00
|
|
|
|
|
|
|
/// TODO: check if possible to always return storage.getSampleBlock()
|
|
|
|
|
|
|
|
Block header;
|
|
|
|
|
|
|
|
for (const auto & column : index_begin->columns)
|
2018-02-21 04:38:26 +00:00
|
|
|
{
|
2020-02-14 10:57:09 +00:00
|
|
|
auto type = DataTypeFactory::instance().get(column.type);
|
|
|
|
header.insert(ColumnWithTypeAndName{ type, column.name });
|
2018-02-21 04:38:26 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-02-14 10:57:09 +00:00
|
|
|
return header;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-06-16 14:25:08 +00:00
|
|
|
StripeLogSource(
|
2021-08-26 22:15:24 +00:00
|
|
|
const StorageStripeLog & storage_,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot_,
|
2020-06-16 14:25:08 +00:00
|
|
|
const Names & column_names,
|
2021-08-24 21:45:58 +00:00
|
|
|
ReadSettings read_settings_,
|
2021-08-26 22:15:24 +00:00
|
|
|
std::shared_ptr<const IndexForNativeFormat> indices_,
|
2020-02-14 10:57:09 +00:00
|
|
|
IndexForNativeFormat::Blocks::const_iterator index_begin_,
|
2021-11-01 00:39:38 +00:00
|
|
|
IndexForNativeFormat::Blocks::const_iterator index_end_,
|
|
|
|
size_t file_size_)
|
2022-05-20 19:49:31 +00:00
|
|
|
: ISource(getHeader(storage_snapshot_, column_names, index_begin_, index_end_))
|
2020-06-16 14:25:08 +00:00
|
|
|
, storage(storage_)
|
2021-07-09 03:15:41 +00:00
|
|
|
, storage_snapshot(storage_snapshot_)
|
2021-08-24 21:45:58 +00:00
|
|
|
, read_settings(std::move(read_settings_))
|
2021-08-26 22:15:24 +00:00
|
|
|
, indices(indices_)
|
2020-06-16 14:25:08 +00:00
|
|
|
, index_begin(index_begin_)
|
|
|
|
, index_end(index_end_)
|
2021-11-01 00:39:38 +00:00
|
|
|
, file_size(file_size_)
|
2018-01-09 01:51:08 +00:00
|
|
|
{
|
2018-08-10 04:02:56 +00:00
|
|
|
}
|
2018-01-06 18:10:44 +00:00
|
|
|
|
2020-02-14 10:57:09 +00:00
|
|
|
String getName() const override { return "StripeLog"; }
|
|
|
|
|
2015-08-16 07:01:41 +00:00
|
|
|
protected:
|
2020-02-14 10:57:09 +00:00
|
|
|
Chunk generate() override
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
2015-10-01 03:30:50 +00:00
|
|
|
Block res;
|
2018-01-09 01:51:08 +00:00
|
|
|
start();
|
2015-12-16 02:32:49 +00:00
|
|
|
|
2015-10-01 03:30:50 +00:00
|
|
|
if (block_in)
|
|
|
|
{
|
|
|
|
res = block_in->read();
|
|
|
|
|
2017-03-12 19:18:07 +00:00
|
|
|
/// Freeing memory before destroying the object.
|
2015-10-01 03:30:50 +00:00
|
|
|
if (!res)
|
|
|
|
{
|
2017-11-20 04:41:56 +00:00
|
|
|
block_in.reset();
|
|
|
|
data_in.reset();
|
2021-08-26 22:15:24 +00:00
|
|
|
indices.reset();
|
2015-10-01 03:30:50 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-02-14 10:57:09 +00:00
|
|
|
return Chunk(res.getColumns(), res.rows());
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
2021-08-26 22:15:24 +00:00
|
|
|
const StorageStripeLog & storage;
|
2021-07-09 03:15:41 +00:00
|
|
|
StorageSnapshotPtr storage_snapshot;
|
2021-08-24 21:45:58 +00:00
|
|
|
ReadSettings read_settings;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
std::shared_ptr<const IndexForNativeFormat> indices;
|
2015-08-16 08:18:34 +00:00
|
|
|
IndexForNativeFormat::Blocks::const_iterator index_begin;
|
|
|
|
IndexForNativeFormat::Blocks::const_iterator index_end;
|
2021-11-01 00:39:38 +00:00
|
|
|
size_t file_size;
|
2021-08-26 22:15:24 +00:00
|
|
|
|
2018-02-21 04:38:26 +00:00
|
|
|
Block header;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-12 19:18:07 +00:00
|
|
|
/** optional - to create objects only on first reading
|
2017-03-13 18:01:46 +00:00
|
|
|
* and delete objects (release buffers) after the source is exhausted
|
|
|
|
* - to save RAM when using a large number of sources.
|
2015-10-01 03:30:50 +00:00
|
|
|
*/
|
2015-12-16 02:32:49 +00:00
|
|
|
bool started = false;
|
2017-11-20 04:15:43 +00:00
|
|
|
std::optional<CompressedReadBufferFromFile> data_in;
|
2021-10-08 17:21:19 +00:00
|
|
|
std::optional<NativeReader> block_in;
|
2018-01-09 01:51:08 +00:00
|
|
|
|
|
|
|
void start()
|
|
|
|
{
|
|
|
|
if (!started)
|
|
|
|
{
|
|
|
|
started = true;
|
|
|
|
|
2019-12-25 08:24:13 +00:00
|
|
|
String data_file_path = storage.table_path + "data.bin";
|
2021-11-01 00:39:38 +00:00
|
|
|
data_in.emplace(storage.disk->readFile(data_file_path, read_settings.adjustBufferSize(file_size)));
|
2018-02-18 02:46:39 +00:00
|
|
|
block_in.emplace(*data_in, 0, index_begin, index_end);
|
2018-01-09 01:51:08 +00:00
|
|
|
}
|
|
|
|
}
|
2015-08-16 07:01:41 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
/// NOTE: The lock `StorageStripeLog::rwlock` is kept locked in exclusive mode while writing.
|
2021-07-23 14:25:35 +00:00
|
|
|
class StripeLogSink final : public SinkToStorage
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-08-26 22:15:24 +00:00
|
|
|
using WriteLock = std::unique_lock<std::shared_timed_mutex>;
|
|
|
|
|
2021-07-23 14:25:35 +00:00
|
|
|
explicit StripeLogSink(
|
2021-08-26 22:15:24 +00:00
|
|
|
StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_, WriteLock && lock_)
|
2021-07-26 10:08:40 +00:00
|
|
|
: SinkToStorage(metadata_snapshot_->getSampleBlock())
|
2021-07-23 14:25:35 +00:00
|
|
|
, storage(storage_)
|
2020-06-16 15:51:29 +00:00
|
|
|
, metadata_snapshot(metadata_snapshot_)
|
2020-09-24 23:29:16 +00:00
|
|
|
, lock(std::move(lock_))
|
2021-08-26 22:15:24 +00:00
|
|
|
, data_out_compressed(storage.disk->writeFile(storage.data_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append))
|
2020-07-12 02:31:58 +00:00
|
|
|
, data_out(std::make_unique<CompressedWriteBuffer>(
|
2021-08-26 22:15:24 +00:00
|
|
|
*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), storage.max_compress_block_size))
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
2020-09-24 23:29:16 +00:00
|
|
|
if (!lock)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
|
2021-01-05 01:49:15 +00:00
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
/// Ensure that indices are loaded because we're going to update them.
|
|
|
|
storage.loadIndices(lock);
|
|
|
|
|
|
|
|
/// If there were no files, save zero file sizes to be able to rollback in case of error.
|
|
|
|
storage.saveFileSizes(lock);
|
|
|
|
|
|
|
|
size_t initial_data_size = storage.file_checker.getFileSize(storage.data_file_path);
|
|
|
|
block_out = std::make_unique<NativeWriter>(*data_out, 0, metadata_snapshot->getSampleBlock(), false, &storage.indices, initial_data_size);
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-07-23 14:25:35 +00:00
|
|
|
String getName() const override { return "StripeLogSink"; }
|
|
|
|
|
|
|
|
~StripeLogSink() override
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2020-07-12 02:31:58 +00:00
|
|
|
if (!done)
|
|
|
|
{
|
|
|
|
/// Rollback partial writes.
|
2021-08-26 22:15:24 +00:00
|
|
|
|
|
|
|
/// No more writing.
|
2020-07-12 02:31:58 +00:00
|
|
|
data_out.reset();
|
|
|
|
data_out_compressed.reset();
|
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
/// Truncate files to the older sizes.
|
2020-07-12 02:31:58 +00:00
|
|
|
storage.file_checker.repair();
|
2021-08-26 22:15:24 +00:00
|
|
|
|
|
|
|
/// Remove excessive indices.
|
|
|
|
storage.removeUnsavedIndices(lock);
|
2020-07-12 02:31:58 +00:00
|
|
|
}
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-07-23 14:25:35 +00:00
|
|
|
void consume(Chunk chunk) override
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
2021-08-26 22:15:24 +00:00
|
|
|
block_out->write(getHeader().cloneWithColumns(chunk.detachColumns()));
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-07-23 14:25:35 +00:00
|
|
|
void onFinish() override
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
|
|
|
if (done)
|
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-07-12 02:31:58 +00:00
|
|
|
data_out->next();
|
2019-12-12 08:57:25 +00:00
|
|
|
data_out_compressed->next();
|
2020-07-30 13:42:05 +00:00
|
|
|
data_out_compressed->finalize();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
/// Save the new indices.
|
|
|
|
storage.saveIndices(lock);
|
|
|
|
|
|
|
|
/// Save the new file sizes.
|
|
|
|
storage.saveFileSizes(lock);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-05-03 17:55:45 +00:00
|
|
|
storage.updateTotalRows(lock);
|
|
|
|
|
2015-08-16 07:01:41 +00:00
|
|
|
done = true;
|
2021-04-04 05:24:01 +00:00
|
|
|
|
|
|
|
/// unlock should be done from the same thread as lock, and dtor may be
|
|
|
|
/// called from different thread, so it should be done here (at least in
|
|
|
|
/// case of no exceptions occurred)
|
|
|
|
lock.unlock();
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
StorageStripeLog & storage;
|
2020-06-16 15:51:29 +00:00
|
|
|
StorageMetadataPtr metadata_snapshot;
|
2021-08-26 22:15:24 +00:00
|
|
|
WriteLock lock;
|
2015-08-16 07:01:41 +00:00
|
|
|
|
2019-12-12 08:57:25 +00:00
|
|
|
std::unique_ptr<WriteBuffer> data_out_compressed;
|
2020-07-12 02:31:58 +00:00
|
|
|
std::unique_ptr<CompressedWriteBuffer> data_out;
|
2021-08-26 22:15:24 +00:00
|
|
|
std::unique_ptr<NativeWriter> block_out;
|
2015-08-16 07:01:41 +00:00
|
|
|
|
|
|
|
bool done = false;
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
StorageStripeLog::StorageStripeLog(
|
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_,
|
2021-04-23 12:18:23 +00:00
|
|
|
const String & comment,
|
2015-08-16 07:01:41 +00:00
|
|
|
bool attach,
|
2022-07-13 20:35:24 +00:00
|
|
|
ContextMutablePtr context_)
|
2019-12-04 16:06:55 +00:00
|
|
|
: IStorage(table_id_)
|
2022-07-13 20:35:24 +00:00
|
|
|
, WithMutableContext(context_)
|
2020-01-13 11:41:42 +00:00
|
|
|
, disk(std::move(disk_))
|
|
|
|
, table_path(relative_path_)
|
2021-08-26 22:15:24 +00:00
|
|
|
, data_file_path(table_path + "data.bin")
|
|
|
|
, index_file_path(table_path + "index.mrk")
|
2020-01-13 11:41:42 +00:00
|
|
|
, file_checker(disk, table_path + "sizes.json")
|
2022-07-13 20:35:24 +00:00
|
|
|
, max_compress_block_size(context_->getSettings().max_compress_block_size)
|
2020-05-30 21:57:37 +00:00
|
|
|
, log(&Poco::Logger::get("StorageStripeLog"))
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
2020-06-19 15:39:41 +00:00
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(columns_);
|
|
|
|
storage_metadata.setConstraints(constraints_);
|
2021-04-23 12:18:23 +00:00
|
|
|
storage_metadata.setComment(comment);
|
2020-06-19 15:39:41 +00:00
|
|
|
setInMemoryMetadata(storage_metadata);
|
2019-08-24 21:20:20 +00:00
|
|
|
|
2019-10-25 19:07:47 +00:00
|
|
|
if (relative_path_.empty())
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::INCORRECT_FILE_NAME, "Storage {} requires data path", getName());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
/// Ensure the file checker is initialized.
|
|
|
|
if (file_checker.empty())
|
|
|
|
{
|
|
|
|
file_checker.setEmpty(data_file_path);
|
|
|
|
file_checker.setEmpty(index_file_path);
|
|
|
|
}
|
|
|
|
|
2015-08-16 07:01:41 +00:00
|
|
|
if (!attach)
|
|
|
|
{
|
2019-12-12 08:57:25 +00:00
|
|
|
/// create directories if they do not exist
|
|
|
|
disk->createDirectories(table_path);
|
2020-07-12 02:31:58 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
file_checker.repair();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|
2022-05-03 17:55:45 +00:00
|
|
|
|
|
|
|
total_bytes = file_checker.getTotalSize();
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
StorageStripeLog::~StorageStripeLog() = default;
|
|
|
|
|
|
|
|
|
2020-04-07 14:05:51 +00:00
|
|
|
void StorageStripeLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
2015-08-16 07:01:41 +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
|
|
|
{
|
2022-08-05 19:41:02 +00:00
|
|
|
disk->createDirectories(new_path_to_table_data);
|
2020-09-17 19:50:43 +00:00
|
|
|
disk->moveDirectory(table_path, new_path_to_table_data);
|
2015-08-16 07:01:41 +00:00
|
|
|
|
2020-09-17 19:50:43 +00:00
|
|
|
table_path = new_path_to_table_data;
|
2021-08-26 22:15:24 +00:00
|
|
|
data_file_path = table_path + "data.bin";
|
|
|
|
index_file_path = table_path + "index.mrk";
|
2020-09-17 19:50:43 +00:00
|
|
|
file_checker.setPath(table_path + "sizes.json");
|
|
|
|
}
|
2020-04-07 14:05:51 +00:00
|
|
|
renameInMemory(new_table_id);
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-07-13 20:35:24 +00:00
|
|
|
static std::chrono::seconds getLockTimeout(ContextPtr local_context)
|
2020-09-24 23:29:16 +00:00
|
|
|
{
|
2022-07-13 20:35:24 +00:00
|
|
|
const Settings & settings = local_context->getSettingsRef();
|
2020-09-24 23:29:16 +00:00
|
|
|
Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds();
|
|
|
|
if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout)
|
|
|
|
lock_timeout = settings.max_execution_time.totalSeconds();
|
|
|
|
return std::chrono::seconds{lock_timeout};
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
Pipe StorageStripeLog::read(
|
2015-08-16 07:01:41 +00:00
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & /*query_info*/,
|
2022-07-13 20:35:24 +00:00
|
|
|
ContextPtr local_context,
|
2018-09-08 11:29:23 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
2017-12-01 21:13:25 +00:00
|
|
|
const size_t /*max_block_size*/,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t num_streams)
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
storage_snapshot->check(column_names);
|
2015-08-16 08:18:34 +00:00
|
|
|
|
2022-07-13 20:35:24 +00:00
|
|
|
auto lock_timeout = getLockTimeout(local_context);
|
2021-08-26 22:15:24 +00:00
|
|
|
loadIndices(lock_timeout);
|
2015-08-16 08:18:34 +00:00
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
ReadLock lock{rwlock, lock_timeout};
|
|
|
|
if (!lock)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
|
2020-02-14 10:57:09 +00:00
|
|
|
|
2021-11-01 00:39:38 +00:00
|
|
|
size_t data_file_size = file_checker.getFileSize(data_file_path);
|
|
|
|
if (!data_file_size)
|
2021-07-09 03:15:41 +00:00
|
|
|
return Pipe(std::make_shared<NullSource>(storage_snapshot->getSampleBlockForColumns(column_names)));
|
2015-12-08 20:04:11 +00:00
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
auto indices_for_selected_columns
|
|
|
|
= std::make_shared<IndexForNativeFormat>(indices.extractIndexForColumns(NameSet{column_names.begin(), column_names.end()}));
|
2021-08-24 21:45:58 +00:00
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
size_t size = indices_for_selected_columns->blocks.size();
|
2017-06-02 15:54:39 +00:00
|
|
|
if (num_streams > size)
|
|
|
|
num_streams = size;
|
2015-08-16 08:18:34 +00:00
|
|
|
|
2022-07-13 20:35:24 +00:00
|
|
|
ReadSettings read_settings = local_context->getReadSettings();
|
2021-08-26 22:15:24 +00:00
|
|
|
Pipes pipes;
|
|
|
|
|
2017-06-02 15:54:39 +00:00
|
|
|
for (size_t stream = 0; stream < num_streams; ++stream)
|
2015-08-16 08:18:34 +00:00
|
|
|
{
|
2021-08-26 22:15:24 +00:00
|
|
|
IndexForNativeFormat::Blocks::const_iterator begin = indices_for_selected_columns->blocks.begin();
|
|
|
|
IndexForNativeFormat::Blocks::const_iterator end = indices_for_selected_columns->blocks.begin();
|
2015-08-16 08:18:34 +00:00
|
|
|
|
2017-06-02 15:54:39 +00:00
|
|
|
std::advance(begin, stream * size / num_streams);
|
|
|
|
std::advance(end, (stream + 1) * size / num_streams);
|
2015-08-16 08:18:34 +00:00
|
|
|
|
2020-02-14 10:57:09 +00:00
|
|
|
pipes.emplace_back(std::make_shared<StripeLogSource>(
|
2021-11-09 12:36:25 +00:00
|
|
|
*this, storage_snapshot, column_names, read_settings, indices_for_selected_columns, begin, end, data_file_size));
|
2015-08-16 08:18:34 +00:00
|
|
|
}
|
|
|
|
|
2017-03-13 18:01:46 +00:00
|
|
|
/// We do not keep read lock directly at the time of reading, because we read ranges of data that do not change.
|
2015-08-16 08:18:34 +00:00
|
|
|
|
2020-08-06 12:24:05 +00:00
|
|
|
return Pipe::unitePipes(std::move(pipes));
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-07-13 20:35:24 +00:00
|
|
|
SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context)
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
2022-07-13 20:35:24 +00:00
|
|
|
WriteLock lock{rwlock, getLockTimeout(local_context)};
|
2020-09-24 23:29:16 +00:00
|
|
|
if (!lock)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
|
2020-09-24 23:29:16 +00:00
|
|
|
|
2021-07-23 14:25:35 +00:00
|
|
|
return std::make_shared<StripeLogSink>(*this, metadata_snapshot, std::move(lock));
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-07-13 20:35:24 +00:00
|
|
|
CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context)
|
2015-08-16 07:01:41 +00:00
|
|
|
{
|
2022-07-13 20:35:24 +00:00
|
|
|
ReadLock lock{rwlock, getLockTimeout(local_context)};
|
2020-09-24 23:29:16 +00:00
|
|
|
if (!lock)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
|
2020-09-24 23:29:16 +00:00
|
|
|
|
2015-08-16 07:01:41 +00:00
|
|
|
return file_checker.check();
|
|
|
|
}
|
|
|
|
|
2021-08-26 22:15:24 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2019-12-12 08:57:25 +00:00
|
|
|
disk->clearDirectory(table_path);
|
2021-08-26 22:15:24 +00:00
|
|
|
|
|
|
|
indices.clear();
|
|
|
|
file_checker.setEmpty(data_file_path);
|
|
|
|
file_checker.setEmpty(index_file_path);
|
|
|
|
|
|
|
|
indices_loaded = true;
|
|
|
|
num_indices_saved = 0;
|
2022-07-13 20:35:24 +00:00
|
|
|
getContext()->dropMMappedFileCache();
|
2021-08-26 22:15:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageStripeLog::loadIndices(std::chrono::seconds lock_timeout)
|
|
|
|
{
|
|
|
|
if (indices_loaded)
|
|
|
|
return;
|
|
|
|
|
|
|
|
/// We load indices with an exclusive lock (i.e. the write lock) because we don't want
|
|
|
|
/// a data race between two threads trying to load indices simultaneously.
|
|
|
|
WriteLock lock{rwlock, lock_timeout};
|
|
|
|
if (!lock)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
|
2021-08-26 22:15:24 +00:00
|
|
|
|
|
|
|
loadIndices(lock);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-05-03 17:55:45 +00:00
|
|
|
void StorageStripeLog::loadIndices(const WriteLock & lock /* already locked exclusively */)
|
2021-08-26 22:15:24 +00:00
|
|
|
{
|
|
|
|
if (indices_loaded)
|
|
|
|
return;
|
|
|
|
|
|
|
|
if (disk->exists(index_file_path))
|
|
|
|
{
|
|
|
|
CompressedReadBufferFromFile index_in(disk->readFile(index_file_path, ReadSettings{}.adjustBufferSize(4096)));
|
|
|
|
indices.read(index_in);
|
|
|
|
}
|
|
|
|
|
|
|
|
indices_loaded = true;
|
|
|
|
num_indices_saved = indices.blocks.size();
|
2022-05-03 17:55:45 +00:00
|
|
|
|
|
|
|
/// We need indices to calculate the number of rows, and now we have the indices.
|
|
|
|
updateTotalRows(lock);
|
2021-08-26 22:15:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageStripeLog::saveIndices(const WriteLock & /* already locked for writing */)
|
|
|
|
{
|
|
|
|
size_t num_indices = indices.blocks.size();
|
|
|
|
if (num_indices_saved == num_indices)
|
|
|
|
return;
|
|
|
|
|
|
|
|
size_t start = num_indices_saved;
|
|
|
|
auto index_out_compressed = disk->writeFile(index_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append);
|
|
|
|
auto index_out = std::make_unique<CompressedWriteBuffer>(*index_out_compressed);
|
|
|
|
|
|
|
|
for (size_t i = start; i != num_indices; ++i)
|
|
|
|
indices.blocks[i].write(*index_out);
|
|
|
|
|
|
|
|
index_out->next();
|
|
|
|
index_out_compressed->next();
|
|
|
|
index_out_compressed->finalize();
|
|
|
|
|
|
|
|
num_indices_saved = num_indices;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageStripeLog::removeUnsavedIndices(const WriteLock & /* already locked for writing */)
|
|
|
|
{
|
|
|
|
if (indices.blocks.size() > num_indices_saved)
|
|
|
|
indices.blocks.resize(num_indices_saved);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageStripeLog::saveFileSizes(const WriteLock & /* already locked for writing */)
|
|
|
|
{
|
|
|
|
file_checker.update(data_file_path);
|
|
|
|
file_checker.update(index_file_path);
|
|
|
|
file_checker.save();
|
2022-05-03 17:55:45 +00:00
|
|
|
total_bytes = file_checker.getTotalSize();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void StorageStripeLog::updateTotalRows(const WriteLock &)
|
|
|
|
{
|
|
|
|
if (!indices_loaded)
|
|
|
|
return;
|
|
|
|
|
|
|
|
size_t new_total_rows = 0;
|
|
|
|
for (const auto & block : indices.blocks)
|
|
|
|
new_total_rows += block.num_rows;
|
|
|
|
total_rows = new_total_rows;
|
|
|
|
}
|
|
|
|
|
|
|
|
std::optional<UInt64> StorageStripeLog::totalRows(const Settings &) const
|
|
|
|
{
|
|
|
|
if (indices_loaded)
|
|
|
|
return total_rows;
|
|
|
|
|
|
|
|
if (!total_bytes)
|
|
|
|
return 0;
|
|
|
|
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
|
|
|
std::optional<UInt64> StorageStripeLog::totalBytes(const Settings &) const
|
|
|
|
{
|
|
|
|
return total_bytes;
|
2018-04-21 00:35:20 +00:00
|
|
|
}
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2022-06-29 12:42:23 +00:00
|
|
|
void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
2021-10-26 09:48:31 +00:00
|
|
|
{
|
2022-05-31 09:33:23 +00:00
|
|
|
auto lock_timeout = getLockTimeout(backup_entries_collector.getContext());
|
2021-10-26 09:48:31 +00:00
|
|
|
loadIndices(lock_timeout);
|
|
|
|
|
|
|
|
ReadLock lock{rwlock, lock_timeout};
|
|
|
|
if (!lock)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
|
2021-10-26 09:48:31 +00:00
|
|
|
|
|
|
|
if (!file_checker.getFileSize(data_file_path))
|
2022-05-29 19:53:56 +00:00
|
|
|
return;
|
2021-10-26 09:48:31 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
fs::path data_path_in_backup_fs = data_path_in_backup;
|
2022-07-03 14:32:11 +00:00
|
|
|
auto temp_dir_owner = std::make_shared<TemporaryFileOnDisk>(disk, "tmp/");
|
2022-05-29 19:53:56 +00:00
|
|
|
fs::path temp_dir = temp_dir_owner->getPath();
|
2021-10-26 09:48:31 +00:00
|
|
|
disk->createDirectories(temp_dir);
|
|
|
|
|
|
|
|
/// data.bin
|
|
|
|
{
|
|
|
|
/// We make a copy of the data file because it can be changed later in write() or in truncate().
|
|
|
|
String data_file_name = fileName(data_file_path);
|
2022-05-29 19:53:56 +00:00
|
|
|
String hardlink_file_path = temp_dir / data_file_name;
|
2022-01-31 06:35:07 +00:00
|
|
|
disk->createHardLink(data_file_path, hardlink_file_path);
|
2022-05-31 09:33:23 +00:00
|
|
|
backup_entries_collector.addBackupEntry(
|
|
|
|
data_path_in_backup_fs / data_file_name,
|
2022-01-31 06:35:07 +00:00
|
|
|
std::make_unique<BackupEntryFromAppendOnlyFile>(
|
|
|
|
disk, hardlink_file_path, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner));
|
2021-10-26 09:48:31 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// index.mrk
|
|
|
|
{
|
|
|
|
/// We make a copy of the data file because it can be changed later in write() or in truncate().
|
|
|
|
String index_file_name = fileName(index_file_path);
|
2022-05-29 19:53:56 +00:00
|
|
|
String hardlink_file_path = temp_dir / index_file_name;
|
2022-01-31 06:35:07 +00:00
|
|
|
disk->createHardLink(index_file_path, hardlink_file_path);
|
2022-05-31 09:33:23 +00:00
|
|
|
backup_entries_collector.addBackupEntry(
|
|
|
|
data_path_in_backup_fs / index_file_name,
|
2022-01-31 06:35:07 +00:00
|
|
|
std::make_unique<BackupEntryFromAppendOnlyFile>(
|
|
|
|
disk, hardlink_file_path, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner));
|
2021-10-26 09:48:31 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// sizes.json
|
|
|
|
String files_info_path = file_checker.getPath();
|
2022-05-31 09:33:23 +00:00
|
|
|
backup_entries_collector.addBackupEntry(
|
|
|
|
data_path_in_backup_fs / fileName(files_info_path), std::make_unique<BackupEntryFromSmallFile>(disk, files_info_path));
|
2021-10-26 09:48:31 +00:00
|
|
|
|
|
|
|
/// columns.txt
|
2022-05-31 09:33:23 +00:00
|
|
|
backup_entries_collector.addBackupEntry(
|
|
|
|
data_path_in_backup_fs / "columns.txt",
|
2022-05-29 19:53:56 +00:00
|
|
|
std::make_unique<BackupEntryFromMemory>(getInMemoryMetadata().getColumns().getAllPhysical().toString()));
|
2021-10-26 09:48:31 +00:00
|
|
|
|
|
|
|
/// count.txt
|
|
|
|
size_t num_rows = 0;
|
|
|
|
for (const auto & block : indices.blocks)
|
|
|
|
num_rows += block.num_rows;
|
2022-05-31 09:33:23 +00:00
|
|
|
backup_entries_collector.addBackupEntry(
|
|
|
|
data_path_in_backup_fs / "count.txt", std::make_unique<BackupEntryFromMemory>(toString(num_rows)));
|
2021-10-26 09:48:31 +00:00
|
|
|
}
|
|
|
|
|
2022-06-29 12:42:23 +00:00
|
|
|
void StorageStripeLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
2021-10-26 09:48:31 +00:00
|
|
|
{
|
2022-05-31 09:33:23 +00:00
|
|
|
auto backup = restorer.getBackup();
|
2022-06-29 12:42:23 +00:00
|
|
|
if (!backup->hasFiles(data_path_in_backup))
|
|
|
|
return;
|
|
|
|
|
|
|
|
if (!restorer.isNonEmptyTableAllowed() && total_bytes)
|
2022-05-31 09:33:23 +00:00
|
|
|
RestorerFromBackup::throwTableIsNotEmpty(getStorageID());
|
2022-01-17 18:55:40 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
auto lock_timeout = getLockTimeout(restorer.getContext());
|
|
|
|
restorer.addDataRestoreTask(
|
|
|
|
[storage = std::static_pointer_cast<StorageStripeLog>(shared_from_this()), backup, data_path_in_backup, lock_timeout]
|
|
|
|
{ storage->restoreDataImpl(backup, data_path_in_backup, lock_timeout); });
|
|
|
|
}
|
2021-10-26 09:48:31 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
void StorageStripeLog::restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, std::chrono::seconds lock_timeout)
|
|
|
|
{
|
|
|
|
WriteLock lock{rwlock, lock_timeout};
|
|
|
|
if (!lock)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
|
2022-01-17 18:55:40 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
/// Load the indices if not loaded yet. We have to do that now because we're going to update these indices.
|
|
|
|
loadIndices(lock);
|
2021-10-26 09:48:31 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
/// If there were no files, save zero file sizes to be able to rollback in case of error.
|
|
|
|
saveFileSizes(lock);
|
2021-10-26 09:48:31 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
fs::path data_path_in_backup_fs = data_path_in_backup;
|
|
|
|
|
|
|
|
/// Append the data file.
|
|
|
|
auto old_data_size = file_checker.getFileSize(data_file_path);
|
2021-10-26 09:48:31 +00:00
|
|
|
{
|
2022-05-31 09:33:23 +00:00
|
|
|
String file_path_in_backup = data_path_in_backup_fs / fileName(data_file_path);
|
2022-06-29 12:42:23 +00:00
|
|
|
if (!backup->fileExists(file_path_in_backup))
|
2022-07-02 16:26:08 +00:00
|
|
|
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", file_path_in_backup);
|
|
|
|
|
2023-03-13 22:43:15 +00:00
|
|
|
backup->copyFileToDisk(file_path_in_backup, disk, data_file_path, WriteMode::Append);
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
2021-10-26 09:48:31 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
/// Append the index.
|
|
|
|
{
|
|
|
|
String index_path_in_backup = data_path_in_backup_fs / fileName(index_file_path);
|
|
|
|
IndexForNativeFormat extra_indices;
|
2022-06-29 12:42:23 +00:00
|
|
|
if (!backup->fileExists(index_path_in_backup))
|
2022-07-02 16:26:08 +00:00
|
|
|
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", index_path_in_backup);
|
|
|
|
|
2023-03-13 22:43:15 +00:00
|
|
|
auto index_in = backup->readFile(index_path_in_backup);
|
2022-05-31 09:33:23 +00:00
|
|
|
CompressedReadBuffer index_compressed_in{*index_in};
|
|
|
|
extra_indices.read(index_compressed_in);
|
|
|
|
|
|
|
|
/// Adjust the offsets.
|
|
|
|
for (auto & block : extra_indices.blocks)
|
2021-10-26 09:48:31 +00:00
|
|
|
{
|
2022-05-31 09:33:23 +00:00
|
|
|
for (auto & column : block.columns)
|
|
|
|
column.location.offset_in_compressed_file += old_data_size;
|
2021-10-26 09:48:31 +00:00
|
|
|
}
|
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
insertAtEnd(indices.blocks, std::move(extra_indices.blocks));
|
2021-10-26 09:48:31 +00:00
|
|
|
}
|
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
/// Finish writing.
|
|
|
|
saveIndices(lock);
|
|
|
|
saveFileSizes(lock);
|
|
|
|
updateTotalRows(lock);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
/// Rollback partial writes.
|
|
|
|
file_checker.repair();
|
|
|
|
removeUnsavedIndices(lock);
|
|
|
|
throw;
|
|
|
|
}
|
2018-04-21 00:35:20 +00:00
|
|
|
}
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
|
|
|
|
void registerStorageStripeLog(StorageFactory & factory)
|
|
|
|
{
|
2020-02-18 14:41:30 +00:00
|
|
|
StorageFactory::StorageFeatures features{
|
|
|
|
.supports_settings = true
|
|
|
|
};
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
factory.registerStorage("StripeLog", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
2020-02-18 14:41:30 +00:00
|
|
|
if (!args.engine_args.empty())
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Engine {} doesn't support any arguments ({} given)",
|
|
|
|
args.engine_name, args.engine_args.size());
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2023-02-04 14:28:31 +00:00
|
|
|
String disk_name = getDiskName(*args.storage_def, args.getContext());
|
2021-04-10 23:33:54 +00:00
|
|
|
DiskPtr disk = args.getContext()->getDisk(disk_name);
|
2020-02-14 14:28:33 +00:00
|
|
|
|
2022-04-19 20:47:29 +00:00
|
|
|
return std::make_shared<StorageStripeLog>(
|
2021-04-23 12:18:23 +00:00
|
|
|
disk,
|
|
|
|
args.relative_data_path,
|
|
|
|
args.table_id,
|
|
|
|
args.columns,
|
|
|
|
args.constraints,
|
|
|
|
args.comment,
|
|
|
|
args.attach,
|
2022-07-13 20:35:24 +00:00
|
|
|
args.getContext());
|
2020-02-18 14:41:30 +00:00
|
|
|
}, features);
|
2017-12-30 00:36:06 +00:00
|
|
|
}
|
|
|
|
|
2015-08-16 07:01:41 +00:00
|
|
|
}
|