ClickHouse/src/Storages/StorageMergeTree.cpp

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

2016 lines
77 KiB
C++
Raw Normal View History

#include "StorageMergeTree.h"
2022-10-23 22:29:24 +00:00
#include "Storages/MergeTree/IMergeTreeDataPart.h"
2018-12-25 23:13:30 +00:00
#include <optional>
2022-01-30 19:49:48 +00:00
#include <base/sort.h>
2022-07-05 07:39:52 +00:00
#include <Backups/BackupEntriesCollector.h>
#include <Databases/IDatabase.h>
#include <Common/escapeForFileName.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
2019-03-09 18:24:11 +00:00
#include <Common/ThreadPool.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/MutationsInterpreter.h>
#include <Interpreters/Context.h>
2021-05-17 11:14:09 +00:00
#include <Interpreters/TransactionLog.h>
2021-08-18 22:19:14 +00:00
#include <IO/copyData.h>
2019-07-03 13:17:19 +00:00
#include <Parsers/ASTCheckQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
2019-07-03 13:17:19 +00:00
#include <Parsers/ASTPartition.h>
2019-07-24 12:56:39 +00:00
#include <Parsers/ASTSetQuery.h>
2018-12-25 23:13:30 +00:00
#include <Parsers/queryToString.h>
#include <Parsers/formatAST.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/ActiveDataPartSet.h>
2018-12-25 23:13:30 +00:00
#include <Storages/AlterCommands.h>
2018-12-25 23:18:07 +00:00
#include <Storages/PartitionCommands.h>
2021-07-26 16:48:25 +00:00
#include <Storages/MergeTree/MergeTreeSink.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
#include <Storages/MergeTree/MergePlainMergeTreeTask.h>
2020-10-22 06:18:10 +00:00
#include <Storages/MergeTree/PartitionPruner.h>
2018-12-25 23:13:30 +00:00
#include <Storages/MergeTree/MergeList.h>
2019-07-03 08:49:52 +00:00
#include <Storages/MergeTree/checkDataPart.h>
2021-10-16 14:03:50 +00:00
#include <QueryPipeline/Pipe.h>
2021-09-08 18:29:38 +00:00
#include <Processors/QueryPlan/QueryPlan.h>
2021-03-04 17:38:12 +00:00
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
2012-07-19 20:32:10 +00:00
2012-07-17 20:04:39 +00:00
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
extern const int NOT_ENOUGH_SPACE;
extern const int BAD_ARGUMENTS;
extern const int INCORRECT_DATA;
extern const int CANNOT_ASSIGN_OPTIMIZE;
extern const int TIMEOUT_EXCEEDED;
2020-10-16 11:58:47 +00:00
extern const int UNKNOWN_POLICY;
extern const int NO_SUCH_DATA_PART;
Fix race between DETACH and merges CI reports failure of the 01442_merge_detach_attach test [1]: <details> 2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference: 2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300 2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300 2021-06-21 02:25:43 @@ -0,0 +1 @@ 2021-06-21 02:25:43 +2 2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple() 2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL 2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t 2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t 2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0 2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB. 2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0 2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact 2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts. 2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal 2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec. 2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection. 2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec. 2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1. 2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2 2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB. 2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB. ... 2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0 2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t 2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown 2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges 2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0 2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating 2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key 2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.) 2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data 2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec. 2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. </details> [1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1 The problem here is that there is a tiny race window between DETACH and OPTIMIZE. DETACH cancel merges, and checks that there no merges for parts currently under lock, and later changes the state of parts. After it allows merges. However OPTIMIZE get parts for processing under lock and only when it starts writing temporary part it checks that merges are not canceled. So suppose the following scenario: T#OPTIMIZE T#DETACH - cancel merges - lock - check that no merges are in progress - unlock - lock - get parts *<--Here, state of the parts are not changed yet, and lock is not held, so OPTIMIZE may (and will) get those parts. -->* - write temporary part - check merges not canceled - unlock - change the parts state - allows merges Plus this patch will also fail merges earlier in case of concurrent DETACH. Refs: #13746 Refs: #23315
2021-06-22 23:19:06 +00:00
extern const int ABORTED;
}
namespace ActionLocks
{
extern const StorageActionBlockType PartsMerge;
2019-08-01 15:36:12 +00:00
extern const StorageActionBlockType PartsTTLMerge;
2019-09-03 14:50:49 +00:00
extern const StorageActionBlockType PartsMove;
}
2021-12-14 20:06:34 +00:00
static MergeTreeTransactionPtr tryGetTransactionForMutation(const MergeTreeMutationEntry & mutation, Poco::Logger * log = nullptr)
{
assert(!mutation.tid.isEmpty());
if (mutation.tid.isPrehistoric())
return {};
auto txn = TransactionLog::instance().tryGetRunningTransaction(mutation.tid.getHash());
if (txn)
return txn;
if (log)
LOG_WARNING(log, "Cannot find transaction {} which had started mutation {}, probably it finished", mutation.tid, mutation.file_name);
return {};
}
StorageMergeTree::StorageMergeTree(
2019-12-04 16:06:55 +00:00
const StorageID & table_id_,
const String & relative_data_path_,
2020-06-09 17:28:29 +00:00
const StorageInMemoryMetadata & metadata_,
bool attach,
2021-05-31 14:49:02 +00:00
ContextMutablePtr context_,
const String & date_column_name,
2019-05-03 02:00:57 +00:00
const MergingParams & merging_params_,
2019-08-26 14:24:29 +00:00
std::unique_ptr<MergeTreeSettings> storage_settings_,
bool has_force_restore_data_flag)
2019-12-27 16:34:50 +00:00
: MergeTreeData(
table_id_,
2020-06-09 17:28:29 +00:00
metadata_,
2019-12-27 16:34:50 +00:00
context_,
date_column_name,
merging_params_,
std::move(storage_settings_),
false, /// require_part_metadata
2019-12-27 16:34:50 +00:00
attach)
, reader(*this)
, writer(*this)
, merger_mutator(*this)
2014-03-13 12:48:07 +00:00
{
2022-12-22 13:31:42 +00:00
initializeDirectoriesAndFormatVersion(relative_data_path_, attach, date_column_name);
2019-05-03 02:00:57 +00:00
loadDataParts(has_force_restore_data_flag);
2021-11-17 18:14:14 +00:00
if (!attach && !getDataPartsForInternalUsage().empty())
throw Exception("Data directory for table already containing data parts - probably it was unclean DROP table or manual intervention. You must either clear directory by hand or use ATTACH TABLE instead of CREATE TABLE if you need to use that parts.", ErrorCodes::INCORRECT_DATA);
2019-05-03 02:00:57 +00:00
increment.set(getMaxBlockNumber());
loadMutations();
2021-04-02 11:46:42 +00:00
2021-04-06 10:14:44 +00:00
loadDeduplicationLog();
2014-03-13 12:48:07 +00:00
}
2012-07-17 20:04:39 +00:00
void StorageMergeTree::startup()
{
clearOldWriteAheadLogs();
clearEmptyParts();
/// Temporary directories contain incomplete results of merges (after forced restart)
/// and don't allow to reinitialize them, so delete each of them immediately
2022-08-09 16:44:51 +00:00
clearOldTemporaryDirectories(0, {"tmp_", "delete_tmp_", "tmp-fetch_"});
/// NOTE background task will also do the above cleanups periodically.
time_after_previous_cleanup_parts.restart();
time_after_previous_cleanup_temporary_directories.restart();
2021-09-04 09:02:07 +00:00
/// Do not schedule any background jobs if current storage has static data files.
if (isStaticStorage())
return;
2020-06-03 22:11:06 +00:00
try
{
2021-09-08 00:21:21 +00:00
background_operations_assignee.start();
startBackgroundMovesIfNeeded();
2020-06-03 22:11:06 +00:00
}
catch (...)
{
2020-06-03 22:11:06 +00:00
/// Exception safety: failed "startup" does not require a call to "shutdown" from the caller.
/// And it should be able to safely destroy table after exception in "startup" method.
2020-06-03 22:15:13 +00:00
/// It means that failed "startup" must not create any background tasks that we will have to wait.
2020-06-03 22:11:06 +00:00
try
{
shutdown();
}
catch (...)
{
std::terminate();
}
/// Note: after failed "startup", the table will be in a state that only allows to destroy the object.
throw;
}
}
void StorageMergeTree::flush()
{
2021-12-28 22:03:55 +00:00
if (flush_called.exchange(true))
2021-12-27 15:54:28 +00:00
return;
flushAllInMemoryPartsIfNeeded();
}
2014-11-12 10:37:47 +00:00
2013-09-30 01:29:19 +00:00
void StorageMergeTree::shutdown()
2012-07-30 20:32:36 +00:00
{
2021-12-28 22:03:55 +00:00
if (shutdown_called.exchange(true))
2014-03-13 12:48:07 +00:00
return;
2020-03-18 14:43:16 +00:00
/// Unlock all waiting mutations
{
std::lock_guard lock(mutation_wait_mutex);
mutation_wait_event.notify_all();
}
2020-03-18 14:43:16 +00:00
2020-06-08 18:08:55 +00:00
merger_mutator.merges_blocker.cancelForever();
parts_mover.moves_blocker.cancelForever();
2021-09-08 00:21:21 +00:00
background_operations_assignee.finish();
background_moves_assignee.finish();
2020-06-08 18:08:55 +00:00
2022-05-27 14:08:49 +00:00
if (deduplication_log)
deduplication_log->shutdown();
2012-07-18 19:44:04 +00:00
}
2014-03-13 12:48:07 +00:00
StorageMergeTree::~StorageMergeTree()
{
shutdown();
}
2012-07-18 19:44:04 +00:00
void StorageMergeTree::read(
QueryPlan & query_plan,
2014-03-09 17:36:01 +00:00
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr local_context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams)
{
/// If true, then we will ask initiator if we can read chosen ranges
bool enable_parallel_reading = local_context->getClientInfo().collaborate_with_initiator;
if (enable_parallel_reading)
2022-11-14 05:41:51 +00:00
LOG_TRACE(log, "Parallel reading from replicas enabled: {}", enable_parallel_reading);
if (auto plan = reader.read(
column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage, nullptr, enable_parallel_reading))
2020-11-10 10:26:26 +00:00
query_plan = std::move(*plan);
/// Now, copy of parts that is required for the query, stored in the processors,
/// while snapshot_data.parts includes all parts, even one that had been filtered out with partition pruning,
/// reset them to avoid holding them.
auto & snapshot_data = assert_cast<MergeTreeData::SnapshotData &>(*storage_snapshot->data);
snapshot_data.parts = {};
}
2020-11-25 13:47:32 +00:00
std::optional<UInt64> StorageMergeTree::totalRows(const Settings &) const
{
return getTotalActiveSizeInRows();
}
std::optional<UInt64> StorageMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const
2020-09-21 10:13:01 +00:00
{
auto parts = getVisibleDataPartsVector(local_context);
return totalRowsByPartitionPredicateImpl(query_info, local_context, parts);
2020-09-21 10:13:01 +00:00
}
2020-11-25 13:47:32 +00:00
std::optional<UInt64> StorageMergeTree::totalBytes(const Settings &) const
{
return getTotalActiveSizeInBytes();
}
2021-07-23 19:33:59 +00:00
SinkToStoragePtr
StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context)
2013-01-23 11:16:32 +00:00
{
const auto & settings = local_context->getSettingsRef();
2021-07-23 19:33:59 +00:00
return std::make_shared<MergeTreeSink>(
*this, metadata_snapshot, settings.max_partitions_per_insert_block, local_context);
2013-01-23 11:16:32 +00:00
}
2018-08-03 13:17:32 +00:00
void StorageMergeTree::checkTableCanBeDropped() const
2012-08-16 18:17:01 +00:00
{
2019-12-03 16:25:32 +00:00
auto table_id = getStorageID();
getContext()->checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes());
}
2020-01-22 11:30:11 +00:00
void StorageMergeTree::drop()
{
2014-04-11 15:53:32 +00:00
shutdown();
2021-08-23 11:26:54 +00:00
/// In case there is read-only disk we cannot allow to call dropAllData(), but dropping tables is allowed.
2021-09-04 09:02:07 +00:00
if (isStaticStorage())
2021-08-23 11:26:54 +00:00
return;
2019-05-03 02:00:57 +00:00
dropAllData();
2018-04-21 00:35:20 +00:00
}
void StorageMergeTree::alter(
2020-03-09 01:22:33 +00:00
const AlterCommands & commands,
ContextPtr local_context,
2021-10-25 17:49:49 +00:00
AlterLockHolder & table_lock_holder)
2013-08-07 13:07:42 +00:00
{
if (local_context->getCurrentTransaction() && local_context->getSettingsRef().throw_on_unsupported_query_inside_transaction)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ALTER METADATA is not supported inside transactions");
auto table_id = getStorageID();
2021-04-06 10:14:44 +00:00
auto old_storage_settings = getSettings();
2020-06-09 17:28:29 +00:00
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
StorageInMemoryMetadata old_metadata = getInMemoryMetadata();
auto maybe_mutation_commands = commands.getMutationCommands(new_metadata, local_context->getSettingsRef().materialize_ttl_after_modify, local_context);
Int64 mutation_version = -1;
commands.apply(new_metadata, local_context);
2020-06-09 17:28:29 +00:00
/// This alter can be performed at new_metadata level only
2020-03-12 15:41:34 +00:00
if (commands.isSettingsAlter())
{
2020-06-09 17:28:29 +00:00
changeSettings(new_metadata.settings_changes, table_lock_holder);
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata);
2019-05-02 15:12:57 +00:00
}
else
{
{
2020-06-09 17:28:29 +00:00
changeSettings(new_metadata.settings_changes, table_lock_holder);
2020-06-18 17:09:06 +00:00
checkTTLExpressions(new_metadata, old_metadata);
/// Reinitialize primary key because primary key column types might have changed.
setProperties(new_metadata, old_metadata);
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata);
2020-04-03 11:09:27 +00:00
if (!maybe_mutation_commands.empty())
mutation_version = startMutation(maybe_mutation_commands, local_context);
}
2022-09-13 22:43:59 +00:00
{
/// Reset Object columns, because column of type
/// Object may be added or dropped by alter.
auto parts_lock = lockParts();
resetObjectColumnsFromActiveParts(parts_lock);
}
2020-03-18 10:02:57 +00:00
/// Always execute required mutations synchronously, because alters
/// should be executed in sequential order.
if (!maybe_mutation_commands.empty())
2021-12-14 20:06:34 +00:00
waitForMutation(mutation_version);
}
2021-04-06 10:14:44 +00:00
{
/// Some additional changes in settings
auto new_storage_settings = getSettings();
if (old_storage_settings->non_replicated_deduplication_window != new_storage_settings->non_replicated_deduplication_window)
{
/// We cannot place this check into settings sanityCheck because it depends on format_version.
/// sanityCheck must work event without storage.
if (new_storage_settings->non_replicated_deduplication_window != 0 && format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
throw Exception("Deduplication for non-replicated MergeTree in old syntax is not supported", ErrorCodes::BAD_ARGUMENTS);
deduplication_log->setDeduplicationWindowSize(new_storage_settings->non_replicated_deduplication_window);
}
}
2014-03-20 13:00:42 +00:00
}
2016-09-02 04:03:40 +00:00
2019-09-02 11:35:53 +00:00
/// While exists, marks parts as 'currently_merging_mutating_parts' and reserves free space on filesystem.
CurrentlyMergingPartsTagger::CurrentlyMergingPartsTagger(
FutureMergedMutatedPartPtr future_part_,
2020-10-23 08:54:00 +00:00
size_t total_size,
StorageMergeTree & storage_,
const StorageMetadataPtr & metadata_snapshot,
bool is_mutation)
: future_part(future_part_), storage(storage_)
2016-09-02 04:03:40 +00:00
{
2020-10-23 08:54:00 +00:00
/// Assume mutex is already locked, because this method is called from mergeTask.
2019-09-04 17:26:53 +00:00
2020-10-23 08:54:00 +00:00
/// if we mutate part, than we should reserve space on the same disk, because mutations possible can create hardlinks
if (is_mutation)
{
reserved_space = storage.tryReserveSpace(total_size, future_part->parts[0]->getDataPartStorage());
}
2020-10-23 08:54:00 +00:00
else
2016-09-02 04:03:40 +00:00
{
2020-10-23 08:54:00 +00:00
IMergeTreeDataPart::TTLInfos ttl_infos;
size_t max_volume_index = 0;
for (auto & part_ptr : future_part->parts)
{
2020-10-23 08:54:00 +00:00
ttl_infos.update(part_ptr->ttl_infos);
2022-10-25 22:14:06 +00:00
auto disk_name = part_ptr->getDataPartStorage().getDiskName();
size_t volume_index = storage.getStoragePolicy()->getVolumeIndexByDiskName(disk_name);
max_volume_index = std::max(max_volume_index, volume_index);
}
2021-02-18 08:50:31 +00:00
reserved_space = storage.balancedReservation(
metadata_snapshot,
total_size,
max_volume_index,
future_part->name,
future_part->part_info,
future_part->parts,
2021-02-18 08:50:31 +00:00
&tagger,
&ttl_infos);
if (!reserved_space)
reserved_space
= storage.tryReserveSpacePreferringTTLRules(metadata_snapshot, total_size, ttl_infos, time(nullptr), max_volume_index);
2016-09-02 04:03:40 +00:00
}
2021-02-18 08:50:31 +00:00
2020-10-23 08:54:00 +00:00
if (!reserved_space)
{
if (is_mutation)
throw Exception("Not enough space for mutating part '" + future_part->parts[0]->name + "'", ErrorCodes::NOT_ENOUGH_SPACE);
2020-10-23 08:54:00 +00:00
else
throw Exception("Not enough space for merging parts", ErrorCodes::NOT_ENOUGH_SPACE);
2016-09-02 04:03:40 +00:00
}
future_part->updatePath(storage, reserved_space.get());
for (const auto & part : future_part->parts)
2016-09-02 04:03:40 +00:00
{
if (storage.currently_merging_mutating_parts.contains(part))
2020-10-23 08:54:00 +00:00
throw Exception("Tagging already tagged part " + part->name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
}
storage.currently_merging_mutating_parts.insert(future_part->parts.begin(), future_part->parts.end());
2020-10-23 08:54:00 +00:00
}
CurrentlyMergingPartsTagger::~CurrentlyMergingPartsTagger()
2020-10-23 08:54:00 +00:00
{
std::lock_guard lock(storage.currently_processing_in_background_mutex);
for (const auto & part : future_part->parts)
2020-10-23 08:54:00 +00:00
{
if (!storage.currently_merging_mutating_parts.contains(part))
2020-10-23 08:54:00 +00:00
std::terminate();
storage.currently_merging_mutating_parts.erase(part);
2016-09-02 04:03:40 +00:00
}
2020-10-23 08:54:00 +00:00
storage.currently_processing_in_background_condition.notify_all();
}
Int64 StorageMergeTree::startMutation(const MutationCommands & commands, ContextPtr query_context)
{
2019-09-04 17:26:53 +00:00
/// Choose any disk, because when we load mutations we search them at each disk
/// where storage can be placed. See loadMutations().
auto disk = getStoragePolicy()->getAnyDisk();
2021-12-14 20:06:34 +00:00
TransactionID current_tid = Tx::PrehistoricTID;
String additional_info;
auto txn = query_context->getCurrentTransaction();
if (txn)
{
current_tid = txn->tid;
additional_info = fmt::format(" (TID: {}; TIDH: {})", current_tid, current_tid.getHash());
}
Int64 version;
2020-10-15 10:54:50 +00:00
{
std::lock_guard lock(currently_processing_in_background_mutex);
2020-03-18 14:43:16 +00:00
MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get(), current_tid, getContext()->getWriteSettings());
2020-10-15 10:54:50 +00:00
version = increment.get();
entry.commit(version);
2021-12-14 20:06:34 +00:00
String mutation_id = entry.file_name;
if (txn)
txn->addMutation(shared_from_this(), mutation_id);
2021-11-19 16:51:03 +00:00
bool inserted = current_mutations_by_version.try_emplace(version, std::move(entry)).second;
if (!inserted)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version);
2021-12-14 20:06:34 +00:00
LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info);
2020-10-15 10:54:50 +00:00
}
2021-09-08 00:21:21 +00:00
background_operations_assignee.trigger();
2020-04-03 11:09:27 +00:00
return version;
}
2020-03-18 10:02:57 +00:00
2020-07-22 19:29:54 +00:00
void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr result_part, bool is_successful, const String & exception_message)
2020-07-22 19:29:54 +00:00
{
/// Update the information about failed parts in the system.mutations table.
Int64 sources_data_version = result_part->parts.at(0)->info.getDataVersion();
Int64 result_data_version = result_part->part_info.getDataVersion();
2020-07-22 19:29:54 +00:00
if (sources_data_version != result_data_version)
{
std::lock_guard lock(currently_processing_in_background_mutex);
auto mutations_begin_it = current_mutations_by_version.upper_bound(sources_data_version);
auto mutations_end_it = current_mutations_by_version.upper_bound(result_data_version);
for (auto it = mutations_begin_it; it != mutations_end_it; ++it)
{
MergeTreeMutationEntry & entry = it->second;
if (is_successful)
{
if (!entry.latest_failed_part.empty() && result_part->part_info.contains(entry.latest_failed_part_info))
2020-07-22 19:29:54 +00:00
{
entry.latest_failed_part.clear();
entry.latest_failed_part_info = MergeTreePartInfo();
entry.latest_fail_time = 0;
entry.latest_fail_reason.clear();
}
}
else
{
entry.latest_failed_part = result_part->parts.at(0)->name;
entry.latest_failed_part_info = result_part->parts.at(0)->info;
2020-07-22 19:29:54 +00:00
entry.latest_fail_time = time(nullptr);
entry.latest_fail_reason = exception_message;
}
}
}
std::unique_lock lock(mutation_wait_mutex);
mutation_wait_event.notify_all();
}
2021-12-14 20:06:34 +00:00
void StorageMergeTree::waitForMutation(Int64 version)
2020-04-03 11:09:27 +00:00
{
String mutation_id = MergeTreeMutationEntry::versionToFileName(version);
waitForMutation(version, mutation_id);
2021-12-14 20:06:34 +00:00
}
void StorageMergeTree::waitForMutation(const String & mutation_id)
{
Int64 version = MergeTreeMutationEntry::parseFileName(mutation_id);
waitForMutation(version, mutation_id);
}
void StorageMergeTree::waitForMutation(Int64 version, const String & mutation_id)
{
2021-12-14 20:06:34 +00:00
LOG_INFO(log, "Waiting mutation: {}", mutation_id);
{
auto check = [version, this]()
{
if (shutdown_called)
return true;
auto mutation_status = getIncompleteMutationsStatus(version);
return !mutation_status || mutation_status->is_done || !mutation_status->latest_fail_reason.empty();
};
std::unique_lock lock(mutation_wait_mutex);
mutation_wait_event.wait(lock, check);
}
2020-07-31 12:22:32 +00:00
/// At least we have our current mutation
2020-07-31 11:37:16 +00:00
std::set<String> mutation_ids;
2021-12-14 20:06:34 +00:00
mutation_ids.insert(mutation_id);
2020-07-31 12:22:32 +00:00
auto mutation_status = getIncompleteMutationsStatus(version, &mutation_ids);
2021-12-14 20:06:34 +00:00
checkMutationStatus(mutation_status, mutation_ids);
2021-12-14 20:06:34 +00:00
LOG_INFO(log, "Mutation {} done", mutation_id);
2020-03-18 10:02:57 +00:00
}
void StorageMergeTree::setMutationCSN(const String & mutation_id, CSN csn)
{
LOG_INFO(log, "Writing CSN {} for mutation {}", csn, mutation_id);
UInt64 version = MergeTreeMutationEntry::parseFileName(mutation_id);
std::lock_guard lock(currently_processing_in_background_mutex);
auto it = current_mutations_by_version.find(version);
if (it == current_mutations_by_version.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find mutation {}", mutation_id);
it->second.writeCSN(csn);
}
void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context)
2020-03-18 10:02:57 +00:00
{
2021-12-15 18:19:29 +00:00
/// Validate partition IDs (if any) before starting mutation
getPartitionIdsAffectedByCommands(commands, query_context);
Int64 version = startMutation(commands, query_context);
2020-04-03 11:09:27 +00:00
2021-12-20 18:53:05 +00:00
if (query_context->getSettingsRef().mutations_sync > 0 || query_context->getCurrentTransaction())
2021-12-14 20:06:34 +00:00
waitForMutation(version);
}
bool StorageMergeTree::hasLightweightDeletedMask() const
{
return has_lightweight_delete_parts.load(std::memory_order_relaxed);
}
namespace
{
struct PartVersionWithName
{
Int64 version;
String name;
};
bool comparator(const PartVersionWithName & f, const PartVersionWithName & s)
{
return f.version < s.version;
}
}
2020-07-31 11:37:16 +00:00
std::optional<MergeTreeMutationStatus> StorageMergeTree::getIncompleteMutationsStatus(Int64 mutation_version, std::set<String> * mutation_ids) const
{
std::lock_guard lock(currently_processing_in_background_mutex);
auto current_mutation_it = current_mutations_by_version.find(mutation_version);
2019-12-19 15:27:56 +00:00
/// Killed
if (current_mutation_it == current_mutations_by_version.end())
return {};
MergeTreeMutationStatus result{.is_done = false};
const auto & mutation_entry = current_mutation_it->second;
2019-12-19 15:27:56 +00:00
2021-12-14 20:06:34 +00:00
auto txn = tryGetTransactionForMutation(mutation_entry, log);
assert(txn || mutation_entry.tid.isPrehistoric());
auto data_parts = getVisibleDataPartsVector(txn);
for (const auto & data_part : data_parts)
{
Int64 data_version = data_part->info.getDataVersion();
2021-03-18 07:38:05 +00:00
if (data_version < mutation_version)
{
if (!mutation_entry.latest_fail_reason.empty())
{
result.latest_failed_part = mutation_entry.latest_failed_part;
result.latest_fail_reason = mutation_entry.latest_fail_reason;
result.latest_fail_time = mutation_entry.latest_fail_time;
/// Fill all mutations which failed with the same error
/// (we can execute several mutations together)
if (mutation_ids)
{
2021-03-18 07:38:05 +00:00
auto mutations_begin_it = current_mutations_by_version.upper_bound(data_version);
for (auto it = mutations_begin_it; it != current_mutations_by_version.end(); ++it)
/// All mutations with the same failure
if (it->second.latest_fail_reason == result.latest_fail_reason)
2020-07-31 11:37:16 +00:00
mutation_ids->insert(it->second.file_name);
}
}
2021-12-14 20:06:34 +00:00
else if (txn)
{
/// Part is locked by concurrent transaction, most likely it will never be mutated
TIDHash part_locked = data_part->version.removal_tid_lock.load();
2021-12-14 20:06:34 +00:00
if (part_locked && part_locked != mutation_entry.tid.getHash())
{
result.latest_failed_part = data_part->name;
result.latest_fail_reason = fmt::format("Serialization error: part {} is locked by transaction {}", data_part->name, part_locked);
result.latest_fail_time = time(nullptr);
}
}
return result;
}
}
result.is_done = true;
return result;
}
std::vector<MergeTreeMutationStatus> StorageMergeTree::getMutationsStatus() const
{
std::lock_guard lock(currently_processing_in_background_mutex);
std::vector<PartVersionWithName> part_versions_with_names;
auto data_parts = getDataPartsVectorForInternalUsage();
part_versions_with_names.reserve(data_parts.size());
for (const auto & part : data_parts)
part_versions_with_names.emplace_back(PartVersionWithName{part->info.getDataVersion(), part->name});
std::sort(part_versions_with_names.begin(), part_versions_with_names.end(), comparator);
std::vector<MergeTreeMutationStatus> result;
for (const auto & kv : current_mutations_by_version)
{
Int64 mutation_version = kv.first;
const MergeTreeMutationEntry & entry = kv.second;
const PartVersionWithName needle{mutation_version, ""};
auto versions_it = std::lower_bound(
part_versions_with_names.begin(), part_versions_with_names.end(), needle, comparator);
size_t parts_to_do = versions_it - part_versions_with_names.begin();
Names parts_to_do_names;
parts_to_do_names.reserve(parts_to_do);
for (size_t i = 0; i < parts_to_do; ++i)
parts_to_do_names.push_back(part_versions_with_names[i].name);
std::map<String, Int64> block_numbers_map({{"", entry.block_number}});
for (const MutationCommand & command : entry.commands)
{
2020-11-09 16:05:40 +00:00
WriteBufferFromOwnString buf;
formatAST(*command.ast, buf, false, true);
result.push_back(MergeTreeMutationStatus
{
entry.file_name,
2020-11-09 16:05:40 +00:00
buf.str(),
entry.create_time,
block_numbers_map,
parts_to_do_names,
/* is_done = */parts_to_do_names.empty(),
entry.latest_failed_part,
entry.latest_fail_time,
entry.latest_fail_reason,
});
}
}
return result;
}
CancellationCode StorageMergeTree::killMutation(const String & mutation_id)
{
2020-05-23 22:24:01 +00:00
LOG_TRACE(log, "Killing mutation {}", mutation_id);
2021-11-19 16:51:03 +00:00
UInt64 mutation_version = MergeTreeMutationEntry::tryParseFileName(mutation_id);
if (!mutation_version)
return CancellationCode::NotFound;
std::optional<MergeTreeMutationEntry> to_kill;
{
2019-08-15 09:43:31 +00:00
std::lock_guard lock(currently_processing_in_background_mutex);
2021-11-19 16:51:03 +00:00
auto it = current_mutations_by_version.find(mutation_version);
if (it != current_mutations_by_version.end())
{
to_kill.emplace(std::move(it->second));
2021-11-19 16:51:03 +00:00
current_mutations_by_version.erase(it);
}
}
if (!to_kill)
return CancellationCode::NotFound;
2021-12-14 20:06:34 +00:00
if (auto txn = tryGetTransactionForMutation(*to_kill, log))
{
LOG_TRACE(log, "Cancelling transaction {} which had started mutation {}", to_kill->tid, mutation_id);
TransactionLog::instance().rollbackTransaction(txn);
}
getContext()->getMergeList().cancelPartMutations(getStorageID(), {}, to_kill->block_number);
to_kill->removeFile();
2020-05-23 22:24:01 +00:00
LOG_TRACE(log, "Cancelled part mutations and removed mutation file {}", mutation_id);
{
std::lock_guard<std::mutex> lock(mutation_wait_mutex);
mutation_wait_event.notify_all();
}
/// Maybe there is another mutation that was blocked by the killed one. Try to execute it immediately.
2021-09-08 00:21:21 +00:00
background_operations_assignee.trigger();
return CancellationCode::CancelSent;
}
2021-04-06 10:14:44 +00:00
void StorageMergeTree::loadDeduplicationLog()
{
auto settings = getSettings();
if (settings->non_replicated_deduplication_window != 0 && format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
throw Exception("Deduplication for non-replicated MergeTree in old syntax is not supported", ErrorCodes::BAD_ARGUMENTS);
2022-05-27 14:08:49 +00:00
auto disk = getDisks()[0];
std::string path = fs::path(relative_data_path) / "deduplication_logs";
deduplication_log = std::make_unique<MergeTreeDeduplicationLog>(path, settings->non_replicated_deduplication_window, format_version, disk);
2021-04-06 10:14:44 +00:00
deduplication_log->load();
}
void StorageMergeTree::loadMutations()
{
2021-11-24 19:45:10 +00:00
for (const auto & disk : getDisks())
{
2021-11-24 19:45:10 +00:00
for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next())
{
if (startsWith(it->name(), "mutation_"))
2019-04-04 13:13:59 +00:00
{
2021-11-24 19:45:10 +00:00
MergeTreeMutationEntry entry(disk, relative_data_path, it->name());
2021-11-19 16:51:03 +00:00
UInt64 block_number = entry.block_number;
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands.size());
if (!entry.tid.isPrehistoric() && !entry.csn)
{
if (auto csn = TransactionLog::getCSN(entry.tid))
{
2022-03-18 11:01:26 +00:00
/// Transaction is committed => mutation is finished, but let's load it anyway (so it will be shown in system.mutations)
entry.writeCSN(csn);
}
else
{
2022-03-14 20:43:34 +00:00
TransactionLog::assertTIDIsNotOutdated(entry.tid);
LOG_DEBUG(log, "Mutation entry {} was created by transaction {}, but it was not committed. Removing mutation entry",
it->name(), entry.tid);
disk->removeFile(it->path());
continue;
}
}
2021-11-19 16:51:03 +00:00
auto inserted = current_mutations_by_version.try_emplace(block_number, std::move(entry)).second;
if (!inserted)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number);
2019-04-04 13:13:59 +00:00
}
else if (startsWith(it->name(), "tmp_mutation_"))
2019-04-04 13:13:59 +00:00
{
disk->removeFile(it->path());
2019-04-04 13:13:59 +00:00
}
}
}
if (!current_mutations_by_version.empty())
2021-11-19 17:05:45 +00:00
increment.value = std::max(increment.value.load(), current_mutations_by_version.rbegin()->first);
}
MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge(
2021-05-17 11:14:09 +00:00
const StorageMetadataPtr & metadata_snapshot,
Fix race between DETACH and merges CI reports failure of the 01442_merge_detach_attach test [1]: <details> 2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference: 2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300 2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300 2021-06-21 02:25:43 @@ -0,0 +1 @@ 2021-06-21 02:25:43 +2 2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple() 2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL 2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t 2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t 2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0 2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB. 2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0 2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact 2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts. 2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal 2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec. 2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection. 2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec. 2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1. 2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2 2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB. 2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB. ... 2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0 2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t 2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown 2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges 2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0 2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating 2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key 2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.) 2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data 2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec. 2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. </details> [1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1 The problem here is that there is a tiny race window between DETACH and OPTIMIZE. DETACH cancel merges, and checks that there no merges for parts currently under lock, and later changes the state of parts. After it allows merges. However OPTIMIZE get parts for processing under lock and only when it starts writing temporary part it checks that merges are not canceled. So suppose the following scenario: T#OPTIMIZE T#DETACH - cancel merges - lock - check that no merges are in progress - unlock - lock - get parts *<--Here, state of the parts are not changed yet, and lock is not held, so OPTIMIZE may (and will) get those parts. -->* - write temporary part - check merges not canceled - unlock - change the parts state - allows merges Plus this patch will also fail merges earlier in case of concurrent DETACH. Refs: #13746 Refs: #23315
2021-06-22 23:19:06 +00:00
bool aggressive,
const String & partition_id,
2021-05-17 11:14:09 +00:00
bool final,
String * out_disable_reason,
TableLockHolder & /* table_lock_holder */,
Fix race between DETACH and merges CI reports failure of the 01442_merge_detach_attach test [1]: <details> 2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference: 2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300 2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300 2021-06-21 02:25:43 @@ -0,0 +1 @@ 2021-06-21 02:25:43 +2 2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple() 2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL 2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t 2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t 2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0 2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB. 2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0 2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact 2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts. 2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal 2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec. 2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection. 2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec. 2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1. 2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2 2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB. 2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB. ... 2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0 2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t 2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown 2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges 2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0 2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating 2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key 2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.) 2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data 2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec. 2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. </details> [1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1 The problem here is that there is a tiny race window between DETACH and OPTIMIZE. DETACH cancel merges, and checks that there no merges for parts currently under lock, and later changes the state of parts. After it allows merges. However OPTIMIZE get parts for processing under lock and only when it starts writing temporary part it checks that merges are not canceled. So suppose the following scenario: T#OPTIMIZE T#DETACH - cancel merges - lock - check that no merges are in progress - unlock - lock - get parts *<--Here, state of the parts are not changed yet, and lock is not held, so OPTIMIZE may (and will) get those parts. -->* - write temporary part - check merges not canceled - unlock - change the parts state - allows merges Plus this patch will also fail merges earlier in case of concurrent DETACH. Refs: #13746 Refs: #23315
2021-06-22 23:19:06 +00:00
std::unique_lock<std::mutex> & lock,
2021-05-17 11:14:09 +00:00
const MergeTreeTransactionPtr & txn,
bool optimize_skip_merged_partitions,
SelectPartsDecision * select_decision_out)
2014-03-13 12:48:07 +00:00
{
2020-09-04 06:55:19 +00:00
auto data_settings = getSettings();
2014-03-13 12:48:07 +00:00
auto future_part = std::make_shared<FutureMergedMutatedPart>();
if (storage_settings.get()->assign_part_uuids)
future_part->uuid = UUIDHelpers::generateV4();
2019-08-15 09:43:31 +00:00
/// You must call destructor with unlocked `currently_processing_in_background_mutex`.
CurrentlyMergingPartsTaggerPtr merging_tagger;
2020-09-04 10:08:09 +00:00
MergeList::EntryPtr merge_entry;
2014-03-13 12:48:07 +00:00
auto can_merge = [this, &lock](const DataPartPtr & left, const DataPartPtr & right, const MergeTreeTransaction * tx, String * disable_reason) -> bool
2014-03-13 12:48:07 +00:00
{
2021-05-18 17:07:29 +00:00
if (tx)
{
2022-03-16 19:16:26 +00:00
/// Cannot merge parts if some of them are not visible in current snapshot
2022-02-03 18:57:09 +00:00
/// TODO Transactions: We can use simplified visibility rules (without CSN lookup) here
if ((left && !left->version.isVisible(tx->getSnapshot(), Tx::EmptyTID))
|| (right && !right->version.isVisible(tx->getSnapshot(), Tx::EmptyTID)))
{
if (disable_reason)
*disable_reason = "Some part is not visible in transaction";
2021-05-18 17:07:29 +00:00
return false;
}
2022-03-18 19:31:44 +00:00
/// Do not try to merge parts that are locked for removal (merge will probably fail)
if ((left && left->version.isRemovalTIDLocked())
|| (right && right->version.isRemovalTIDLocked()))
{
if (disable_reason)
*disable_reason = "Some part is locked for removal in another cuncurrent transaction";
2022-03-18 19:31:44 +00:00
return false;
}
2021-05-18 17:07:29 +00:00
}
2021-05-10 18:03:37 +00:00
/// This predicate is checked for the first part of each range.
/// (left = nullptr, right = "first part of partition")
if (!left)
return !currently_merging_mutating_parts.contains(right);
return !currently_merging_mutating_parts.contains(left) && !currently_merging_mutating_parts.contains(right)
&& getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock) && partsContainSameProjections(left, right);
};
2014-03-27 11:30:54 +00:00
2020-11-11 10:34:32 +00:00
SelectPartsDecision select_decision = SelectPartsDecision::CANNOT_SELECT;
2016-08-13 01:59:09 +00:00
if (partition_id.empty())
{
UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge();
bool merge_with_ttl_allowed = getTotalMergesWithTTLInMergeList() < data_settings->max_number_of_merges_with_ttl_in_pool;
/// TTL requirements is much more strict than for regular merge, so
/// if regular not possible, than merge with ttl is not also not
/// possible.
if (max_source_parts_size > 0)
2014-04-11 13:05:17 +00:00
{
select_decision = merger_mutator.selectPartsToMerge(
future_part,
aggressive,
max_source_parts_size,
can_merge,
merge_with_ttl_allowed,
2021-05-17 11:14:09 +00:00
txn,
out_disable_reason);
}
else if (out_disable_reason)
*out_disable_reason = "Current value of max_source_parts_size is zero";
}
else
{
while (true)
{
select_decision = merger_mutator.selectAllPartsToMergeWithinPartition(
future_part, can_merge, partition_id, final, metadata_snapshot, txn, out_disable_reason, optimize_skip_merged_partitions);
auto timeout_ms = getSettings()->lock_acquire_timeout_for_background_operations.totalMilliseconds();
auto timeout = std::chrono::milliseconds(timeout_ms);
/// If final - we will wait for currently processing merges to finish and continue.
if (final
&& select_decision != SelectPartsDecision::SELECTED
&& !currently_merging_mutating_parts.empty()
&& out_disable_reason
&& out_disable_reason->empty())
2020-06-02 00:41:52 +00:00
{
LOG_DEBUG(log, "Waiting for currently running merges ({} parts are merging right now) to perform OPTIMIZE FINAL",
2020-11-10 21:02:11 +00:00
currently_merging_mutating_parts.size());
2020-06-02 00:41:52 +00:00
if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for(lock, timeout))
2020-06-02 00:41:52 +00:00
{
*out_disable_reason = fmt::format("Timeout ({} ms) while waiting for already running merges before running OPTIMIZE with FINAL", timeout_ms);
2020-06-02 00:41:52 +00:00
break;
}
2020-06-02 00:41:52 +00:00
}
else
break;
2014-04-11 13:05:17 +00:00
}
}
2014-03-13 12:48:07 +00:00
/// In case of final we need to know the decision of select in StorageMergeTree::merge
/// to treat NOTHING_TO_MERGE as successful merge (otherwise optimize final will be uncompleted)
if (select_decision_out)
*select_decision_out = select_decision;
2020-11-10 14:42:56 +00:00
if (select_decision != SelectPartsDecision::SELECTED)
{
if (out_disable_reason)
{
if (!out_disable_reason->empty())
{
*out_disable_reason += ". ";
}
*out_disable_reason += "Cannot select parts for optimization";
}
return {};
2014-03-13 12:48:07 +00:00
}
/// Account TTL merge here to avoid exceeding the max_number_of_merges_with_ttl_in_pool limit
if (isTTLMergeType(future_part->merge_type))
getContext()->getMergeList().bookMergeWithTTL();
merging_tagger = std::make_unique<CurrentlyMergingPartsTagger>(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part->parts), *this, metadata_snapshot, false);
return std::make_shared<MergeMutateSelectedEntry>(future_part, std::move(merging_tagger), std::make_shared<MutationCommands>());
}
bool StorageMergeTree::merge(
bool aggressive,
const String & partition_id,
bool final,
bool deduplicate,
const Names & deduplicate_by_columns,
2021-05-17 11:14:09 +00:00
const MergeTreeTransactionPtr & txn,
2020-11-20 14:29:13 +00:00
String * out_disable_reason,
2020-12-04 14:01:59 +00:00
bool optimize_skip_merged_partitions)
{
2020-10-20 21:10:55 +00:00
auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations);
auto metadata_snapshot = getInMemoryMetadataPtr();
2020-11-10 20:01:43 +00:00
SelectPartsDecision select_decision;
MergeMutateSelectedEntryPtr merge_mutate_entry;
Fix race between DETACH and merges CI reports failure of the 01442_merge_detach_attach test [1]: <details> 2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference: 2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300 2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300 2021-06-21 02:25:43 @@ -0,0 +1 @@ 2021-06-21 02:25:43 +2 2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple() 2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL 2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t 2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t 2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0 2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB. 2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0 2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact 2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts. 2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal 2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec. 2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection. 2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec. 2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1. 2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2 2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB. 2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB. ... 2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0 2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t 2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown 2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges 2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0 2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating 2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key 2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.) 2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data 2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec. 2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. </details> [1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1 The problem here is that there is a tiny race window between DETACH and OPTIMIZE. DETACH cancel merges, and checks that there no merges for parts currently under lock, and later changes the state of parts. After it allows merges. However OPTIMIZE get parts for processing under lock and only when it starts writing temporary part it checks that merges are not canceled. So suppose the following scenario: T#OPTIMIZE T#DETACH - cancel merges - lock - check that no merges are in progress - unlock - lock - get parts *<--Here, state of the parts are not changed yet, and lock is not held, so OPTIMIZE may (and will) get those parts. -->* - write temporary part - check merges not canceled - unlock - change the parts state - allows merges Plus this patch will also fail merges earlier in case of concurrent DETACH. Refs: #13746 Refs: #23315
2021-06-22 23:19:06 +00:00
{
std::unique_lock lock(currently_processing_in_background_mutex);
if (merger_mutator.merges_blocker.isCancelled())
throw Exception("Cancelled merging parts", ErrorCodes::ABORTED);
merge_mutate_entry = selectPartsToMerge(
metadata_snapshot,
aggressive,
partition_id,
final,
out_disable_reason,
table_lock_holder,
lock,
txn,
Fix race between DETACH and merges CI reports failure of the 01442_merge_detach_attach test [1]: <details> 2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference: 2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300 2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300 2021-06-21 02:25:43 @@ -0,0 +1 @@ 2021-06-21 02:25:43 +2 2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple() 2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL 2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t 2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t 2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0 2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB. 2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0 2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact 2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts. 2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal 2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec. 2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection. 2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec. 2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1. 2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2 2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB. 2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB. ... 2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0 2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t 2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown 2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges 2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0 2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating 2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key 2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.) 2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data 2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec. 2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. </details> [1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1 The problem here is that there is a tiny race window between DETACH and OPTIMIZE. DETACH cancel merges, and checks that there no merges for parts currently under lock, and later changes the state of parts. After it allows merges. However OPTIMIZE get parts for processing under lock and only when it starts writing temporary part it checks that merges are not canceled. So suppose the following scenario: T#OPTIMIZE T#DETACH - cancel merges - lock - check that no merges are in progress - unlock - lock - get parts *<--Here, state of the parts are not changed yet, and lock is not held, so OPTIMIZE may (and will) get those parts. -->* - write temporary part - check merges not canceled - unlock - change the parts state - allows merges Plus this patch will also fail merges earlier in case of concurrent DETACH. Refs: #13746 Refs: #23315
2021-06-22 23:19:06 +00:00
optimize_skip_merged_partitions,
&select_decision);
}
2020-11-10 20:01:43 +00:00
2020-11-19 22:22:40 +00:00
/// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization)
if (select_decision == SelectPartsDecision::NOTHING_TO_MERGE)
2020-11-10 20:01:43 +00:00
return true;
if (!merge_mutate_entry)
return false;
2022-09-02 08:54:48 +00:00
/// Copying a vector of columns `deduplicate by columns.
IExecutableTask::TaskResultCallback f = [](bool) {};
auto task = std::make_shared<MergePlainMergeTreeTask>(
*this, metadata_snapshot, deduplicate, deduplicate_by_columns, merge_mutate_entry, table_lock_holder, f);
2020-10-13 14:25:42 +00:00
2021-09-23 20:15:49 +00:00
task->setCurrentTransaction(MergeTreeTransactionHolder{}, MergeTreeTransactionPtr{txn});
executeHere(task);
2014-04-11 13:05:17 +00:00
return true;
}
2019-09-05 13:12:29 +00:00
bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & part) const
{
std::lock_guard background_processing_lock(currently_processing_in_background_mutex);
return currently_merging_mutating_parts.contains(part);
2019-08-16 15:57:19 +00:00
}
MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate(
2021-07-29 17:37:06 +00:00
const StorageMetadataPtr & metadata_snapshot, String * /* disable_reason */, TableLockHolder & /* table_lock_holder */,
std::unique_lock<std::mutex> & /*currently_processing_in_background_mutex_lock*/)
2019-06-19 17:56:41 +00:00
{
if (current_mutations_by_version.empty())
return {};
size_t max_source_part_size = merger_mutator.getMaxSourcePartSizeForMutation();
if (max_source_part_size == 0)
{
LOG_DEBUG(
log,
2021-05-03 10:50:44 +00:00
"Not enough idle threads to apply mutations at the moment. See settings 'number_of_free_entries_in_pool_to_execute_mutation' "
"and 'background_pool_size'");
return {};
}
2022-10-12 04:18:17 +00:00
size_t max_ast_elements = getContext()->getSettingsRef().max_expanded_ast_elements;
auto future_part = std::make_shared<FutureMergedMutatedPart>();
if (storage_settings.get()->assign_part_uuids)
future_part->uuid = UUIDHelpers::generateV4();
CurrentlyMergingPartsTaggerPtr tagger;
auto mutations_end_it = current_mutations_by_version.end();
for (const auto & part : getDataPartsVectorForInternalUsage())
{
if (currently_merging_mutating_parts.contains(part))
continue;
auto mutations_begin_it = current_mutations_by_version.upper_bound(part->info.getDataVersion());
if (mutations_begin_it == mutations_end_it)
continue;
if (max_source_part_size < part->getBytesOnDisk())
{
LOG_DEBUG(
log,
"Current max source part size for mutation is {} but part size {}. Will not mutate part {} yet",
max_source_part_size,
part->getBytesOnDisk(),
part->name);
continue;
}
2021-12-14 20:06:34 +00:00
TransactionID first_mutation_tid = mutations_begin_it->second.tid;
MergeTreeTransactionPtr txn = tryGetTransactionForMutation(mutations_begin_it->second, log);
assert(txn || first_mutation_tid.isPrehistoric());
2021-05-13 07:23:39 +00:00
2021-12-14 20:06:34 +00:00
if (txn)
{
/// Mutate visible parts only
/// NOTE Do not mutate visible parts in Outdated state, because it does not make sense:
/// mutation will fail anyway due to serialization error.
if (!part->version.isVisible(*txn))
2021-12-14 20:06:34 +00:00
continue;
}
auto commands = std::make_shared<MutationCommands>();
size_t current_ast_elements = 0;
2021-12-15 18:19:29 +00:00
auto last_mutation_to_apply = mutations_end_it;
for (auto it = mutations_begin_it; it != mutations_end_it; ++it)
{
2022-03-18 11:01:26 +00:00
/// Do not squash mutations from different transactions to be able to commit/rollback them independently.
2021-12-14 20:06:34 +00:00
if (first_mutation_tid != it->second.tid)
break;
size_t commands_size = 0;
MutationCommands commands_for_size_validation;
for (const auto & command : it->second.commands)
{
if (command.type != MutationCommand::Type::DROP_COLUMN
&& command.type != MutationCommand::Type::DROP_INDEX
&& command.type != MutationCommand::Type::DROP_PROJECTION
&& command.type != MutationCommand::Type::RENAME_COLUMN)
{
commands_for_size_validation.push_back(command);
}
else
{
commands_size += command.ast->size();
}
}
if (!commands_for_size_validation.empty())
{
try
{
2021-10-04 21:13:18 +00:00
auto fake_query_context = Context::createCopy(getContext());
fake_query_context->makeQueryContext();
fake_query_context->setCurrentQueryId("");
MutationsInterpreter interpreter(
shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, false);
commands_size += interpreter.evaluateCommandsSize();
}
catch (...)
{
2021-10-04 21:13:18 +00:00
tryLogCurrentException(log);
MergeTreeMutationEntry & entry = it->second;
entry.latest_fail_time = time(nullptr);
entry.latest_fail_reason = getCurrentExceptionMessage(false);
2021-12-15 18:19:29 +00:00
/// NOTE we should not skip mutations, because exception may be retryable (e.g. MEMORY_LIMIT_EXCEEDED)
break;
}
}
if (current_ast_elements + commands_size >= max_ast_elements)
break;
current_ast_elements += commands_size;
commands->insert(commands->end(), it->second.commands.begin(), it->second.commands.end());
2021-12-15 18:19:29 +00:00
last_mutation_to_apply = it;
}
2021-12-15 18:19:29 +00:00
assert(commands->empty() == (last_mutation_to_apply == mutations_end_it));
if (!commands->empty())
{
auto new_part_info = part->info;
2021-12-15 18:19:29 +00:00
new_part_info.mutation = last_mutation_to_apply->first;
future_part->parts.push_back(part);
future_part->part_info = new_part_info;
future_part->name = part->getNewName(new_part_info);
future_part->type = part->getType();
tagger = std::make_unique<CurrentlyMergingPartsTagger>(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true);
2021-12-14 20:06:34 +00:00
return std::make_shared<MergeMutateSelectedEntry>(future_part, std::move(tagger), commands, txn);
}
}
return {};
}
2021-09-08 00:21:21 +00:00
bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) //-V657
2014-03-13 12:48:07 +00:00
{
2014-04-11 13:05:17 +00:00
if (shutdown_called)
2021-06-21 13:36:21 +00:00
return false;
2021-09-04 09:02:07 +00:00
assert(!isStaticStorage());
2020-10-13 14:25:42 +00:00
auto metadata_snapshot = getInMemoryMetadataPtr();
MergeMutateSelectedEntryPtr merge_entry, mutate_entry;
2019-09-03 17:06:36 +00:00
auto shared_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations);
2019-09-03 17:06:36 +00:00
2022-02-03 18:57:09 +00:00
MergeTreeTransactionHolder transaction_for_merge;
MergeTreeTransactionPtr txn;
if (transactions_enabled.load(std::memory_order_relaxed))
{
/// TODO Transactions: avoid beginning transaction if there is nothing to merge.
txn = TransactionLog::instance().beginTransaction();
transaction_for_merge = MergeTreeTransactionHolder{txn, /* autocommit = */ true};
}
2021-09-23 20:15:49 +00:00
bool has_mutations = false;
2020-10-13 14:25:42 +00:00
{
Fix race between DETACH and merges CI reports failure of the 01442_merge_detach_attach test [1]: <details> 2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference: 2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300 2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300 2021-06-21 02:25:43 @@ -0,0 +1 @@ 2021-06-21 02:25:43 +2 2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple() 2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL 2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t 2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t 2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0 2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB. 2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0 2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact 2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts. 2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal 2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec. 2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection. 2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec. 2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1. 2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2 2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB. 2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB. ... 2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0 2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t 2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown 2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges 2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0 2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating 2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key 2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.) 2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data 2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec. 2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. </details> [1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1 The problem here is that there is a tiny race window between DETACH and OPTIMIZE. DETACH cancel merges, and checks that there no merges for parts currently under lock, and later changes the state of parts. After it allows merges. However OPTIMIZE get parts for processing under lock and only when it starts writing temporary part it checks that merges are not canceled. So suppose the following scenario: T#OPTIMIZE T#DETACH - cancel merges - lock - check that no merges are in progress - unlock - lock - get parts *<--Here, state of the parts are not changed yet, and lock is not held, so OPTIMIZE may (and will) get those parts. -->* - write temporary part - check merges not canceled - unlock - change the parts state - allows merges Plus this patch will also fail merges earlier in case of concurrent DETACH. Refs: #13746 Refs: #23315
2021-06-22 23:19:06 +00:00
std::unique_lock lock(currently_processing_in_background_mutex);
if (merger_mutator.merges_blocker.isCancelled())
return false;
merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, shared_lock, lock, txn);
2022-10-12 04:18:17 +00:00
if (!merge_entry && !current_mutations_by_version.empty())
mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, shared_lock, lock);
2021-07-30 20:42:01 +00:00
has_mutations = !current_mutations_by_version.empty();
}
2021-06-21 13:36:21 +00:00
if (merge_entry)
2020-10-13 14:25:42 +00:00
{
auto task = std::make_shared<MergePlainMergeTreeTask>(*this, metadata_snapshot, false, Names{}, merge_entry, shared_lock, common_assignee_trigger);
2022-02-03 18:57:09 +00:00
task->setCurrentTransaction(std::move(transaction_for_merge), std::move(txn));
2022-06-01 19:09:53 +00:00
bool scheduled = assignee.scheduleMergeMutateTask(task);
/// The problem that we already booked a slot for TTL merge, but a merge list entry will be created only in a prepare method
/// in MergePlainMergeTreeTask. So, this slot will never be freed.
if (!scheduled && isTTLMergeType(merge_entry->future_part->merge_type))
getContext()->getMergeList().cancelMergeWithTTL();
return scheduled;
2014-04-11 18:04:21 +00:00
}
2021-06-21 13:36:21 +00:00
if (mutate_entry)
2014-04-11 18:04:21 +00:00
{
auto task = std::make_shared<MutatePlainMergeTreeTask>(*this, metadata_snapshot, mutate_entry, shared_lock, common_assignee_trigger);
assignee.scheduleMergeMutateTask(task);
2021-06-21 13:36:21 +00:00
return true;
2014-04-11 18:04:21 +00:00
}
if (has_mutations)
2014-04-11 18:04:21 +00:00
{
/// Notify in case of errors if no mutation was successfully selected.
/// Otherwise, notification will occur after any of mutations complete.
std::lock_guard lock(mutation_wait_mutex);
mutation_wait_event.notify_all();
2014-04-11 18:04:21 +00:00
}
2021-09-08 00:21:21 +00:00
bool scheduled = false;
2022-03-02 15:23:29 +00:00
if (auto lock = time_after_previous_cleanup_temporary_directories.compareAndRestartDeferred(
2021-11-09 12:26:51 +00:00
getSettings()->merge_tree_clear_old_temporary_directories_interval_seconds))
{
assignee.scheduleCommonTask(std::make_shared<ExecutableLambdaAdapter>(
[this, shared_lock] ()
2021-08-30 19:37:03 +00:00
{
return clearOldTemporaryDirectories(getSettings()->temporary_directories_lifetime.totalSeconds());
2021-11-09 12:26:51 +00:00
}, common_assignee_trigger, getStorageID()), /* need_trigger */ false);
2021-09-08 00:21:21 +00:00
scheduled = true;
}
2021-09-25 05:38:56 +00:00
if (auto lock = time_after_previous_cleanup_parts.compareAndRestartDeferred(
2021-11-09 12:26:51 +00:00
getSettings()->merge_tree_clear_old_parts_interval_seconds))
2014-04-11 18:04:21 +00:00
{
assignee.scheduleCommonTask(std::make_shared<ExecutableLambdaAdapter>(
[this, shared_lock] ()
2021-08-30 19:37:03 +00:00
{
/// All use relative_data_path which changes during rename
/// so execute under share lock.
2021-11-09 12:26:51 +00:00
size_t cleared_count = 0;
cleared_count += clearOldPartsFromFilesystem();
cleared_count += clearOldWriteAheadLogs();
cleared_count += clearOldMutations();
cleared_count += clearEmptyParts();
2022-06-10 11:19:37 +00:00
if (getSettings()->merge_tree_enable_clear_old_broken_detached)
2022-09-05 01:50:24 +00:00
cleared_count += clearOldBrokenPartsFromDetachedDirectory();
2021-11-09 12:26:51 +00:00
return cleared_count;
/// TODO maybe take into account number of cleared objects when calculating backoff
}, common_assignee_trigger, getStorageID()), /* need_trigger */ false);
2021-09-08 00:21:21 +00:00
scheduled = true;
2021-09-25 05:38:56 +00:00
}
2021-09-08 00:21:21 +00:00
return scheduled;
2014-03-13 12:48:07 +00:00
}
2021-12-01 08:18:07 +00:00
UInt64 StorageMergeTree::getCurrentMutationVersion(
2021-03-18 07:38:05 +00:00
const DataPartPtr & part,
std::unique_lock<std::mutex> & /*currently_processing_in_background_mutex_lock*/) const
2021-03-18 07:38:05 +00:00
{
auto it = current_mutations_by_version.upper_bound(part->info.getDataVersion());
if (it == current_mutations_by_version.begin())
return 0;
--it;
return it->first;
}
2021-11-09 12:26:51 +00:00
size_t StorageMergeTree::clearOldMutations(bool truncate)
{
2021-12-14 20:06:34 +00:00
size_t finished_mutations_to_keep = truncate ? 0 : getSettings()->finished_mutations_to_keep;
std::vector<MergeTreeMutationEntry> mutations_to_delete;
{
std::lock_guard<std::mutex> lock(currently_processing_in_background_mutex);
2021-12-14 20:06:34 +00:00
if (current_mutations_by_version.size() <= finished_mutations_to_keep)
2021-11-09 12:26:51 +00:00
return 0;
2019-07-29 09:15:46 +00:00
auto end_it = current_mutations_by_version.end();
auto begin_it = current_mutations_by_version.begin();
2021-12-14 20:06:34 +00:00
if (std::optional<Int64> min_version = getMinPartDataVersion())
end_it = current_mutations_by_version.upper_bound(*min_version);
2021-12-14 20:06:34 +00:00
size_t done_count = std::distance(begin_it, end_it);
2022-08-25 12:22:27 +00:00
if (done_count <= finished_mutations_to_keep)
return 0;
for (auto it = begin_it; it != end_it; ++it)
{
if (!it->second.tid.isPrehistoric())
{
done_count = std::distance(begin_it, it);
break;
}
}
2021-12-14 20:06:34 +00:00
if (done_count <= finished_mutations_to_keep)
return 0;
2022-01-10 17:07:01 +00:00
size_t to_delete_count = done_count - finished_mutations_to_keep;
auto it = begin_it;
for (size_t i = 0; i < to_delete_count; ++i)
{
2021-12-14 20:06:34 +00:00
const auto & tid = it->second.tid;
if (!tid.isPrehistoric() && !TransactionLog::getCSN(tid))
2021-12-22 16:34:02 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove mutation {}, because transaction {} is not committed. It's a bug",
it->first, tid);
mutations_to_delete.push_back(std::move(it->second));
it = current_mutations_by_version.erase(it);
}
}
for (auto & mutation : mutations_to_delete)
{
2020-05-23 22:24:01 +00:00
LOG_TRACE(log, "Removing mutation: {}", mutation.file_name);
mutation.removeFile();
}
2021-11-09 12:26:51 +00:00
return mutations_to_delete.size();
}
bool StorageMergeTree::optimize(
2020-06-17 13:39:26 +00:00
const ASTPtr & /*query*/,
const StorageMetadataPtr & /*metadata_snapshot*/,
const ASTPtr & partition,
bool final,
bool deduplicate,
const Names & deduplicate_by_columns,
ContextPtr local_context)
{
if (deduplicate)
{
if (deduplicate_by_columns.empty())
LOG_DEBUG(log, "DEDUPLICATE BY all columns");
else
LOG_DEBUG(log, "DEDUPLICATE BY ('{}')", fmt::join(deduplicate_by_columns, "', '"));
}
2021-05-17 11:14:09 +00:00
auto txn = local_context->getCurrentTransaction();
String disable_reason;
if (!partition && final)
{
DataPartsVector data_parts = getVisibleDataPartsVector(local_context);
std::unordered_set<String> partition_ids;
2019-05-03 02:00:57 +00:00
for (const DataPartPtr & part : data_parts)
2018-07-05 18:45:18 +00:00
partition_ids.emplace(part->info.partition_id);
for (const String & partition_id : partition_ids)
{
if (!merge(
true,
partition_id,
true,
deduplicate,
deduplicate_by_columns,
2021-05-17 11:14:09 +00:00
txn,
&disable_reason,
local_context->getSettingsRef().optimize_skip_merged_partitions))
{
2020-11-10 18:22:26 +00:00
constexpr const char * message = "Cannot OPTIMIZE table: {}";
if (disable_reason.empty())
disable_reason = "unknown reason";
LOG_INFO(log, fmt::runtime(message), disable_reason);
if (local_context->getSettingsRef().optimize_throw_if_noop)
2020-11-10 18:22:26 +00:00
throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason);
return false;
}
}
}
else
{
String partition_id;
if (partition)
partition_id = getPartitionIDFromQuery(partition, local_context);
if (!merge(
true,
partition_id,
final,
deduplicate,
deduplicate_by_columns,
2021-05-17 11:14:09 +00:00
txn,
&disable_reason,
local_context->getSettingsRef().optimize_skip_merged_partitions))
{
2020-11-10 18:22:26 +00:00
constexpr const char * message = "Cannot OPTIMIZE table: {}";
if (disable_reason.empty())
disable_reason = "unknown reason";
LOG_INFO(log, fmt::runtime(message), disable_reason);
if (local_context->getSettingsRef().optimize_throw_if_noop)
2020-11-10 18:22:26 +00:00
throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason);
return false;
}
}
return true;
}
ActionLock StorageMergeTree::stopMergesAndWait()
{
/// TODO allow to stop merges in specific partition only (like it's done in ReplicatedMergeTree)
Fix race between DETACH and merges CI reports failure of the 01442_merge_detach_attach test [1]: <details> 2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference: 2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300 2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300 2021-06-21 02:25:43 @@ -0,0 +1 @@ 2021-06-21 02:25:43 +2 2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple() 2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL 2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t 2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t 2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0 2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB. 2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0 2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact 2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts. 2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal 2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec. 2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection. 2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec. 2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1. 2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2 2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB. 2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB. ... 2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0 2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t 2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown 2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges 2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0 2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating 2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key 2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.) 2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data 2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec. 2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. </details> [1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1 The problem here is that there is a tiny race window between DETACH and OPTIMIZE. DETACH cancel merges, and checks that there no merges for parts currently under lock, and later changes the state of parts. After it allows merges. However OPTIMIZE get parts for processing under lock and only when it starts writing temporary part it checks that merges are not canceled. So suppose the following scenario: T#OPTIMIZE T#DETACH - cancel merges - lock - check that no merges are in progress - unlock - lock - get parts *<--Here, state of the parts are not changed yet, and lock is not held, so OPTIMIZE may (and will) get those parts. -->* - write temporary part - check merges not canceled - unlock - change the parts state - allows merges Plus this patch will also fail merges earlier in case of concurrent DETACH. Refs: #13746 Refs: #23315
2021-06-22 23:19:06 +00:00
std::unique_lock lock(currently_processing_in_background_mutex);
/// Asks to complete merges and does not allow them to start.
/// This protects against "revival" of data for a removed partition after completion of merge.
auto merge_blocker = merger_mutator.merges_blocker.cancel();
Fix race between DETACH and merges CI reports failure of the 01442_merge_detach_attach test [1]: <details> 2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference: 2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300 2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300 2021-06-21 02:25:43 @@ -0,0 +1 @@ 2021-06-21 02:25:43 +2 2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple() 2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL 2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t 2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t 2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0 2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB. 2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0 2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact 2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts. 2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal 2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec. 2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection. 2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec. 2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1. 2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2 2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB. 2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB. ... 2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0 2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t 2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown 2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges 2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0 2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating 2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key 2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.) 2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data 2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec. 2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. </details> [1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1 The problem here is that there is a tiny race window between DETACH and OPTIMIZE. DETACH cancel merges, and checks that there no merges for parts currently under lock, and later changes the state of parts. After it allows merges. However OPTIMIZE get parts for processing under lock and only when it starts writing temporary part it checks that merges are not canceled. So suppose the following scenario: T#OPTIMIZE T#DETACH - cancel merges - lock - check that no merges are in progress - unlock - lock - get parts *<--Here, state of the parts are not changed yet, and lock is not held, so OPTIMIZE may (and will) get those parts. -->* - write temporary part - check merges not canceled - unlock - change the parts state - allows merges Plus this patch will also fail merges earlier in case of concurrent DETACH. Refs: #13746 Refs: #23315
2021-06-22 23:19:06 +00:00
while (!currently_merging_mutating_parts.empty())
{
Fix race between DETACH and merges CI reports failure of the 01442_merge_detach_attach test [1]: <details> 2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference: 2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300 2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300 2021-06-21 02:25:43 @@ -0,0 +1 @@ 2021-06-21 02:25:43 +2 2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple() 2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL 2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t 2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t 2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0 2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB. 2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0 2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact 2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts. 2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal 2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec. 2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection. 2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec. 2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1. 2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2 2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB. 2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB. ... 2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0 2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t 2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown 2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges 2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0 2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating 2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key 2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.) 2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data 2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec. 2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. </details> [1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1 The problem here is that there is a tiny race window between DETACH and OPTIMIZE. DETACH cancel merges, and checks that there no merges for parts currently under lock, and later changes the state of parts. After it allows merges. However OPTIMIZE get parts for processing under lock and only when it starts writing temporary part it checks that merges are not canceled. So suppose the following scenario: T#OPTIMIZE T#DETACH - cancel merges - lock - check that no merges are in progress - unlock - lock - get parts *<--Here, state of the parts are not changed yet, and lock is not held, so OPTIMIZE may (and will) get those parts. -->* - write temporary part - check merges not canceled - unlock - change the parts state - allows merges Plus this patch will also fail merges earlier in case of concurrent DETACH. Refs: #13746 Refs: #23315
2021-06-22 23:19:06 +00:00
LOG_DEBUG(log, "Waiting for currently running merges ({} parts are merging right now)",
currently_merging_mutating_parts.size());
Fix race between DETACH and merges CI reports failure of the 01442_merge_detach_attach test [1]: <details> 2021-06-21 02:25:43 01442_merge_detach_attach: [ FAIL ] 122.37 sec. - result differs with reference: 2021-06-21 02:25:43 --- /usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.reference 2021-06-21 00:43:12.000000000 +0300 2021-06-21 02:25:43 +++ /tmp/clickhouse-test/0_stateless/01442_merge_detach_attach.stdout 2021-06-21 02:25:43.211212197 +0300 2021-06-21 02:25:43 @@ -0,0 +1 @@ 2021-06-21 02:25:43 +2 2021.06.21 02:25:08.930896 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> executeQuery: (from [::1]:36540, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') ALTER TABLE t DETACH PARTITION tuple() 2021.06.21 02:25:08.931245 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> executeQuery: (from [::1]:36542, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') OPTIMIZE TABLE t FINAL 2021.06.21 02:25:08.931826 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Trace> ContextAccess (default): Access granted: ALTER DELETE ON test_89nl0v.t 2021.06.21 02:25:08.932159 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> ContextAccess (default): Access granted: OPTIMIZE ON test_89nl0v.t 2021.06.21 02:25:08.932889 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_143_143_0 2021.06.21 02:25:08.932921 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected 2 parts from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.933530 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> DiskLocal: Reserving 1.00 MiB on disk `default`, having unreserved 4.60 TiB. 2021.06.21 02:25:08.933705 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detaching all_144_144_0 2021.06.21 02:25:08.934215 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merging 2 parts: from all_143_143_0 to all_144_144_0 into Compact 2021.06.21 02:25:08.934280 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Information> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Detached 2 parts. 2021.06.21 02:25:08.934948 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Selected MergeAlgorithm: Horizontal 2021.06.21 02:25:08.936090 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_143_143_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.937621 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MergeTreeSequentialSource: Reading 2 marks from part all_144_144_0, total 1 rows starting from the beginning of the part, column x 2021.06.21 02:25:08.938124 [ 100543 ] {16eb4fe5-2d6b-4c81-a6be-c6a3b293edd3} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.06.21 02:25:08.939928 [ 100543 ] {} <Debug> TCPHandler: Processed in 0.018537432 sec. 2021.06.21 02:25:08.942140 [ 100543 ] {} <Debug> TCPHandler: Done processing connection. 2021.06.21 02:25:08.948343 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merge sorted 2 rows, containing 1 columns (1 merged, 0 gathered) in 0.014203821 sec., 140.80718139154246 rows/sec., 140.81 B/sec. 2021.06.21 02:25:08.952021 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249): Renaming temporary part tmp_merge_all_143_144_1 to all_143_144_1. 2021.06.21 02:25:08.952869 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Warning> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Unexpected number of parts removed when adding all_143_144_1: 0 instead of 2 2021.06.21 02:25:08.953264 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Trace> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (MergerMutator): Merged 2 parts: from all_143_143_0 to all_144_144_0 2021.06.21 02:25:08.953913 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage: 4.01 MiB. 2021.06.21 02:25:08.958369 [ 100540 ] {bd4b710d-cd27-44ea-9dc9-e45d6ace0bd7} <Debug> MemoryTracker: Peak memory usage (for query): 4.01 MiB. ... 2021.06.21 02:25:09.216075 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> executeQuery: (from [::1]:36544, using production parser) (comment: '/usr/share/clickhouse-test/queries/0_stateless/01442_merge_detach_attach.sh') SELECT count() FROM t HAVING count() > 0 2021.06.21 02:25:09.229491 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> ContextAccess (default): Access granted: SELECT(x) ON test_89nl0v.t 2021.06.21 02:25:09.232000 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> InterpreterSelectQuery: FetchColumns -> Complete 2021.06.21 02:25:09.239907 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Key condition: unknown 2021.06.21 02:25:09.240358 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> test_89nl0v.t (efd364af-6721-4249-afd3-64af67214249) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 1/1 marks by primary key, 1 marks to read from 1 ranges 2021.06.21 02:25:09.241560 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MergeTreeSelectProcessor: Reading 1 ranges from part all_143_144_1, approx. 2 rows starting from 0 2021.06.21 02:25:09.256053 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> AggregatingTransform: Aggregating 2021.06.21 02:25:09.256410 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Aggregation method: without_key 2021.06.21 02:25:09.257576 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> AggregatingTransform: Aggregated. 2 to 1 rows (from 2.00 B) in 0.013910412 sec. (143.777 rows/sec., 143.78 B/sec.) 2021.06.21 02:25:09.257911 [ 58403 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Trace> Aggregator: Merging aggregated data 2021.06.21 02:25:09.262595 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Information> executeQuery: Read 2 rows, 2.00 B in 0.045805192 sec., 43 rows/sec., 43.66 B/sec. 2021.06.21 02:25:09.263337 [ 100543 ] {1699b404-b132-4b96-b04a-1c28cd062a79} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B. </details> [1]: https://clickhouse-test-reports.s3.yandex.net/25513/b96df135aee40b1a54a4fc3f0d4db89e11385564/functional_stateless_tests_(memory).html#fail1 The problem here is that there is a tiny race window between DETACH and OPTIMIZE. DETACH cancel merges, and checks that there no merges for parts currently under lock, and later changes the state of parts. After it allows merges. However OPTIMIZE get parts for processing under lock and only when it starts writing temporary part it checks that merges are not canceled. So suppose the following scenario: T#OPTIMIZE T#DETACH - cancel merges - lock - check that no merges are in progress - unlock - lock - get parts *<--Here, state of the parts are not changed yet, and lock is not held, so OPTIMIZE may (and will) get those parts. -->* - write temporary part - check merges not canceled - unlock - change the parts state - allows merges Plus this patch will also fail merges earlier in case of concurrent DETACH. Refs: #13746 Refs: #23315
2021-06-22 23:19:06 +00:00
if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for(
lock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC)))
{
throw Exception("Timeout while waiting for already running merges", ErrorCodes::TIMEOUT_EXCEEDED);
}
}
return merge_blocker;
}
2021-06-08 18:17:18 +00:00
MergeTreeDataPartPtr StorageMergeTree::outdatePart(MergeTreeTransaction * txn, const String & part_name, bool force)
2014-10-03 17:57:01 +00:00
{
if (force)
2014-10-03 17:57:01 +00:00
{
/// Forcefully stop merges and make part outdated
auto merge_blocker = stopMergesAndWait();
2022-12-13 19:57:15 +00:00
auto parts_lock = lockParts();
2022-12-22 13:31:42 +00:00
auto part = getPartIfExistsUnlocked(part_name, {MergeTreeDataPartState::Active}, parts_lock);
if (!part)
2022-11-18 15:33:43 +00:00
throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Part {} not found, won't try to drop it.", part_name);
2022-12-13 19:57:15 +00:00
removePartsFromWorkingSet(txn, {part}, true, &parts_lock);
return part;
}
else
{
/// Wait merges selector
std::unique_lock lock(currently_processing_in_background_mutex);
2022-12-13 19:57:15 +00:00
auto parts_lock = lockParts();
2022-12-22 13:31:42 +00:00
auto part = getPartIfExistsUnlocked(part_name, {MergeTreeDataPartState::Active}, parts_lock);
/// It's okay, part was already removed
if (!part)
return nullptr;
2014-10-03 17:57:01 +00:00
/// Part will be "removed" by merge or mutation, it's OK in case of some
/// background cleanup processes like removing of empty parts.
if (currently_merging_mutating_parts.contains(part))
return nullptr;
2022-12-13 19:57:15 +00:00
removePartsFromWorkingSet(txn, {part}, true, &parts_lock);
return part;
}
}
2021-05-17 14:26:36 +00:00
void StorageMergeTree::dropPartNoWaitNoThrow(const String & part_name)
{
2022-03-18 11:01:26 +00:00
if (auto part = outdatePart(NO_TRANSACTION_RAW, part_name, /*force=*/ false))
{
if (deduplication_log)
{
deduplication_log->dropPart(part->info);
}
/// Need to destroy part objects before clearing them from filesystem.
part.reset();
clearOldPartsFromFilesystem();
2022-09-22 22:51:13 +00:00
LOG_INFO(log, "Removed 1 part {}.", part_name);
}
2021-04-20 02:31:08 +00:00
2021-05-17 14:26:36 +00:00
/// Else nothing to do, part was removed in some different way
2021-04-20 02:31:08 +00:00
}
2022-09-22 22:51:13 +00:00
struct FutureNewEmptyPart
2021-04-20 02:31:08 +00:00
{
2022-09-22 22:51:13 +00:00
MergeTreePartInfo part_info;
MergeTreePartition partition;
std::string part_name;
scope_guard tmp_dir_guard;
2022-09-22 22:51:13 +00:00
StorageMergeTree::MutableDataPartPtr data_part;
std::string getDirName() const { return StorageMergeTree::EMPTY_PART_TMP_PREFIX + part_name; }
};
2022-09-22 22:51:13 +00:00
using FutureNewEmptyParts = std::vector<FutureNewEmptyPart>;
2022-09-22 22:51:13 +00:00
Strings getPartsNames(const FutureNewEmptyParts & parts)
{
Strings part_names;
for (const auto & p : parts)
part_names.push_back(p.part_name);
return part_names;
2021-04-20 02:31:08 +00:00
}
2014-10-03 17:57:01 +00:00
FutureNewEmptyParts initCoverageWithNewEmptyParts(const DataPartsVector & old_parts)
2021-04-20 02:31:08 +00:00
{
FutureNewEmptyParts future_parts;
for (const auto & old_part : old_parts)
2021-04-20 02:31:08 +00:00
{
future_parts.emplace_back();
auto & new_part = future_parts.back();
new_part.part_info = old_part->info;
new_part.part_info.level += 1;
new_part.partition = old_part->partition;
new_part.part_name = old_part->getNewName(new_part.part_info);
}
return future_parts;
2022-09-22 22:51:13 +00:00
}
StorageMergeTree::MutableDataPartsVector createEmptyDataParts(MergeTreeData & data, FutureNewEmptyParts & future_parts, const MergeTreeTransactionPtr & txn)
2022-09-22 22:51:13 +00:00
{
StorageMergeTree::MutableDataPartsVector data_parts;
for (auto & part: future_parts)
data_parts.push_back(data.createEmptyPart(part.part_info, part.partition, part.part_name, txn));
2022-09-22 22:51:13 +00:00
return data_parts;
}
void captureTmpDirectoryHolders(MergeTreeData & data, FutureNewEmptyParts & future_parts)
2022-09-22 22:51:13 +00:00
{
for (auto & part : future_parts)
part.tmp_dir_guard = data.getTemporaryPartDirectoryHolder(part.getDirName());
2022-09-22 22:51:13 +00:00
}
void StorageMergeTree::renameAndCommitEmptyParts(MutableDataPartsVector & new_parts, Transaction & transaction)
{
DataPartsVector covered_parts;
2022-09-22 22:51:13 +00:00
for (auto & part: new_parts)
{
DataPartsVector covered_parts_by_one_part = renameTempPartAndReplace(part, transaction);
if (covered_parts_by_one_part.size() > 1)
2022-11-18 15:33:43 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} expected to cover not more then 1 part. {} covered parts have been found. This is a bug.",
part->name, covered_parts_by_one_part.size());
std::move(covered_parts_by_one_part.begin(), covered_parts_by_one_part.end(), std::back_inserter(covered_parts));
}
LOG_INFO(log, "Remove {} parts by covering them with empty {} parts. With txn {}.",
covered_parts.size(), new_parts.size(), transaction.getTID());
transaction.commit();
2022-09-22 22:51:13 +00:00
/// Remove covered parts without waiting for old_parts_lifetime seconds.
for (auto & part: covered_parts)
part->remove_time.store(0, std::memory_order_relaxed);
2022-09-22 22:51:13 +00:00
if (deduplication_log)
for (const auto & part : covered_parts)
deduplication_log->dropPart(part->info);
}
void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr query_context, TableExclusiveLockHolder &)
{
/// Asks to complete merges and does not allow them to start.
/// This protects against "revival" of data for a removed partition after completion of merge.
auto merge_blocker = stopMergesAndWait();
Stopwatch watch;
auto txn = query_context->getCurrentTransaction();
MergeTreeData::Transaction transaction(*this, txn.get());
{
auto operation_data_parts_lock = lockOperationsWithParts();
2022-09-22 22:51:13 +00:00
auto parts = getVisibleDataPartsVector(query_context);
auto future_parts = initCoverageWithNewEmptyParts(parts);
LOG_TEST(log, "Made {} empty parts in order to cover {} parts. Empty parts: {}, covered parts: {}. With txn {}",
future_parts.size(), parts.size(),
fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNames(parts), ", "),
transaction.getTID());
captureTmpDirectoryHolders(*this, future_parts);
auto new_data_parts = createEmptyDataParts(*this, future_parts, txn);
renameAndCommitEmptyParts(new_data_parts, transaction);
2022-09-22 22:51:13 +00:00
PartLog::addNewParts(query_context, new_data_parts, watch.elapsed());
LOG_INFO(log, "Truncated table with {} parts by replacing them with new empty {} parts. With txn {}",
parts.size(), future_parts.size(),
transaction.getTID());
}
/// Old parts are needed to be destroyed before clearing them from filesystem.
clearOldMutations(true);
clearOldPartsFromFilesystem();
2022-09-22 22:51:13 +00:00
clearEmptyParts();
}
void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPtr query_context)
{
/// Asks to complete merges and does not allow them to start.
/// This protects against "revival" of data for a removed partition after completion of merge.
auto merge_blocker = stopMergesAndWait();
Stopwatch watch;
/// It's important to create it outside of lock scope because
/// otherwise it can lock parts in destructor and deadlock is possible.
auto txn = query_context->getCurrentTransaction();
MergeTreeData::Transaction transaction(*this, txn.get());
{
auto operation_data_parts_lock = lockOperationsWithParts();
auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active});
if (!part)
2022-11-18 15:33:43 +00:00
throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Part {} not found, won't try to drop it.", part_name);
if (detach)
2021-04-02 11:46:42 +00:00
{
auto metadata_snapshot = getInMemoryMetadataPtr();
LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory());
2021-04-20 02:31:08 +00:00
part->makeCloneInDetached("", metadata_snapshot);
2021-04-02 11:46:42 +00:00
}
2022-09-14 13:04:24 +00:00
{
auto future_parts = initCoverageWithNewEmptyParts({part});
LOG_TEST(log, "Made {} empty parts in order to cover {} part. With txn {}",
fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNames({part}), ", "),
transaction.getTID());
2022-09-22 22:51:13 +00:00
captureTmpDirectoryHolders(*this, future_parts);
auto new_data_parts = createEmptyDataParts(*this, future_parts, txn);
renameAndCommitEmptyParts(new_data_parts, transaction);
2022-09-22 22:51:13 +00:00
PartLog::addNewParts(query_context, new_data_parts, watch.elapsed());
2022-09-16 12:47:16 +00:00
const auto * op = detach ? "Detached" : "Dropped";
LOG_INFO(log, "{} {} part by replacing it with new empty {} part. With txn {}",
op, part->name, future_parts[0].part_name,
transaction.getTID());
2022-09-14 13:04:24 +00:00
}
2021-04-20 02:31:08 +00:00
}
2021-04-02 11:46:42 +00:00
/// Old part objects is needed to be destroyed before clearing them from filesystem.
clearOldMutations(true);
clearOldPartsFromFilesystem();
2022-09-22 22:51:13 +00:00
clearEmptyParts();
}
2022-09-16 12:47:16 +00:00
void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, ContextPtr query_context)
{
2022-09-16 12:47:16 +00:00
const auto * partition_ast = partition->as<ASTPartition>();
/// Asks to complete merges and does not allow them to start.
/// This protects against "revival" of data for a removed partition after completion of merge.
auto merge_blocker = stopMergesAndWait();
Stopwatch watch;
/// It's important to create it outside of lock scope because
/// otherwise it can lock parts in destructor and deadlock is possible.
auto txn = query_context->getCurrentTransaction();
MergeTreeData::Transaction transaction(*this, txn.get());
2021-04-20 02:31:08 +00:00
{
auto operation_data_parts_lock = lockOperationsWithParts();
DataPartsVector parts;
{
2022-09-14 13:04:24 +00:00
if (partition_ast && partition_ast->all)
parts = getVisibleDataPartsVector(query_context);
else
{
2022-09-16 12:47:16 +00:00
String partition_id = getPartitionIDFromQuery(partition, query_context);
2022-09-14 13:04:24 +00:00
parts = getVisibleDataPartsVectorInPartition(query_context, partition_id);
}
}
2014-10-03 17:57:01 +00:00
if (detach)
for (const auto & part : parts)
{
auto metadata_snapshot = getInMemoryMetadataPtr();
LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory());
part->makeCloneInDetached("", metadata_snapshot);
}
auto future_parts = initCoverageWithNewEmptyParts(parts);
LOG_TEST(log, "Made {} empty parts in order to cover {} parts. Empty parts: {}, covered parts: {}. With txn {}",
future_parts.size(), parts.size(),
fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNames(parts), ", "),
transaction.getTID());
2021-04-20 02:31:08 +00:00
captureTmpDirectoryHolders(*this, future_parts);
auto new_data_parts = createEmptyDataParts(*this, future_parts, txn);
renameAndCommitEmptyParts(new_data_parts, transaction);
2022-09-22 22:51:13 +00:00
PartLog::addNewParts(query_context, new_data_parts, watch.elapsed());
2022-09-16 12:47:16 +00:00
const auto * op = detach ? "Detached" : "Dropped";
LOG_INFO(log, "{} partition with {} parts by replacing them with new empty {} parts. With txn {}",
op, parts.size(), future_parts.size(),
transaction.getTID());
}
/// Old parts are needed to be destroyed before clearing them from filesystem.
clearOldMutations(true);
2019-05-03 02:00:57 +00:00
clearOldPartsFromFilesystem();
2022-09-22 22:51:13 +00:00
clearEmptyParts();
2014-10-03 17:57:01 +00:00
}
PartitionCommandsResultInfo StorageMergeTree::attachPartition(
const ASTPtr & partition, const StorageMetadataPtr & /* metadata_snapshot */,
bool attach_part, ContextPtr local_context)
2014-10-03 18:41:16 +00:00
{
2020-07-28 15:10:36 +00:00
PartitionCommandsResultInfo results;
2019-08-29 16:17:47 +00:00
PartsTemporaryRename renamed_parts(*this, "detached/");
MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, local_context, renamed_parts);
2019-07-30 17:24:40 +00:00
for (size_t i = 0; i < loaded_parts.size(); ++i)
2014-10-03 18:41:16 +00:00
{
2021-11-24 19:45:10 +00:00
LOG_INFO(log, "Attaching part {} from {}", loaded_parts[i]->name, renamed_parts.old_and_new_names[i].new_name);
2022-02-15 15:00:45 +00:00
/// We should write version metadata on part creation to distinguish it from parts that were created without transaction.
auto txn = local_context->getCurrentTransaction();
TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID;
loaded_parts[i]->version.setCreationTID(tid, nullptr);
loaded_parts[i]->storeVersionMetadata();
2021-11-24 19:45:10 +00:00
String old_name = renamed_parts.old_and_new_names[i].old_name;
/// 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(*this, local_context->getCurrentTransaction().get());
2022-06-24 15:19:59 +00:00
{
auto lock = lockParts();
fillNewPartName(loaded_parts[i], lock);
2022-10-22 22:51:59 +00:00
renameTempPartAndAdd(loaded_parts[i], transaction, lock);
2022-06-24 15:19:59 +00:00
transaction.commit(&lock);
}
2022-06-24 11:19:29 +00:00
2021-11-24 19:45:10 +00:00
renamed_parts.old_and_new_names[i].old_name.clear();
2020-07-28 15:10:36 +00:00
results.push_back(PartitionCommandResultInfo{
.partition_id = loaded_parts[i]->info.partition_id,
.part_name = loaded_parts[i]->name,
.old_part_name = old_name,
});
2015-06-02 20:22:53 +00:00
LOG_INFO(log, "Finished attaching part");
2014-10-03 18:41:16 +00:00
}
2017-04-16 15:00:33 +00:00
/// New parts with other data may appear in place of deleted parts.
local_context->dropCaches();
2020-07-28 15:10:36 +00:00
return results;
2014-10-03 18:41:16 +00:00
}
void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr local_context)
{
auto lock1 = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout);
auto lock2 = source_table->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout);
auto merges_blocker = stopMergesAndWait();
auto source_metadata_snapshot = source_table->getInMemoryMetadataPtr();
auto my_metadata_snapshot = getInMemoryMetadataPtr();
Stopwatch watch;
MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot);
String partition_id = getPartitionIDFromQuery(partition, local_context);
2021-11-17 18:14:14 +00:00
DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id);
2019-05-03 02:00:57 +00:00
MutableDataPartsVector dst_parts;
2022-08-09 16:44:51 +00:00
std::vector<scope_guard> dst_parts_locks;
static const String TMP_PREFIX = "tmp_replace_from_";
2019-05-03 02:00:57 +00:00
for (const DataPartPtr & src_part : src_parts)
{
2019-06-19 16:16:13 +00:00
if (!canReplacePartition(src_part))
throw Exception(
"Cannot replace partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
ErrorCodes::BAD_ARGUMENTS);
2019-06-19 16:16:13 +00:00
/// This will generate unique name in scope of current server process.
2019-05-03 02:00:57 +00:00
Int64 temp_index = insert_increment.get();
MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level);
2022-09-27 13:23:02 +00:00
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, my_metadata_snapshot, local_context->getCurrentTransaction(), {}, false, {});
2022-02-14 19:50:08 +00:00
dst_parts.emplace_back(std::move(dst_part));
2022-08-09 16:44:51 +00:00
dst_parts_locks.emplace_back(std::move(part_lock));
}
/// ATTACH empty part set
if (!replace && dst_parts.empty())
return;
MergeTreePartInfo drop_range;
if (replace)
{
drop_range.partition_id = partition_id;
drop_range.min_block = 0;
drop_range.max_block = increment.get(); // there will be a "hole" in block numbers
drop_range.level = std::numeric_limits<decltype(drop_range.level)>::max();
}
/// Atomically add new parts and remove old ones
try
{
{
/// Here we use the transaction just like RAII since rare errors in renameTempPartAndReplace() are possible
/// and we should be able to rollback already added (Precomitted) parts
2021-06-04 09:26:47 +00:00
Transaction transaction(*this, local_context->getCurrentTransaction().get());
2019-05-03 02:00:57 +00:00
auto data_parts_lock = lockParts();
2022-06-24 15:19:59 +00:00
/** It is important that obtaining new block number and adding that block to parts set is done atomically.
* Otherwise there is race condition - merge of blocks could happen in interval that doesn't yet contain new part.
*/
for (auto part : dst_parts)
{
fillNewPartName(part, data_parts_lock);
2022-10-22 22:51:59 +00:00
renameTempPartAndReplaceUnlocked(part, transaction, data_parts_lock);
2022-06-24 15:19:59 +00:00
}
/// Populate transaction
transaction.commit(&data_parts_lock);
/// If it is REPLACE (not ATTACH), remove all parts which max_block_number less then min_block_number of the first new block
if (replace)
removePartsInRangeFromWorkingSet(local_context->getCurrentTransaction().get(), drop_range, data_parts_lock);
}
PartLog::addNewParts(getContext(), dst_parts, watch.elapsed());
}
catch (...)
{
PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException());
throw;
}
}
void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr local_context)
{
auto lock1 = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout);
auto lock2 = dest_table->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout);
auto merges_blocker = stopMergesAndWait();
2019-09-17 09:00:20 +00:00
auto dest_table_storage = std::dynamic_pointer_cast<StorageMergeTree>(dest_table);
if (!dest_table_storage)
throw Exception("Table " + getStorageID().getNameForLogs() + " supports movePartitionToTable only for MergeTree family of table engines."
" Got " + dest_table->getName(), ErrorCodes::NOT_IMPLEMENTED);
if (dest_table_storage->getStoragePolicy() != this->getStoragePolicy())
throw Exception("Destination table " + dest_table_storage->getStorageID().getNameForLogs() +
" should have the same storage policy of source table " + getStorageID().getNameForLogs() + ". " +
getStorageID().getNameForLogs() + ": " + this->getStoragePolicy()->getName() + ", " +
2020-10-16 11:58:47 +00:00
dest_table_storage->getStorageID().getNameForLogs() + ": " + dest_table_storage->getStoragePolicy()->getName(), ErrorCodes::UNKNOWN_POLICY);
auto dest_metadata_snapshot = dest_table->getInMemoryMetadataPtr();
auto metadata_snapshot = getInMemoryMetadataPtr();
Stopwatch watch;
MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this, metadata_snapshot, dest_metadata_snapshot);
String partition_id = getPartitionIDFromQuery(partition, local_context);
2021-11-17 18:14:14 +00:00
DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id);
MutableDataPartsVector dst_parts;
2022-08-09 16:44:51 +00:00
std::vector<scope_guard> dst_parts_locks;
static const String TMP_PREFIX = "tmp_move_from_";
for (const DataPartPtr & src_part : src_parts)
{
if (!dest_table_storage->canReplacePartition(src_part))
throw Exception(
"Cannot move partition '" + partition_id + "' because part '" + src_part->name + "' has inconsistent granularity with table",
ErrorCodes::LOGICAL_ERROR);
/// This will generate unique name in scope of current server process.
Int64 temp_index = insert_increment.get();
MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level);
2022-09-27 13:23:02 +00:00
auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, local_context->getCurrentTransaction(), {}, false, {});
2022-02-14 19:50:08 +00:00
dst_parts.emplace_back(std::move(dst_part));
2022-08-09 16:44:51 +00:00
dst_parts_locks.emplace_back(std::move(part_lock));
}
/// empty part set
if (dst_parts.empty())
return;
/// Move new parts to the destination table. NOTE It doesn't look atomic.
try
{
{
2021-06-04 09:26:47 +00:00
Transaction transaction(*dest_table_storage, local_context->getCurrentTransaction().get());
auto src_data_parts_lock = lockParts();
auto dest_data_parts_lock = dest_table_storage->lockParts();
2022-06-24 15:19:59 +00:00
for (auto & part : dst_parts)
{
dest_table_storage->fillNewPartName(part, dest_data_parts_lock);
2022-10-22 22:51:59 +00:00
dest_table_storage->renameTempPartAndReplaceUnlocked(part, transaction, dest_data_parts_lock);
2022-06-24 15:19:59 +00:00
}
2019-10-22 07:15:58 +00:00
2022-06-24 15:19:59 +00:00
removePartsFromWorkingSet(local_context->getCurrentTransaction().get(), src_parts, true, src_data_parts_lock);
transaction.commit(&src_data_parts_lock);
}
2019-09-16 08:56:30 +00:00
clearOldPartsFromFilesystem();
PartLog::addNewParts(getContext(), dst_parts, watch.elapsed());
}
catch (...)
{
PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException());
throw;
}
}
2019-07-25 10:46:07 +00:00
ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type)
{
if (action_type == ActionLocks::PartsMerge)
2019-08-01 15:36:12 +00:00
return merger_mutator.merges_blocker.cancel();
else if (action_type == ActionLocks::PartsTTLMerge)
2019-09-03 14:50:49 +00:00
return merger_mutator.ttl_merges_blocker.cancel();
else if (action_type == ActionLocks::PartsMove)
return parts_mover.moves_blocker.cancel();
return {};
}
2020-10-15 16:10:22 +00:00
void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type)
{
if (action_type == ActionLocks::PartsMerge || action_type == ActionLocks::PartsTTLMerge)
2021-09-08 00:21:21 +00:00
background_operations_assignee.trigger();
2020-10-15 16:10:22 +00:00
else if (action_type == ActionLocks::PartsMove)
2021-09-08 00:21:21 +00:00
background_moves_assignee.trigger();
2020-10-15 16:10:22 +00:00
}
CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context)
2019-07-03 08:49:52 +00:00
{
2019-07-03 13:17:19 +00:00
CheckResults results;
DataPartsVector data_parts;
if (const auto & check_query = query->as<ASTCheckQuery &>(); check_query.partition)
{
String partition_id = getPartitionIDFromQuery(check_query.partition, local_context);
2021-11-17 18:14:14 +00:00
data_parts = getVisibleDataPartsVectorInPartition(local_context, partition_id);
2019-07-03 13:17:19 +00:00
}
else
data_parts = getVisibleDataPartsVector(local_context);
2019-07-03 13:17:19 +00:00
2019-07-03 08:49:52 +00:00
for (auto & part : data_parts)
2019-07-03 13:17:19 +00:00
{
/// If the checksums file is not present, calculate the checksums and write them to disk.
2022-10-23 22:29:24 +00:00
static constexpr auto checksums_path = "checksums.txt";
if (part->isStoredOnDisk() && !part->getDataPartStorage().exists(checksums_path))
2019-07-03 13:17:19 +00:00
{
try
{
2020-01-13 14:53:32 +00:00
auto calculated_checksums = checkDataPart(part, false);
2019-07-09 09:02:52 +00:00
calculated_checksums.checkEqual(part->checksums, true);
2022-04-21 19:19:13 +00:00
2022-10-23 22:29:24 +00:00
auto & part_mutable = const_cast<IMergeTreeDataPart &>(*part);
part_mutable.writeChecksums(part->checksums, local_context->getWriteSettings());
2022-01-07 10:37:08 +00:00
part->checkMetadata();
2019-07-03 20:51:13 +00:00
results.emplace_back(part->name, true, "Checksums recounted and written to disk.");
2019-07-03 13:17:19 +00:00
}
2019-07-09 09:02:52 +00:00
catch (const Exception & ex)
2019-07-03 13:17:19 +00:00
{
results.emplace_back(part->name, false,
2019-07-03 20:51:13 +00:00
"Check of part finished with error: '" + ex.message() + "'");
2019-07-03 13:17:19 +00:00
}
}
else
{
try
{
2020-01-13 14:53:32 +00:00
checkDataPart(part, true);
2022-01-07 10:37:08 +00:00
part->checkMetadata();
2019-07-03 13:17:19 +00:00
results.emplace_back(part->name, true, "");
}
2019-07-09 09:02:52 +00:00
catch (const Exception & ex)
2019-07-03 13:17:19 +00:00
{
results.emplace_back(part->name, false, ex.message());
}
}
}
return results;
2019-07-03 08:49:52 +00:00
}
2020-04-02 18:24:11 +00:00
2022-07-05 07:39:52 +00:00
void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
{
auto local_context = backup_entries_collector.getContext();
DataPartsVector data_parts;
if (partitions)
data_parts = getVisibleDataPartsVectorInPartitions(local_context, getPartitionIDsFromQuery(*partitions, local_context));
else
data_parts = getVisibleDataPartsVector(local_context);
Int64 min_data_version = std::numeric_limits<Int64>::max();
for (const auto & data_part : data_parts)
min_data_version = std::min(min_data_version, data_part->info.getDataVersion());
backup_entries_collector.addBackupEntries(backupParts(data_parts, data_path_in_backup, local_context));
2022-07-05 07:39:52 +00:00
backup_entries_collector.addBackupEntries(backupMutations(min_data_version + 1, data_path_in_backup));
}
BackupEntries StorageMergeTree::backupMutations(UInt64 version, const String & data_path_in_backup) const
{
2022-07-06 10:03:10 +00:00
fs::path mutations_path_in_backup = fs::path{data_path_in_backup} / "mutations";
2022-07-05 07:39:52 +00:00
BackupEntries backup_entries;
for (auto it = current_mutations_by_version.lower_bound(version); it != current_mutations_by_version.end(); ++it)
backup_entries.emplace_back(mutations_path_in_backup / fmt::format("{:010}.txt", it->first), it->second.backup());
return backup_entries;
}
void StorageMergeTree::attachRestoredParts(MutableDataPartsVector && parts)
2021-08-18 22:19:14 +00:00
{
for (auto part : parts)
2022-06-24 11:19:29 +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-06-24 11:19:29 +00:00
MergeTreeData::Transaction transaction(*this, NO_TRANSACTION_RAW);
{
auto lock = lockParts();
fillNewPartName(part, lock);
2022-10-22 22:51:59 +00:00
renameTempPartAndAdd(part, transaction, lock);
transaction.commit(&lock);
}
2022-06-24 11:19:29 +00:00
}
2021-08-18 22:19:14 +00:00
}
2020-11-28 08:17:20 +00:00
MutationCommands StorageMergeTree::getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const
2020-04-02 18:24:11 +00:00
{
std::lock_guard lock(currently_processing_in_background_mutex);
2020-04-02 18:24:11 +00:00
auto it = current_mutations_by_version.upper_bound(part->info.getDataVersion());
2020-04-02 18:24:11 +00:00
if (it == current_mutations_by_version.end())
return {};
return it->second.commands;
}
2020-10-14 07:22:48 +00:00
void StorageMergeTree::startBackgroundMovesIfNeeded()
{
2020-10-20 11:27:50 +00:00
if (areBackgroundMovesNeeded())
2021-09-08 00:21:21 +00:00
background_moves_assignee.start();
2020-10-14 07:22:48 +00:00
}
2021-07-05 12:44:58 +00:00
std::unique_ptr<MergeTreeSettings> StorageMergeTree::getDefaultSettings() const
{
return std::make_unique<MergeTreeSettings>(getContext()->getMergeTreeSettings());
}
2022-06-24 15:19:59 +00:00
void StorageMergeTree::fillNewPartName(MutableDataPartPtr & part, DataPartsLock &)
{
part->info.min_block = part->info.max_block = increment.get();
part->info.mutation = 0;
part->name = part->getNewName(part->info);
}
2012-07-17 20:04:39 +00:00
}