Merge pull request #50489 from ClickHouse/vdimir/alter_moving_garbage

Cleanup moving parts
This commit is contained in:
Nikolai Kochetov 2023-06-16 15:39:32 +02:00 committed by GitHub
commit df1ea0b572
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 313 additions and 34 deletions

View File

@ -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();

View File

@ -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);

View File

@ -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;

View File

@ -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 = {};
}
}

View File

@ -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;

View File

@ -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>

View File

@ -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>

View File

@ -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>

View 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"

View File

@ -16,4 +16,5 @@
</shard>
</test_cluster>
</remote_servers>
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
</clickhouse>

View File

@ -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
)

View File

@ -105,4 +105,5 @@
</s3_encrypted_cache_policy>
</policies>
</storage_configuration>
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
</clickhouse>

View File

@ -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
)

View File

@ -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>

View File

@ -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)

View File

@ -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>

View File

@ -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
)

View File

@ -8,4 +8,5 @@
</s3>
<enable_system_unfreeze>true</enable_system_unfreeze>
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
</clickhouse>

View File

@ -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)

View File

@ -24,5 +24,6 @@
</policies>
</storage_configuration>
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
</clickhouse>

View File

@ -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)")

View File

@ -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>

View File

@ -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>

View File

@ -89,4 +89,5 @@
<cluster>test_cluster</cluster>
<shard>1</shard>
</macros>
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
</clickhouse>

View File

@ -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)
)

View File

@ -93,4 +93,5 @@
<cluster>test_cluster</cluster>
</macros>
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
</clickhouse>

View File

@ -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}"
)

View File

@ -107,4 +107,5 @@
</storage_configuration>
<allow_remove_stale_moving_parts>true</allow_remove_stale_moving_parts>
</clickhouse>

View File

@ -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
)

View File

@ -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>

View File

@ -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(