enable some tests

This commit is contained in:
Alexander Tokmakov 2021-03-09 03:48:37 +03:00
parent 163e27b299
commit 5c5d11cfce
7 changed files with 50 additions and 150 deletions

View File

@ -231,8 +231,8 @@ void DatabaseOnDisk::createTable(
if (create.attach_short_syntax)
{
/// Metadata already exists, table was detached
removeDetachedPermanentlyFlag(context, table_name, table_metadata_path);
attachTable(table_name, table, getTableDataPath(create));
removeDetachedPermanentlyFlag(table_name, table_metadata_path);
return;
}
@ -270,12 +270,12 @@ void DatabaseOnDisk::createTable(
commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, context);
removeDetachedPermanentlyFlag(table_name, table_metadata_path);
removeDetachedPermanentlyFlag(context, table_name, table_metadata_path);
}
/// If the table was detached permanently we will have a flag file with
/// .sql.detached extension, is not needed anymore since we attached the table back
void DatabaseOnDisk::removeDetachedPermanentlyFlag(const String & table_name, const String & table_metadata_path) const
void DatabaseOnDisk::removeDetachedPermanentlyFlag(const Context &, const String & table_name, const String & table_metadata_path) const
{
try
{

View File

@ -94,11 +94,10 @@ protected:
virtual void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path, const Context & query_context);
virtual void removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path) const;
const String metadata_path;
const String data_path;
private:
void removeDetachedPermanentlyFlag(const String & table_name, const String & table_metadata_path) const;
};
}

View File

@ -712,12 +712,28 @@ void DatabaseReplicated::detachTablePermanently(const Context & context, const S
assert(!ddl_worker->isCurrentlyActive() || txn);
if (txn && txn->isInitialQuery())
{
/// We have to remove metadata from zookeeper, because we do not distinguish permanently detached tables
/// from attached tables when recovering replica.
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name);
txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1));
}
DatabaseAtomic::detachTablePermanently(context, table_name);
}
void DatabaseReplicated::removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path) const
{
auto txn = context.getZooKeeperMetadataTransaction();
assert(!ddl_worker->isCurrentlyActive() || txn);
if (txn && txn->isInitialQuery() && !txn->isExecuted())
{
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name);
String statement = readMetadataFile(table_name);
txn->addOp(zkutil::makeCreateRequest(metadata_zk_path, statement, zkutil::CreateMode::Persistent));
}
DatabaseAtomic::removeDetachedPermanentlyFlag(context, table_name, table_metadata_path);
}
String DatabaseReplicated::readMetadataFile(const String & table_name) const
{
String statement;

View File

@ -45,6 +45,7 @@ public:
const ASTPtr & query) override;
void removeDictionary(const Context & context, const String & dictionary_name) override;
void detachTablePermanently(const Context & context, const String & table_name) override;
void removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path) const override;
/// Try to execute DLL query on current host as initial query. If query is succeed,
/// then it will be executed on all replicas.

View File

@ -554,9 +554,9 @@ SELECT count() FROM test;
DROP TABLE IF EXISTS test_r1;
DROP TABLE IF EXISTS test_r2;
CREATE TABLE test_r1 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test', 'r1') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G';
CREATE TABLE test_r1 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test_01666', 'r1') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G';
INSERT INTO test_r1 SELECT * FROM test;
CREATE TABLE test_r2 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test', 'r2') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G';
CREATE TABLE test_r2 AS test ENGINE = ReplicatedMergeTree('/clickhouse/test_01666', 'r2') ORDER BY "\\" SETTINGS min_bytes_for_wide_part = '100G';
SYSTEM SYNC REPLICA test_r2;

View File

