提交 066fb4c8 编写于 作者: A Alexander Tokmakov

fix

上级 87502d02
......@@ -93,7 +93,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
LOG_DEBUG(log, "Waiting for worker thread to process all entries before {}", entry_name);
{
std::unique_lock lock{mutex};
wait_current_task_change.wait(lock, [&]() { assert(current_task <= entry_name); return zookeeper->expired() || current_task == entry_name; });
wait_current_task_change.wait(lock, [&]() { assert(zookeeper->expired() || current_task <= entry_name); return zookeeper->expired() || current_task == entry_name; });
}
if (zookeeper->expired())
......
......@@ -357,7 +357,7 @@ void DDLWorker::scheduleTasks()
if (!task)
{
LOG_DEBUG(log, "Will not execute task {}: {}", entry_name, reason);
updateMaxDDLEntryID(*task);
updateMaxDDLEntryID(entry_name);
continue;
}
......@@ -449,9 +449,9 @@ bool DDLWorker::tryExecuteQuery(const String & query, DDLTaskBase & task)
return true;
}
void DDLWorker::updateMaxDDLEntryID(const DDLTaskBase & task)
void DDLWorker::updateMaxDDLEntryID(const String & entry_name)
{
DB::ReadBufferFromString in(task.entry_name);
DB::ReadBufferFromString in(entry_name);
DB::assertString("query-", in);
UInt64 id;
readText(id, in);
......@@ -511,6 +511,7 @@ void DDLWorker::processTask(DDLTaskBase & task)
if (task.execute_on_leader)
{
tryExecuteQueryOnLeaderReplica(task, storage, rewritten_query, task.entry_path, zookeeper);
}
else
{
......@@ -549,7 +550,7 @@ void DDLWorker::processTask(DDLTaskBase & task)
task.was_executed = true;
}
updateMaxDDLEntryID(task);
updateMaxDDLEntryID(task.entry_name);
/// FIXME: if server fails right here, the task will be executed twice. We need WAL here.
/// If ZooKeeper connection is lost here, we will try again to write query status.
......
......@@ -73,7 +73,7 @@ protected:
virtual DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper);
void processTask(DDLTaskBase & task);
void updateMaxDDLEntryID(const DDLTaskBase & task);
void updateMaxDDLEntryID(const String & entry_name);
/// Check that query should be executed on leader replica only
static bool taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, StoragePtr storage);
......
......@@ -976,12 +976,10 @@ void DDLGuard::releaseTableLock() noexcept
table_lock_removed = true;
guards_lock.lock();
--it->second.counter;
if (!it->second.counter)
{
UInt32 counter = --it->second.counter;
table_lock.unlock();
if (counter == 0)
map.erase(it);
}
guards_lock.unlock();
}
......
......@@ -33,7 +33,7 @@ bool isSupportedAlterType(int type)
{
assert(type != ASTAlterCommand::NO_TYPE);
static const std::unordered_set<int> unsupported_alter_types{
/// It's dangerous, because it may duplicate data if executed on multiple replicas
/// It's dangerous, because it may duplicate data if executed on multiple replicas. We can allow it after #18978
ASTAlterCommand::ATTACH_PARTITION,
/// Usually followed by ATTACH PARTITION
ASTAlterCommand::FETCH_PARTITION,
......
......@@ -18,3 +18,6 @@ yes 'SELECT 1' 2>/dev/null | {
} | grep -x -c 1
wait
# Reset max_memory_usage_for_user, so it will not affect other tests
${CLICKHOUSE_CLIENT} --max_memory_usage_for_user=0 -q "SELECT 1 FORMAT Null"
......@@ -42,3 +42,6 @@ execute_group_by
# if memory accounting will be incorrect, the second query will be failed with MEMORY_LIMIT_EXCEEDED
execute_group_by
wait
# Reset max_memory_usage_for_user, so it will not affect other tests
${CLICKHOUSE_CLIENT} --max_memory_usage_for_user=0 -q "SELECT 1 FORMAT Null"
......@@ -66,4 +66,7 @@ echo 'OK'
${CLICKHOUSE_CLIENT} --query "DROP USER test_01541";
# Reset max_memory_usage_for_user, so it will not affect other tests
${CLICKHOUSE_CLIENT} --max_memory_usage_for_user=0 -q "SELECT 1 FORMAT Null"
exit 0
......@@ -100,7 +100,133 @@
"00604_show_create_database",
"00609_mv_index_in_in",
"00510_materizlized_view_and_deduplication_zookeeper",
"00738_lock_for_inner_table"
"memory_tracking", /// FIXME remove it before merge
"memory_tracking",
"memory_usage",
"00738_lock_for_inner_table",
"01666_blns",
"01652_ignore_and_low_cardinality",
"01651_map_functions",
"01650_fetch_patition_with_macro_in_zk_path",
"01648_mutations_and_escaping",
"01640_marks_corruption_regression",
"01622_byte_size",
"01611_string_to_low_cardinality_key_alter",
"01602_show_create_view",
"01600_log_queries_with_extensive_info",
"01560_ttl_remove_empty_parts",
"01554_bloom_filter_index_big_integer_uuid",
"01550_type_map_formats_input",
"01550_type_map_formats",
"01550_create_map_type",
"01532_primary_key_without_order_by_zookeeper",
"01511_alter_version_versioned_collapsing_merge_tree_zookeeper",
"01509_parallel_quorum_insert_no_replicas",
"01504_compression_multiple_streams",
"01494_storage_join_persistency",
"01493_storage_set_persistency",
"01493_alter_remove_properties_zookeeper",
"01475_read_subcolumns_storages",
"01475_read_subcolumns",
"01463_test_alter_live_view_refresh",
"01451_replicated_detach_drop_part",
"01451_detach_drop_part",
"01440_big_int_exotic_casts",
"01430_modify_sample_by_zookeeper",
"01417_freeze_partition_verbose_zookeeper",
"01417_freeze_partition_verbose",
"01396_inactive_replica_cleanup_nodes_zookeeper",
"01375_compact_parts_codecs",
"01357_version_collapsing_attach_detach_zookeeper",
"01355_alter_column_with_order",
"01291_geo_types",
"01270_optimize_skip_unused_shards_low_cardinality",
"01237_live_view_over_distributed_with_subquery_select_table_alias",
"01236_distributed_over_live_view_over_distributed",
"01235_live_view_over_distributed",
"01182_materialized_view_different_structure",
"01150_ddl_guard_rwr",
"01148_zookeeper_path_macros_unfolding",
"01135_default_and_alter_zookeeper",
"01130_in_memory_parts_partitons",
"01127_month_partitioning_consistency_select",
"01114_database_atomic",
"01083_expressions_in_engine_arguments",
"01073_attach_if_not_exists",
"01072_optimize_skip_unused_shards_const_expr_eval",
"01071_prohibition_secondary_index_with_old_format_merge_tree",
"01071_live_view_detach_dependency",
"01062_alter_on_mutataion_zookeeper",
"01060_shutdown_table_after_detach",
"01056_create_table_as",
"01035_avg",
"01021_only_tuple_columns",
"01019_alter_materialized_view_query",
"01019_alter_materialized_view_consistent",
"01019_alter_materialized_view_atomic",
"01015_attach_part",
"00989_parallel_parts_loading",
"00980_zookeeper_merge_tree_alter_settings",
"00980_merge_alter_settings",
"00980_create_temporary_live_view",
"00978_live_view_watch",
"00977_live_view_watch_events",
"00976_live_view_select_version",
"00975_live_view_create",
"00974_live_view_select_with_aggregation",
"00973_live_view_with_subquery_select_with_aggregation_in_subquery",
"00973_live_view_with_subquery_select_with_aggregation",
"00973_live_view_with_subquery_select_table_alias",
"00973_live_view_with_subquery_select_nested_with_aggregation_table_alias",
"00973_live_view_with_subquery_select_nested_with_aggregation",
"00973_live_view_with_subquery_select_nested",
"00973_live_view_with_subquery_select_join_no_alias",
"00973_live_view_with_subquery_select_join",
"00973_live_view_with_subquery_select",
"00973_live_view_select_prewhere",
"00973_live_view_select",
"00972_live_view_select_1",
"00969_live_view_watch_format_jsoneachrowwithprogress",
"00968_live_view_select_format_jsoneachrowwithprogress",
"00961_temporary_live_view_watch",
"00955_test_final_mark",
"00933_reserved_word",
"00926_zookeeper_adaptive_index_granularity_replicated_merge_tree",
"00926_adaptive_index_granularity_replacing_merge_tree",
"00926_adaptive_index_granularity_merge_tree",
"00925_zookeeper_empty_replicated_merge_tree_optimize_final",
"00800_low_cardinality_distinct_numeric",
"00754_alter_modify_order_by_replicated_zookeeper",
"00751_low_cardinality_nullable_group_by",
"00751_default_databasename_for_view",
"00719_parallel_ddl_table",
"00718_low_cardinaliry_alter",
"00717_low_cardinaliry_distributed_group_by",
"00688_low_cardinality_syntax",
"00688_low_cardinality_nullable_cast",
"00688_low_cardinality_in",
"00652_replicated_mutations_zookeeper",
"00634_rename_view",
"00626_replace_partition_from_table",
"00625_arrays_in_nested",
"00623_replicated_truncate_table_zookeeper",
"00619_union_highlite",
"00599_create_view_with_subquery",
"00571_non_exist_database_when_create_materializ_view",
"00553_buff_exists_materlized_column",
"00516_deduplication_after_drop_partition_zookeeper",
"00508_materialized_view_to",
"00446_clear_column_in_partition_concurrent_zookeeper",
"00423_storage_log_single_thread",
"00311_array_primary_key",
"00236_replicated_drop_on_non_leader_zookeeper",
"00226_zookeeper_deduplication_and_unexpected_parts",
"00215_primary_key_order_zookeeper",
"00180_attach_materialized_view",
"00121_drop_column_zookeeper",
"00116_storage_set",
"00083_create_merge_tree_zookeeper",
"00062_replicated_merge_tree_alter_zookeeper"
],
"polymorphic-parts": [
"01508_partition_pruning", /// bug, shoud be fixed
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册