diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 534d53164ad..2172adec83c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -393,8 +393,8 @@ struct Settings : public SettingsCollection M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \ M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ - M(SettingDefaultDatabaseEngine, default_database_engine, /*DefaultDatabaseEngine::Ordinary*/ DefaultDatabaseEngine::Atomic, "Default database engine.", 0) \ - M(SettingBool, allow_experimental_database_atomic, /*false*/ true, "Allow to create database with Engine=Atomic.", 0) \ + M(SettingDefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \ + M(SettingBool, allow_experimental_database_atomic, false, "Allow to create database with Engine=Atomic.", 0) \ M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(SettingUInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 8ab5e22225a..99c7201993e 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -10,8 +10,6 @@ #include #include #include -#include -#include #include #include @@ -501,18 +499,25 @@ void DatabaseCatalog::loadMarkedAsDroppedTables() continue; /// Process .sql files with metadata of tables which were marked as dropped - String full_path = path + it.name(); - - Strings name_parts; - boost::split(name_parts, it.name(), boost::is_any_of(".")); // NOLINT: LLVM Bug 41141 - if (name_parts.size() != 4) /// Unexpected file - continue; - StorageID dropped_id = StorageID::createEmpty(); - dropped_id.database_name = unescapeForFileName(name_parts[0]); - dropped_id.table_name = unescapeForFileName(name_parts[1]); - dropped_id.uuid = parse(name_parts[2]); + size_t dot_pos = it.name().find('.'); + if (dot_pos == std::string::npos) + continue; + dropped_id.database_name = unescapeForFileName(it.name().substr(0, dot_pos)); + size_t prev_dot_pos = dot_pos; + dot_pos = it.name().find('.', prev_dot_pos + 1); + if (dot_pos == std::string::npos) + continue; + dropped_id.table_name = unescapeForFileName(it.name().substr(prev_dot_pos + 1, dot_pos - prev_dot_pos - 1)); + + prev_dot_pos = dot_pos; + dot_pos = it.name().find('.', prev_dot_pos + 1); + if (dot_pos == std::string::npos) + continue; + dropped_id.uuid = parse(it.name().substr(prev_dot_pos + 1, dot_pos - prev_dot_pos - 1)); + + String full_path = path + it.name(); dropped_metadata.emplace(std::move(full_path), std::move(dropped_id)); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6df8d7d9298..a1a4654a1d6 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -102,10 +102,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) engine->name = old_style_database ? "Ordinary" : "Atomic"; storage->set(storage->engine, engine); create.set(create.storage, storage); - - if (!context.getSettingsRef().allow_experimental_database_atomic) - throw Exception("Atomic is an experimental database engine. Enable allow_experimental_database_atomic to use it.", - ErrorCodes::UNKNOWN_DATABASE_ENGINE); } else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty())) { @@ -115,6 +111,9 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE); } + if (create.storage->engine->name == "Atomic" && !context.getSettingsRef().allow_experimental_database_atomic && !internal) + throw Exception("Atomic is an experimental database engine. Enable allow_experimental_database_atomic to use it.", + ErrorCodes::UNKNOWN_DATABASE_ENGINE); String database_name_escaped = escapeForFileName(database_name); String path = context.getPath(); diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 11d73159775..7f26666a1ae 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -14,8 +14,6 @@ #include #include -#include - namespace DB { @@ -73,10 +71,6 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() if (!create_query && show_query && show_query->temporary) throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY); - //FIXME temporary print create query without UUID for tests (remove it) - auto & create = create_query->as(); - create.uuid = UUIDHelpers::Nil; - std::stringstream stream; formatAST(*create_query, stream, false, false); String res = stream.str(); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index ab1c1865411..cf633e913e6 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -221,6 +221,7 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, drop_query->database = target_table_id.database_name; drop_query->table = target_table_id.table_name; drop_query->kind = kind; + drop_query->no_delay = true; ASTPtr ast_drop_query = drop_query; InterpreterDropQuery drop_interpreter(ast_drop_query, global_context); drop_interpreter.execute(); diff --git a/tests/queries/0_stateless/00116_storage_set.sql b/tests/queries/0_stateless/00116_storage_set.sql index 2b573c87f4a..aa93a0620d0 100644 --- a/tests/queries/0_stateless/00116_storage_set.sql +++ b/tests/queries/0_stateless/00116_storage_set.sql @@ -19,7 +19,7 @@ INSERT INTO set2 VALUES ('abc'), ('World'); SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2; DETACH TABLE set2; -ATTACH TABLE set2; -- (x String) ENGINE = Set; +ATTACH TABLE set2 (x String) ENGINE = Set; SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2; diff --git a/tests/queries/0_stateless/00180_attach_materialized_view.sql b/tests/queries/0_stateless/00180_attach_materialized_view.sql index 0e20b15a438..089e4926bcf 100644 --- a/tests/queries/0_stateless/00180_attach_materialized_view.sql +++ b/tests/queries/0_stateless/00180_attach_materialized_view.sql @@ -6,8 +6,7 @@ CREATE TABLE t_00180 (x UInt8) ENGINE = Null; CREATE MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; DETACH TABLE mv_00180; ---ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; -ATTACH TABLE mv_00180; +ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180; DROP TABLE t_00180; DROP TABLE mv_00180; diff --git a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql index 1abe7d54610..6b29d0a8cd3 100644 --- a/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql +++ b/tests/queries/0_stateless/00226_zookeeper_deduplication_and_unexpected_parts.sql @@ -21,7 +21,7 @@ INSERT INTO deduplication (x) VALUES (1); SELECT * FROM deduplication; DETACH TABLE deduplication; -ATTACH TABLE deduplication;-- (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1); +ATTACH TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1); SELECT * FROM deduplication; diff --git a/tests/queries/0_stateless/00311_array_primary_key.sql b/tests/queries/0_stateless/00311_array_primary_key.sql index 0598583c411..0ea368609da 100644 --- a/tests/queries/0_stateless/00311_array_primary_key.sql +++ b/tests/queries/0_stateless/00311_array_primary_key.sql @@ -11,7 +11,7 @@ INSERT INTO array_pk VALUES ([5, 6], 'ghi', 6); SELECT * FROM array_pk ORDER BY n; DETACH TABLE array_pk; -ATTACH TABLE array_pk; -- (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); +ATTACH TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1); SELECT * FROM array_pk ORDER BY n; diff --git a/tests/queries/0_stateless/00423_storage_log_single_thread.sql b/tests/queries/0_stateless/00423_storage_log_single_thread.sql index 512118dd55b..7d5e14c9ee5 100644 --- a/tests/queries/0_stateless/00423_storage_log_single_thread.sql +++ b/tests/queries/0_stateless/00423_storage_log_single_thread.sql @@ -5,7 +5,7 @@ SELECT * FROM log LIMIT 1; SELECT * FROM log; DETACH TABLE log; -ATTACH TABLE log; -- (s String) ENGINE = Log; +ATTACH TABLE log (s String) ENGINE = Log; SELECT * FROM log; SELECT * FROM log LIMIT 1; @@ -15,13 +15,13 @@ INSERT INTO log VALUES ('Hello'), ('World'); SELECT * FROM log LIMIT 1; DETACH TABLE log; -ATTACH TABLE log; -- (s String) ENGINE = Log; +ATTACH TABLE log (s String) ENGINE = Log; SELECT * FROM log LIMIT 1; SELECT * FROM log; DETACH TABLE log; -ATTACH TABLE log; -- (s String) ENGINE = Log; +ATTACH TABLE log (s String) ENGINE = Log; SELECT * FROM log; SELECT * FROM log LIMIT 1; diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index 3720a8907b5..6ff36318d30 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" +$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" $CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & @@ -18,7 +18,7 @@ wait $CLICKHOUSE_CLIENT -q "ATTACH TABLE test_01107.mt" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" $CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" -$CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107 ENGINE=Atomic" +$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "ATTACH DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" $CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(100)" & diff --git a/tests/queries/0_stateless/01109_exchange_tables.sql b/tests/queries/0_stateless/01109_exchange_tables.sql index 7fbb36e8ce9..7125bfea851 100644 --- a/tests/queries/0_stateless/01109_exchange_tables.sql +++ b/tests/queries/0_stateless/01109_exchange_tables.sql @@ -1,4 +1,5 @@ DROP DATABASE IF EXISTS test_01109; +SET allow_experimental_database_atomic=1; CREATE DATABASE test_01109 ENGINE=Atomic; USE test_01109; diff --git a/tests/queries/0_stateless/01114_database_atomic.reference b/tests/queries/0_stateless/01114_database_atomic.reference index 3cef8bd8bc1..a6c9fc3c4f4 100644 --- a/tests/queries/0_stateless/01114_database_atomic.reference +++ b/tests/queries/0_stateless/01114_database_atomic.reference @@ -6,11 +6,11 @@ test_01114_2 Atomic store metadata test_01114_2 test_01114_3 Ordinary test_01114_3 metadata test_01114_3 20 100 -CREATE TABLE test_01114_2.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -mt 00001114-0000-4000-8000-000000000001 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000001\' (`n` UInt64) ENGINE = MergeTree() PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 +CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +mt 00001114-0000-4000-8000-000000000002 CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\' (`n` UInt64) ENGINE = MergeTree() PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192 20 -CREATE TABLE test_01114_1.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01114_2.mt\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01114_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree()\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192 5 20 190 30 435 diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 6754a79beda..9cd624ad4b0 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -8,7 +8,7 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_2" $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_3" -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" +$CLICKHOUSE_CLIENT --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" $CLICKHOUSE_CLIENT --default_database_engine=Atomic --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_2" $CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01114_3" @@ -29,13 +29,13 @@ $CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_3.mt_tmp" $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_3" -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '00001114-0000-4000-8000-000000000001' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '00001114-0000-4000-8000-000000000002' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" $CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_2.mt" $CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" -$CLICKHOUSE_CLIENT -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1) AS col FROM test_01114_1.mt)" & # 20s, result: 20, 190 -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1) FROM numbers(30)" & # 30s +$CLICKHOUSE_CLIENT -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 30s, result: 20, 190 +$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s sleep 1 # SELECT and INSERT should start before the following RENAMEs $CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp" @@ -46,7 +46,8 @@ $CLICKHOUSE_CLIENT -q "EXCHANGE TABLES test_01114_1.mt AND test_01114_2.mt" # Check that nothing changed $CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" -$CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_1.mt" +uuid_mt1=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE database='test_01114_1' AND name='mt'"` +$CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_1.mt" | sed "s/$uuid_mt1/00001114-0000-4000-8000-000000000001/g" $CLICKHOUSE_CLIENT -q "SHOW CREATE TABLE test_01114_2.mt" $CLICKHOUSE_CLIENT -q "DROP TABLE test_01114_1.mt" diff --git a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql index 7550d5292d0..6c1ae2b907b 100644 --- a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql +++ b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql @@ -15,7 +15,7 @@ LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.dict` FORMAT TSVRaw; -SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.no_such_dict`; -- { serverError 36; } +SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.no_such_dict`; -- { serverError 60; } DROP DATABASE dict_db_01225; DROP DATABASE dict_db_01225_dictionary; diff --git a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql index b03013c11fe..d3e73faa7be 100644 --- a/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql +++ b/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql @@ -24,7 +24,7 @@ GROUP BY loyalty ORDER BY loyalty ASC; DETACH TABLE join; -ATTACH TABLE join;-- (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); +ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID); SELECT loyalty,