mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
enable more tests with Atomic database
This commit is contained in:
parent
37236e5923
commit
b68782d285
@ -28,7 +28,7 @@ def get_options(i):
|
||||
options = ""
|
||||
if 0 < i:
|
||||
options += " --order=random"
|
||||
if i == 1:
|
||||
if i % 2 == 1:
|
||||
options += " --atomic-db-engine"
|
||||
return options
|
||||
|
||||
|
@ -919,7 +919,33 @@ private:
|
||||
while (begin < end)
|
||||
{
|
||||
const char * pos = begin;
|
||||
ASTPtr orig_ast = parseQuery(pos, end, true);
|
||||
|
||||
ASTPtr orig_ast;
|
||||
try
|
||||
{
|
||||
orig_ast = parseQuery(pos, end, true);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
if (!test_mode)
|
||||
throw;
|
||||
|
||||
/// Try find test hint for syntax error
|
||||
const char * end_of_line = find_first_symbols<'\n'>(begin, end);
|
||||
TestHint hint(true, String(begin, end_of_line - begin));
|
||||
if (hint.serverError()) /// Syntax errors are considered as client errors
|
||||
throw;
|
||||
if (hint.clientError() != e.code())
|
||||
{
|
||||
if (hint.clientError())
|
||||
e.addMessage("\nExpected clinet error: " + std::to_string(hint.clientError()));
|
||||
throw;
|
||||
}
|
||||
|
||||
/// It's expected syntax error, skip the line
|
||||
begin = end_of_line;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!orig_ast)
|
||||
{
|
||||
|
@ -657,7 +657,10 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr
|
||||
/// Table was removed from database. Enqueue removal of its data from disk.
|
||||
time_t drop_time;
|
||||
if (table)
|
||||
{
|
||||
drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
|
||||
table->is_dropped = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Try load table from metadata to drop it correctly (e.g. remove metadata from zk or remove data from all volumes)
|
||||
@ -674,6 +677,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr
|
||||
try
|
||||
{
|
||||
table = createTableFromAST(*create, table_id.getDatabaseName(), data_path, *global_context, false).second;
|
||||
table->is_dropped = true;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -763,7 +767,6 @@ void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) const
|
||||
if (table.table)
|
||||
{
|
||||
table.table->drop();
|
||||
table.table->is_dropped = true;
|
||||
}
|
||||
|
||||
/// Even if table is not loaded, try remove its data from disk.
|
||||
|
@ -673,6 +673,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
create.attach_short_syntax = true;
|
||||
create.if_not_exists = if_not_exists;
|
||||
}
|
||||
/// TODO maybe assert table structure if create.attach_short_syntax is false?
|
||||
|
||||
if (!create.temporary && create.database.empty())
|
||||
create.database = current_database;
|
||||
|
@ -66,6 +66,7 @@ void MergeTreeWriteAheadLog::dropPart(const String & part_name)
|
||||
writeIntBinary(static_cast<UInt8>(0), *out);
|
||||
writeIntBinary(static_cast<UInt8>(ActionType::DROP_PART), *out);
|
||||
writeStringBinary(part_name, *out);
|
||||
out->next();
|
||||
}
|
||||
|
||||
void MergeTreeWriteAheadLog::rotate(const std::lock_guard<std::mutex> &)
|
||||
|
@ -4260,9 +4260,13 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(
|
||||
* To do this, check its node `log_pointer` - the maximum number of the element taken from `log` + 1.
|
||||
*/
|
||||
|
||||
const auto & check_replica_become_inactive = [this, &replica]()
|
||||
bool waiting_itself = replica == replica_name;
|
||||
|
||||
const auto & stop_waiting = [&]()
|
||||
{
|
||||
return !getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/is_active");
|
||||
bool stop_waiting_itself = waiting_itself && is_dropped;
|
||||
bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/is_active");
|
||||
return stop_waiting_itself || stop_waiting_non_active;
|
||||
};
|
||||
constexpr auto event_wait_timeout_ms = 1000;
|
||||
|
||||
@ -4277,7 +4281,7 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(
|
||||
LOG_DEBUG(log, "Waiting for {} to pull {} to queue", replica, log_node_name);
|
||||
|
||||
/// Let's wait until entry gets into the replica queue.
|
||||
while (wait_for_non_active || !check_replica_become_inactive())
|
||||
while (!stop_waiting())
|
||||
{
|
||||
zkutil::EventPtr event = std::make_shared<Poco::Event>();
|
||||
|
||||
@ -4325,7 +4329,7 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(
|
||||
LOG_DEBUG(log, "Waiting for {} to pull {} to queue", replica, log_node_name);
|
||||
|
||||
/// Let's wait until the entry gets into the replica queue.
|
||||
while (wait_for_non_active || !check_replica_become_inactive())
|
||||
while (!stop_waiting())
|
||||
{
|
||||
zkutil::EventPtr event = std::make_shared<Poco::Event>();
|
||||
|
||||
@ -4378,10 +4382,8 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(
|
||||
|
||||
/// Third - wait until the entry disappears from the replica queue or replica become inactive.
|
||||
String path_to_wait_on = zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for;
|
||||
if (wait_for_non_active)
|
||||
return getZooKeeper()->waitForDisappear(path_to_wait_on);
|
||||
|
||||
return getZooKeeper()->waitForDisappear(path_to_wait_on, check_replica_become_inactive);
|
||||
return getZooKeeper()->waitForDisappear(path_to_wait_on, stop_waiting);
|
||||
}
|
||||
|
||||
|
||||
|
@ -344,6 +344,12 @@ protected:
|
||||
{
|
||||
ASTPtr ast = database->tryGetCreateTableQuery(table_name, context);
|
||||
|
||||
if (ast && !context.getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil)
|
||||
{
|
||||
auto & create = ast->as<ASTCreateQuery &>();
|
||||
create.uuid = UUIDHelpers::Nil;
|
||||
}
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(ast ? queryToString(ast) : "");
|
||||
|
||||
|
@ -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;
|
||||
|
||||
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
|
||||
|
||||
|
@ -6,7 +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;
|
||||
|
||||
DROP TABLE t_00180;
|
||||
DROP TABLE mv_00180;
|
||||
|
@ -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_00226/deduplication', 'r1', d, x, 1);
|
||||
ATTACH TABLE deduplication;
|
||||
|
||||
SELECT * FROM deduplication;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
SELECT * FROM array_pk ORDER BY n;
|
||||
|
||||
|
@ -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;
|
||||
|
||||
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;
|
||||
|
||||
SELECT * FROM log LIMIT 1;
|
||||
SELECT * FROM log;
|
||||
|
||||
DETACH TABLE log;
|
||||
ATTACH TABLE log (s String) ENGINE = Log;
|
||||
ATTACH TABLE log;
|
||||
|
||||
SELECT * FROM log;
|
||||
SELECT * FROM log LIMIT 1;
|
||||
|
@ -11,34 +11,34 @@ echo "CREATE TABLE concurrent_alter_column (ts DATETIME) ENGINE = MergeTree PART
|
||||
function thread1()
|
||||
{
|
||||
while true; do
|
||||
for i in {1..500}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN c$i DOUBLE;"; done | ${CLICKHOUSE_CLIENT} -n --query_id=alter1
|
||||
for i in {1..500}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN c$i DOUBLE;"; done | ${CLICKHOUSE_CLIENT} -n --query_id=alter_00816_1
|
||||
done
|
||||
}
|
||||
|
||||
function thread2()
|
||||
{
|
||||
while true; do
|
||||
echo "ALTER TABLE concurrent_alter_column ADD COLUMN d DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter2;
|
||||
echo "ALTER TABLE concurrent_alter_column ADD COLUMN d DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_2;
|
||||
sleep "$(echo 0.0$RANDOM)";
|
||||
echo "ALTER TABLE concurrent_alter_column DROP COLUMN d" | ${CLICKHOUSE_CLIENT} --query_id=alter2;
|
||||
echo "ALTER TABLE concurrent_alter_column DROP COLUMN d" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_2;
|
||||
done
|
||||
}
|
||||
|
||||
function thread3()
|
||||
{
|
||||
while true; do
|
||||
echo "ALTER TABLE concurrent_alter_column ADD COLUMN e DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter3;
|
||||
echo "ALTER TABLE concurrent_alter_column ADD COLUMN e DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_3;
|
||||
sleep "$(echo 0.0$RANDOM)";
|
||||
echo "ALTER TABLE concurrent_alter_column DROP COLUMN e" | ${CLICKHOUSE_CLIENT} --query_id=alter3;
|
||||
echo "ALTER TABLE concurrent_alter_column DROP COLUMN e" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_3;
|
||||
done
|
||||
}
|
||||
|
||||
function thread4()
|
||||
{
|
||||
while true; do
|
||||
echo "ALTER TABLE concurrent_alter_column ADD COLUMN f DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter4;
|
||||
echo "ALTER TABLE concurrent_alter_column ADD COLUMN f DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_4;
|
||||
sleep "$(echo 0.0$RANDOM)";
|
||||
echo "ALTER TABLE concurrent_alter_column DROP COLUMN f" | ${CLICKHOUSE_CLIENT} --query_id=alter4;
|
||||
echo "ALTER TABLE concurrent_alter_column DROP COLUMN f" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_4;
|
||||
done
|
||||
}
|
||||
|
||||
@ -57,9 +57,18 @@ timeout $TIMEOUT bash -c thread4 2> /dev/null &
|
||||
|
||||
wait
|
||||
|
||||
echo "DROP TABLE concurrent_alter_column" | ${CLICKHOUSE_CLIENT}
|
||||
echo "DROP TABLE concurrent_alter_column NO DELAY" | ${CLICKHOUSE_CLIENT} # NO DELAY has effect only for Atomic database
|
||||
|
||||
db_engine=`$CLICKHOUSE_CLIENT -q "SELECT engine FROM system.databases WHERE name=currentDatabase()"`
|
||||
if [[ $db_engine == "Atomic" ]]; then
|
||||
# DROP is non-blocking, so wait for alters
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT -q "SELECT c = 0 FROM (SELECT count() as c FROM system.processes WHERE query_id LIKE 'alter_00816_%')" | grep 1 > /dev/null && break;
|
||||
sleep 1;
|
||||
done
|
||||
fi
|
||||
|
||||
# Check for deadlocks
|
||||
echo "SELECT * FROM system.processes WHERE query_id LIKE 'alter%'" | ${CLICKHOUSE_CLIENT}
|
||||
echo "SELECT * FROM system.processes WHERE query_id LIKE 'alter_00816_%'" | ${CLICKHOUSE_CLIENT}
|
||||
|
||||
echo 'did not crash'
|
||||
|
13
tests/queries/0_stateless/01190_full_attach_syntax.reference
Normal file
13
tests/queries/0_stateless/01190_full_attach_syntax.reference
Normal file
@ -0,0 +1,13 @@
|
||||
CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
|
||||
CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
|
||||
CREATE TABLE default.log\n(\n `s` String\n)\nENGINE = Log
|
||||
CREATE TABLE default.log\n(\n `s` String\n)\nENGINE = Log()
|
||||
test
|
||||
CREATE TABLE default.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date MATERIALIZED \'2000-01-01\'\n)\nENGINE = MergeTree(d, (key, s, n), 1)
|
||||
[1,2] Hello 2
|
||||
CREATE TABLE default.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = MergeTree(d, (key, s, n), 1)
|
||||
CREATE MATERIALIZED VIEW default.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM default.log
|
||||
CREATE MATERIALIZED VIEW default.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM default.log
|
||||
CREATE MATERIALIZED VIEW default.mv\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = Null AS\nSELECT *\nFROM default.mt
|
||||
CREATE LIVE VIEW default.lv\n(\n `1` UInt8\n) AS\nSELECT 1
|
||||
CREATE LIVE VIEW default.lv\n(\n `1` UInt8\n) AS\nSELECT 1
|
66
tests/queries/0_stateless/01190_full_attach_syntax.sql
Normal file
66
tests/queries/0_stateless/01190_full_attach_syntax.sql
Normal file
@ -0,0 +1,66 @@
|
||||
DROP DATABASE IF EXISTS test_01190;
|
||||
CREATE DATABASE test_01190;
|
||||
|
||||
CREATE TABLE test_01190.table_for_dict (key UInt64, col UInt8) ENGINE = Memory;
|
||||
|
||||
CREATE DICTIONARY test_01190.dict (key UInt64 DEFAULT 0, col UInt8 DEFAULT 1) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'test_01190')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());
|
||||
|
||||
SHOW CREATE DICTIONARY test_01190.dict;
|
||||
|
||||
DETACH DICTIONARY test_01190.dict;
|
||||
ATTACH TABLE test_01190.dict; -- { serverError 80 }
|
||||
-- Full ATTACH syntax is not allowed for dictionaries
|
||||
ATTACH DICTIONARY test_01190.dict (key UInt64 DEFAULT 0, col UInt8 DEFAULT 42) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'test_01190')) LIFETIME(MIN 1 MAX 100) LAYOUT(FLAT()); -- { clientError 62 }
|
||||
ATTACH DICTIONARY test_01190.dict;
|
||||
SHOW CREATE DICTIONARY test_01190.dict;
|
||||
|
||||
DROP DATABASE test_01190;
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS log;
|
||||
DROP TABLE IF EXISTS mt;
|
||||
DROP TABLE IF EXISTS mv;
|
||||
DROP TABLE IF EXISTS lv;
|
||||
|
||||
CREATE TABLE log ENGINE = Log AS SELECT 'test' AS s;
|
||||
SHOW CREATE log;
|
||||
DETACH TABLE log;
|
||||
ATTACH DICTIONARY log; -- { serverError 487 }
|
||||
ATTACH TABLE log (s String) ENGINE = Log();
|
||||
SHOW CREATE log;
|
||||
SELECT * FROM log;
|
||||
|
||||
DROP TABLE IF EXISTS mt;
|
||||
CREATE TABLE mt (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1);
|
||||
INSERT INTO mt VALUES ([1, 2], 'Hello', 2);
|
||||
DETACH TABLE mt;
|
||||
ATTACH TABLE mt (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree ORDER BY (key, s, n) PARTITION BY toYYYYMM(d); -- { serverError 342 }
|
||||
ATTACH TABLE mt (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1);
|
||||
SHOW CREATE mt;
|
||||
SELECT * FROM mt;
|
||||
DETACH TABLE mt;
|
||||
ATTACH TABLE mt (key Array(UInt8), s String, n UInt64, d Date) ENGINE = MergeTree(d, (key, s, n), 1); -- It works (with Ordinary database), but probably it shouldn't
|
||||
SHOW CREATE mt;
|
||||
|
||||
CREATE MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM log;
|
||||
SHOW CREATE mv;
|
||||
DETACH VIEW mv;
|
||||
ATTACH MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM log;
|
||||
SHOW CREATE mv;
|
||||
DETACH VIEW mv;
|
||||
ATTACH MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM mt; -- It works (with Ordinary database), but probably it shouldn't
|
||||
SHOW CREATE mv;
|
||||
|
||||
SET allow_experimental_live_view = 1;
|
||||
CREATE LIVE VIEW lv AS SELECT 1;
|
||||
SHOW CREATE lv;
|
||||
DETACH VIEW lv;
|
||||
ATTACH LIVE VIEW lv AS SELECT 1;
|
||||
SHOW CREATE lv;
|
||||
|
||||
DROP TABLE log;
|
||||
DROP TABLE mt;
|
||||
DROP TABLE mv;
|
||||
DROP TABLE lv;
|
||||
|
||||
|
@ -7,11 +7,21 @@ set -e
|
||||
|
||||
function thread()
|
||||
{
|
||||
db_engine=`$CLICKHOUSE_CLIENT -q "SELECT engine FROM system.databases WHERE name=currentDatabase()"`
|
||||
if [[ $db_engine == "Atomic" ]]; then
|
||||
# Ignore "Replica already exists" exception
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1 NO DELAY;
|
||||
CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01305/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 |
|
||||
grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now|is already finished removing by another replica right now|Removing leftovers from table|Another replica was suddenly created|was successfully removed from ZooKeeper|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time|already exists'
|
||||
done
|
||||
else
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1;
|
||||
CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01305/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 |
|
||||
grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now|is already finished removing by another replica right now|Removing leftovers from table|Another replica was suddenly created|was successfully removed from ZooKeeper|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time'
|
||||
done
|
||||
fi
|
||||
}
|
||||
|
||||
|
||||
|
@ -22,7 +22,7 @@ GROUP BY loyalty
|
||||
ORDER BY loyalty ASC;
|
||||
|
||||
DETACH TABLE join;
|
||||
ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(SEMI, LEFT, UserID);
|
||||
ATTACH TABLE join;
|
||||
|
||||
SELECT
|
||||
loyalty,
|
||||
|
@ -3,10 +3,8 @@
|
||||
*/
|
||||
{
|
||||
"thread-sanitizer": [
|
||||
"00281",
|
||||
"00877",
|
||||
"00985",
|
||||
"avx2",
|
||||
"query_profiler",
|
||||
"memory_profiler",
|
||||
/// 01083 and 00505 and 00505 are critical and temproray disabled
|
||||
@ -21,9 +19,7 @@
|
||||
"01193_metadata_loading"
|
||||
],
|
||||
"address-sanitizer": [
|
||||
"00281",
|
||||
"00877",
|
||||
"avx2",
|
||||
"query_profiler",
|
||||
"memory_profiler",
|
||||
"odbc_roundtrip",
|
||||
@ -31,9 +27,7 @@
|
||||
"01193_metadata_loading"
|
||||
],
|
||||
"ub-sanitizer": [
|
||||
"00281",
|
||||
"capnproto",
|
||||
"avx2",
|
||||
"query_profiler",
|
||||
"memory_profiler",
|
||||
"01103_check_cpu_instructions_at_startup",
|
||||
@ -41,9 +35,7 @@
|
||||
"01193_metadata_loading"
|
||||
],
|
||||
"memory-sanitizer": [
|
||||
"00281",
|
||||
"capnproto",
|
||||
"avx2",
|
||||
"query_profiler",
|
||||
"memory_profiler",
|
||||
"01103_check_cpu_instructions_at_startup",
|
||||
@ -53,8 +45,6 @@
|
||||
"01193_metadata_loading"
|
||||
],
|
||||
"debug-build": [
|
||||
"00281",
|
||||
"avx2",
|
||||
"query_profiler",
|
||||
"memory_profiler",
|
||||
"00899_long_attach",
|
||||
@ -70,12 +60,10 @@
|
||||
],
|
||||
"unbundled-build": [
|
||||
"00429",
|
||||
"00428",
|
||||
"00877",
|
||||
"pocopatch",
|
||||
"parquet",
|
||||
"xxhash",
|
||||
"avx2",
|
||||
"_h3",
|
||||
"query_profiler",
|
||||
"memory_profiler",
|
||||
@ -98,33 +86,19 @@
|
||||
"01455_time_zones"
|
||||
],
|
||||
"release-build": [
|
||||
"avx2"
|
||||
],
|
||||
"database-atomic": [
|
||||
"00065_loyalty_with_storage_join",
|
||||
"avx",
|
||||
/// Inner tables of materialized views have different names
|
||||
"00738_lock_for_inner_table",
|
||||
"00699_materialized_view_mutations",
|
||||
"00609_mv_index_in_in",
|
||||
"00510_materizlized_view_and_deduplication_zookeeper",
|
||||
/// Create queries contain UUID
|
||||
/// Different database engine
|
||||
"00604_show_create_database",
|
||||
"00080_show_tables_and_system_tables",
|
||||
"01272_suspicious_codecs",
|
||||
/// UUID must be specified in ATTACH TABLE
|
||||
"01249_bad_arguments_for_bloom_filter",
|
||||
"00423_storage_log_single_thread",
|
||||
"00311_array_primary_key",
|
||||
"00226_zookeeper_deduplication_and_unexpected_parts",
|
||||
"00180_attach_materialized_view",
|
||||
"00116_storage_set",
|
||||
"01190_full_attach_syntax",
|
||||
/// Assumes blocking DROP
|
||||
"00816_long_concurrent_alter_column",
|
||||
"00992_system_parts_race_condition_zookeeper", /// FIXME
|
||||
"01320_create_sync_race_condition",
|
||||
"01305_replica_create_drop_zookeeper",
|
||||
"01130_in_memory_parts_partitons",
|
||||
/// Internal distionary name is different
|
||||
"01225_show_create_table_from_dictionary",
|
||||
"01224_no_superfluous_dict_reload"
|
||||
],
|
||||
@ -132,7 +106,6 @@
|
||||
/// These tests fail with compact parts, because they
|
||||
/// check some implementation defined things
|
||||
/// like checksums, computed granularity, ProfileEvents, etc.
|
||||
"avx",
|
||||
"01045_order_by_pk_special_storages",
|
||||
"01042_check_query_and_last_granule_size",
|
||||
"00961_checksums_in_system_parts_columns_table",
|
||||
|
Loading…
Reference in New Issue
Block a user