ClickHouse/src/Storages/MergeTree/MergeTreeSink.cpp

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

182 lines
6.0 KiB
C++
Raw Normal View History

2021-07-26 16:48:25 +00:00
#include <Storages/MergeTree/MergeTreeSink.h>
2020-05-20 12:02:02 +00:00
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Storages/StorageMergeTree.h>
2022-09-13 22:43:59 +00:00
#include <DataTypes/ObjectUtils.h>
#include <Interpreters/PartLog.h>
2022-06-24 15:19:59 +00:00
namespace ProfileEvents
{
extern const Event DuplicatedInsertedBlocks;
}
namespace DB
{
MergeTreeSink::~MergeTreeSink() = default;
MergeTreeSink::MergeTreeSink(
StorageMergeTree & storage_,
StorageMetadataPtr metadata_snapshot_,
size_t max_parts_per_block_,
ContextPtr context_)
: SinkToStorage(metadata_snapshot_->getSampleBlock())
, storage(storage_)
, metadata_snapshot(metadata_snapshot_)
, max_parts_per_block(max_parts_per_block_)
, context(context_)
, storage_snapshot(storage.getStorageSnapshot(metadata_snapshot, context))
{
}
2021-07-23 19:33:59 +00:00
void MergeTreeSink::onStart()
{
/// Only check "too many parts" before write,
/// because interrupting long-running INSERT query in the middle is not convenient for users.
2022-04-18 04:15:41 +00:00
storage.delayInsertOrThrowIfNeeded(nullptr, context);
}
void MergeTreeSink::onFinish()
{
finishDelayedChunk();
}
struct MergeTreeSink::DelayedChunk
{
struct Partition
{
MergeTreeDataWriter::TemporaryPart temp_part;
UInt64 elapsed_ns;
String block_dedup_token;
};
std::vector<Partition> partitions;
};
2021-07-23 19:33:59 +00:00
void MergeTreeSink::consume(Chunk chunk)
{
auto block = getHeader().cloneWithColumns(chunk.detachColumns());
2021-07-23 19:33:59 +00:00
2022-09-13 22:43:59 +00:00
deduceTypesOfObjectColumns(storage_snapshot, block);
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context);
using DelayedPartitions = std::vector<MergeTreeSink::DelayedChunk::Partition>;
DelayedPartitions partitions;
const Settings & settings = context->getSettingsRef();
size_t streams = 0;
bool support_parallel_write = false;
for (auto & current_block : part_blocks)
{
Stopwatch watch;
String block_dedup_token;
auto temp_part = storage.writer.writeTempPart(current_block, metadata_snapshot, context);
2022-01-14 19:53:55 +00:00
UInt64 elapsed_ns = watch.elapsed();
2021-02-12 14:02:04 +00:00
/// If optimize_on_insert setting is true, current_block could become empty after merge
/// and we didn't create part.
if (!temp_part.part)
2021-02-12 14:02:04 +00:00
continue;
2022-04-22 16:58:09 +00:00
if (!support_parallel_write && temp_part.part->data_part_storage->supportParallelWrite())
support_parallel_write = true;
if (storage.getDeduplicationLog())
{
const String & dedup_token = settings.insert_deduplication_token;
if (!dedup_token.empty())
{
/// multiple blocks can be inserted within the same insert query
/// an ordinal number is added to dedup token to generate a distinctive block id for each block
block_dedup_token = fmt::format("{}_{}", dedup_token, chunk_dedup_seqnum);
++chunk_dedup_seqnum;
}
}
size_t max_insert_delayed_streams_for_parallel_write = DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE;
if (!support_parallel_write || settings.max_insert_delayed_streams_for_parallel_write.changed)
max_insert_delayed_streams_for_parallel_write = settings.max_insert_delayed_streams_for_parallel_write;
/// In case of too much columns/parts in block, flush explicitly.
streams += temp_part.streams.size();
if (streams > max_insert_delayed_streams_for_parallel_write)
{
finishDelayedChunk();
delayed_chunk = std::make_unique<MergeTreeSink::DelayedChunk>();
delayed_chunk->partitions = std::move(partitions);
finishDelayedChunk();
streams = 0;
support_parallel_write = false;
partitions = DelayedPartitions{};
}
partitions.emplace_back(MergeTreeSink::DelayedChunk::Partition
{
.temp_part = std::move(temp_part),
.elapsed_ns = elapsed_ns,
.block_dedup_token = std::move(block_dedup_token)
});
}
finishDelayedChunk();
delayed_chunk = std::make_unique<MergeTreeSink::DelayedChunk>();
delayed_chunk->partitions = std::move(partitions);
}
void MergeTreeSink::finishDelayedChunk()
{
if (!delayed_chunk)
return;
for (auto & partition : delayed_chunk->partitions)
{
partition.temp_part.finalize();
auto & part = partition.temp_part.part;
2022-06-24 15:19:59 +00:00
bool added = false;
/// It's important to create it outside of lock scope because
2022-07-01 13:26:27 +00:00
/// otherwise it can lock parts in destructor and deadlock is possible.
MergeTreeData::Transaction transaction(storage, context->getCurrentTransaction().get());
2022-06-24 15:19:59 +00:00
{
auto lock = storage.lockParts();
storage.fillNewPartName(part, lock);
auto * deduplication_log = storage.getDeduplicationLog();
if (deduplication_log)
{
const String block_id = part->getZeroLevelPartBlockID(partition.block_dedup_token);
auto res = deduplication_log->addPart(block_id, part->info);
if (!res.second)
{
ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks);
LOG_INFO(storage.log, "Block with ID {} already exists as part {}; ignoring it", block_id, res.first.getPartName());
2022-07-22 18:22:04 +00:00
continue;
2022-06-24 15:19:59 +00:00
}
}
2022-07-22 18:22:04 +00:00
2022-10-22 22:51:59 +00:00
added = storage.renameTempPartAndAdd(part, transaction, lock);
2022-07-22 18:22:04 +00:00
transaction.commit(&lock);
2022-06-24 15:19:59 +00:00
}
2021-04-02 17:54:24 +00:00
/// Part can be deduplicated, so increment counters and add to part log only if it's really added
2022-06-24 15:19:59 +00:00
if (added)
2021-04-02 11:46:42 +00:00
{
PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns);
storage.incrementInsertedPartsProfileEvent(part->getType());
2021-04-02 16:45:18 +00:00
/// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'.
2021-09-08 00:21:21 +00:00
storage.background_operations_assignee.trigger();
2021-04-02 16:45:18 +00:00
}
}
delayed_chunk.reset();
}
}