mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #50489 from ClickHouse/vdimir/alter_moving_garbage
Cleanup moving parts
This commit is contained in:
commit
df1ea0b572
@ -492,13 +492,17 @@ void IMergeTreeDataPart::removeIfNeeded()
|
||||
|
||||
if (is_temp)
|
||||
{
|
||||
String file_name = fileName(getDataPartStorage().getPartDirectory());
|
||||
const auto & part_directory = getDataPartStorage().getPartDirectory();
|
||||
|
||||
String file_name = fileName(part_directory);
|
||||
|
||||
if (file_name.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "relative_path {} of part {} is invalid or not set",
|
||||
getDataPartStorage().getPartDirectory(), name);
|
||||
|
||||
if (!startsWith(file_name, "tmp") && !endsWith(file_name, ".tmp_proj"))
|
||||
const auto part_parent_directory = directoryPath(part_directory);
|
||||
bool is_moving_part = part_parent_directory.ends_with("moving/");
|
||||
if (!startsWith(file_name, "tmp") && !endsWith(file_name, ".tmp_proj") && !is_moving_part)
|
||||
{
|
||||
LOG_ERROR(
|
||||
storage.log,
|
||||
@ -507,6 +511,11 @@ void IMergeTreeDataPart::removeIfNeeded()
|
||||
path);
|
||||
return;
|
||||
}
|
||||
|
||||
if (is_moving_part)
|
||||
{
|
||||
LOG_TRACE(storage.log, "Removing unneeded moved part from {}", path);
|
||||
}
|
||||
}
|
||||
|
||||
remove();
|
||||
|
@ -19,6 +19,7 @@
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/Config/ConfigHelper.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
@ -2014,6 +2015,21 @@ static bool isOldPartDirectory(const DiskPtr & disk, const String & directory_pa
|
||||
|
||||
|
||||
size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes)
|
||||
{
|
||||
size_t cleared_count = 0;
|
||||
|
||||
cleared_count += clearOldTemporaryDirectories(relative_data_path, custom_directories_lifetime_seconds, valid_prefixes);
|
||||
|
||||
if (allowRemoveStaleMovingParts())
|
||||
{
|
||||
/// Clear _all_ parts from the `moving` directory
|
||||
cleared_count += clearOldTemporaryDirectories(fs::path(relative_data_path) / "moving", custom_directories_lifetime_seconds, {""});
|
||||
}
|
||||
|
||||
return cleared_count;
|
||||
}
|
||||
|
||||
size_t MergeTreeData::clearOldTemporaryDirectories(const String & root_path, size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes)
|
||||
{
|
||||
/// If the method is already called from another thread, then we don't need to do anything.
|
||||
std::unique_lock lock(clear_old_temporary_directories_mutex, std::defer_lock);
|
||||
@ -2032,7 +2048,7 @@ size_t MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lif
|
||||
if (disk->isBroken())
|
||||
continue;
|
||||
|
||||
for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next())
|
||||
for (auto it = disk->iterateDirectory(root_path); it->isValid(); it->next())
|
||||
{
|
||||
const std::string & basename = it->name();
|
||||
bool start_with_valid_prefix = false;
|
||||
@ -7857,7 +7873,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr &
|
||||
for (const auto & moving_part : moving_tagger->parts_to_move)
|
||||
{
|
||||
Stopwatch stopwatch;
|
||||
MutableDataPartPtr cloned_part;
|
||||
MergeTreePartsMover::TemporaryClonedPart cloned_part;
|
||||
ProfileEventsScope profile_events_scope;
|
||||
|
||||
auto write_part_log = [&](const ExecutionStatus & execution_status)
|
||||
@ -7867,7 +7883,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr &
|
||||
execution_status,
|
||||
stopwatch.elapsed(),
|
||||
moving_part.part->name,
|
||||
cloned_part,
|
||||
cloned_part.part,
|
||||
{moving_part.part},
|
||||
nullptr,
|
||||
profile_events_scope.getSnapshot());
|
||||
@ -7943,9 +7959,6 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr &
|
||||
catch (...)
|
||||
{
|
||||
write_part_log(ExecutionStatus::fromCurrentException("", true));
|
||||
if (cloned_part)
|
||||
cloned_part->remove();
|
||||
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -8460,6 +8473,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createEmptyPart(
|
||||
return new_data_part;
|
||||
}
|
||||
|
||||
bool MergeTreeData::allowRemoveStaleMovingParts() const
|
||||
{
|
||||
return ConfigHelper::getBool(getContext()->getConfigRef(), "allow_remove_stale_moving_parts");
|
||||
}
|
||||
|
||||
CurrentlySubmergingEmergingTagger::~CurrentlySubmergingEmergingTagger()
|
||||
{
|
||||
std::lock_guard lock(storage.currently_submerging_emerging_mutex);
|
||||
|
@ -679,6 +679,7 @@ public:
|
||||
/// Delete all directories which names begin with "tmp"
|
||||
/// Must be called with locked lockForShare() because it's using relative_data_path.
|
||||
size_t clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes = {"tmp_", "tmp-fetch_"});
|
||||
size_t clearOldTemporaryDirectories(const String & root_path, size_t custom_directories_lifetime_seconds, const NameSet & valid_prefixes);
|
||||
|
||||
size_t clearEmptyParts();
|
||||
|
||||
@ -1064,6 +1065,9 @@ public:
|
||||
void waitForOutdatedPartsToBeLoaded() const;
|
||||
bool canUsePolymorphicParts() const;
|
||||
|
||||
/// TODO: make enabled by default in the next release if no problems found.
|
||||
bool allowRemoveStaleMovingParts() const;
|
||||
|
||||
protected:
|
||||
friend class IMergeTreeDataPart;
|
||||
friend class MergeTreeDataMergerMutator;
|
||||
|
@ -11,6 +11,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ABORTED;
|
||||
extern const int DIRECTORY_ALREADY_EXISTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -203,7 +204,7 @@ bool MergeTreePartsMover::selectPartsForMove(
|
||||
return false;
|
||||
}
|
||||
|
||||
MergeTreeMutableDataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEntry & moving_part) const
|
||||
MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const MergeTreeMoveEntry & moving_part) const
|
||||
{
|
||||
if (moves_blocker.isCancelled())
|
||||
throw Exception(ErrorCodes::ABORTED, "Cancelled moving parts.");
|
||||
@ -212,6 +213,8 @@ MergeTreeMutableDataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEn
|
||||
auto part = moving_part.part;
|
||||
auto disk = moving_part.reserved_space->getDisk();
|
||||
LOG_DEBUG(log, "Cloning part {} from '{}' to '{}'", part->name, part->getDataPartStorage().getDiskName(), disk->getName());
|
||||
TemporaryClonedPart cloned_part;
|
||||
cloned_part.temporary_directory_lock = data->getTemporaryPartDirectoryHolder(part->name);
|
||||
|
||||
MutableDataPartStoragePtr cloned_part_storage;
|
||||
if (disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication)
|
||||
@ -222,8 +225,10 @@ MergeTreeMutableDataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEn
|
||||
String relative_path = part->getDataPartStorage().getPartDirectory();
|
||||
if (disk->exists(path_to_clone + relative_path))
|
||||
{
|
||||
LOG_WARNING(log, "Path {} already exists. Will remove it and clone again.", fullPath(disk, path_to_clone + relative_path));
|
||||
disk->removeRecursive(fs::path(path_to_clone) / relative_path / "");
|
||||
throw Exception(ErrorCodes::DIRECTORY_ALREADY_EXISTS,
|
||||
"Cannot clone part {} from '{}' to '{}': path '{}' already exists",
|
||||
part->name, part->getDataPartStorage().getDiskName(), disk->getName(),
|
||||
fullPath(disk, path_to_clone + relative_path));
|
||||
}
|
||||
|
||||
disk->createDirectories(path_to_clone);
|
||||
@ -242,37 +247,48 @@ MergeTreeMutableDataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEn
|
||||
}
|
||||
|
||||
MergeTreeDataPartBuilder builder(*data, part->name, cloned_part_storage);
|
||||
auto cloned_part = std::move(builder).withPartFormatFromDisk().build();
|
||||
LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part->getDataPartStorage().getFullPath());
|
||||
cloned_part.part = std::move(builder).withPartFormatFromDisk().build();
|
||||
LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part.part->getDataPartStorage().getFullPath());
|
||||
|
||||
cloned_part->loadColumnsChecksumsIndexes(true, true);
|
||||
cloned_part->loadVersionMetadata();
|
||||
cloned_part->modification_time = cloned_part->getDataPartStorage().getLastModified().epochTime();
|
||||
cloned_part.part->is_temp = data->allowRemoveStaleMovingParts();
|
||||
cloned_part.part->loadColumnsChecksumsIndexes(true, true);
|
||||
cloned_part.part->loadVersionMetadata();
|
||||
cloned_part.part->modification_time = cloned_part.part->getDataPartStorage().getLastModified().epochTime();
|
||||
return cloned_part;
|
||||
}
|
||||
|
||||
|
||||
void MergeTreePartsMover::swapClonedPart(const MergeTreeMutableDataPartPtr & cloned_part) const
|
||||
void MergeTreePartsMover::swapClonedPart(TemporaryClonedPart & cloned_part) const
|
||||
{
|
||||
if (moves_blocker.isCancelled())
|
||||
throw Exception(ErrorCodes::ABORTED, "Cancelled moving parts.");
|
||||
|
||||
auto active_part = data->getActiveContainingPart(cloned_part->name);
|
||||
auto active_part = data->getActiveContainingPart(cloned_part.part->name);
|
||||
|
||||
/// It's ok, because we don't block moving parts for merges or mutations
|
||||
if (!active_part || active_part->name != cloned_part->name)
|
||||
if (!active_part || active_part->name != cloned_part.part->name)
|
||||
{
|
||||
LOG_INFO(log, "Failed to swap {}. Active part doesn't exist. Possible it was merged or mutated. Will remove copy on path '{}'.", cloned_part->name, cloned_part->getDataPartStorage().getFullPath());
|
||||
LOG_INFO(log,
|
||||
"Failed to swap {}. Active part doesn't exist (containing part {}). "
|
||||
"Possible it was merged or mutated. Part on path '{}' {}",
|
||||
cloned_part.part->name,
|
||||
active_part ? active_part->name : "doesn't exist",
|
||||
cloned_part.part->getDataPartStorage().getFullPath(),
|
||||
data->allowRemoveStaleMovingParts() ? "will be removed" : "will remain intact (set <allow_remove_stale_moving_parts> in config.xml, exercise caution when using)");
|
||||
return;
|
||||
}
|
||||
|
||||
cloned_part.part->is_temp = false;
|
||||
|
||||
/// Don't remove new directory but throw an error because it may contain part which is currently in use.
|
||||
cloned_part->renameTo(active_part->name, false);
|
||||
cloned_part.part->renameTo(active_part->name, false);
|
||||
|
||||
/// TODO what happen if server goes down here?
|
||||
data->swapActivePart(cloned_part);
|
||||
data->swapActivePart(cloned_part.part);
|
||||
|
||||
LOG_TRACE(log, "Part {} was moved to {}", cloned_part->name, cloned_part->getDataPartStorage().getFullPath());
|
||||
LOG_TRACE(log, "Part {} was moved to {}", cloned_part.part->name, cloned_part.part->getDataPartStorage().getFullPath());
|
||||
|
||||
cloned_part.temporary_directory_lock = {};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <functional>
|
||||
#include <optional>
|
||||
#include <vector>
|
||||
#include <base/scope_guard.h>
|
||||
#include <Disks/StoragePolicy.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MovesList.h>
|
||||
@ -43,12 +44,19 @@ private:
|
||||
using AllowedMovingPredicate = std::function<bool(const std::shared_ptr<const IMergeTreeDataPart> &, String * reason)>;
|
||||
|
||||
public:
|
||||
|
||||
explicit MergeTreePartsMover(MergeTreeData * data_)
|
||||
: data(data_)
|
||||
, log(&Poco::Logger::get("MergeTreePartsMover"))
|
||||
{
|
||||
}
|
||||
|
||||
struct TemporaryClonedPart
|
||||
{
|
||||
MergeTreeMutableDataPartPtr part;
|
||||
scope_guard temporary_directory_lock;
|
||||
};
|
||||
|
||||
/// Select parts for background moves according to storage_policy configuration.
|
||||
/// Returns true if at least one part was selected for move.
|
||||
bool selectPartsForMove(
|
||||
@ -57,14 +65,14 @@ public:
|
||||
const std::lock_guard<std::mutex> & moving_parts_lock);
|
||||
|
||||
/// Copies part to selected reservation in detached folder. Throws exception if part already exists.
|
||||
MergeTreeMutableDataPartPtr clonePart(const MergeTreeMoveEntry & moving_part) const;
|
||||
TemporaryClonedPart clonePart(const MergeTreeMoveEntry & moving_part) const;
|
||||
|
||||
/// Replaces cloned part from detached directory into active data parts set.
|
||||
/// Replacing part changes state to DeleteOnDestroy and will be removed from disk after destructor of
|
||||
/// IMergeTreeDataPart called. If replacing part doesn't exists or not active (committed) than
|
||||
/// cloned part will be removed and log message will be reported. It may happen in case of concurrent
|
||||
/// merge or mutation.
|
||||
void swapClonedPart(const MergeTreeMutableDataPartPtr & cloned_parts) const;
|
||||
void swapClonedPart(TemporaryClonedPart & cloned_part) const;
|
||||
|
||||
/// Can stop background moves and moves from queries
|
||||
ActionBlocker moves_blocker;
|
||||
|
@ -0,0 +1,16 @@
|
||||
<clickhouse>
|
||||
<remote_servers>
|
||||
<test_cluster>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>node1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster>
|
||||
</remote_servers>
|
||||
</clickhouse>
|
@ -0,0 +1,26 @@
|
||||
<clickhouse>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<s3>
|
||||
<type>s3</type>
|
||||
<endpoint>http://minio1:9001/root/data/</endpoint>
|
||||
<access_key_id>minio</access_key_id>
|
||||
<secret_access_key>minio123</secret_access_key>
|
||||
</s3>
|
||||
</disks>
|
||||
<policies>
|
||||
<two_disks>
|
||||
<volumes>
|
||||
<default>
|
||||
<disk>default</disk>
|
||||
</default>
|
||||
<external>
|
||||
<disk>s3</disk>
|
||||
</external>
|
||||
</volumes>
|
||||
</two_disks>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
</clickhouse>
|
@ -0,0 +1,7 @@
|
||||
<clickhouse>
|
||||
<tcp_port>9000</tcp_port>
|
||||
<listen_host>127.0.0.1</listen_host>
|
||||
<max_concurrent_queries>500</max_concurrent_queries>
|
||||
<path>./clickhouse/</path>
|
||||
<users_config>users.xml</users_config>
|
||||
</clickhouse>
|
155
tests/integration/test_alter_moving_garbage/test.py
Normal file
155
tests/integration/test_alter_moving_garbage/test.py
Normal file
@ -0,0 +1,155 @@
|
||||
import logging
|
||||
import time
|
||||
|
||||
import pytest
|
||||
import threading
|
||||
import random
|
||||
|
||||
from helpers.client import QueryRuntimeException
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
# two replicas in remote_servers.xml
|
||||
REPLICA_COUNT = 2
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def cluster():
|
||||
try:
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
for i in range(1, REPLICA_COUNT + 1):
|
||||
cluster.add_instance(
|
||||
f"node{i}",
|
||||
main_configs=[
|
||||
"configs/config.d/storage_conf.xml",
|
||||
"configs/config.d/remote_servers.xml",
|
||||
],
|
||||
with_minio=True,
|
||||
with_zookeeper=True,
|
||||
)
|
||||
|
||||
logging.info("Starting cluster...")
|
||||
cluster.start()
|
||||
logging.info("Cluster started")
|
||||
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def create_table(node, table_name, replicated, additional_settings):
|
||||
settings = {
|
||||
"storage_policy": "two_disks",
|
||||
"old_parts_lifetime": 1,
|
||||
"index_granularity": 512,
|
||||
"temporary_directories_lifetime": 0,
|
||||
"merge_tree_clear_old_temporary_directories_interval_seconds": 1,
|
||||
}
|
||||
settings.update(additional_settings)
|
||||
|
||||
table_engine = (
|
||||
f"ReplicatedMergeTree('/clickhouse/tables/0/{table_name}', '{node.name}')"
|
||||
if replicated
|
||||
else "MergeTree()"
|
||||
)
|
||||
|
||||
create_table_statement = f"""
|
||||
CREATE TABLE {table_name} (
|
||||
dt Date,
|
||||
id Int64,
|
||||
data String,
|
||||
INDEX min_max (id) TYPE minmax GRANULARITY 3
|
||||
) ENGINE = {table_engine}
|
||||
PARTITION BY dt
|
||||
ORDER BY (dt, id)
|
||||
SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))}"""
|
||||
|
||||
if replicated:
|
||||
node.query_with_retry(create_table_statement)
|
||||
else:
|
||||
node.query(create_table_statement)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
"allow_remote_fs_zero_copy_replication,replicated_engine",
|
||||
[(False, False), (False, True), (True, True)],
|
||||
)
|
||||
def test_create_table(
|
||||
cluster, allow_remote_fs_zero_copy_replication, replicated_engine
|
||||
):
|
||||
if replicated_engine:
|
||||
nodes = list(cluster.instances.values())
|
||||
else:
|
||||
nodes = [cluster.instances["node1"]]
|
||||
|
||||
additional_settings = {}
|
||||
|
||||
# Different names for logs readability
|
||||
table_name = "test_table"
|
||||
if allow_remote_fs_zero_copy_replication:
|
||||
table_name = "test_table_zero_copy"
|
||||
additional_settings["allow_remote_fs_zero_copy_replication"] = 1
|
||||
if replicated_engine:
|
||||
table_name = table_name + "_replicated"
|
||||
|
||||
for node in nodes:
|
||||
create_table(node, table_name, replicated_engine, additional_settings)
|
||||
|
||||
for i in range(1, 11):
|
||||
partition = f"2021-01-{i:02d}"
|
||||
random.choice(nodes).query(
|
||||
f"INSERT INTO {table_name} SELECT toDate('{partition}'), number as id, toString(sipHash64(number, {i})) FROM numbers(10_000)"
|
||||
)
|
||||
|
||||
# Run ALTER in parallel with moving parts
|
||||
|
||||
stop_alter = False
|
||||
|
||||
def alter():
|
||||
random.choice(nodes).query(f"ALTER TABLE {table_name} ADD COLUMN col0 String")
|
||||
for d in range(1, 100):
|
||||
if stop_alter:
|
||||
break
|
||||
|
||||
# Some lightweight mutation should change moving part before it is swapped, then we will have to cleanup it.
|
||||
# Messages `Failed to swap {}. Active part doesn't exist` should appear in logs.
|
||||
#
|
||||
# I managed to reproduce issue with DELETE (`ALTER TABLE {table_name} ADD/DROP COLUMN` also works on real s3 instead of minio)
|
||||
# Note: do not delete rows with id % 100 = 0, because they are used in `check_count` to use them in check that data is not corrupted
|
||||
random.choice(nodes).query(f"DELETE FROM {table_name} WHERE id % 100 = {d}")
|
||||
|
||||
time.sleep(0.1)
|
||||
|
||||
alter_thread = threading.Thread(target=alter)
|
||||
alter_thread.start()
|
||||
|
||||
for i in range(1, 11):
|
||||
partition = f"2021-01-{i:02d}"
|
||||
try:
|
||||
random.choice(nodes).query(
|
||||
f"ALTER TABLE {table_name} MOVE PARTITION '{partition}' TO DISK 's3'",
|
||||
)
|
||||
except QueryRuntimeException as e:
|
||||
if "PART_IS_TEMPORARILY_LOCKED" in str(e):
|
||||
continue
|
||||
raise e
|
||||
|
||||
# Function to clear old temporary directories wakes up every 1 second, sleep to make sure it is called
|
||||
time.sleep(0.5)
|
||||
|
||||
stop_alter = True
|
||||
alter_thread.join()
|
||||
|
||||
# Check that no data was lost
|
||||
|
||||
data_digest = None
|
||||
if replicated_engine:
|
||||
# We don't know what data was replicated, so we need to check all replicas and take unique values
|
||||
data_digest = random.choice(nodes).query_with_retry(
|
||||
f"SELECT countDistinct(dt, data) FROM clusterAllReplicas(test_cluster, default.{table_name}) WHERE id % 100 == 0"
|
||||
)
|
||||
else:
|
||||
data_digest = random.choice(nodes).query(
|
||||
f"SELECT countDistinct(dt, data) FROM {table_name} WHERE id % 100 == 0"
|
||||
)
|
||||
|
||||
assert data_digest == "1000\n"
|
@ -16,4 +16,5 @@
|
||||
</shard>
|
||||
</test_cluster>
|
||||
</remote_servers>
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
</clickhouse>
|
||||
|
@ -18,7 +18,7 @@ def initialize_database(nodes, shard):
|
||||
CREATE TABLE `{database}`.dest (p UInt64, d UInt64)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/{database}/tables/test_consistent_shard2{shard}/replicated', '{replica}')
|
||||
ORDER BY d PARTITION BY p
|
||||
SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0;
|
||||
SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0, temporary_directories_lifetime=1;
|
||||
""".format(
|
||||
shard=shard, replica=node.name, database=CLICKHOUSE_DATABASE
|
||||
)
|
||||
|
@ -105,4 +105,5 @@
|
||||
</s3_encrypted_cache_policy>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
</clickhouse>
|
||||
|
@ -96,7 +96,7 @@ def test_part_move(policy, destination_disks):
|
||||
data String
|
||||
) ENGINE=MergeTree()
|
||||
ORDER BY id
|
||||
SETTINGS storage_policy='{}'
|
||||
SETTINGS storage_policy='{}', temporary_directories_lifetime=1
|
||||
""".format(
|
||||
policy
|
||||
)
|
||||
|
@ -15,4 +15,5 @@
|
||||
<max_concurrent_queries>500</max_concurrent_queries>
|
||||
<path>./clickhouse/</path>
|
||||
<users_config>users.xml</users_config>
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
</clickhouse>
|
||||
|
@ -66,6 +66,7 @@ def create_table(node, table_name, **additional_settings):
|
||||
"storage_policy": "blob_storage_policy",
|
||||
"old_parts_lifetime": 1,
|
||||
"index_granularity": 512,
|
||||
"temporary_directories_lifetime": 1,
|
||||
}
|
||||
settings.update(additional_settings)
|
||||
|
||||
|
@ -29,4 +29,5 @@
|
||||
<merge_tree>
|
||||
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
|
||||
</merge_tree>
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
</clickhouse>
|
||||
|
@ -29,7 +29,8 @@ def create_table(cluster, table_name, additional_settings=None):
|
||||
SETTINGS
|
||||
storage_policy='hdfs',
|
||||
old_parts_lifetime=0,
|
||||
index_granularity=512
|
||||
index_granularity=512,
|
||||
temporary_directories_lifetime=1
|
||||
""".format(
|
||||
table_name
|
||||
)
|
||||
|
@ -8,4 +8,5 @@
|
||||
</s3>
|
||||
|
||||
<enable_system_unfreeze>true</enable_system_unfreeze>
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
</clickhouse>
|
||||
|
@ -75,6 +75,7 @@ def create_table(node, table_name, **additional_settings):
|
||||
"storage_policy": "s3",
|
||||
"old_parts_lifetime": 0,
|
||||
"index_granularity": 512,
|
||||
"temporary_directories_lifetime": 1,
|
||||
}
|
||||
settings.update(additional_settings)
|
||||
|
||||
|
@ -24,5 +24,6 @@
|
||||
</policies>
|
||||
|
||||
</storage_configuration>
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
|
||||
</clickhouse>
|
||||
|
@ -46,7 +46,7 @@ def test_move_partition_to_disk_on_cluster(start_cluster):
|
||||
"(x UInt64) "
|
||||
"ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_local_table', '{replica}') "
|
||||
"ORDER BY tuple()"
|
||||
"SETTINGS storage_policy = 'jbod_with_external';",
|
||||
"SETTINGS storage_policy = 'jbod_with_external', temporary_directories_lifetime=1;",
|
||||
)
|
||||
|
||||
node1.query("INSERT INTO test_local_table VALUES (0)")
|
||||
|
@ -122,5 +122,10 @@
|
||||
</policies>
|
||||
|
||||
</storage_configuration>
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
|
||||
<merge_tree>
|
||||
<temporary_directories_lifetime>1</temporary_directories_lifetime>
|
||||
</merge_tree>
|
||||
|
||||
</clickhouse>
|
||||
|
@ -24,9 +24,11 @@
|
||||
</default_with_external>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
|
||||
|
||||
<merge_tree>
|
||||
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
|
||||
<temporary_directories_lifetime>1</temporary_directories_lifetime>
|
||||
</merge_tree>
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
|
||||
</clickhouse>
|
||||
|
@ -89,4 +89,5 @@
|
||||
<cluster>test_cluster</cluster>
|
||||
<shard>1</shard>
|
||||
</macros>
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
</clickhouse>
|
||||
|
@ -128,7 +128,7 @@ def test_hdfs_zero_copy_replication_single_move(cluster, storage_policy, init_ob
|
||||
CREATE TABLE single_node_move_test (dt DateTime, id Int64)
|
||||
ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/single_node_move_test', '{replica}')
|
||||
ORDER BY (dt, id)
|
||||
SETTINGS storage_policy='$policy'
|
||||
SETTINGS storage_policy='$policy',temporary_directories_lifetime=1
|
||||
"""
|
||||
).substitute(policy=storage_policy)
|
||||
)
|
||||
|
@ -93,4 +93,5 @@
|
||||
<cluster>test_cluster</cluster>
|
||||
</macros>
|
||||
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
</clickhouse>
|
||||
|
@ -163,7 +163,7 @@ def test_s3_zero_copy_on_hybrid_storage(started_cluster):
|
||||
CREATE TABLE hybrid_test ON CLUSTER test_cluster (id UInt32, value String)
|
||||
ENGINE=ReplicatedMergeTree('/clickhouse/tables/hybrid_test', '{}')
|
||||
ORDER BY id
|
||||
SETTINGS storage_policy='hybrid'
|
||||
SETTINGS storage_policy='hybrid',temporary_directories_lifetime=1
|
||||
""".format(
|
||||
"{replica}"
|
||||
)
|
||||
|
@ -107,4 +107,5 @@
|
||||
|
||||
</storage_configuration>
|
||||
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
</clickhouse>
|
||||
|
@ -1549,7 +1549,7 @@ def test_double_move_while_select(started_cluster, name, positive):
|
||||
) ENGINE = MergeTree
|
||||
ORDER BY tuple()
|
||||
PARTITION BY n
|
||||
SETTINGS storage_policy='small_jbod_with_external'
|
||||
SETTINGS storage_policy='small_jbod_with_external',temporary_directories_lifetime=1
|
||||
""".format(
|
||||
name=name
|
||||
)
|
||||
|
@ -38,4 +38,5 @@
|
||||
<merge_tree>
|
||||
<allow_remote_fs_zero_copy_replication>true</allow_remote_fs_zero_copy_replication>
|
||||
</merge_tree>
|
||||
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
|
||||
</clickhouse>
|
||||
|
@ -45,7 +45,7 @@ CREATE TABLE test1 (EventDate Date, CounterID UInt32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse-tables/test1', 'r1')
|
||||
PARTITION BY toMonday(EventDate)
|
||||
ORDER BY (CounterID, EventDate)
|
||||
SETTINGS index_granularity = 8192, storage_policy = 's3'"""
|
||||
SETTINGS index_granularity = 8192, storage_policy = 's3', temporary_directories_lifetime=1"""
|
||||
)
|
||||
|
||||
node1.query(
|
||||
|
Loading…
Reference in New Issue
Block a user