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>
|
2017-06-25 00:01:10 +00:00
|
|
|
#include <Storages/StorageMergeTree.h>
|
|
|
|
#include <Interpreters/PartLog.h>
|
2022-05-06 14:44:00 +00:00
|
|
|
#include <DataTypes/ObjectUtils.h>
|
2017-06-25 00:01:10 +00:00
|
|
|
|
2022-06-24 15:19:59 +00:00
|
|
|
namespace ProfileEvents
|
|
|
|
{
|
|
|
|
extern const Event DuplicatedInsertedBlocks;
|
|
|
|
}
|
2017-06-25 00:01:10 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2022-02-01 10:36:51 +00:00
|
|
|
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_)
|
2022-12-15 23:54:46 +00:00
|
|
|
, storage_snapshot(storage.getStorageSnapshotWithoutParts(metadata_snapshot))
|
2022-02-01 10:36:51 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
void MergeTreeSink::onStart()
|
2017-06-25 00:01:10 +00:00
|
|
|
{
|
2020-11-29 15:08:02 +00:00
|
|
|
/// 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);
|
2020-11-29 15:08:02 +00:00
|
|
|
}
|
|
|
|
|
2022-02-01 10:36:51 +00:00
|
|
|
void MergeTreeSink::onFinish()
|
|
|
|
{
|
|
|
|
finishDelayedChunk();
|
|
|
|
}
|
|
|
|
|
|
|
|
struct MergeTreeSink::DelayedChunk
|
|
|
|
{
|
|
|
|
struct Partition
|
|
|
|
{
|
|
|
|
MergeTreeDataWriter::TemporaryPart temp_part;
|
|
|
|
UInt64 elapsed_ns;
|
|
|
|
String block_dedup_token;
|
|
|
|
};
|
|
|
|
|
|
|
|
std::vector<Partition> partitions;
|
|
|
|
};
|
|
|
|
|
2017-06-25 00:01:10 +00:00
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
void MergeTreeSink::consume(Chunk chunk)
|
2020-11-29 15:08:02 +00:00
|
|
|
{
|
2021-09-03 17:29:36 +00:00
|
|
|
auto block = getHeader().cloneWithColumns(chunk.detachColumns());
|
2022-05-06 14:44:00 +00:00
|
|
|
if (!storage_snapshot->object_columns.empty())
|
|
|
|
convertDynamicColumnsToTuples(block, storage_snapshot);
|
2021-07-23 19:33:59 +00:00
|
|
|
|
2021-05-21 16:14:01 +00:00
|
|
|
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context);
|
2022-02-20 22:04:45 +00:00
|
|
|
|
|
|
|
using DelayedPartitions = std::vector<MergeTreeSink::DelayedChunk::Partition>;
|
|
|
|
DelayedPartitions partitions;
|
|
|
|
|
|
|
|
const Settings & settings = context->getSettingsRef();
|
|
|
|
size_t streams = 0;
|
|
|
|
bool support_parallel_write = false;
|
|
|
|
|
2017-06-25 00:01:10 +00:00
|
|
|
for (auto & current_block : part_blocks)
|
|
|
|
{
|
|
|
|
Stopwatch watch;
|
2022-02-01 10:36:51 +00:00
|
|
|
String block_dedup_token;
|
|
|
|
|
|
|
|
auto temp_part = storage.writer.writeTempPart(current_block, metadata_snapshot, context);
|
2017-06-25 00:01:10 +00:00
|
|
|
|
2022-02-01 10:36:51 +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.
|
2022-02-01 10:36:51 +00:00
|
|
|
if (!temp_part.part)
|
2021-02-12 14:02:04 +00:00
|
|
|
continue;
|
|
|
|
|
2022-10-23 03:29:26 +00:00
|
|
|
if (!support_parallel_write && temp_part.part->getDataPartStorage().supportParallelWrite())
|
2022-02-20 22:04:45 +00:00
|
|
|
support_parallel_write = true;
|
|
|
|
|
2022-01-03 23:04:56 +00:00
|
|
|
if (storage.getDeduplicationLog())
|
|
|
|
{
|
2022-02-20 22:04:45 +00:00
|
|
|
const String & dedup_token = settings.insert_deduplication_token;
|
2022-01-03 23:04:56 +00:00
|
|
|
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;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-02-20 22:04:45 +00:00
|
|
|
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{};
|
|
|
|
}
|
|
|
|
|
2022-02-01 10:36:51 +00:00
|
|
|
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;
|
|
|
|
|
2022-07-01 13:16:32 +00:00
|
|
|
/// 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.
|
2022-07-01 13:16:32 +00:00
|
|
|
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);
|
2022-12-20 21:44:27 +00:00
|
|
|
LOG_INFO(storage.log, "Block with ID {} already exists as part {}; ignoring it", block_id, res.first.getPartNameForLogs());
|
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
|
|
|
{
|
2022-02-01 10:36:51 +00:00
|
|
|
PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns);
|
2022-05-25 14:54:49 +00:00
|
|
|
storage.incrementInsertedPartsProfileEvent(part->getType());
|
2017-06-25 00:01:10 +00:00
|
|
|
|
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
|
|
|
}
|
2017-06-25 00:01:10 +00:00
|
|
|
}
|
2022-02-01 10:36:51 +00:00
|
|
|
|
|
|
|
delayed_chunk.reset();
|
2017-06-25 00:01:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|