mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Make setting global
This commit is contained in:
parent
4c30857759
commit
ca3fe49a2a
@ -396,6 +396,7 @@ class IColumn;
|
||||
M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \
|
||||
M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \
|
||||
M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \
|
||||
M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \
|
||||
\
|
||||
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
|
||||
\
|
||||
|
@ -365,7 +365,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
String * out_disable_reason)
|
||||
String * out_disable_reason,
|
||||
const Context * context)
|
||||
{
|
||||
MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id);
|
||||
|
||||
@ -381,7 +382,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti
|
||||
|
||||
/// If final, optimize_skip_merged_partitions is true and we have only one part in partition with level > 0
|
||||
/// than we don't select it to merge. But if there are some expired TTL then merge is needed
|
||||
if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 &&
|
||||
if (final && context && context->getSettingsRef().optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 &&
|
||||
(!metadata_snapshot->hasAnyTTL() || parts[0]->checkAllTTLCalculated(metadata_snapshot)))
|
||||
{
|
||||
return SelectPartsDecision::NOTHING_TO_MERGE;
|
||||
|
@ -105,7 +105,8 @@ public:
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
String * out_disable_reason = nullptr);
|
||||
String * out_disable_reason = nullptr,
|
||||
const Context * context = nullptr);
|
||||
|
||||
/** Merge the parts.
|
||||
* If `reservation != nullptr`, now and then reduces the size of the reserved space
|
||||
|
@ -48,7 +48,6 @@ struct Settings;
|
||||
M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \
|
||||
M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \
|
||||
M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 0) \
|
||||
M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \
|
||||
\
|
||||
/** Inserts settings. */ \
|
||||
M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \
|
||||
|
@ -635,7 +635,7 @@ void StorageMergeTree::loadMutations()
|
||||
}
|
||||
|
||||
std::shared_ptr<StorageMergeTree::MergeMutateSelectedEntry> StorageMergeTree::selectPartsToMerge(
|
||||
const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */, SelectPartsDecision * select_decision_out)
|
||||
const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */, const Context * context, SelectPartsDecision * select_decision_out)
|
||||
{
|
||||
std::unique_lock lock(currently_processing_in_background_mutex);
|
||||
auto data_settings = getSettings();
|
||||
@ -685,7 +685,7 @@ std::shared_ptr<StorageMergeTree::MergeMutateSelectedEntry> StorageMergeTree::se
|
||||
{
|
||||
UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace();
|
||||
select_decision = merger_mutator.selectAllPartsToMergeWithinPartition(
|
||||
future_part, disk_space, can_merge, partition_id, final, metadata_snapshot, out_disable_reason);
|
||||
future_part, disk_space, can_merge, partition_id, final, metadata_snapshot, out_disable_reason, context);
|
||||
|
||||
/// If final - we will wait for currently processing merges to finish and continue.
|
||||
/// TODO Respect query settings for timeout
|
||||
@ -738,14 +738,15 @@ bool StorageMergeTree::merge(
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
bool deduplicate,
|
||||
String * out_disable_reason)
|
||||
String * out_disable_reason,
|
||||
const Context * context)
|
||||
{
|
||||
auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations);
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
|
||||
SelectPartsDecision select_decision;
|
||||
|
||||
auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, &select_decision);
|
||||
auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, context, &select_decision);
|
||||
|
||||
/// 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)
|
||||
@ -1040,7 +1041,7 @@ bool StorageMergeTree::optimize(
|
||||
|
||||
for (const String & partition_id : partition_ids)
|
||||
{
|
||||
if (!merge(true, partition_id, true, deduplicate, &disable_reason))
|
||||
if (!merge(true, partition_id, true, deduplicate, &disable_reason, &context))
|
||||
{
|
||||
std::stringstream message;
|
||||
message.exceptions(std::ios::failbit);
|
||||
@ -1063,7 +1064,7 @@ bool StorageMergeTree::optimize(
|
||||
if (partition)
|
||||
partition_id = getPartitionIDFromQuery(partition, context);
|
||||
|
||||
if (!merge(true, partition_id, final, deduplicate, &disable_reason))
|
||||
if (!merge(true, partition_id, final, deduplicate, &disable_reason, &context))
|
||||
{
|
||||
std::stringstream message;
|
||||
message.exceptions(std::ios::failbit);
|
||||
|
@ -137,7 +137,7 @@ private:
|
||||
* If aggressive - when selects parts don't takes into account their ratio size and novelty (used for OPTIMIZE query).
|
||||
* Returns true if merge is finished successfully.
|
||||
*/
|
||||
bool merge(bool aggressive, const String & partition_id, bool final, bool deduplicate, String * out_disable_reason = nullptr);
|
||||
bool merge(bool aggressive, const String & partition_id, bool final, bool deduplicate, String * out_disable_reason = nullptr, const Context * context = nullptr);
|
||||
|
||||
ActionLock stopMergesAndWait();
|
||||
|
||||
@ -186,6 +186,7 @@ private:
|
||||
bool final,
|
||||
String * disable_reason,
|
||||
TableLockHolder & table_lock_holder,
|
||||
const Context * context = nullptr,
|
||||
SelectPartsDecision * select_decision_out = nullptr);
|
||||
bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder);
|
||||
|
||||
|
@ -3809,7 +3809,7 @@ bool StorageReplicatedMergeTree::optimize(
|
||||
|
||||
FutureMergedMutatedPart future_merged_part;
|
||||
SelectPartsDecision select_decision = merger_mutator.selectAllPartsToMergeWithinPartition(
|
||||
future_merged_part, disk_space, can_merge, partition_id, true, metadata_snapshot, nullptr);
|
||||
future_merged_part, disk_space, can_merge, partition_id, true, metadata_snapshot, nullptr, &query_context);
|
||||
|
||||
if (select_decision != SelectPartsDecision::SELECTED)
|
||||
break;
|
||||
@ -3856,7 +3856,7 @@ bool StorageReplicatedMergeTree::optimize(
|
||||
UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace();
|
||||
String partition_id = getPartitionIDFromQuery(partition, query_context);
|
||||
select_decision = merger_mutator.selectAllPartsToMergeWithinPartition(
|
||||
future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, &disable_reason);
|
||||
future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, &disable_reason, &query_context);
|
||||
}
|
||||
|
||||
/// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization)
|
||||
|
@ -1,6 +1,8 @@
|
||||
DROP TABLE IF EXISTS optimize_final;
|
||||
|
||||
CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x SETTINGS optimize_skip_merged_partitions=1;
|
||||
SET optimize_skip_merged_partitions=1;
|
||||
|
||||
CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x;
|
||||
|
||||
INSERT INTO optimize_final SELECT toDate('2020-01-01'), number FROM numbers(5);
|
||||
INSERT INTO optimize_final SELECT toDate('2020-01-01'), number + 5 FROM numbers(5);
|
||||
|
Loading…
Reference in New Issue
Block a user