mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
disable by default
This commit is contained in:
parent
5e6d4b9449
commit
20e6593f96
@ -393,8 +393,8 @@ struct Settings : public SettingsCollection<Settings>
|
|||||||
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, 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(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(SettingDefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \
|
||||||
M(SettingBool, allow_experimental_database_atomic, /*false*/ true, "Allow to create database with Engine=Atomic.", 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, 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(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) \
|
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) \
|
||||||
|
@ -10,8 +10,6 @@
|
|||||||
#include <Storages/StorageMemory.h>
|
#include <Storages/StorageMemory.h>
|
||||||
#include <Core/BackgroundSchedulePool.h>
|
#include <Core/BackgroundSchedulePool.h>
|
||||||
#include <Parsers/formatAST.h>
|
#include <Parsers/formatAST.h>
|
||||||
#include <boost/algorithm/string.hpp>
|
|
||||||
#include <boost/algorithm/string/split.hpp>
|
|
||||||
#include <IO/ReadHelpers.h>
|
#include <IO/ReadHelpers.h>
|
||||||
#include <Poco/DirectoryIterator.h>
|
#include <Poco/DirectoryIterator.h>
|
||||||
|
|
||||||
@ -501,18 +499,25 @@ void DatabaseCatalog::loadMarkedAsDroppedTables()
|
|||||||
continue;
|
continue;
|
||||||
|
|
||||||
/// Process .sql files with metadata of tables which were marked as dropped
|
/// 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();
|
StorageID dropped_id = StorageID::createEmpty();
|
||||||
dropped_id.database_name = unescapeForFileName(name_parts[0]);
|
size_t dot_pos = it.name().find('.');
|
||||||
dropped_id.table_name = unescapeForFileName(name_parts[1]);
|
if (dot_pos == std::string::npos)
|
||||||
dropped_id.uuid = parse<UUID>(name_parts[2]);
|
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<UUID>(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));
|
dropped_metadata.emplace(std::move(full_path), std::move(dropped_id));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -102,10 +102,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
|||||||
engine->name = old_style_database ? "Ordinary" : "Atomic";
|
engine->name = old_style_database ? "Ordinary" : "Atomic";
|
||||||
storage->set(storage->engine, engine);
|
storage->set(storage->engine, engine);
|
||||||
create.set(create.storage, storage);
|
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()))
|
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);
|
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 database_name_escaped = escapeForFileName(database_name);
|
||||||
String path = context.getPath();
|
String path = context.getPath();
|
||||||
|
@ -14,8 +14,6 @@
|
|||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Interpreters/InterpreterShowCreateQuery.h>
|
#include <Interpreters/InterpreterShowCreateQuery.h>
|
||||||
|
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -73,10 +71,6 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
|||||||
if (!create_query && show_query && show_query->temporary)
|
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);
|
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<ASTCreateQuery &>();
|
|
||||||
create.uuid = UUIDHelpers::Nil;
|
|
||||||
|
|
||||||
std::stringstream stream;
|
std::stringstream stream;
|
||||||
formatAST(*create_query, stream, false, false);
|
formatAST(*create_query, stream, false, false);
|
||||||
String res = stream.str();
|
String res = stream.str();
|
||||||
|
@ -221,6 +221,7 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context,
|
|||||||
drop_query->database = target_table_id.database_name;
|
drop_query->database = target_table_id.database_name;
|
||||||
drop_query->table = target_table_id.table_name;
|
drop_query->table = target_table_id.table_name;
|
||||||
drop_query->kind = kind;
|
drop_query->kind = kind;
|
||||||
|
drop_query->no_delay = true;
|
||||||
ASTPtr ast_drop_query = drop_query;
|
ASTPtr ast_drop_query = drop_query;
|
||||||
InterpreterDropQuery drop_interpreter(ast_drop_query, global_context);
|
InterpreterDropQuery drop_interpreter(ast_drop_query, global_context);
|
||||||
drop_interpreter.execute();
|
drop_interpreter.execute();
|
||||||
|
@ -19,7 +19,7 @@ INSERT INTO set2 VALUES ('abc'), ('World');
|
|||||||
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
|
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
|
||||||
|
|
||||||
DETACH TABLE 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;
|
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
|
||||||
|
|
||||||
|
@ -6,8 +6,7 @@ CREATE TABLE t_00180 (x UInt8) ENGINE = Null;
|
|||||||
CREATE MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180;
|
CREATE MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180;
|
||||||
|
|
||||||
DETACH TABLE mv_00180;
|
DETACH TABLE mv_00180;
|
||||||
--ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180;
|
ATTACH MATERIALIZED VIEW mv_00180 ENGINE = Null AS SELECT * FROM t_00180;
|
||||||
ATTACH TABLE mv_00180;
|
|
||||||
|
|
||||||
DROP TABLE t_00180;
|
DROP TABLE t_00180;
|
||||||
DROP TABLE mv_00180;
|
DROP TABLE mv_00180;
|
||||||
|
@ -21,7 +21,7 @@ INSERT INTO deduplication (x) VALUES (1);
|
|||||||
SELECT * FROM deduplication;
|
SELECT * FROM deduplication;
|
||||||
|
|
||||||
DETACH TABLE 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;
|
SELECT * FROM deduplication;
|
||||||
|
|
||||||
|
@ -11,7 +11,7 @@ INSERT INTO array_pk VALUES ([5, 6], 'ghi', 6);
|
|||||||
SELECT * FROM array_pk ORDER BY n;
|
SELECT * FROM array_pk ORDER BY n;
|
||||||
|
|
||||||
DETACH TABLE array_pk;
|
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;
|
SELECT * FROM array_pk ORDER BY n;
|
||||||
|
|
||||||
|
@ -5,7 +5,7 @@ SELECT * FROM log LIMIT 1;
|
|||||||
SELECT * FROM log;
|
SELECT * FROM log;
|
||||||
|
|
||||||
DETACH TABLE log;
|
DETACH TABLE log;
|
||||||
ATTACH TABLE log; -- (s String) ENGINE = Log;
|
ATTACH TABLE log (s String) ENGINE = Log;
|
||||||
|
|
||||||
SELECT * FROM log;
|
SELECT * FROM log;
|
||||||
SELECT * FROM log LIMIT 1;
|
SELECT * FROM log LIMIT 1;
|
||||||
@ -15,13 +15,13 @@ INSERT INTO log VALUES ('Hello'), ('World');
|
|||||||
SELECT * FROM log LIMIT 1;
|
SELECT * FROM log LIMIT 1;
|
||||||
|
|
||||||
DETACH TABLE log;
|
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 LIMIT 1;
|
||||||
SELECT * FROM log;
|
SELECT * FROM log;
|
||||||
|
|
||||||
DETACH TABLE log;
|
DETACH TABLE log;
|
||||||
ATTACH TABLE log; -- (s String) ENGINE = Log;
|
ATTACH TABLE log (s String) ENGINE = Log;
|
||||||
|
|
||||||
SELECT * FROM log;
|
SELECT * FROM log;
|
||||||
SELECT * FROM log LIMIT 1;
|
SELECT * FROM log LIMIT 1;
|
||||||
|
@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|||||||
. $CURDIR/../shell_config.sh
|
. $CURDIR/../shell_config.sh
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107"
|
$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 "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)" &
|
$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 "ATTACH TABLE test_01107.mt"
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM 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 "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 "SELECT count(n), sum(n) FROM test_01107.mt"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(100)" &
|
$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(100)" &
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
DROP DATABASE IF EXISTS test_01109;
|
DROP DATABASE IF EXISTS test_01109;
|
||||||
|
SET allow_experimental_database_atomic=1;
|
||||||
CREATE DATABASE test_01109 ENGINE=Atomic;
|
CREATE DATABASE test_01109 ENGINE=Atomic;
|
||||||
|
|
||||||
USE test_01109;
|
USE test_01109;
|
||||||
|
@ -6,11 +6,11 @@ test_01114_2 Atomic store metadata test_01114_2
|
|||||||
test_01114_3 Ordinary test_01114_3 metadata test_01114_3
|
test_01114_3 Ordinary test_01114_3 metadata test_01114_3
|
||||||
20
|
20
|
||||||
100
|
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
|
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-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
|
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
|
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_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\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
|
5
|
||||||
20 190
|
20 190
|
||||||
30 435
|
30 435
|
||||||
|
@ -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 "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=Atomic --allow_experimental_database_atomic=1 -q "CREATE DATABASE test_01114_2"
|
||||||
$CLICKHOUSE_CLIENT --default_database_engine=Ordinary -q "CREATE DATABASE test_01114_3"
|
$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 "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 "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 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 "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) FROM numbers(30)" & # 30s
|
$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
|
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"
|
$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
|
# Check that nothing changed
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt"
|
$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 "SHOW CREATE TABLE test_01114_2.mt"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "DROP TABLE test_01114_1.mt"
|
$CLICKHOUSE_CLIENT -q "DROP TABLE test_01114_1.mt"
|
||||||
|
@ -15,7 +15,7 @@ LIFETIME(MIN 0 MAX 0)
|
|||||||
LAYOUT(FLAT());
|
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.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;
|
||||||
DROP DATABASE dict_db_01225_dictionary;
|
DROP DATABASE dict_db_01225_dictionary;
|
||||||
|
@ -24,7 +24,7 @@ GROUP BY loyalty
|
|||||||
ORDER BY loyalty ASC;
|
ORDER BY loyalty ASC;
|
||||||
|
|
||||||
DETACH TABLE join;
|
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
|
SELECT
|
||||||
loyalty,
|
loyalty,
|
||||||
|
Loading…
Reference in New Issue
Block a user