mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
Merge pull request #27931 from ClickHouse/wait_for_all_replicas_timeouts
Avoid too long waiting for inactive replicas
This commit is contained in:
commit
703101fe4d
@ -97,7 +97,7 @@ class IColumn;
|
||||
M(Bool, optimize_move_to_prewhere_if_final, false, "If query has `FINAL`, the optimization `move_to_prewhere` is not always correct and it is enabled only if both settings `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are turned on", 0) \
|
||||
\
|
||||
M(UInt64, replication_alter_partitions_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) \
|
||||
M(UInt64, replication_alter_columns_timeout, 60, "Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time.", 0) \
|
||||
M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \
|
||||
\
|
||||
M(LoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 0) \
|
||||
M(UInt64, load_balancing_first_offset, 0, "Which replica to preferably send a query when FIRST_OR_RANDOM load balancing strategy is used.", 0) \
|
||||
@ -514,6 +514,7 @@ class IColumn;
|
||||
M(Bool, allow_experimental_window_functions, true, "Obsolete setting, does nothing.", 0) \
|
||||
M(HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT, "Obsolete setting, does nothing.", 0) \
|
||||
M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing.", 0) \
|
||||
M(UInt64, replication_alter_columns_timeout, 60, "Obsolete setting, does nothing.", 0) \
|
||||
/** The section above is for obsolete settings. Do not add anything there. */
|
||||
|
||||
|
||||
|
@ -194,7 +194,7 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil::
|
||||
/// This wait in background schedule pool is useless. It'd be
|
||||
/// better to have some notification which will call `step`
|
||||
/// function when all replicated will finish. TODO.
|
||||
storage.waitForAllReplicasToProcessLogEntry(log_entry, true);
|
||||
storage.waitForAllReplicasToProcessLogEntry(zookeeper_path, log_entry, -1);
|
||||
}
|
||||
|
||||
{
|
||||
@ -231,7 +231,7 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil::
|
||||
String log_znode_path = dynamic_cast<const Coordination::CreateResponse &>(*responses.back()).path_created;
|
||||
log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
|
||||
|
||||
storage.waitForAllTableReplicasToProcessLogEntry(entry.to_shard, log_entry, true);
|
||||
storage.waitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, -1);
|
||||
}
|
||||
|
||||
{
|
||||
@ -269,7 +269,7 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil::
|
||||
String log_znode_path = dynamic_cast<const Coordination::CreateResponse &>(*responses.back()).path_created;
|
||||
log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
|
||||
|
||||
storage.waitForAllTableReplicasToProcessLogEntry(entry.to_shard, log_entry, true);
|
||||
storage.waitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, -1);
|
||||
}
|
||||
|
||||
{
|
||||
@ -318,7 +318,7 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil::
|
||||
String log_znode_path = dynamic_cast<const Coordination::CreateResponse &>(*responses.back()).path_created;
|
||||
log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
|
||||
|
||||
storage.waitForAllTableReplicasToProcessLogEntry(entry.to_shard, log_entry, true);
|
||||
storage.waitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, -1);
|
||||
}
|
||||
|
||||
{
|
||||
@ -348,7 +348,7 @@ void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil::
|
||||
{
|
||||
ReplicatedMergeTreeLogEntry log_entry;
|
||||
if (storage.dropPartImpl(zk, entry.part_name, log_entry, false, false))
|
||||
storage.waitForAllReplicasToProcessLogEntry(log_entry, true);
|
||||
storage.waitForAllReplicasToProcessLogEntry(zookeeper_path, log_entry, -1);
|
||||
}
|
||||
|
||||
{
|
||||
|
@ -4732,12 +4732,10 @@ bool StorageReplicatedMergeTree::optimize(
|
||||
}
|
||||
}
|
||||
|
||||
if (query_context->getSettingsRef().replication_alter_partitions_sync != 0)
|
||||
{
|
||||
/// NOTE Table lock must not be held while waiting. Some combination of R-W-R locks from different threads will yield to deadlock.
|
||||
for (auto & merge_entry : merge_entries)
|
||||
waitForAllReplicasToProcessLogEntry(merge_entry, false);
|
||||
}
|
||||
table_lock.reset();
|
||||
|
||||
for (auto & merge_entry : merge_entries)
|
||||
waitForLogEntryToBeProcessedIfNecessary(merge_entry, query_context);
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -5048,20 +5046,8 @@ void StorageReplicatedMergeTree::alter(
|
||||
|
||||
table_lock_holder.reset();
|
||||
|
||||
std::vector<String> unwaited;
|
||||
if (query_context->getSettingsRef().replication_alter_partitions_sync == 2)
|
||||
{
|
||||
LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
|
||||
unwaited = waitForAllReplicasToProcessLogEntry(*alter_entry, false);
|
||||
}
|
||||
else if (query_context->getSettingsRef().replication_alter_partitions_sync == 1)
|
||||
{
|
||||
LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
|
||||
waitForReplicaToProcessLogEntry(replica_name, *alter_entry);
|
||||
}
|
||||
|
||||
if (!unwaited.empty())
|
||||
throw Exception("Some replicas doesn't finish metadata alter", ErrorCodes::UNFINISHED);
|
||||
LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
|
||||
waitForLogEntryToBeProcessedIfNecessary(*alter_entry, query_context, "Some replicas doesn't finish metadata alter: ");
|
||||
|
||||
if (mutation_znode)
|
||||
{
|
||||
@ -5212,11 +5198,7 @@ void StorageReplicatedMergeTree::dropPart(const String & part_name, bool detach,
|
||||
|
||||
dropPartImpl(zookeeper, part_name, entry, detach, /*throw_if_noop=*/ true);
|
||||
|
||||
/// If necessary, wait until the operation is performed on itself or on all replicas.
|
||||
if (query_context->getSettingsRef().replication_alter_partitions_sync == 1)
|
||||
waitForReplicaToProcessLogEntry(replica_name, entry);
|
||||
else if (query_context->getSettingsRef().replication_alter_partitions_sync == 2)
|
||||
waitForAllReplicasToProcessLogEntry(entry);
|
||||
waitForLogEntryToBeProcessedIfNecessary(entry, query_context);
|
||||
}
|
||||
|
||||
void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool detach, ContextPtr query_context)
|
||||
@ -5233,12 +5215,7 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool de
|
||||
|
||||
if (did_drop)
|
||||
{
|
||||
/// If necessary, wait until the operation is performed on itself or on all replicas.
|
||||
if (query_context->getSettingsRef().replication_alter_partitions_sync == 1)
|
||||
waitForReplicaToProcessLogEntry(replica_name, entry);
|
||||
else if (query_context->getSettingsRef().replication_alter_partitions_sync == 2)
|
||||
waitForAllReplicasToProcessLogEntry(entry);
|
||||
|
||||
waitForLogEntryToBeProcessedIfNecessary(entry, query_context);
|
||||
cleanLastPartNode(partition_id);
|
||||
}
|
||||
}
|
||||
@ -5257,13 +5234,17 @@ void StorageReplicatedMergeTree::truncate(
|
||||
|
||||
Strings partitions = zookeeper->getChildren(fs::path(zookeeper_path) / "block_numbers");
|
||||
|
||||
std::vector<std::unique_ptr<LogEntry>> entries_to_wait;
|
||||
entries_to_wait.reserve(partitions.size());
|
||||
for (String & partition_id : partitions)
|
||||
{
|
||||
LogEntry entry;
|
||||
|
||||
if (dropAllPartsInPartition(*zookeeper, partition_id, entry, query_context, false))
|
||||
waitForAllReplicasToProcessLogEntry(entry);
|
||||
auto entry = std::make_unique<LogEntry>();
|
||||
if (dropAllPartsInPartition(*zookeeper, partition_id, *entry, query_context, false))
|
||||
entries_to_wait.push_back(std::move(entry));
|
||||
}
|
||||
|
||||
for (const auto & entry : entries_to_wait)
|
||||
waitForLogEntryToBeProcessedIfNecessary(*entry, query_context);
|
||||
}
|
||||
|
||||
|
||||
@ -5421,19 +5402,20 @@ StorageReplicatedMergeTree::allocateBlockNumber(
|
||||
}
|
||||
|
||||
|
||||
Strings StorageReplicatedMergeTree::waitForAllTableReplicasToProcessLogEntry(
|
||||
const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active)
|
||||
Strings StorageReplicatedMergeTree::tryWaitForAllReplicasToProcessLogEntry(
|
||||
const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry, Int64 wait_for_inactive_timeout)
|
||||
{
|
||||
LOG_DEBUG(log, "Waiting for all replicas to process {}", entry.znode_name);
|
||||
|
||||
auto zookeeper = getZooKeeper();
|
||||
Strings replicas = zookeeper->getChildren(fs::path(table_zookeeper_path) / "replicas");
|
||||
Strings unwaited;
|
||||
bool wait_for_inactive = wait_for_inactive_timeout != 0;
|
||||
for (const String & replica : replicas)
|
||||
{
|
||||
if (wait_for_non_active || zookeeper->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active"))
|
||||
if (wait_for_inactive || zookeeper->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active"))
|
||||
{
|
||||
if (!waitForTableReplicaToProcessLogEntry(table_zookeeper_path, replica, entry, wait_for_non_active))
|
||||
if (!tryWaitForReplicaToProcessLogEntry(table_zookeeper_path, replica, entry, wait_for_inactive_timeout))
|
||||
unwaited.push_back(replica);
|
||||
}
|
||||
else
|
||||
@ -5446,16 +5428,38 @@ Strings StorageReplicatedMergeTree::waitForAllTableReplicasToProcessLogEntry(
|
||||
return unwaited;
|
||||
}
|
||||
|
||||
|
||||
Strings StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(
|
||||
const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active)
|
||||
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(
|
||||
const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry, Int64 wait_for_inactive_timeout, const String & error_context)
|
||||
{
|
||||
return waitForAllTableReplicasToProcessLogEntry(zookeeper_path, entry, wait_for_non_active);
|
||||
Strings unfinished_replicas = tryWaitForAllReplicasToProcessLogEntry(table_zookeeper_path, entry, wait_for_inactive_timeout);
|
||||
if (unfinished_replicas.empty())
|
||||
return;
|
||||
|
||||
throw Exception(ErrorCodes::UNFINISHED, "{}Timeout exceeded while waiting for replicas {} to process entry {}. "
|
||||
"Probably some replicas are inactive", error_context, fmt::join(unfinished_replicas, ", "), entry.znode_name);
|
||||
}
|
||||
|
||||
void StorageReplicatedMergeTree::waitForLogEntryToBeProcessedIfNecessary(const ReplicatedMergeTreeLogEntryData & entry, ContextPtr query_context, const String & error_context)
|
||||
{
|
||||
/// If necessary, wait until the operation is performed on itself or on all replicas.
|
||||
Int64 wait_for_inactive_timeout = query_context->getSettingsRef().replication_wait_for_inactive_replica_timeout;
|
||||
if (query_context->getSettingsRef().replication_alter_partitions_sync == 1)
|
||||
{
|
||||
bool finished = tryWaitForReplicaToProcessLogEntry(zookeeper_path, replica_name, entry, wait_for_inactive_timeout);
|
||||
if (!finished)
|
||||
{
|
||||
throw Exception(ErrorCodes::UNFINISHED, "{}Log entry {} is not precessed on local replica, "
|
||||
"most likely because the replica was shut down.", error_context, entry.znode_name);
|
||||
}
|
||||
}
|
||||
else if (query_context->getSettingsRef().replication_alter_partitions_sync == 2)
|
||||
{
|
||||
waitForAllReplicasToProcessLogEntry(zookeeper_path, entry, wait_for_inactive_timeout, error_context);
|
||||
}
|
||||
}
|
||||
|
||||
bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry(
|
||||
const String & table_zookeeper_path, const String & replica, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active)
|
||||
bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry(
|
||||
const String & table_zookeeper_path, const String & replica, const ReplicatedMergeTreeLogEntryData & entry, Int64 wait_for_inactive_timeout)
|
||||
{
|
||||
String entry_str = entry.toString();
|
||||
String log_node_name;
|
||||
@ -5473,18 +5477,27 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry(
|
||||
*/
|
||||
|
||||
bool waiting_itself = replica == replica_name;
|
||||
/// Do not wait if timeout is zero
|
||||
bool wait_for_inactive = wait_for_inactive_timeout != 0;
|
||||
/// Wait for unlimited time if timeout is negative
|
||||
bool check_timeout = wait_for_inactive_timeout > 0;
|
||||
Stopwatch time_waiting;
|
||||
|
||||
const auto & stop_waiting = [&]()
|
||||
{
|
||||
bool stop_waiting_itself = waiting_itself && partial_shutdown_called;
|
||||
bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active");
|
||||
return is_dropped || stop_waiting_itself || stop_waiting_non_active;
|
||||
bool timeout_exceeded = check_timeout && wait_for_inactive_timeout < time_waiting.elapsedSeconds();
|
||||
bool stop_waiting_inactive = (!wait_for_inactive || timeout_exceeded)
|
||||
&& !getZooKeeper()->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active");
|
||||
return is_dropped || stop_waiting_itself || stop_waiting_inactive;
|
||||
};
|
||||
|
||||
/// Don't recheck ZooKeeper too often
|
||||
constexpr auto event_wait_timeout_ms = 3000;
|
||||
|
||||
if (startsWith(entry.znode_name, "log-"))
|
||||
if (!startsWith(entry.znode_name, "log-"))
|
||||
throw Exception("Logical error: unexpected name of log node: " + entry.znode_name, ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
{
|
||||
/// Take the number from the node name `log-xxxxxxxxxx`.
|
||||
UInt64 log_index = parse<UInt64>(entry.znode_name.substr(entry.znode_name.size() - 10));
|
||||
@ -5493,13 +5506,17 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry(
|
||||
LOG_DEBUG(log, "Waiting for {} to pull {} to queue", replica, log_node_name);
|
||||
|
||||
/// Let's wait until entry gets into the replica queue.
|
||||
bool pulled_to_queue = false;
|
||||
while (!stop_waiting())
|
||||
{
|
||||
zkutil::EventPtr event = std::make_shared<Poco::Event>();
|
||||
|
||||
String log_pointer = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer", nullptr, event);
|
||||
if (!log_pointer.empty() && parse<UInt64>(log_pointer) > log_index)
|
||||
{
|
||||
pulled_to_queue = true;
|
||||
break;
|
||||
}
|
||||
|
||||
/// Wait with timeout because we can be already shut down, but not dropped.
|
||||
/// So log_pointer node will exist, but we will never update it because all background threads already stopped.
|
||||
@ -5507,9 +5524,10 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry(
|
||||
/// but the query will never finish because the drop already shut down the table.
|
||||
event->tryWait(event_wait_timeout_ms);
|
||||
}
|
||||
|
||||
if (!pulled_to_queue)
|
||||
return false;
|
||||
}
|
||||
else
|
||||
throw Exception("Logical error: unexpected name of log node: " + entry.znode_name, ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
LOG_DEBUG(log, "Looking for node corresponding to {} in {} queue", log_node_name, replica);
|
||||
|
||||
@ -5547,13 +5565,6 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry(
|
||||
}
|
||||
|
||||
|
||||
bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(
|
||||
const String & replica, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active)
|
||||
{
|
||||
return waitForTableReplicaToProcessLogEntry(zookeeper_path, replica, entry, wait_for_non_active);
|
||||
}
|
||||
|
||||
|
||||
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
|
||||
{
|
||||
auto zookeeper = tryGetZooKeeper();
|
||||
@ -6562,13 +6573,10 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
|
||||
parts_to_remove.clear();
|
||||
cleanup_thread.wakeup();
|
||||
|
||||
/// If necessary, wait until the operation is performed on all replicas.
|
||||
if (query_context->getSettingsRef().replication_alter_partitions_sync > 1)
|
||||
{
|
||||
lock2.reset();
|
||||
lock1.reset();
|
||||
waitForAllReplicasToProcessLogEntry(entry);
|
||||
}
|
||||
lock2.reset();
|
||||
lock1.reset();
|
||||
|
||||
waitForLogEntryToBeProcessedIfNecessary(entry, query_context);
|
||||
}
|
||||
|
||||
void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context)
|
||||
@ -6767,12 +6775,9 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
|
||||
|
||||
parts_to_remove.clear();
|
||||
cleanup_thread.wakeup();
|
||||
lock2.reset();
|
||||
|
||||
if (query_context->getSettingsRef().replication_alter_partitions_sync > 1)
|
||||
{
|
||||
lock2.reset();
|
||||
dest_table_storage->waitForAllReplicasToProcessLogEntry(entry);
|
||||
}
|
||||
dest_table_storage->waitForLogEntryToBeProcessedIfNecessary(entry, query_context);
|
||||
|
||||
/// Create DROP_RANGE for the source table
|
||||
Coordination::Requests ops_src;
|
||||
@ -6787,11 +6792,8 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
|
||||
log_znode_path = dynamic_cast<const Coordination::CreateResponse &>(*op_results.front()).path_created;
|
||||
entry_delete.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
|
||||
|
||||
if (query_context->getSettingsRef().replication_alter_partitions_sync > 1)
|
||||
{
|
||||
lock1.reset();
|
||||
waitForAllReplicasToProcessLogEntry(entry_delete);
|
||||
}
|
||||
lock1.reset();
|
||||
waitForLogEntryToBeProcessedIfNecessary(entry_delete, query_context);
|
||||
|
||||
/// Cleaning possibly stored information about parts from /quorum/last_part node in ZooKeeper.
|
||||
cleanLastPartNode(partition_id);
|
||||
|
@ -635,22 +635,27 @@ private:
|
||||
const String & zookeeper_block_id_path = "", const String & zookeeper_path_prefix = "") const;
|
||||
|
||||
/** Wait until all replicas, including this, execute the specified action from the log.
|
||||
* If replicas are added at the same time, it can not wait the added replica .
|
||||
* If replicas are added at the same time, it can not wait the added replica.
|
||||
*
|
||||
* Waits for inactive replicas no more than wait_for_inactive_timeout.
|
||||
* Returns list of inactive replicas that have not executed entry or throws exception.
|
||||
*
|
||||
* NOTE: This method must be called without table lock held.
|
||||
* Because it effectively waits for other thread that usually has to also acquire a lock to proceed and this yields deadlock.
|
||||
* TODO: There are wrong usages of this method that are not fixed yet.
|
||||
*
|
||||
* One method for convenient use on current table, another for waiting on foreign shards.
|
||||
*/
|
||||
Strings waitForAllTableReplicasToProcessLogEntry(const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true);
|
||||
Strings waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true);
|
||||
void waitForAllReplicasToProcessLogEntry(const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry,
|
||||
Int64 wait_for_inactive_timeout, const String & error_context = {});
|
||||
Strings tryWaitForAllReplicasToProcessLogEntry(const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry,
|
||||
Int64 wait_for_inactive_timeout);
|
||||
|
||||
/** Wait until the specified replica executes the specified action from the log.
|
||||
* NOTE: See comment about locks above.
|
||||
*/
|
||||
bool waitForTableReplicaToProcessLogEntry(const String & table_zookeeper_path, const String & replica_name, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true);
|
||||
bool waitForReplicaToProcessLogEntry(const String & replica_name, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true);
|
||||
bool tryWaitForReplicaToProcessLogEntry(const String & table_zookeeper_path, const String & replica_name,
|
||||
const ReplicatedMergeTreeLogEntryData & entry, Int64 wait_for_inactive_timeout = 0);
|
||||
|
||||
/// Depending on settings, do nothing or wait for this replica or all replicas process log entry.
|
||||
void waitForLogEntryToBeProcessedIfNecessary(const ReplicatedMergeTreeLogEntryData & entry, ContextPtr query_context, const String & error_context = {});
|
||||
|
||||
/// Throw an exception if the table is readonly.
|
||||
void assertNotReadonly() const;
|
||||
|
6
tests/config/config.d/merge_tree_settings.xml
Normal file
6
tests/config/config.d/merge_tree_settings.xml
Normal file
@ -0,0 +1,6 @@
|
||||
<yandex>
|
||||
<merge_tree>
|
||||
<!-- 10 seconds (default is 1 minute) -->
|
||||
<zookeeper_session_expiration_check_period>10</zookeeper_session_expiration_check_period>
|
||||
</merge_tree>
|
||||
</yandex>
|
@ -28,6 +28,7 @@ ln -sf $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/graphite.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/database_atomic.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/max_concurrent_queries.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/merge_tree_settings.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/test_cluster_with_incorrect_pw.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/keeper_port.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/
|
||||
|
@ -6,6 +6,8 @@
|
||||
<http_receive_timeout>60</http_receive_timeout>
|
||||
<!-- 1 minute (default is 10 minutes) -->
|
||||
<insert_quorum_timeout>60000</insert_quorum_timeout>
|
||||
<!-- 30 seconds (default is 2 minutes) -->
|
||||
<replication_wait_for_inactive_replica_timeout>30</replication_wait_for_inactive_replica_timeout>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>
|
||||
|
@ -12,6 +12,7 @@ SYSTEM SYNC REPLICA byte_identical_r2;
|
||||
ALTER TABLE byte_identical_r1 ADD COLUMN y UInt64 DEFAULT rand();
|
||||
SYSTEM SYNC REPLICA byte_identical_r1;
|
||||
SYSTEM SYNC REPLICA byte_identical_r2;
|
||||
SET replication_alter_partitions_sync=2;
|
||||
OPTIMIZE TABLE byte_identical_r1 PARTITION tuple() FINAL;
|
||||
|
||||
SELECT x, t1.y - t2.y FROM byte_identical_r1 t1 SEMI LEFT JOIN byte_identical_r2 t2 USING x ORDER BY x;
|
||||
|
@ -98,6 +98,7 @@ SELECT '*** disable the feature';
|
||||
ALTER TABLE execute_on_single_replica_r1 MODIFY SETTING execute_merges_on_single_replica_time_threshold=0;
|
||||
ALTER TABLE execute_on_single_replica_r2 MODIFY SETTING execute_merges_on_single_replica_time_threshold=0;
|
||||
|
||||
SET replication_alter_partitions_sync=2;
|
||||
/* all_0_0_6 - we disabled the feature, both replicas will merge */
|
||||
OPTIMIZE TABLE execute_on_single_replica_r2 FINAL;
|
||||
/* all_0_0_7 - same */
|
||||
|
Loading…
Reference in New Issue
Block a user