2020-11-23 17:45:59 +00:00
|
|
|
#include <cassert>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/Exception.h>
|
2011-10-31 17:55:06 +00:00
|
|
|
|
2022-04-19 20:47:29 +00:00
|
|
|
#include <boost/noncopyable.hpp>
|
2020-09-22 09:23:46 +00:00
|
|
|
#include <Interpreters/MutationsInterpreter.h>
|
2022-02-09 00:18:53 +00:00
|
|
|
#include <Interpreters/getColumnFromBlock.h>
|
|
|
|
#include <Interpreters/inplaceBlockConversions.h>
|
2023-11-16 21:24:58 +00:00
|
|
|
#include <Storages/AlterCommands.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2020-09-22 09:23:46 +00:00
|
|
|
#include <Storages/StorageMemory.h>
|
2021-02-07 01:41:31 +00:00
|
|
|
#include <Storages/MemorySettings.h>
|
2022-02-09 00:18:53 +00:00
|
|
|
#include <DataTypes/ObjectUtils.h>
|
2011-10-31 17:55:06 +00:00
|
|
|
|
2018-06-05 19:46:49 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2022-05-24 20:06:08 +00:00
|
|
|
#include <QueryPipeline/QueryPipelineBuilder.h>
|
2021-07-23 19:33:59 +00:00
|
|
|
#include <Processors/Sinks/SinkToStorage.h>
|
2021-10-06 17:59:27 +00:00
|
|
|
#include <Processors/Executors/PullingPipelineExecutor.h>
|
2023-03-19 19:35:24 +00:00
|
|
|
#include <Processors/QueryPlan/ReadFromMemoryStorageStep.h>
|
2023-06-16 19:38:50 +00:00
|
|
|
#include <Processors/QueryPlan/QueryPlan.h>
|
2022-01-10 19:01:41 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2024-03-10 05:44:26 +00:00
|
|
|
#include <Formats/NativeReader.h>
|
|
|
|
#include <Formats/NativeWriter.h>
|
2018-06-05 19:46:49 +00:00
|
|
|
|
2022-01-24 17:41:13 +00:00
|
|
|
#include <Common/FileChecker.h>
|
|
|
|
#include <Compression/CompressedReadBuffer.h>
|
|
|
|
#include <Compression/CompressedReadBufferFromFile.h>
|
|
|
|
#include <Compression/CompressedWriteBuffer.h>
|
2022-05-29 19:53:56 +00:00
|
|
|
#include <Backups/BackupEntriesCollector.h>
|
2023-05-03 23:27:16 +00:00
|
|
|
#include <Backups/BackupEntryFromAppendOnlyFile.h>
|
2023-04-25 17:44:03 +00:00
|
|
|
#include <Backups/BackupEntryFromMemory.h>
|
2022-07-05 17:03:20 +00:00
|
|
|
#include <Backups/BackupEntryFromSmallFile.h>
|
2022-01-24 17:41:13 +00:00
|
|
|
#include <Backups/IBackup.h>
|
2022-07-05 17:03:20 +00:00
|
|
|
#include <Backups/IBackupEntriesLazyBatch.h>
|
2022-05-31 09:33:23 +00:00
|
|
|
#include <Backups/RestorerFromBackup.h>
|
2022-02-22 17:05:52 +00:00
|
|
|
#include <Disks/IO/createReadBufferFromFileBase.h>
|
2022-07-03 14:32:11 +00:00
|
|
|
#include <Disks/TemporaryFileOnDisk.h>
|
2022-01-24 17:41:13 +00:00
|
|
|
#include <IO/copyData.h>
|
|
|
|
|
2011-10-31 17:55:06 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2022-06-29 12:42:23 +00:00
|
|
|
extern const int CANNOT_RESTORE_TABLE;
|
2023-11-17 22:23:04 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2024-02-29 22:42:23 +00:00
|
|
|
extern const int SETTING_CONSTRAINT_VIOLATION;
|
2017-12-30 00:36:06 +00:00
|
|
|
}
|
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
class MemorySink : public SinkToStorage
|
2011-10-31 17:55:06 +00:00
|
|
|
{
|
2015-01-18 08:25:56 +00:00
|
|
|
public:
|
2021-07-23 19:33:59 +00:00
|
|
|
MemorySink(
|
2020-06-16 14:25:08 +00:00
|
|
|
StorageMemory & storage_,
|
2022-03-17 17:26:18 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
2024-02-29 22:42:23 +00:00
|
|
|
ContextPtr context_)
|
2021-07-26 10:08:40 +00:00
|
|
|
: SinkToStorage(metadata_snapshot_->getSampleBlock())
|
2021-07-23 19:33:59 +00:00
|
|
|
, storage(storage_)
|
2024-02-29 22:42:23 +00:00
|
|
|
, storage_snapshot(storage_.getStorageSnapshot(metadata_snapshot_, context_))
|
|
|
|
, context(context_)
|
2021-02-10 17:48:39 +00:00
|
|
|
{
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
String getName() const override { return "MemorySink"; }
|
2018-02-19 00:45:32 +00:00
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
void consume(Chunk chunk) override
|
2015-01-18 08:25:56 +00:00
|
|
|
{
|
2021-09-03 17:29:36 +00:00
|
|
|
auto block = getHeader().cloneWithColumns(chunk.getColumns());
|
2022-02-09 00:18:53 +00:00
|
|
|
storage_snapshot->metadata->check(block, true);
|
|
|
|
if (!storage_snapshot->object_columns.empty())
|
|
|
|
{
|
|
|
|
auto extended_storage_columns = storage_snapshot->getColumns(
|
|
|
|
GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects());
|
|
|
|
|
2022-05-06 14:44:00 +00:00
|
|
|
convertDynamicColumnsToTuples(block, storage_snapshot);
|
2022-02-09 00:18:53 +00:00
|
|
|
}
|
2021-02-07 02:40:06 +00:00
|
|
|
|
2021-02-12 00:25:00 +00:00
|
|
|
if (storage.compress)
|
2021-02-07 02:40:06 +00:00
|
|
|
{
|
2021-02-12 00:25:00 +00:00
|
|
|
Block compressed_block;
|
2021-02-12 21:26:12 +00:00
|
|
|
for (const auto & elem : block)
|
2021-02-12 00:25:00 +00:00
|
|
|
compressed_block.insert({ elem.column->compress(), elem.type, elem.name });
|
2021-02-07 02:40:06 +00:00
|
|
|
|
2024-01-28 22:06:50 +00:00
|
|
|
new_blocks.push_back(std::move(compressed_block));
|
2021-02-12 00:25:00 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2024-01-28 22:06:50 +00:00
|
|
|
new_blocks.push_back(std::move(block));
|
2020-10-06 13:45:17 +00:00
|
|
|
}
|
2021-02-07 02:40:06 +00:00
|
|
|
}
|
2021-02-07 02:40:06 +00:00
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
void onFinish() override
|
2021-02-07 02:40:06 +00:00
|
|
|
{
|
2021-02-12 00:25:00 +00:00
|
|
|
size_t inserted_bytes = 0;
|
|
|
|
size_t inserted_rows = 0;
|
|
|
|
|
|
|
|
for (const auto & block : new_blocks)
|
|
|
|
{
|
|
|
|
inserted_bytes += block.allocatedBytes();
|
|
|
|
inserted_rows += block.rows();
|
|
|
|
}
|
|
|
|
|
2024-02-29 22:42:23 +00:00
|
|
|
Settings settings = context->getSettings();
|
|
|
|
if ((settings.min_bytes_to_keep && settings.min_bytes_to_keep > settings.max_bytes_to_keep)
|
|
|
|
|| (settings.min_rows_to_keep && settings.min_rows_to_keep > settings.max_rows_to_keep)) {
|
|
|
|
throw Exception(ErrorCodes::SETTING_CONSTRAINT_VIOLATION, "Min. bytes / rows must be set with a max.");
|
|
|
|
}
|
|
|
|
|
2021-02-07 02:40:06 +00:00
|
|
|
std::lock_guard lock(storage.mutex);
|
2021-02-12 00:25:00 +00:00
|
|
|
|
|
|
|
auto new_data = std::make_unique<Blocks>(*(storage.data.get()));
|
2024-02-29 22:42:23 +00:00
|
|
|
UInt64 new_total_rows = storage.total_size_rows.load(std::memory_order_relaxed) + inserted_rows;
|
|
|
|
UInt64 new_total_bytes = storage.total_size_bytes.load(std::memory_order_relaxed) + inserted_bytes;
|
|
|
|
while (!new_data->empty()
|
|
|
|
&& ((settings.max_bytes_to_keep && new_total_bytes > settings.max_bytes_to_keep)
|
|
|
|
|| (settings.max_rows_to_keep && new_total_rows > settings.max_rows_to_keep)))
|
|
|
|
{
|
|
|
|
Block oldest_block = new_data->front();
|
|
|
|
UInt64 rows_to_remove = oldest_block.rows();
|
|
|
|
UInt64 bytes_to_remove = oldest_block.allocatedBytes();
|
|
|
|
if (new_total_bytes - bytes_to_remove < settings.min_bytes_to_keep
|
|
|
|
|| new_total_rows - rows_to_remove < settings.min_rows_to_keep)
|
|
|
|
{
|
|
|
|
break; // stop - removing next block will put us under min_bytes / min_rows threshold
|
|
|
|
}
|
|
|
|
|
|
|
|
// delete old block from current storage table
|
|
|
|
new_total_rows -= rows_to_remove;
|
|
|
|
new_total_bytes -= bytes_to_remove;
|
|
|
|
new_data->erase(new_data->begin());
|
|
|
|
}
|
|
|
|
|
2024-03-01 01:01:40 +00:00
|
|
|
// append new data to modified storage table and commit
|
2021-02-07 02:40:06 +00:00
|
|
|
new_data->insert(new_data->end(), new_blocks.begin(), new_blocks.end());
|
|
|
|
|
|
|
|
storage.data.set(std::move(new_data));
|
2024-02-29 22:42:23 +00:00
|
|
|
storage.total_size_rows.store(new_total_rows, std::memory_order_relaxed);
|
|
|
|
storage.total_size_bytes.store(new_total_bytes, std::memory_order_relaxed);
|
2015-01-18 08:25:56 +00:00
|
|
|
}
|
2021-02-07 02:40:06 +00:00
|
|
|
|
2015-01-18 08:25:56 +00:00
|
|
|
private:
|
2021-02-12 00:25:00 +00:00
|
|
|
Blocks new_blocks;
|
2015-01-18 08:25:56 +00:00
|
|
|
StorageMemory & storage;
|
2022-02-09 00:18:53 +00:00
|
|
|
StorageSnapshotPtr storage_snapshot;
|
2024-02-29 22:42:23 +00:00
|
|
|
ContextPtr context;
|
2015-01-18 08:25:56 +00:00
|
|
|
};
|
2011-10-31 17:55:06 +00:00
|
|
|
|
|
|
|
|
2021-02-07 01:41:31 +00:00
|
|
|
StorageMemory::StorageMemory(
|
|
|
|
const StorageID & table_id_,
|
|
|
|
ColumnsDescription columns_description_,
|
|
|
|
ConstraintsDescription constraints_,
|
2021-04-23 12:18:23 +00:00
|
|
|
const String & comment,
|
2021-02-07 01:41:31 +00:00
|
|
|
bool compress_)
|
2021-02-12 00:25:00 +00:00
|
|
|
: IStorage(table_id_), data(std::make_unique<const Blocks>()), compress(compress_)
|
2014-09-30 03:08:47 +00:00
|
|
|
{
|
2020-06-19 15:39:41 +00:00
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(std::move(columns_description_));
|
|
|
|
storage_metadata.setConstraints(std::move(constraints_));
|
2021-04-23 12:18:23 +00:00
|
|
|
storage_metadata.setComment(comment);
|
2020-06-19 15:39:41 +00:00
|
|
|
setInMemoryMetadata(storage_metadata);
|
2014-09-30 03:08:47 +00:00
|
|
|
}
|
|
|
|
|
2022-03-17 17:26:18 +00:00
|
|
|
StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr /*query_context*/) const
|
2022-02-09 00:18:53 +00:00
|
|
|
{
|
|
|
|
auto snapshot_data = std::make_unique<SnapshotData>();
|
|
|
|
snapshot_data->blocks = data.get();
|
|
|
|
|
2022-05-06 14:44:00 +00:00
|
|
|
if (!hasDynamicSubcolumns(metadata_snapshot->getColumns()))
|
2022-02-09 00:18:53 +00:00
|
|
|
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, ColumnsDescription{}, std::move(snapshot_data));
|
|
|
|
|
2022-05-06 14:44:00 +00:00
|
|
|
auto object_columns = getConcreteObjectColumns(
|
2022-02-09 20:47:53 +00:00
|
|
|
snapshot_data->blocks->begin(),
|
|
|
|
snapshot_data->blocks->end(),
|
|
|
|
metadata_snapshot->getColumns(),
|
2022-03-01 17:20:53 +00:00
|
|
|
[](const auto & block) -> const auto & { return block.getColumnsWithTypeAndName(); });
|
2022-02-09 00:18:53 +00:00
|
|
|
|
2023-05-25 22:54:54 +00:00
|
|
|
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, std::move(object_columns), std::move(snapshot_data));
|
2022-02-09 00:18:53 +00:00
|
|
|
}
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2023-03-03 05:41:48 +00:00
|
|
|
void StorageMemory::read(
|
2023-05-01 19:34:25 +00:00
|
|
|
QueryPlan & query_plan,
|
2023-03-03 05:41:48 +00:00
|
|
|
const Names & column_names,
|
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2024-02-20 09:34:10 +00:00
|
|
|
SelectQueryInfo & query_info,
|
|
|
|
ContextPtr context,
|
2023-04-29 05:15:33 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
|
|
size_t /*max_block_size*/,
|
2023-03-03 05:41:48 +00:00
|
|
|
size_t num_streams)
|
|
|
|
{
|
2024-02-20 09:34:10 +00:00
|
|
|
query_plan.addStep(std::make_unique<ReadFromMemoryStorageStep>(
|
|
|
|
column_names, query_info, storage_snapshot, context, shared_from_this(), num_streams, delay_read_for_global_subqueries));
|
2011-10-31 17:55:06 +00:00
|
|
|
}
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2023-06-07 18:33:08 +00:00
|
|
|
SinkToStoragePtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool /*async_insert*/)
|
2011-10-31 17:55:06 +00:00
|
|
|
{
|
2022-03-17 17:26:18 +00:00
|
|
|
return std::make_shared<MemorySink>(*this, metadata_snapshot, context);
|
2011-10-31 17:55:06 +00:00
|
|
|
}
|
|
|
|
|
2011-11-05 23:31:19 +00:00
|
|
|
|
2020-01-22 11:30:11 +00:00
|
|
|
void StorageMemory::drop()
|
2011-11-05 23:31:19 +00:00
|
|
|
{
|
2021-02-12 00:25:00 +00:00
|
|
|
data.set(std::make_unique<Blocks>());
|
2020-10-06 13:45:17 +00:00
|
|
|
total_size_bytes.store(0, std::memory_order_relaxed);
|
|
|
|
total_size_rows.store(0, std::memory_order_relaxed);
|
2011-11-05 23:31:19 +00:00
|
|
|
}
|
|
|
|
|
2021-02-12 00:25:00 +00:00
|
|
|
static inline void updateBlockData(Block & old_block, const Block & new_block)
|
2020-09-22 09:23:46 +00:00
|
|
|
{
|
2021-02-12 00:25:00 +00:00
|
|
|
for (const auto & it : new_block)
|
2020-09-22 09:23:46 +00:00
|
|
|
{
|
2021-02-12 00:25:00 +00:00
|
|
|
auto col_name = it.name;
|
|
|
|
auto & col_with_type_name = old_block.getByName(col_name);
|
|
|
|
col_with_type_name.column = it.column;
|
2020-09-22 09:23:46 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-02-25 10:07:48 +00:00
|
|
|
void StorageMemory::checkMutationIsPossible(const MutationCommands & /*commands*/, const Settings & /*settings*/) const
|
|
|
|
{
|
|
|
|
/// Some validation will be added
|
|
|
|
}
|
|
|
|
|
2023-01-30 17:38:28 +00:00
|
|
|
void StorageMemory::mutate(const MutationCommands & commands, ContextPtr context)
|
2020-09-22 09:23:46 +00:00
|
|
|
{
|
2020-10-08 07:19:37 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2020-09-22 09:29:57 +00:00
|
|
|
auto metadata_snapshot = getInMemoryMetadataPtr();
|
|
|
|
auto storage = getStorageID();
|
|
|
|
auto storage_ptr = DatabaseCatalog::instance().getTable(storage, context);
|
2021-05-19 08:56:08 +00:00
|
|
|
|
2021-06-28 17:02:22 +00:00
|
|
|
/// When max_threads > 1, the order of returning blocks is uncertain,
|
2021-05-19 08:56:08 +00:00
|
|
|
/// which will lead to inconsistency after updateBlockData.
|
|
|
|
auto new_context = Context::createCopy(context);
|
|
|
|
new_context->setSetting("max_streams_to_max_threads_ratio", 1);
|
|
|
|
new_context->setSetting("max_threads", 1);
|
|
|
|
|
2023-05-25 22:54:54 +00:00
|
|
|
MutationsInterpreter::Settings settings(true);
|
|
|
|
auto interpreter = std::make_unique<MutationsInterpreter>(storage_ptr, metadata_snapshot, commands, new_context, settings);
|
2022-05-24 20:06:08 +00:00
|
|
|
auto pipeline = QueryPipelineBuilder::getPipeline(interpreter->execute());
|
2021-10-06 17:59:27 +00:00
|
|
|
PullingPipelineExecutor executor(pipeline);
|
2020-09-22 09:23:46 +00:00
|
|
|
|
2021-02-12 00:25:00 +00:00
|
|
|
Blocks out;
|
2021-10-06 17:59:27 +00:00
|
|
|
Block block;
|
|
|
|
while (executor.pull(block))
|
2020-09-22 09:23:46 +00:00
|
|
|
{
|
2021-02-12 00:25:00 +00:00
|
|
|
if (compress)
|
|
|
|
for (auto & elem : block)
|
|
|
|
elem.column = elem.column->compress();
|
2021-02-07 01:41:31 +00:00
|
|
|
|
2021-02-12 00:25:00 +00:00
|
|
|
out.push_back(block);
|
2020-09-22 09:23:46 +00:00
|
|
|
}
|
|
|
|
|
2021-02-12 00:25:00 +00:00
|
|
|
std::unique_ptr<Blocks> new_data;
|
2020-11-23 05:30:36 +00:00
|
|
|
|
2021-02-12 00:25:00 +00:00
|
|
|
// all column affected
|
2020-09-22 09:23:46 +00:00
|
|
|
if (interpreter->isAffectingAllColumns())
|
|
|
|
{
|
2021-02-12 00:25:00 +00:00
|
|
|
new_data = std::make_unique<Blocks>(out);
|
2020-09-22 09:23:46 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-02-12 00:25:00 +00:00
|
|
|
/// just some of the column affected, we need update it with new column
|
|
|
|
new_data = std::make_unique<Blocks>(*(data.get()));
|
2020-10-04 16:28:36 +00:00
|
|
|
auto data_it = new_data->begin();
|
2020-09-22 09:23:46 +00:00
|
|
|
auto out_it = out.begin();
|
2020-11-26 19:22:26 +00:00
|
|
|
|
2020-11-23 17:45:59 +00:00
|
|
|
while (data_it != new_data->end())
|
2020-09-22 09:23:46 +00:00
|
|
|
{
|
2021-02-12 00:25:00 +00:00
|
|
|
/// Mutation does not change the number of blocks
|
2020-11-23 17:45:59 +00:00
|
|
|
assert(out_it != out.end());
|
|
|
|
|
2021-02-12 00:25:00 +00:00
|
|
|
updateBlockData(*data_it, *out_it);
|
2020-09-22 09:23:46 +00:00
|
|
|
++data_it;
|
|
|
|
++out_it;
|
|
|
|
}
|
2020-11-23 17:45:59 +00:00
|
|
|
|
|
|
|
assert(out_it == out.end());
|
2020-09-22 09:23:46 +00:00
|
|
|
}
|
2020-11-23 05:30:36 +00:00
|
|
|
|
2021-02-12 00:25:00 +00:00
|
|
|
size_t rows = 0;
|
2020-11-23 05:30:36 +00:00
|
|
|
size_t bytes = 0;
|
|
|
|
for (const auto & buffer : *new_data)
|
|
|
|
{
|
|
|
|
rows += buffer.rows();
|
|
|
|
bytes += buffer.bytes();
|
|
|
|
}
|
2021-05-17 10:24:01 +00:00
|
|
|
total_size_bytes.store(bytes, std::memory_order_relaxed);
|
|
|
|
total_size_rows.store(rows, std::memory_order_relaxed);
|
2020-11-23 05:30:36 +00:00
|
|
|
data.set(std::move(new_data));
|
2020-09-22 09:23:46 +00:00
|
|
|
}
|
|
|
|
|
2020-10-06 13:45:17 +00:00
|
|
|
|
2020-06-18 10:29:13 +00:00
|
|
|
void StorageMemory::truncate(
|
2021-04-10 23:33:54 +00:00
|
|
|
const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &)
|
2018-04-21 00:35:20 +00:00
|
|
|
{
|
2021-02-12 00:25:00 +00:00
|
|
|
data.set(std::make_unique<Blocks>());
|
2020-10-06 13:45:17 +00:00
|
|
|
total_size_bytes.store(0, std::memory_order_relaxed);
|
|
|
|
total_size_rows.store(0, std::memory_order_relaxed);
|
2018-04-21 00:35:20 +00:00
|
|
|
}
|
|
|
|
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
namespace
|
2022-01-24 17:41:13 +00:00
|
|
|
{
|
2022-07-05 17:03:20 +00:00
|
|
|
class MemoryBackup : public IBackupEntriesLazyBatch, boost::noncopyable
|
2022-01-31 06:35:07 +00:00
|
|
|
{
|
2022-05-31 09:33:23 +00:00
|
|
|
public:
|
2022-07-05 17:03:20 +00:00
|
|
|
MemoryBackup(
|
2023-03-30 17:06:49 +00:00
|
|
|
ContextPtr context_,
|
2022-05-31 09:33:23 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
|
|
|
const std::shared_ptr<const Blocks> blocks_,
|
|
|
|
const String & data_path_in_backup,
|
2022-07-03 14:32:11 +00:00
|
|
|
const DiskPtr & temp_disk_,
|
2023-07-23 09:23:36 +00:00
|
|
|
const ReadSettings & read_settings_,
|
2022-05-31 09:33:23 +00:00
|
|
|
UInt64 max_compress_block_size_)
|
2023-03-30 17:06:49 +00:00
|
|
|
: context(context_)
|
|
|
|
, metadata_snapshot(metadata_snapshot_)
|
2022-05-31 09:33:23 +00:00
|
|
|
, blocks(blocks_)
|
2022-07-03 14:32:11 +00:00
|
|
|
, temp_disk(temp_disk_)
|
2023-07-23 09:23:36 +00:00
|
|
|
, read_settings(read_settings_)
|
2022-05-31 09:33:23 +00:00
|
|
|
, max_compress_block_size(max_compress_block_size_)
|
|
|
|
{
|
2022-07-05 17:03:20 +00:00
|
|
|
fs::path data_path_in_backup_fs = data_path_in_backup;
|
|
|
|
data_bin_pos = file_paths.size();
|
|
|
|
file_paths.emplace_back(data_path_in_backup_fs / "data.bin");
|
|
|
|
index_mrk_pos= file_paths.size();
|
|
|
|
file_paths.emplace_back(data_path_in_backup_fs / "index.mrk");
|
|
|
|
columns_txt_pos = file_paths.size();
|
|
|
|
file_paths.emplace_back(data_path_in_backup_fs / "columns.txt");
|
|
|
|
count_txt_pos = file_paths.size();
|
|
|
|
file_paths.emplace_back(data_path_in_backup_fs / "count.txt");
|
|
|
|
sizes_json_pos = file_paths.size();
|
|
|
|
file_paths.emplace_back(data_path_in_backup_fs / "sizes.json");
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
private:
|
2022-07-05 17:03:20 +00:00
|
|
|
size_t getSize() const override
|
|
|
|
{
|
|
|
|
return file_paths.size();
|
|
|
|
}
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2022-07-05 17:03:20 +00:00
|
|
|
const String & getName(size_t i) const override
|
|
|
|
{
|
|
|
|
return file_paths[i];
|
|
|
|
}
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2022-07-05 17:03:20 +00:00
|
|
|
BackupEntries generate() override
|
2022-01-31 06:35:07 +00:00
|
|
|
{
|
2022-07-05 17:03:20 +00:00
|
|
|
BackupEntries backup_entries;
|
|
|
|
backup_entries.resize(file_paths.size());
|
|
|
|
|
|
|
|
temp_dir_owner.emplace(temp_disk);
|
2023-06-27 15:10:48 +00:00
|
|
|
fs::path temp_dir = temp_dir_owner->getRelativePath();
|
2022-07-05 17:03:20 +00:00
|
|
|
temp_disk->createDirectories(temp_dir);
|
2022-01-31 06:35:07 +00:00
|
|
|
|
|
|
|
/// Writing data.bin
|
|
|
|
IndexForNativeFormat index;
|
|
|
|
{
|
2022-07-05 17:03:20 +00:00
|
|
|
auto data_file_path = temp_dir / fs::path{file_paths[data_bin_pos]}.filename();
|
2023-05-17 01:14:30 +00:00
|
|
|
auto data_out_compressed = temp_disk->writeFile(data_file_path);
|
|
|
|
auto data_out = std::make_unique<CompressedWriteBuffer>(*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size);
|
|
|
|
NativeWriter block_out{*data_out, 0, metadata_snapshot->getSampleBlock(), false, &index};
|
|
|
|
for (const auto & block : *blocks)
|
|
|
|
block_out.write(block);
|
|
|
|
data_out->finalize();
|
|
|
|
data_out.reset();
|
|
|
|
data_out_compressed->finalize();
|
|
|
|
data_out_compressed.reset();
|
2023-05-03 23:27:16 +00:00
|
|
|
backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared<BackupEntryFromAppendOnlyFile>(temp_disk, data_file_path)};
|
2022-01-31 06:35:07 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Writing index.mrk
|
|
|
|
{
|
2022-07-05 17:03:20 +00:00
|
|
|
auto index_mrk_path = temp_dir / fs::path{file_paths[index_mrk_pos]}.filename();
|
2023-05-17 01:14:30 +00:00
|
|
|
auto index_mrk_out_compressed = temp_disk->writeFile(index_mrk_path);
|
|
|
|
auto index_mrk_out = std::make_unique<CompressedWriteBuffer>(*index_mrk_out_compressed);
|
|
|
|
index.write(*index_mrk_out);
|
|
|
|
index_mrk_out->finalize();
|
|
|
|
index_mrk_out.reset();
|
|
|
|
index_mrk_out_compressed->finalize();
|
|
|
|
index_mrk_out_compressed.reset();
|
2023-05-03 23:27:16 +00:00
|
|
|
backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared<BackupEntryFromAppendOnlyFile>(temp_disk, index_mrk_path)};
|
2022-01-31 06:35:07 +00:00
|
|
|
}
|
|
|
|
|
2022-07-05 17:03:20 +00:00
|
|
|
/// Writing columns.txt
|
|
|
|
{
|
|
|
|
auto columns_desc = metadata_snapshot->getColumns().getAllPhysical().toString();
|
|
|
|
backup_entries[columns_txt_pos] = {file_paths[columns_txt_pos], std::make_shared<BackupEntryFromMemory>(columns_desc)};
|
|
|
|
}
|
2022-01-31 06:35:07 +00:00
|
|
|
|
2022-07-05 17:03:20 +00:00
|
|
|
/// Writing count.txt
|
|
|
|
{
|
|
|
|
size_t num_rows = 0;
|
|
|
|
for (const auto & block : *blocks)
|
|
|
|
num_rows += block.rows();
|
|
|
|
backup_entries[count_txt_pos] = {file_paths[count_txt_pos], std::make_shared<BackupEntryFromMemory>(toString(num_rows))};
|
|
|
|
}
|
2022-01-31 06:35:07 +00:00
|
|
|
|
2022-07-05 17:03:20 +00:00
|
|
|
/// Writing sizes.json
|
|
|
|
{
|
|
|
|
auto sizes_json_path = temp_dir / fs::path{file_paths[sizes_json_pos]}.filename();
|
|
|
|
FileChecker file_checker{temp_disk, sizes_json_path};
|
|
|
|
for (size_t i = 0; i != file_paths.size(); ++i)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-07-05 17:03:20 +00:00
|
|
|
if (i == sizes_json_pos)
|
|
|
|
continue;
|
|
|
|
file_checker.update(temp_dir / fs::path{file_paths[i]}.filename());
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
file_checker.save();
|
2023-07-23 09:23:36 +00:00
|
|
|
backup_entries[sizes_json_pos] = {file_paths[sizes_json_pos], std::make_shared<BackupEntryFromSmallFile>(temp_disk, sizes_json_path, read_settings)};
|
2022-07-05 17:03:20 +00:00
|
|
|
}
|
2022-01-31 06:35:07 +00:00
|
|
|
|
|
|
|
/// We don't need to keep `blocks` any longer.
|
|
|
|
blocks.reset();
|
|
|
|
metadata_snapshot.reset();
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2022-07-05 17:03:20 +00:00
|
|
|
return backup_entries;
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2023-03-30 17:06:49 +00:00
|
|
|
ContextPtr context;
|
2022-05-31 09:33:23 +00:00
|
|
|
StorageMetadataPtr metadata_snapshot;
|
|
|
|
std::shared_ptr<const Blocks> blocks;
|
2022-07-03 14:32:11 +00:00
|
|
|
DiskPtr temp_disk;
|
|
|
|
std::optional<TemporaryFileOnDisk> temp_dir_owner;
|
2023-07-23 09:23:36 +00:00
|
|
|
ReadSettings read_settings;
|
2022-07-05 17:03:20 +00:00
|
|
|
UInt64 max_compress_block_size;
|
|
|
|
Strings file_paths;
|
|
|
|
size_t data_bin_pos, index_mrk_pos, columns_txt_pos, count_txt_pos, sizes_json_pos;
|
2022-05-31 09:33:23 +00:00
|
|
|
};
|
|
|
|
}
|
2022-01-31 06:35:07 +00:00
|
|
|
|
2022-06-29 12:42:23 +00:00
|
|
|
void StorageMemory::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
2022-01-31 06:35:07 +00:00
|
|
|
{
|
2023-04-05 13:42:13 +00:00
|
|
|
auto temp_disk = backup_entries_collector.getContext()->getGlobalTemporaryVolume()->getDisk(0);
|
2023-07-23 09:23:36 +00:00
|
|
|
const auto & read_settings = backup_entries_collector.getReadSettings();
|
2022-05-31 09:33:23 +00:00
|
|
|
auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size;
|
2023-07-23 09:23:36 +00:00
|
|
|
|
2023-03-30 17:06:49 +00:00
|
|
|
backup_entries_collector.addBackupEntries(std::make_shared<MemoryBackup>(
|
|
|
|
backup_entries_collector.getContext(),
|
|
|
|
getInMemoryMetadataPtr(),
|
|
|
|
data.get(),
|
|
|
|
data_path_in_backup,
|
|
|
|
temp_disk,
|
2023-07-23 09:23:36 +00:00
|
|
|
read_settings,
|
2023-03-30 17:06:49 +00:00
|
|
|
max_compress_block_size)->getBackupEntries());
|
2022-01-24 17:41:13 +00:00
|
|
|
}
|
2022-01-31 06:35:07 +00:00
|
|
|
|
2022-06-29 12:42:23 +00:00
|
|
|
void StorageMemory::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
2022-01-31 06:35:07 +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;
|
2022-01-31 06:35:07 +00:00
|
|
|
|
2022-06-29 12:42:23 +00:00
|
|
|
if (!restorer.isNonEmptyTableAllowed() && total_size_bytes)
|
2022-05-31 09:33:23 +00:00
|
|
|
RestorerFromBackup::throwTableIsNotEmpty(getStorageID());
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2023-04-05 13:42:13 +00:00
|
|
|
auto temp_disk = restorer.getContext()->getGlobalTemporaryVolume()->getDisk(0);
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
restorer.addDataRestoreTask(
|
2022-07-03 14:32:11 +00:00
|
|
|
[storage = std::static_pointer_cast<StorageMemory>(shared_from_this()), backup, data_path_in_backup, temp_disk]
|
|
|
|
{ storage->restoreDataImpl(backup, data_path_in_backup, temp_disk); });
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2022-07-06 10:03:10 +00:00
|
|
|
void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, const DiskPtr & temporary_disk)
|
2022-01-24 17:41:13 +00:00
|
|
|
{
|
2022-05-31 09:33:23 +00:00
|
|
|
/// Our data are in the StripeLog format.
|
|
|
|
|
|
|
|
fs::path data_path_in_backup_fs = data_path_in_backup;
|
|
|
|
|
|
|
|
/// Reading index.mrk
|
|
|
|
IndexForNativeFormat index;
|
2022-01-24 17:41:13 +00:00
|
|
|
{
|
2022-05-31 09:33:23 +00:00
|
|
|
String index_file_path = data_path_in_backup_fs / "index.mrk";
|
2022-06-29 12:42:23 +00:00
|
|
|
if (!backup->fileExists(index_file_path))
|
2022-07-02 16:26:08 +00:00
|
|
|
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", index_file_path);
|
|
|
|
|
2023-03-13 22:43:15 +00:00
|
|
|
auto in = backup->readFile(index_file_path);
|
2022-05-31 09:33:23 +00:00
|
|
|
CompressedReadBuffer compressed_in{*in};
|
|
|
|
index.read(compressed_in);
|
2022-01-24 17:41:13 +00:00
|
|
|
}
|
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
/// Reading data.bin
|
|
|
|
Blocks new_blocks;
|
|
|
|
size_t new_bytes = 0;
|
|
|
|
size_t new_rows = 0;
|
2022-01-24 17:41:13 +00:00
|
|
|
{
|
2022-05-31 09:33:23 +00:00
|
|
|
String data_file_path = data_path_in_backup_fs / "data.bin";
|
2022-06-29 12:42:23 +00:00
|
|
|
if (!backup->fileExists(data_file_path))
|
2022-07-02 16:26:08 +00:00
|
|
|
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "File {} in backup is required to restore table", data_file_path);
|
|
|
|
|
2023-03-13 22:43:15 +00:00
|
|
|
auto in = backup->readFile(data_file_path);
|
2022-07-03 14:32:11 +00:00
|
|
|
std::optional<TemporaryFileOnDisk> temp_data_file;
|
2022-05-31 09:33:23 +00:00
|
|
|
if (!dynamic_cast<ReadBufferFromFileBase *>(in.get()))
|
2022-01-24 17:41:13 +00:00
|
|
|
{
|
2022-07-06 10:03:10 +00:00
|
|
|
temp_data_file.emplace(temporary_disk);
|
2023-06-27 15:10:48 +00:00
|
|
|
auto out = std::make_unique<WriteBufferFromFile>(temp_data_file->getAbsolutePath());
|
2022-07-03 14:32:11 +00:00
|
|
|
copyData(*in, *out);
|
|
|
|
out.reset();
|
2023-06-27 15:10:48 +00:00
|
|
|
in = createReadBufferFromFileBase(temp_data_file->getAbsolutePath(), {});
|
2022-01-24 17:41:13 +00:00
|
|
|
}
|
2022-05-31 09:33:23 +00:00
|
|
|
std::unique_ptr<ReadBufferFromFileBase> in_from_file{static_cast<ReadBufferFromFileBase *>(in.release())};
|
|
|
|
CompressedReadBufferFromFile compressed_in{std::move(in_from_file)};
|
|
|
|
NativeReader block_in{compressed_in, 0, index.blocks.begin(), index.blocks.end()};
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
while (auto block = block_in.read())
|
2022-01-24 17:41:13 +00:00
|
|
|
{
|
2024-01-28 22:06:50 +00:00
|
|
|
if (compress)
|
|
|
|
{
|
|
|
|
Block compressed_block;
|
|
|
|
for (const auto & elem : block)
|
|
|
|
compressed_block.insert({ elem.column->compress(), elem.type, elem.name });
|
|
|
|
|
|
|
|
new_blocks.push_back(std::move(compressed_block));
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
new_blocks.push_back(std::move(block));
|
|
|
|
}
|
|
|
|
|
|
|
|
new_bytes += new_blocks.back().bytes();
|
|
|
|
new_rows += new_blocks.back().rows();
|
2022-01-24 17:41:13 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
/// Append old blocks with the new ones.
|
|
|
|
auto old_blocks = data.get();
|
|
|
|
Blocks old_and_new_blocks = *old_blocks;
|
|
|
|
old_and_new_blocks.insert(old_and_new_blocks.end(), std::make_move_iterator(new_blocks.begin()), std::make_move_iterator(new_blocks.end()));
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
/// Finish restoring.
|
|
|
|
data.set(std::make_unique<Blocks>(std::move(old_and_new_blocks)));
|
|
|
|
total_size_bytes += new_bytes;
|
|
|
|
total_size_rows += new_rows;
|
2022-01-24 17:41:13 +00:00
|
|
|
}
|
|
|
|
|
2023-11-16 21:24:58 +00:00
|
|
|
void StorageMemory::checkAlterIsPossible(const AlterCommands & commands, ContextPtr) const
|
|
|
|
{
|
|
|
|
for (const auto & command : commands)
|
|
|
|
{
|
|
|
|
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
|
|
|
|
&& command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN
|
2023-11-17 22:23:04 +00:00
|
|
|
&& command.type != AlterCommand::Type::COMMENT_TABLE && command.type != AlterCommand::Type::RENAME_COLUMN)
|
2023-11-16 21:24:58 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
|
|
|
|
command.type, getName());
|
|
|
|
}
|
|
|
|
}
|
2022-01-24 17:41:13 +00:00
|
|
|
|
2020-11-25 13:47:32 +00:00
|
|
|
std::optional<UInt64> StorageMemory::totalRows(const Settings &) const
|
2020-03-29 08:02:35 +00:00
|
|
|
{
|
2020-10-06 13:45:17 +00:00
|
|
|
/// All modifications of these counters are done under mutex which automatically guarantees synchronization/consistency
|
|
|
|
/// When run concurrently we are fine with any value: "before" or "after"
|
|
|
|
return total_size_rows.load(std::memory_order_relaxed);
|
2020-03-29 08:02:35 +00:00
|
|
|
}
|
|
|
|
|
2020-11-25 13:47:32 +00:00
|
|
|
std::optional<UInt64> StorageMemory::totalBytes(const Settings &) const
|
2020-03-29 08:52:10 +00:00
|
|
|
{
|
2020-10-06 13:45:17 +00:00
|
|
|
return total_size_bytes.load(std::memory_order_relaxed);
|
2020-03-29 08:52:10 +00:00
|
|
|
}
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
void registerStorageMemory(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("Memory", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
if (!args.engine_args.empty())
|
2021-02-07 01:41:31 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
|
|
|
"Engine {} doesn't support any arguments ({} given)",
|
|
|
|
args.engine_name, args.engine_args.size());
|
|
|
|
|
|
|
|
bool has_settings = args.storage_def->settings;
|
|
|
|
MemorySettings settings;
|
|
|
|
if (has_settings)
|
|
|
|
settings.loadFromQuery(*args.storage_def);
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2022-04-19 20:47:29 +00:00
|
|
|
return std::make_shared<StorageMemory>(args.table_id, args.columns, args.constraints, args.comment, settings.compress);
|
2021-01-08 11:42:17 +00:00
|
|
|
},
|
|
|
|
{
|
2021-02-12 21:26:12 +00:00
|
|
|
.supports_settings = true,
|
2021-01-08 11:42:17 +00:00
|
|
|
.supports_parallel_insert = true,
|
2017-12-30 00:36:06 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2011-10-31 17:55:06 +00:00
|
|
|
}
|