@ -106,163 +106,47 @@
"00738_lock_for_inner_table"
],
"database-replicated": [
/// Tests with DETACH TABLE (it's not allowed)
/// and tests with SET (session and query settings are not supported)
"memory_tracking",
"memory_usage",
"live_view",
"01413_alter_update_supertype",
"01149_zookeeper_mutation_stuck_after_replace_partition",
"00836_indices_alter_replicated_zookeeper",
"00652_mutations_alter_update",
"01715_tuple_insert_null_as_default",
"00825_protobuf_format_map",
"00152_insert_different_granularity",
"01715_background_checker_blather_zookeeper",
"01714_alter_drop_version",
"01114_materialize_clear_index_compact_parts",
"00814_replicated_minimalistic_part_header_zookeeper",
"01188_attach_table_from_pat",
"01415_sticking_mutations",
"01130_in_memory_parts",
"01110_dictionary_layout_without_arguments",
"01018_ddl_dictionaries_create",
"01018_ddl_dictionaries_select",
"01414_freeze_does_not_prevent_alters",
"01018_ddl_dictionaries_bad_queries",
"01686_rocksdb",
"01550_mutation_subquery",
"01070_mutations_with_dependencies",
"01070_materialize_ttl",
"01055_compact_parts",
"01017_mutations_with_nondeterministic_functions_zookeeper",
"00926_adaptive_index_granularity_pk",
"00910_zookeeper_test_alter_compression_codecs",
"00908_bloom_filter_index",
"00616_final_single_part",
"00446_clear_column_in_partition_zookeeper",
"01533_multiple_nested",
"01213_alter_rename_column_zookeeper",
"01575_disable_detach_table_of_dictionary",
"01457_create_as_table_function_structure",
"01415_inconsistent_merge_tree_settings",
"01413_allow_non_metadata_alters",
"01378_alter_rename_with_ttl_zookeeper",
"01349_mutation_datetime_key",
"01325_freeze_mutation_stuck",
"01272_suspicious_codecs",
"01181_db_atomic_drop_on_cluster",
"00957_delta_diff_bug",
"00910_zookeeper_custom_compression_codecs_replicated",
"00899_long_attach_memory_limit",
"00804_test_custom_compression_codes_log_storages",
"00804_test_alter_compression_codecs",
"00804_test_delta_codec_no_type_alter",
"00804_test_custom_compression_codecs",
"00753_alter_attach",
"00715_fetch_merged_or_mutated_part_zookeeper",
"00688_low_cardinality_serialization",
"01575_disable_detach_table_of_dictionary",
"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",
"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",
"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",
"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",
"01175_distributed_ddl_output_mode",
"01415_sticking_mutations",
"00980_zookeeper_merge_tree_alter_settings",
"00980_merge_alter_settings",
"01148_zookeeper_path_macros_unfolding",
"01019_alter_materialized_view_atomic",
"01019_alter_materialized_view_consistent",
"01018_ddl_dictionaries_bad_queries",
"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",
"00180_attach_materialized_view",
/// Unsupported type of ALTER query
"01650_fetch_patition_with_macro_in_zk_path",
"01451_detach_drop_part",
"01451_replicated_detach_drop_part",
"01417_freeze_partition_verbose",
"01417_freeze_partition_verbose_zookeeper",
"01130_in_memory_parts_partitons",
"01060_shutdown_table_after_detach",
"01021_only_tuple_columns",
"01015_attach_part",
"00626_replace_partition_from_table_zookeeper",
"00626_replace_partition_from_table",
"00625_arrays_in_nested",
/// Old syntax is not allowed
"01062_alter_on_mutataion_zookeeper",
"00925_zookeeper_empty_replicated_merge_tree_optimize_final",
"00754_alter_modify_order_by_replicated_zookeeper",
"00652_replicated_mutations_zookeeper",
"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",
"01720_constraints_complex_types",
"01747_alter_partition_key_enum_zookeeper"
/// Does not support renaming of multiple tables in single query
"00634_rename_view"
],
"polymorphic-parts": [
"01508_partition_pruning_long", /// bug, shoud be fixed