ClickHouse/src/Storages/StorageMemory.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

518 lines
19 KiB
C++
Raw Normal View History

2020-11-23 17:45:59 +00:00
#include <cassert>
#include <Common/Exception.h>
2011-10-31 17:55:06 +00:00
#include <boost/noncopyable.hpp>
2020-09-22 09:23:46 +00:00
#include <Interpreters/MutationsInterpreter.h>
#include <Interpreters/getColumnFromBlock.h>
#include <Interpreters/inplaceBlockConversions.h>
#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>
#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>
#include <Parsers/ASTCreateQuery.h>
2018-06-05 19:46:49 +00:00
#include <Common/FileChecker.h>
#include <Compression/CompressedReadBuffer.h>
#include <Compression/CompressedReadBufferFromFile.h>
#include <Compression/CompressedWriteBuffer.h>
#include <Backups/BackupEntriesCollector.h>
#include <Backups/BackupEntryFromAppendOnlyFile.h>
#include <Backups/BackupEntryFromMemory.h>
#include <Backups/BackupEntryFromSmallFile.h>
#include <Backups/IBackup.h>
#include <Backups/IBackupEntriesLazyBatch.h>
#include <Backups/RestorerFromBackup.h>
#include <Disks/IO/createReadBufferFromFileBase.h>
2022-07-03 14:32:11 +00:00
#include <Disks/TemporaryFileOnDisk.h>
#include <IO/copyData.h>
2011-10-31 17:55:06 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int CANNOT_RESTORE_TABLE;
}
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(
StorageMemory & storage_,
const StorageMetadataPtr & metadata_snapshot_,
ContextPtr context)
2021-07-26 10:08:40 +00:00
: SinkToStorage(metadata_snapshot_->getSampleBlock())
2021-07-23 19:33:59 +00:00
, storage(storage_)
, storage_snapshot(storage_.getStorageSnapshot(metadata_snapshot_, context))
2021-02-10 17:48:39 +00:00
{
}
2021-07-23 19:33:59 +00:00
String getName() const override { return "MemorySink"; }
2021-07-23 19:33:59 +00:00
void consume(Chunk chunk) override
2015-01-18 08:25:56 +00:00
{
auto block = getHeader().cloneWithColumns(chunk.getColumns());
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);
}
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
2021-02-12 00:25:00 +00:00
new_blocks.emplace_back(compressed_block);
}
else
{
new_blocks.emplace_back(block);
}
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();
}
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()));
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));
storage.total_size_bytes.fetch_add(inserted_bytes, std::memory_order_relaxed);
storage.total_size_rows.fetch_add(inserted_rows, 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;
2021-02-07 02:40:06 +00:00
2015-01-18 08:25:56 +00:00
StorageMemory & storage;
StorageSnapshotPtr storage_snapshot;
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_)
{
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);
}
StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr /*query_context*/) const
{
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()))
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(
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(); });
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, object_columns, std::move(snapshot_data));
}
void StorageMemory::read(
2023-05-01 19:34:25 +00:00
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
2023-05-01 18:10:05 +00:00
SelectQueryInfo & /*query_info*/,
ContextPtr /*context*/,
QueryProcessingStage::Enum /*processed_stage*/,
size_t /*max_block_size*/,
size_t num_streams)
{
2023-05-01 18:10:05 +00:00
query_plan.addStep(std::make_unique<ReadFromMemoryStorageStep>(column_names, storage_snapshot, num_streams, delay_read_for_global_subqueries));
2011-10-31 17:55:06 +00:00
}
SinkToStoragePtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
2011-10-31 17:55:06 +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>());
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
}
}
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);
auto interpreter = std::make_unique<MutationsInterpreter>(storage_ptr, metadata_snapshot, commands, new_context, true);
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-06-18 10:29:13 +00:00
void StorageMemory::truncate(
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>());
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
}
namespace
{
class MemoryBackup : public IBackupEntriesLazyBatch, boost::noncopyable
{
public:
MemoryBackup(
ContextPtr context_,
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_,
UInt64 max_compress_block_size_)
: context(context_)
, metadata_snapshot(metadata_snapshot_)
, blocks(blocks_)
2022-07-03 14:32:11 +00:00
, temp_disk(temp_disk_)
, max_compress_block_size(max_compress_block_size_)
{
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");
}
private:
size_t getSize() const override
{
return file_paths.size();
}
const String & getName(size_t i) const override
{
return file_paths[i];
}
BackupEntries generate() override
{
BackupEntries backup_entries;
backup_entries.resize(file_paths.size());
temp_dir_owner.emplace(temp_disk);
fs::path temp_dir = temp_dir_owner->getPath();
temp_disk->createDirectories(temp_dir);
/// Writing data.bin
IndexForNativeFormat index;
{
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();
backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared<BackupEntryFromAppendOnlyFile>(temp_disk, data_file_path)};
}
/// Writing index.mrk
{
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();
backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared<BackupEntryFromAppendOnlyFile>(temp_disk, index_mrk_path)};
}
/// 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)};
}
/// 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))};
}
/// 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)
{
if (i == sizes_json_pos)
continue;
file_checker.update(temp_dir / fs::path{file_paths[i]}.filename());
}
file_checker.save();
backup_entries[sizes_json_pos] = {file_paths[sizes_json_pos], std::make_shared<BackupEntryFromSmallFile>(temp_disk, sizes_json_path)};
}
/// We don't need to keep `blocks` any longer.
blocks.reset();
metadata_snapshot.reset();
return backup_entries;
}
ContextPtr context;
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;
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;
};
}
void StorageMemory::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
{
auto temp_disk = backup_entries_collector.getContext()->getGlobalTemporaryVolume()->getDisk(0);
auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size;
backup_entries_collector.addBackupEntries(std::make_shared<MemoryBackup>(
backup_entries_collector.getContext(),
getInMemoryMetadataPtr(),
data.get(),
data_path_in_backup,
temp_disk,
max_compress_block_size)->getBackupEntries());
}
void StorageMemory::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
{
auto backup = restorer.getBackup();
if (!backup->hasFiles(data_path_in_backup))
return;
if (!restorer.isNonEmptyTableAllowed() && total_size_bytes)
RestorerFromBackup::throwTableIsNotEmpty(getStorageID());
auto temp_disk = restorer.getContext()->getGlobalTemporaryVolume()->getDisk(0);
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-07-06 10:03:10 +00:00
void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & data_path_in_backup, const DiskPtr & temporary_disk)
{
/// Our data are in the StripeLog format.
fs::path data_path_in_backup_fs = data_path_in_backup;
/// Reading index.mrk
IndexForNativeFormat index;
{
String index_file_path = data_path_in_backup_fs / "index.mrk";
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);
auto in = backup->readFile(index_file_path);
CompressedReadBuffer compressed_in{*in};
index.read(compressed_in);
}
/// Reading data.bin
Blocks new_blocks;
size_t new_bytes = 0;
size_t new_rows = 0;
{
String data_file_path = data_path_in_backup_fs / "data.bin";
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);
auto in = backup->readFile(data_file_path);
2022-07-03 14:32:11 +00:00
std::optional<TemporaryFileOnDisk> temp_data_file;
if (!dynamic_cast<ReadBufferFromFileBase *>(in.get()))
{
2022-07-06 10:03:10 +00:00
temp_data_file.emplace(temporary_disk);
2022-07-03 14:32:11 +00:00
auto out = std::make_unique<WriteBufferFromFile>(temp_data_file->getPath());
copyData(*in, *out);
out.reset();
in = createReadBufferFromFileBase(temp_data_file->getPath(), {});
}
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()};
while (auto block = block_in.read())
{
new_bytes += block.bytes();
new_rows += block.rows();
new_blocks.push_back(std::move(block));
}
}
/// 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()));
/// Finish restoring.
data.set(std::make_unique<Blocks>(std::move(old_and_new_blocks)));
total_size_bytes += new_bytes;
total_size_rows += new_rows;
}
2020-11-25 13:47:32 +00:00
std::optional<UInt64> StorageMemory::totalRows(const Settings &) const
{
/// 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-11-25 13:47:32 +00:00
std::optional<UInt64> StorageMemory::totalBytes(const Settings &) const
{
return total_size_bytes.load(std::memory_order_relaxed);
}
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);
return std::make_shared<StorageMemory>(args.table_id, args.columns, args.constraints, args.comment, settings.compress);
},
{
2021-02-12 21:26:12 +00:00
.supports_settings = true,
.supports_parallel_insert = true,
});
}
2011-10-31 17:55:06 +00:00
}