mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
rename invalid identifiers
This commit is contained in:
parent
b8fc666fcd
commit
7c208abc58
@ -9,13 +9,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||
|
||||
$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -n --query="
|
||||
DROP DATABASE IF EXISTS 01053_db;
|
||||
DROP DATABASE IF EXISTS _01053_db;
|
||||
|
||||
CREATE DATABASE 01053_db Engine = Ordinary;
|
||||
CREATE DATABASE _01053_db Engine = Ordinary;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.table_for_dict;
|
||||
DROP TABLE IF EXISTS _01053_db.table_for_dict;
|
||||
|
||||
CREATE TABLE 01053_db.table_for_dict
|
||||
CREATE TABLE _01053_db.table_for_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64,
|
||||
@ -25,15 +25,15 @@ $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -n --query="
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY id;
|
||||
|
||||
INSERT INTO 01053_db.table_for_dict VALUES (1, 100, -100, 'clickhouse'), (2, 3, 4, 'database'), (5, 6, 7, 'columns'), (10, 9, 8, '');
|
||||
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'a' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370;
|
||||
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'b' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370, 370;
|
||||
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'c' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 700, 370;
|
||||
INSERT INTO _01053_db.table_for_dict VALUES (1, 100, -100, 'clickhouse'), (2, 3, 4, 'database'), (5, 6, 7, 'columns'), (10, 9, 8, '');
|
||||
INSERT INTO _01053_db.table_for_dict SELECT number, 0, -1, 'a' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370;
|
||||
INSERT INTO _01053_db.table_for_dict SELECT number, 0, -1, 'b' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370, 370;
|
||||
INSERT INTO _01053_db.table_for_dict SELECT number, 0, -1, 'c' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 700, 370;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
DROP DICTIONARY IF EXISTS _01053_db.ssd_dict;
|
||||
|
||||
-- Probably we need rewrite it to integration test
|
||||
CREATE DICTIONARY 01053_db.ssd_dict
|
||||
CREATE DICTIONARY _01053_db.ssd_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64 DEFAULT 0,
|
||||
@ -41,40 +41,40 @@ $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -n --query="
|
||||
c String DEFAULT 'none'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '_01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '$USER_FILES_PATH/0d'));
|
||||
|
||||
SELECT 'TEST_SMALL';
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(4));
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(5));
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(6));
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(2));
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(3));
|
||||
SELECT dictGetInt32('_01053_db.ssd_dict', 'b', toUInt64(1));
|
||||
SELECT dictGetInt32('_01053_db.ssd_dict', 'b', toUInt64(4));
|
||||
SELECT dictGetUInt64('_01053_db.ssd_dict', 'a', toUInt64(5));
|
||||
SELECT dictGetUInt64('_01053_db.ssd_dict', 'a', toUInt64(6));
|
||||
SELECT dictGetString('_01053_db.ssd_dict', 'c', toUInt64(2));
|
||||
SELECT dictGetString('_01053_db.ssd_dict', 'c', toUInt64(3));
|
||||
|
||||
SELECT * FROM 01053_db.ssd_dict ORDER BY id;
|
||||
DROP DICTIONARY 01053_db.ssd_dict;
|
||||
SELECT * FROM _01053_db.ssd_dict ORDER BY id;
|
||||
DROP DICTIONARY _01053_db.ssd_dict;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.keys_table;
|
||||
DROP TABLE IF EXISTS _01053_db.keys_table;
|
||||
|
||||
CREATE TABLE 01053_db.keys_table
|
||||
CREATE TABLE _01053_db.keys_table
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
ENGINE = StripeLog();
|
||||
|
||||
INSERT INTO 01053_db.keys_table VALUES (1);
|
||||
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (2);
|
||||
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (5);
|
||||
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 700, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (10);
|
||||
INSERT INTO _01053_db.keys_table VALUES (1);
|
||||
INSERT INTO _01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370;
|
||||
INSERT INTO _01053_db.keys_table VALUES (2);
|
||||
INSERT INTO _01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370, 370;
|
||||
INSERT INTO _01053_db.keys_table VALUES (5);
|
||||
INSERT INTO _01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 700, 370;
|
||||
INSERT INTO _01053_db.keys_table VALUES (10);
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
DROP DICTIONARY IF EXISTS _01053_db.ssd_dict;
|
||||
|
||||
CREATE DICTIONARY 01053_db.ssd_dict
|
||||
CREATE DICTIONARY _01053_db.ssd_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64 DEFAULT 0,
|
||||
@ -82,76 +82,76 @@ $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -n --query="
|
||||
c String DEFAULT 'none'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '_01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '$USER_FILES_PATH/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
|
||||
|
||||
SELECT 'UPDATE DICTIONARY';
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
SELECT sum(dictGetUInt64('_01053_db.ssd_dict', 'a', toUInt64(id))) FROM _01053_db.keys_table;
|
||||
|
||||
SELECT 'VALUE FROM DISK';
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
|
||||
SELECT dictGetInt32('_01053_db.ssd_dict', 'b', toUInt64(1));
|
||||
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(1));
|
||||
SELECT dictGetString('_01053_db.ssd_dict', 'c', toUInt64(1));
|
||||
|
||||
SELECT 'VALUE FROM RAM BUFFER';
|
||||
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(10));
|
||||
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(10));
|
||||
SELECT dictGetInt32('_01053_db.ssd_dict', 'b', toUInt64(10));
|
||||
SELECT dictGetString('_01053_db.ssd_dict', 'c', toUInt64(10));
|
||||
|
||||
SELECT 'VALUES FROM DISK AND RAM BUFFER';
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
SELECT sum(dictGetUInt64('_01053_db.ssd_dict', 'a', toUInt64(id))) FROM _01053_db.keys_table;
|
||||
|
||||
SELECT 'HAS';
|
||||
SELECT count() FROM 01053_db.keys_table WHERE dictHas('01053_db.ssd_dict', toUInt64(id));
|
||||
SELECT count() FROM _01053_db.keys_table WHERE dictHas('_01053_db.ssd_dict', toUInt64(id));
|
||||
|
||||
SELECT 'VALUES NOT FROM TABLE';
|
||||
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('01053_db.ssd_dict', 'c', toUInt64(1000000));
|
||||
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('01053_db.ssd_dict', 'c', toUInt64(1000000));
|
||||
SELECT dictGetUInt64('_01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('_01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('_01053_db.ssd_dict', 'c', toUInt64(1000000));
|
||||
SELECT dictGetUInt64('_01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('_01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('_01053_db.ssd_dict', 'c', toUInt64(1000000));
|
||||
|
||||
SELECT 'DUPLICATE KEYS';
|
||||
SELECT arrayJoin([1, 2, 3, 3, 2, 1]) AS id, dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(id));
|
||||
SELECT arrayJoin([1, 2, 3, 3, 2, 1]) AS id, dictGetInt32('_01053_db.ssd_dict', 'b', toUInt64(id));
|
||||
--SELECT
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
DROP DICTIONARY IF EXISTS _01053_db.ssd_dict;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.keys_table;
|
||||
DROP TABLE IF EXISTS _01053_db.keys_table;
|
||||
|
||||
CREATE TABLE 01053_db.keys_table
|
||||
CREATE TABLE _01053_db.keys_table
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY id;
|
||||
|
||||
INSERT INTO 01053_db.keys_table VALUES (1);
|
||||
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (2);
|
||||
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (5);
|
||||
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 700, 370;
|
||||
INSERT INTO 01053_db.keys_table VALUES (10);
|
||||
INSERT INTO _01053_db.keys_table VALUES (1);
|
||||
INSERT INTO _01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370;
|
||||
INSERT INTO _01053_db.keys_table VALUES (2);
|
||||
INSERT INTO _01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370, 370;
|
||||
INSERT INTO _01053_db.keys_table VALUES (5);
|
||||
INSERT INTO _01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 700, 370;
|
||||
INSERT INTO _01053_db.keys_table VALUES (10);
|
||||
|
||||
OPTIMIZE TABLE 01053_db.keys_table;
|
||||
OPTIMIZE TABLE _01053_db.keys_table;
|
||||
|
||||
CREATE DICTIONARY 01053_db.ssd_dict
|
||||
CREATE DICTIONARY _01053_db.ssd_dict
|
||||
(
|
||||
id UInt64,
|
||||
a UInt64 DEFAULT 0,
|
||||
b Int32 DEFAULT -1
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '_01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '$USER_FILES_PATH/2d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 1024));
|
||||
|
||||
SELECT 'UPDATE DICTIONARY (MT)';
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
SELECT sum(dictGetUInt64('_01053_db.ssd_dict', 'a', toUInt64(id))) FROM _01053_db.keys_table;
|
||||
|
||||
SELECT 'VALUES FROM DISK AND RAM BUFFER (MT)';
|
||||
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
|
||||
SELECT sum(dictGetUInt64('_01053_db.ssd_dict', 'a', toUInt64(id))) FROM _01053_db.keys_table;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
|
||||
DROP DICTIONARY IF EXISTS _01053_db.ssd_dict;
|
||||
|
||||
DROP TABLE IF EXISTS 01053_db.table_for_dict;
|
||||
DROP TABLE IF EXISTS _01053_db.table_for_dict;
|
||||
|
||||
DROP DATABASE IF EXISTS 01053_db;"
|
||||
DROP DATABASE IF EXISTS _01053_db;"
|
||||
|
@ -27,27 +27,27 @@ with client(name="client1>", log=log) as client1, client(
|
||||
client2.send("SET allow_experimental_window_view = 1")
|
||||
client2.expect(prompt)
|
||||
|
||||
client1.send("CREATE DATABASE IF NOT EXISTS 01056_window_view_proc_hop_watch")
|
||||
client1.send("CREATE DATABASE IF NOT EXISTS _01056_window_view_proc_hop_watch")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01056_window_view_proc_hop_watch.mt")
|
||||
client1.send("DROP TABLE IF EXISTS _01056_window_view_proc_hop_watch.mt")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01056_window_view_proc_hop_watch.wv")
|
||||
client1.send("DROP TABLE IF EXISTS _01056_window_view_proc_hop_watch.wv")
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send(
|
||||
"CREATE TABLE 01056_window_view_proc_hop_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
"CREATE TABLE _01056_window_view_proc_hop_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv ENGINE Memory AS SELECT count(a) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;"
|
||||
"CREATE WINDOW VIEW _01056_window_view_proc_hop_watch.wv ENGINE Memory AS SELECT count(a) AS count FROM _01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send("WATCH 01056_window_view_proc_hop_watch.wv")
|
||||
client1.send("WATCH _01056_window_view_proc_hop_watch.wv")
|
||||
client1.expect("Query id" + end_of_block)
|
||||
client1.expect("Progress: 0.00 rows.*\)")
|
||||
client2.send(
|
||||
"INSERT INTO 01056_window_view_proc_hop_watch.mt VALUES (1, now('US/Samoa') + 3)"
|
||||
"INSERT INTO _01056_window_view_proc_hop_watch.mt VALUES (1, now('US/Samoa') + 3)"
|
||||
)
|
||||
client1.expect("1" + end_of_block)
|
||||
client1.expect("Progress: 1.00 rows.*\)")
|
||||
@ -58,9 +58,9 @@ with client(name="client1>", log=log) as client1, client(
|
||||
if match.groups()[1]:
|
||||
client1.send(client1.command)
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01056_window_view_proc_hop_watch.wv")
|
||||
client1.send("DROP TABLE _01056_window_view_proc_hop_watch.wv")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01056_window_view_proc_hop_watch.mt")
|
||||
client1.send("DROP TABLE _01056_window_view_proc_hop_watch.mt")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP DATABASE IF EXISTS 01056_window_view_proc_hop_watch")
|
||||
client1.send("DROP DATABASE IF EXISTS _01056_window_view_proc_hop_watch")
|
||||
client1.expect(prompt)
|
||||
|
@ -27,42 +27,42 @@ with client(name="client1>", log=log) as client1, client(
|
||||
client2.send("SET allow_experimental_window_view = 1")
|
||||
client2.expect(prompt)
|
||||
|
||||
client1.send("CREATE DATABASE IF NOT EXISTS 01062_window_view_event_hop_watch_asc")
|
||||
client1.send("CREATE DATABASE IF NOT EXISTS _01062_window_view_event_hop_watch_asc")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01062_window_view_event_hop_watch_asc.mt")
|
||||
client1.send("DROP TABLE IF EXISTS _01062_window_view_event_hop_watch_asc.mt")
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"DROP TABLE IF EXISTS 01062_window_view_event_hop_watch_asc.wv NO DELAY"
|
||||
"DROP TABLE IF EXISTS _01062_window_view_event_hop_watch_asc.wv NO DELAY"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send(
|
||||
"CREATE TABLE 01062_window_view_event_hop_watch_asc.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
"CREATE TABLE _01062_window_view_event_hop_watch_asc.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01062_window_view_event_hop_watch_asc.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM 01062_window_view_event_hop_watch_asc.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
"CREATE WINDOW VIEW _01062_window_view_event_hop_watch_asc.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM _01062_window_view_event_hop_watch_asc.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send("WATCH 01062_window_view_event_hop_watch_asc.wv")
|
||||
client1.send("WATCH _01062_window_view_event_hop_watch_asc.wv")
|
||||
client1.expect("Query id" + end_of_block)
|
||||
client1.expect("Progress: 0.00 rows.*\)")
|
||||
client2.send(
|
||||
"INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));"
|
||||
"INSERT INTO _01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));"
|
||||
)
|
||||
client2.expect(prompt)
|
||||
client2.send(
|
||||
"INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:05', 'US/Samoa'));"
|
||||
"INSERT INTO _01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:05', 'US/Samoa'));"
|
||||
)
|
||||
client2.expect(prompt)
|
||||
client1.expect("1*" + end_of_block)
|
||||
client2.send(
|
||||
"INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));"
|
||||
"INSERT INTO _01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));"
|
||||
)
|
||||
client2.expect(prompt)
|
||||
client2.send(
|
||||
"INSERT INTO 01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:10', 'US/Samoa'));"
|
||||
"INSERT INTO _01062_window_view_event_hop_watch_asc.mt VALUES (1, toDateTime('1990/01/01 12:00:10', 'US/Samoa'));"
|
||||
)
|
||||
client2.expect(prompt)
|
||||
client1.expect("1" + end_of_block)
|
||||
@ -75,9 +75,9 @@ with client(name="client1>", log=log) as client1, client(
|
||||
if match.groups()[1]:
|
||||
client1.send(client1.command)
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01062_window_view_event_hop_watch_asc.wv NO DELAY")
|
||||
client1.send("DROP TABLE _01062_window_view_event_hop_watch_asc.wv NO DELAY")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01062_window_view_event_hop_watch_asc.mt")
|
||||
client1.send("DROP TABLE _01062_window_view_event_hop_watch_asc.mt")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP DATABASE IF EXISTS 01062_window_view_event_hop_watch_asc")
|
||||
client1.send("DROP DATABASE IF EXISTS _01062_window_view_event_hop_watch_asc")
|
||||
client1.expect(prompt)
|
||||
|
@ -27,41 +27,41 @@ with client(name="client1>", log=log) as client1, client(
|
||||
client2.expect(prompt)
|
||||
|
||||
client1.send(
|
||||
"CREATE DATABASE IF NOT EXISTS 01065_window_view_event_hop_watch_bounded"
|
||||
"CREATE DATABASE IF NOT EXISTS _01065_window_view_event_hop_watch_bounded"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01065_window_view_event_hop_watch_bounded.mt")
|
||||
client1.send("DROP TABLE IF EXISTS _01065_window_view_event_hop_watch_bounded.mt")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01065_window_view_event_hop_watch_bounded.wv")
|
||||
client1.send("DROP TABLE IF EXISTS _01065_window_view_event_hop_watch_bounded.wv")
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send(
|
||||
"CREATE TABLE 01065_window_view_event_hop_watch_bounded.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
"CREATE TABLE _01065_window_view_event_hop_watch_bounded.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01065_window_view_event_hop_watch_bounded.wv ENGINE Memory WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM 01065_window_view_event_hop_watch_bounded.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
"CREATE WINDOW VIEW _01065_window_view_event_hop_watch_bounded.wv ENGINE Memory WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM _01065_window_view_event_hop_watch_bounded.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
)
|
||||
client1.expect("Ok.")
|
||||
|
||||
client1.send("WATCH 01065_window_view_event_hop_watch_bounded.wv")
|
||||
client1.send("WATCH _01065_window_view_event_hop_watch_bounded.wv")
|
||||
client1.expect("Query id" + end_of_block)
|
||||
client1.expect("Progress: 0.00 rows.*\)")
|
||||
client2.send(
|
||||
"INSERT INTO 01065_window_view_event_hop_watch_bounded.mt VALUES (1, '1990/01/01 12:00:00');"
|
||||
"INSERT INTO _01065_window_view_event_hop_watch_bounded.mt VALUES (1, '1990/01/01 12:00:00');"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client2.send(
|
||||
"INSERT INTO 01065_window_view_event_hop_watch_bounded.mt VALUES (1, '1990/01/01 12:00:05');"
|
||||
"INSERT INTO _01065_window_view_event_hop_watch_bounded.mt VALUES (1, '1990/01/01 12:00:05');"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client1.expect("1" + end_of_block)
|
||||
client2.send(
|
||||
"INSERT INTO 01065_window_view_event_hop_watch_bounded.mt VALUES (1, '1990/01/01 12:00:06');"
|
||||
"INSERT INTO _01065_window_view_event_hop_watch_bounded.mt VALUES (1, '1990/01/01 12:00:06');"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client2.send(
|
||||
"INSERT INTO 01065_window_view_event_hop_watch_bounded.mt VALUES (1, '1990/01/01 12:00:10');"
|
||||
"INSERT INTO _01065_window_view_event_hop_watch_bounded.mt VALUES (1, '1990/01/01 12:00:10');"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client1.expect("2" + end_of_block)
|
||||
@ -72,9 +72,9 @@ with client(name="client1>", log=log) as client1, client(
|
||||
if match.groups()[1]:
|
||||
client1.send(client1.command)
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01065_window_view_event_hop_watch_bounded.wv")
|
||||
client1.send("DROP TABLE _01065_window_view_event_hop_watch_bounded.wv")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01065_window_view_event_hop_watch_bounded.mt")
|
||||
client1.send("DROP TABLE _01065_window_view_event_hop_watch_bounded.mt")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP DATABASE IF EXISTS 01065_window_view_event_hop_watch_bounded")
|
||||
client1.send("DROP DATABASE IF EXISTS _01065_window_view_event_hop_watch_bounded")
|
||||
client1.expect(prompt)
|
||||
|
@ -27,33 +27,33 @@ with client(name="client1>", log=log) as client1, client(
|
||||
client2.send("SET allow_experimental_window_view = 1")
|
||||
client2.expect(prompt)
|
||||
|
||||
client1.send("CREATE DATABASE 01069_window_view_proc_tumble_watch")
|
||||
client1.send("CREATE DATABASE _01069_window_view_proc_tumble_watch")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01069_window_view_proc_tumble_watch.mt")
|
||||
client1.send("DROP TABLE IF EXISTS _01069_window_view_proc_tumble_watch.mt")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01069_window_view_proc_tumble_watch.wv NO DELAY")
|
||||
client1.send("DROP TABLE IF EXISTS _01069_window_view_proc_tumble_watch.wv NO DELAY")
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send(
|
||||
"CREATE TABLE 01069_window_view_proc_tumble_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
"CREATE TABLE _01069_window_view_proc_tumble_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01069_window_view_proc_tumble_watch.wv ENGINE Memory AS SELECT count(a) AS count FROM 01069_window_view_proc_tumble_watch.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND, 'US/Samoa') AS wid;"
|
||||
"CREATE WINDOW VIEW _01069_window_view_proc_tumble_watch.wv ENGINE Memory AS SELECT count(a) AS count FROM _01069_window_view_proc_tumble_watch.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND, 'US/Samoa') AS wid;"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send("WATCH 01069_window_view_proc_tumble_watch.wv")
|
||||
client1.send("WATCH _01069_window_view_proc_tumble_watch.wv")
|
||||
client1.expect("Query id" + end_of_block)
|
||||
client1.expect("Progress: 0.00 rows.*\)")
|
||||
client2.send(
|
||||
"INSERT INTO 01069_window_view_proc_tumble_watch.mt VALUES (1, now('US/Samoa') + 3)"
|
||||
"INSERT INTO _01069_window_view_proc_tumble_watch.mt VALUES (1, now('US/Samoa') + 3)"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client1.expect("1" + end_of_block)
|
||||
client1.expect("Progress: 1.00 rows.*\)")
|
||||
client2.send(
|
||||
"INSERT INTO 01069_window_view_proc_tumble_watch.mt VALUES (1, now('US/Samoa') + 3)"
|
||||
"INSERT INTO _01069_window_view_proc_tumble_watch.mt VALUES (1, now('US/Samoa') + 3)"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client1.expect("1" + end_of_block)
|
||||
@ -65,9 +65,9 @@ with client(name="client1>", log=log) as client1, client(
|
||||
if match.groups()[1]:
|
||||
client1.send(client1.command)
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01069_window_view_proc_tumble_watch.wv NO DELAY")
|
||||
client1.send("DROP TABLE _01069_window_view_proc_tumble_watch.wv NO DELAY")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01069_window_view_proc_tumble_watch.mt")
|
||||
client1.send("DROP TABLE _01069_window_view_proc_tumble_watch.mt")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP DATABASE IF EXISTS 01069_window_view_proc_tumble_watch")
|
||||
client1.send("DROP DATABASE IF EXISTS _01069_window_view_proc_tumble_watch")
|
||||
client1.expect(prompt)
|
||||
|
@ -27,31 +27,31 @@ with client(name="client1>", log=log) as client1, client(
|
||||
client2.send("SET allow_experimental_window_view = 1")
|
||||
client2.expect(prompt)
|
||||
|
||||
client1.send("CREATE DATABASE IF NOT EXISTS 01070_window_view_watch_events")
|
||||
client1.send("CREATE DATABASE IF NOT EXISTS _01070_window_view_watch_events")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01070_window_view_watch_events.mt NO DELAY")
|
||||
client1.send("DROP TABLE IF EXISTS _01070_window_view_watch_events.mt NO DELAY")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01070_window_view_watch_events.wv NO DELAY")
|
||||
client1.send("DROP TABLE IF EXISTS _01070_window_view_watch_events.wv NO DELAY")
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send(
|
||||
"CREATE TABLE 01070_window_view_watch_events.mt(a Int32, timestamp DateTime('US/Samoa')) ENGINE=MergeTree ORDER BY tuple()"
|
||||
"CREATE TABLE _01070_window_view_watch_events.mt(a Int32, timestamp DateTime('US/Samoa')) ENGINE=MergeTree ORDER BY tuple()"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01070_window_view_watch_events.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM 01070_window_view_watch_events.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid"
|
||||
"CREATE WINDOW VIEW _01070_window_view_watch_events.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM _01070_window_view_watch_events.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send("WATCH 01070_window_view_watch_events.wv EVENTS")
|
||||
client1.send("WATCH _01070_window_view_watch_events.wv EVENTS")
|
||||
client1.expect("Query id" + end_of_block)
|
||||
client1.expect("Progress: 0.00 rows.*\)")
|
||||
client2.send(
|
||||
"INSERT INTO 01070_window_view_watch_events.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));"
|
||||
"INSERT INTO _01070_window_view_watch_events.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client2.send(
|
||||
"INSERT INTO 01070_window_view_watch_events.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));"
|
||||
"INSERT INTO _01070_window_view_watch_events.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client1.expect("1990-01-01 12:00:05" + end_of_block)
|
||||
@ -63,9 +63,9 @@ with client(name="client1>", log=log) as client1, client(
|
||||
if match.groups()[1]:
|
||||
client1.send(client1.command)
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01070_window_view_watch_events.wv NO DELAY;")
|
||||
client1.send("DROP TABLE _01070_window_view_watch_events.wv NO DELAY;")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01070_window_view_watch_events.mt;")
|
||||
client1.send("DROP TABLE _01070_window_view_watch_events.mt;")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP DATABASE IF EXISTS 01070_window_view_watch_events;")
|
||||
client1.send("DROP DATABASE IF EXISTS _01070_window_view_watch_events;")
|
||||
client1.expect(prompt)
|
||||
|
@ -31,51 +31,51 @@ with client(name="client1>", log=log) as client1, client(
|
||||
client3.send("SET window_view_heartbeat_interval = 1")
|
||||
client3.expect(prompt)
|
||||
|
||||
client1.send("CREATE DATABASE IF NOT EXISTS 01078_window_view_alter_query_watch")
|
||||
client1.send("CREATE DATABASE IF NOT EXISTS _01078_window_view_alter_query_watch")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01078_window_view_alter_query_watch.mt NO DELAY")
|
||||
client1.send("DROP TABLE IF EXISTS _01078_window_view_alter_query_watch.mt NO DELAY")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01078_window_view_alter_query_watch.wv NO DELAY")
|
||||
client1.send("DROP TABLE IF EXISTS _01078_window_view_alter_query_watch.wv NO DELAY")
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send(
|
||||
"CREATE TABLE 01078_window_view_alter_query_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
"CREATE TABLE _01078_window_view_alter_query_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01078_window_view_alter_query_watch.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM 01078_window_view_alter_query_watch.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
"CREATE WINDOW VIEW _01078_window_view_alter_query_watch.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM _01078_window_view_alter_query_watch.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send("WATCH 01078_window_view_alter_query_watch.wv")
|
||||
client1.send("WATCH _01078_window_view_alter_query_watch.wv")
|
||||
client1.expect("Query id" + end_of_block)
|
||||
client1.expect("Progress: 0.00 rows.*\)")
|
||||
client2.send(
|
||||
"INSERT INTO 01078_window_view_alter_query_watch.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));"
|
||||
"INSERT INTO _01078_window_view_alter_query_watch.mt VALUES (1, toDateTime('1990/01/01 12:00:00', 'US/Samoa'));"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client2.send(
|
||||
"INSERT INTO 01078_window_view_alter_query_watch.mt VALUES (1, toDateTime('1990/01/01 12:00:05', 'US/Samoa'));"
|
||||
"INSERT INTO _01078_window_view_alter_query_watch.mt VALUES (1, toDateTime('1990/01/01 12:00:05', 'US/Samoa'));"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client1.expect("1" + end_of_block)
|
||||
client1.expect("Progress: 1.00 rows.*\)")
|
||||
client2.send(
|
||||
"ALTER TABLE 01078_window_view_alter_query_watch.wv MODIFY QUERY SELECT count(a) * 2 AS count, hopEnd(wid) AS w_end FROM 01078_window_view_alter_query_watch.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
"ALTER TABLE _01078_window_view_alter_query_watch.wv MODIFY QUERY SELECT count(a) * 2 AS count, hopEnd(wid) AS w_end FROM _01078_window_view_alter_query_watch.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client2.expect(prompt)
|
||||
client1.expect("1 row" + end_of_block)
|
||||
client1.expect(prompt)
|
||||
client3.send("WATCH 01078_window_view_alter_query_watch.wv")
|
||||
client3.send("WATCH _01078_window_view_alter_query_watch.wv")
|
||||
client3.expect("Query id" + end_of_block)
|
||||
client3.expect("Progress: 0.00 rows.*\)")
|
||||
client2.send(
|
||||
"INSERT INTO 01078_window_view_alter_query_watch.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));"
|
||||
"INSERT INTO _01078_window_view_alter_query_watch.mt VALUES (1, toDateTime('1990/01/01 12:00:06', 'US/Samoa'));"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client2.send(
|
||||
"INSERT INTO 01078_window_view_alter_query_watch.mt VALUES (1, toDateTime('1990/01/01 12:00:10', 'US/Samoa'));"
|
||||
"INSERT INTO _01078_window_view_alter_query_watch.mt VALUES (1, toDateTime('1990/01/01 12:00:10', 'US/Samoa'));"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client3.expect("2" + end_of_block)
|
||||
@ -87,9 +87,9 @@ with client(name="client1>", log=log) as client1, client(
|
||||
if match.groups()[1]:
|
||||
client3.send(client3.command)
|
||||
client3.expect(prompt)
|
||||
client3.send("DROP TABLE 01078_window_view_alter_query_watch.wv NO DELAY;")
|
||||
client3.send("DROP TABLE _01078_window_view_alter_query_watch.wv NO DELAY;")
|
||||
client3.expect(prompt)
|
||||
client3.send("DROP TABLE 01078_window_view_alter_query_watch.mt;")
|
||||
client3.send("DROP TABLE _01078_window_view_alter_query_watch.mt;")
|
||||
client3.expect(prompt)
|
||||
client3.send("DROP DATABASE IF EXISTS 01078_window_view_alter_query_watch;")
|
||||
client3.send("DROP DATABASE IF EXISTS _01078_window_view_alter_query_watch;")
|
||||
client3.expect(prompt)
|
||||
|
@ -26,32 +26,32 @@ with client(name="client1>", log=log) as client1, client(
|
||||
client2.send("SET allow_experimental_window_view = 1")
|
||||
client2.expect(prompt)
|
||||
|
||||
client1.send("CREATE DATABASE IF NOT EXISTS 01082_window_view_watch_limit")
|
||||
client1.send("CREATE DATABASE IF NOT EXISTS _01082_window_view_watch_limit")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01082_window_view_watch_limit.mt")
|
||||
client1.send("DROP TABLE IF EXISTS _01082_window_view_watch_limit.mt")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE IF EXISTS 01082_window_view_watch_limit.wv NO DELAY")
|
||||
client1.send("DROP TABLE IF EXISTS _01082_window_view_watch_limit.wv NO DELAY")
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send(
|
||||
"CREATE TABLE 01082_window_view_watch_limit.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
"CREATE TABLE _01082_window_view_watch_limit.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()"
|
||||
)
|
||||
client1.expect(prompt)
|
||||
client1.send(
|
||||
"CREATE WINDOW VIEW 01082_window_view_watch_limit.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM 01082_window_view_watch_limit.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
"CREATE WINDOW VIEW _01082_window_view_watch_limit.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM _01082_window_view_watch_limit.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid"
|
||||
)
|
||||
client1.expect("Ok.")
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send("WATCH 01082_window_view_watch_limit.wv LIMIT 1")
|
||||
client1.send("WATCH _01082_window_view_watch_limit.wv LIMIT 1")
|
||||
client1.expect("Query id" + end_of_block)
|
||||
client1.expect("Progress: 0.00 rows.*\)")
|
||||
client2.send(
|
||||
"INSERT INTO 01082_window_view_watch_limit.mt VALUES (1, '1990/01/01 12:00:00');"
|
||||
"INSERT INTO _01082_window_view_watch_limit.mt VALUES (1, '1990/01/01 12:00:00');"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client2.send(
|
||||
"INSERT INTO 01082_window_view_watch_limit.mt VALUES (1, '1990/01/01 12:00:05');"
|
||||
"INSERT INTO _01082_window_view_watch_limit.mt VALUES (1, '1990/01/01 12:00:05');"
|
||||
)
|
||||
client2.expect("Ok.")
|
||||
client1.expect("1" + end_of_block)
|
||||
@ -59,9 +59,9 @@ with client(name="client1>", log=log) as client1, client(
|
||||
client1.expect("1 row" + end_of_block)
|
||||
client1.expect(prompt)
|
||||
|
||||
client1.send("DROP TABLE 01082_window_view_watch_limit.wv NO DELAY")
|
||||
client1.send("DROP TABLE _01082_window_view_watch_limit.wv NO DELAY")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP TABLE 01082_window_view_watch_limit.mt")
|
||||
client1.send("DROP TABLE _01082_window_view_watch_limit.mt")
|
||||
client1.expect(prompt)
|
||||
client1.send("DROP DATABASE IF EXISTS 01082_window_view_watch_limit")
|
||||
client1.send("DROP DATABASE IF EXISTS _01082_window_view_watch_limit")
|
||||
client1.expect(prompt)
|
||||
|
@ -9,10 +9,10 @@ USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonex
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
DROP DATABASE IF EXISTS 01280_db;
|
||||
CREATE DATABASE 01280_db Engine = Ordinary;
|
||||
DROP TABLE IF EXISTS 01280_db.table_for_dict;
|
||||
CREATE TABLE 01280_db.table_for_dict
|
||||
DROP DATABASE IF EXISTS _01280_db;
|
||||
CREATE DATABASE _01280_db Engine = Ordinary;
|
||||
DROP TABLE IF EXISTS _01280_db.table_for_dict;
|
||||
CREATE TABLE _01280_db.table_for_dict
|
||||
(
|
||||
k1 String,
|
||||
k2 Int32,
|
||||
@ -23,13 +23,13 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY (k1, k2);
|
||||
|
||||
INSERT INTO 01280_db.table_for_dict VALUES (toString(1), 3, 100, -100, 'clickhouse'), (toString(2), -1, 3, 4, 'database'), (toString(5), -3, 6, 7, 'columns'), (toString(10), -20, 9, 8, '');
|
||||
INSERT INTO 01280_db.table_for_dict SELECT toString(number), number + 1, 0, -1, 'a' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370;
|
||||
INSERT INTO 01280_db.table_for_dict SELECT toString(number), number + 10, 0, -1, 'b' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370, 370;
|
||||
INSERT INTO 01280_db.table_for_dict SELECT toString(number), number + 100, 0, -1, 'c' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 700, 370;
|
||||
INSERT INTO _01280_db.table_for_dict VALUES (toString(1), 3, 100, -100, 'clickhouse'), (toString(2), -1, 3, 4, 'database'), (toString(5), -3, 6, 7, 'columns'), (toString(10), -20, 9, 8, '');
|
||||
INSERT INTO _01280_db.table_for_dict SELECT toString(number), number + 1, 0, -1, 'a' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370;
|
||||
INSERT INTO _01280_db.table_for_dict SELECT toString(number), number + 10, 0, -1, 'b' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370, 370;
|
||||
INSERT INTO _01280_db.table_for_dict SELECT toString(number), number + 100, 0, -1, 'c' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 700, 370;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01280_db.ssd_dict;
|
||||
CREATE DICTIONARY 01280_db.ssd_dict
|
||||
DROP DICTIONARY IF EXISTS _01280_db.ssd_dict;
|
||||
CREATE DICTIONARY _01280_db.ssd_dict
|
||||
(
|
||||
k1 String,
|
||||
k2 Int32,
|
||||
@ -38,54 +38,54 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
c String DEFAULT 'none'
|
||||
)
|
||||
PRIMARY KEY k1, k2
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01280_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '_01280_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH '$USER_FILES_PATH/0d'));"
|
||||
|
||||
$CLICKHOUSE_CLIENT -nq "SELECT dictHas('01280_db.ssd_dict', 'a', tuple('1')); -- { serverError 43 }"
|
||||
$CLICKHOUSE_CLIENT -nq "SELECT dictHas('_01280_db.ssd_dict', 'a', tuple('1')); -- { serverError 43 }"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
SELECT 'TEST_SMALL';
|
||||
SELECT 'VALUE FROM RAM BUFFER';
|
||||
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('1', toInt32(3)));
|
||||
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('1', toInt32(3)));
|
||||
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('1', toInt32(3)));
|
||||
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('1', toInt32(3)));
|
||||
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('1', toInt32(3)));
|
||||
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('1', toInt32(3)));
|
||||
SELECT dictGetUInt64('_01280_db.ssd_dict', 'a', tuple('1', toInt32(3)));
|
||||
SELECT dictGetInt32('_01280_db.ssd_dict', 'b', tuple('1', toInt32(3)));
|
||||
SELECT dictGetString('_01280_db.ssd_dict', 'c', tuple('1', toInt32(3)));
|
||||
SELECT dictGetUInt64('_01280_db.ssd_dict', 'a', tuple('1', toInt32(3)));
|
||||
SELECT dictGetInt32('_01280_db.ssd_dict', 'b', tuple('1', toInt32(3)));
|
||||
SELECT dictGetString('_01280_db.ssd_dict', 'c', tuple('1', toInt32(3)));
|
||||
|
||||
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('2', toInt32(-1)));
|
||||
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('2', toInt32(-1)));
|
||||
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('2', toInt32(-1)));
|
||||
SELECT dictGetUInt64('_01280_db.ssd_dict', 'a', tuple('2', toInt32(-1)));
|
||||
SELECT dictGetInt32('_01280_db.ssd_dict', 'b', tuple('2', toInt32(-1)));
|
||||
SELECT dictGetString('_01280_db.ssd_dict', 'c', tuple('2', toInt32(-1)));
|
||||
|
||||
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('5', toInt32(-3)));
|
||||
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('5', toInt32(-3)));
|
||||
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('5', toInt32(-3)));
|
||||
SELECT dictGetUInt64('_01280_db.ssd_dict', 'a', tuple('5', toInt32(-3)));
|
||||
SELECT dictGetInt32('_01280_db.ssd_dict', 'b', tuple('5', toInt32(-3)));
|
||||
SELECT dictGetString('_01280_db.ssd_dict', 'c', tuple('5', toInt32(-3)));
|
||||
|
||||
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('10', toInt32(-20)));
|
||||
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('10', toInt32(-20)));
|
||||
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('10', toInt32(-20)));"
|
||||
SELECT dictGetUInt64('_01280_db.ssd_dict', 'a', tuple('10', toInt32(-20)));
|
||||
SELECT dictGetInt32('_01280_db.ssd_dict', 'b', tuple('10', toInt32(-20)));
|
||||
SELECT dictGetString('_01280_db.ssd_dict', 'c', tuple('10', toInt32(-20)));"
|
||||
|
||||
$CLICKHOUSE_CLIENT -nq "SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple(toInt32(3))); -- { serverError 53 }"
|
||||
$CLICKHOUSE_CLIENT -nq "SELECT dictGetUInt64('_01280_db.ssd_dict', 'a', tuple(toInt32(3))); -- { serverError 53 }"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="DROP DICTIONARY 01280_db.ssd_dict;
|
||||
DROP TABLE IF EXISTS 01280_db.keys_table;
|
||||
CREATE TABLE 01280_db.keys_table
|
||||
$CLICKHOUSE_CLIENT -n --query="DROP DICTIONARY _01280_db.ssd_dict;
|
||||
DROP TABLE IF EXISTS _01280_db.keys_table;
|
||||
CREATE TABLE _01280_db.keys_table
|
||||
(
|
||||
k1 String,
|
||||
k2 Int32
|
||||
)
|
||||
ENGINE = StripeLog();
|
||||
|
||||
INSERT INTO 01280_db.keys_table VALUES ('1', 3);
|
||||
INSERT INTO 01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370;
|
||||
INSERT INTO 01280_db.keys_table VALUES ('2', -1);
|
||||
INSERT INTO 01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370, 370;
|
||||
INSERT INTO 01280_db.keys_table VALUES ('5', -3);
|
||||
INSERT INTO 01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 700, 370;
|
||||
INSERT INTO 01280_db.keys_table VALUES ('10', -20);
|
||||
INSERT INTO _01280_db.keys_table VALUES ('1', 3);
|
||||
INSERT INTO _01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370;
|
||||
INSERT INTO _01280_db.keys_table VALUES ('2', -1);
|
||||
INSERT INTO _01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370, 370;
|
||||
INSERT INTO _01280_db.keys_table VALUES ('5', -3);
|
||||
INSERT INTO _01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 700, 370;
|
||||
INSERT INTO _01280_db.keys_table VALUES ('10', -20);
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01280_db.ssd_dict;CREATE DICTIONARY 01280_db.ssd_dict
|
||||
DROP DICTIONARY IF EXISTS _01280_db.ssd_dict;CREATE DICTIONARY _01280_db.ssd_dict
|
||||
(
|
||||
k1 String,
|
||||
k2 Int32,
|
||||
@ -94,35 +94,35 @@ $CLICKHOUSE_CLIENT -n --query="DROP DICTIONARY 01280_db.ssd_dict;
|
||||
c String DEFAULT 'none'
|
||||
)
|
||||
PRIMARY KEY k1, k2
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01280_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '_01280_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH '$USER_FILES_PATH/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
|
||||
|
||||
SELECT 'UPDATE DICTIONARY';
|
||||
|
||||
SELECT sum(dictGetUInt64('01280_db.ssd_dict', 'a', (k1, k2))) FROM 01280_db.keys_table;
|
||||
SELECT sum(dictGetUInt64('_01280_db.ssd_dict', 'a', (k1, k2))) FROM _01280_db.keys_table;
|
||||
|
||||
SELECT 'VALUE FROM DISK';
|
||||
SELECT dictGetInt32('01280_db.ssd_dict', 'b', ('1', toInt32(3)));
|
||||
SELECT dictGetString('01280_db.ssd_dict', 'c', ('1', toInt32(3)));
|
||||
SELECT dictGetInt32('_01280_db.ssd_dict', 'b', ('1', toInt32(3)));
|
||||
SELECT dictGetString('_01280_db.ssd_dict', 'c', ('1', toInt32(3)));
|
||||
|
||||
SELECT 'VALUE FROM RAM BUFFER';
|
||||
SELECT dictGetInt32('01280_db.ssd_dict', 'b', ('10', toInt32(-20)));
|
||||
SELECT dictGetString('01280_db.ssd_dict', 'c', ('10', toInt32(-20)));
|
||||
SELECT dictGetInt32('_01280_db.ssd_dict', 'b', ('10', toInt32(-20)));
|
||||
SELECT dictGetString('_01280_db.ssd_dict', 'c', ('10', toInt32(-20)));
|
||||
|
||||
SELECT 'VALUES FROM DISK AND RAM BUFFER';
|
||||
SELECT sum(dictGetUInt64('01280_db.ssd_dict', 'a', (k1, k2))) FROM 01280_db.keys_table;
|
||||
SELECT sum(dictGetUInt64('_01280_db.ssd_dict', 'a', (k1, k2))) FROM _01280_db.keys_table;
|
||||
|
||||
SELECT 'HAS';
|
||||
SELECT count() FROM 01280_db.keys_table WHERE dictHas('01280_db.ssd_dict', (k1, k2));
|
||||
SELECT count() FROM _01280_db.keys_table WHERE dictHas('_01280_db.ssd_dict', (k1, k2));
|
||||
|
||||
SELECT 'VALUES NOT FROM TABLE';
|
||||
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', ('unknown', toInt32(0))), dictGetInt32('01280_db.ssd_dict', 'b', ('unknown', toInt32(0))), dictGetString('01280_db.ssd_dict', 'c', ('unknown', toInt32(0)));
|
||||
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', ('unknown', toInt32(0))), dictGetInt32('01280_db.ssd_dict', 'b', ('unknown', toInt32(0))), dictGetString('01280_db.ssd_dict', 'c', ('unknown', toInt32(0)));
|
||||
SELECT dictGetUInt64('_01280_db.ssd_dict', 'a', ('unknown', toInt32(0))), dictGetInt32('_01280_db.ssd_dict', 'b', ('unknown', toInt32(0))), dictGetString('_01280_db.ssd_dict', 'c', ('unknown', toInt32(0)));
|
||||
SELECT dictGetUInt64('_01280_db.ssd_dict', 'a', ('unknown', toInt32(0))), dictGetInt32('_01280_db.ssd_dict', 'b', ('unknown', toInt32(0))), dictGetString('_01280_db.ssd_dict', 'c', ('unknown', toInt32(0)));
|
||||
|
||||
SELECT 'DUPLICATE KEYS';
|
||||
SELECT arrayJoin([('1', toInt32(3)), ('2', toInt32(-1)), ('', toInt32(0)), ('', toInt32(0)), ('2', toInt32(-1)), ('1', toInt32(3))]) AS keys, dictGetInt32('01280_db.ssd_dict', 'b', keys);
|
||||
SELECT arrayJoin([('1', toInt32(3)), ('2', toInt32(-1)), ('', toInt32(0)), ('', toInt32(0)), ('2', toInt32(-1)), ('1', toInt32(3))]) AS keys, dictGetInt32('_01280_db.ssd_dict', 'b', keys);
|
||||
DROP DICTIONARY IF EXISTS database_for_dict.ssd_dict;
|
||||
DROP TABLE IF EXISTS database_for_dict.keys_table;"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="DROP DATABASE IF EXISTS 01280_db;"
|
||||
$CLICKHOUSE_CLIENT -n --query="DROP DATABASE IF EXISTS _01280_db;"
|
||||
|
@ -2,49 +2,49 @@
|
||||
-- Tag no-ordinary-database: Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database
|
||||
-- Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default
|
||||
|
||||
DROP TABLE IF EXISTS 01504_test;
|
||||
DROP TABLE IF EXISTS _01504_test;
|
||||
|
||||
CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB; -- { serverError 36 }
|
||||
CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key2); -- { serverError 47 }
|
||||
CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key, value); -- { serverError 36 }
|
||||
CREATE TABLE 01504_test (key Tuple(String, UInt32), value UInt64) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
CREATE TABLE _01504_test (key String, value UInt32) Engine=EmbeddedRocksDB; -- { serverError 36 }
|
||||
CREATE TABLE _01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key2); -- { serverError 47 }
|
||||
CREATE TABLE _01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key, value); -- { serverError 36 }
|
||||
CREATE TABLE _01504_test (key Tuple(String, UInt32), value UInt64) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
|
||||
DROP TABLE IF EXISTS 01504_test;
|
||||
CREATE TABLE 01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
DROP TABLE IF EXISTS _01504_test;
|
||||
CREATE TABLE _01504_test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
|
||||
INSERT INTO 01504_test SELECT '1_1', number FROM numbers(10000);
|
||||
SELECT COUNT(1) == 1 FROM 01504_test;
|
||||
INSERT INTO _01504_test SELECT '1_1', number FROM numbers(10000);
|
||||
SELECT COUNT(1) == 1 FROM _01504_test;
|
||||
|
||||
INSERT INTO 01504_test SELECT concat(toString(number), '_1'), number FROM numbers(10000);
|
||||
SELECT COUNT(1) == 10000 FROM 01504_test;
|
||||
SELECT uniqExact(key) == 32 FROM (SELECT * FROM 01504_test LIMIT 32 SETTINGS max_block_size = 1);
|
||||
SELECT SUM(value) == 1 + 99 + 900 FROM 01504_test WHERE key IN ('1_1', '99_1', '900_1');
|
||||
INSERT INTO _01504_test SELECT concat(toString(number), '_1'), number FROM numbers(10000);
|
||||
SELECT COUNT(1) == 10000 FROM _01504_test;
|
||||
SELECT uniqExact(key) == 32 FROM (SELECT * FROM _01504_test LIMIT 32 SETTINGS max_block_size = 1);
|
||||
SELECT SUM(value) == 1 + 99 + 900 FROM _01504_test WHERE key IN ('1_1', '99_1', '900_1');
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS 01504_test;
|
||||
DROP TABLE IF EXISTS 01504_test_memory;
|
||||
DROP TABLE IF EXISTS _01504_test;
|
||||
DROP TABLE IF EXISTS _01504_test_memory;
|
||||
|
||||
CREATE TABLE 01504_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksDB PRIMARY KEY(k);
|
||||
CREATE TABLE 01504_test_memory AS 01504_test Engine = Memory;
|
||||
CREATE TABLE _01504_test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksDB PRIMARY KEY(k);
|
||||
CREATE TABLE _01504_test_memory AS _01504_test Engine = Memory;
|
||||
|
||||
INSERT INTO 01504_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k;
|
||||
INSERT INTO _01504_test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k;
|
||||
|
||||
INSERT INTO 01504_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k;
|
||||
INSERT INTO _01504_test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k;
|
||||
|
||||
|
||||
SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 01504_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM 01504_test_memory) B USING a ORDER BY a;
|
||||
SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM _01504_test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM _01504_test_memory) B USING a ORDER BY a;
|
||||
|
||||
CREATE TEMPORARY TABLE keys AS SELECT * FROM system.numbers LIMIT 1 OFFSET 4;
|
||||
|
||||
SET max_rows_to_read = 2;
|
||||
SELECT dummy == (1,1.2) FROM 01504_test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ;
|
||||
SELECT k == 4 FROM 01504_test WHERE k = 4 OR k IN [4] OR k in (4, 10000001, 10000002) AND value > 0;
|
||||
SELECT k == 4 FROM 01504_test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4);
|
||||
SELECT k, value FROM 01504_test WHERE k = 0 OR value > 0; -- { serverError 158 }
|
||||
SELECT k, value FROM 01504_test WHERE k = 0 AND k IN (1, 3) OR k > 8; -- { serverError 158 }
|
||||
SELECT dummy == (1,1.2) FROM _01504_test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ;
|
||||
SELECT k == 4 FROM _01504_test WHERE k = 4 OR k IN [4] OR k in (4, 10000001, 10000002) AND value > 0;
|
||||
SELECT k == 4 FROM _01504_test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4);
|
||||
SELECT k, value FROM _01504_test WHERE k = 0 OR value > 0; -- { serverError 158 }
|
||||
SELECT k, value FROM _01504_test WHERE k = 0 AND k IN (1, 3) OR k > 8; -- { serverError 158 }
|
||||
|
||||
TRUNCATE TABLE 01504_test;
|
||||
SELECT 0 == COUNT(1) FROM 01504_test;
|
||||
TRUNCATE TABLE _01504_test;
|
||||
SELECT 0 == COUNT(1) FROM _01504_test;
|
||||
|
||||
DROP TABLE IF EXISTS 01504_test;
|
||||
DROP TABLE IF EXISTS 01504_test_memory;
|
||||
DROP TABLE IF EXISTS _01504_test;
|
||||
DROP TABLE IF EXISTS _01504_test_memory;
|
||||
|
@ -5,20 +5,20 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
echo "SELECT 1;" > 01523_client_local_queries_file_parameter_tmp.sql
|
||||
$CLICKHOUSE_CLIENT --queries-file=01523_client_local_queries_file_parameter_tmp.sql 2>&1
|
||||
echo "SELECT 1;" > _01523_client_local_queries_file_parameter_tmp.sql
|
||||
$CLICKHOUSE_CLIENT --queries-file=_01523_client_local_queries_file_parameter_tmp.sql 2>&1
|
||||
|
||||
echo "CREATE TABLE 01523_test(value Int32) ENGINE=Log;
|
||||
INSERT INTO 01523_test
|
||||
echo "CREATE TABLE _01523_test(value Int32) ENGINE=Log;
|
||||
INSERT INTO _01523_test
|
||||
VALUES (1), (2), (3);
|
||||
SELECT * FROM 01523_test;
|
||||
DROP TABLE 01523_test;" > 01523_client_local_queries_file_parameter_tmp.sql
|
||||
$CLICKHOUSE_CLIENT --queries-file=01523_client_local_queries_file_parameter_tmp.sql 2>&1
|
||||
SELECT * FROM _01523_test;
|
||||
DROP TABLE _01523_test;" > _01523_client_local_queries_file_parameter_tmp.sql
|
||||
$CLICKHOUSE_CLIENT --queries-file=_01523_client_local_queries_file_parameter_tmp.sql 2>&1
|
||||
|
||||
echo "CREATE TABLE 01523_test (a Int64, b Int64) ENGINE = File(CSV, stdin);
|
||||
SELECT a, b FROM 01523_test;
|
||||
DROP TABLE 01523_test;" > 01523_client_local_queries_file_parameter_tmp.sql
|
||||
echo "CREATE TABLE _01523_test (a Int64, b Int64) ENGINE = File(CSV, stdin);
|
||||
SELECT a, b FROM _01523_test;
|
||||
DROP TABLE _01523_test;" > _01523_client_local_queries_file_parameter_tmp.sql
|
||||
|
||||
echo -e "1,2\n3,4" | $CLICKHOUSE_LOCAL --queries-file=01523_client_local_queries_file_parameter_tmp.sql 2>&1
|
||||
echo -e "1,2\n3,4" | $CLICKHOUSE_LOCAL --queries-file=_01523_client_local_queries_file_parameter_tmp.sql 2>&1
|
||||
|
||||
rm 01523_client_local_queries_file_parameter_tmp.sql
|
||||
rm _01523_client_local_queries_file_parameter_tmp.sql
|
||||
|
@ -1,25 +1,25 @@
|
||||
SET allow_experimental_bigint_types = 1;
|
||||
|
||||
CREATE TABLE 01154_test (x Int128, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192;
|
||||
INSERT INTO 01154_test VALUES (1), (2), (3);
|
||||
SELECT x FROM 01154_test WHERE x = 1;
|
||||
SELECT x FROM 01154_test WHERE x IN (1, 2);
|
||||
DROP TABLE 01154_test;
|
||||
CREATE TABLE _01154_test (x Int128, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192;
|
||||
INSERT INTO _01154_test VALUES (1), (2), (3);
|
||||
SELECT x FROM _01154_test WHERE x = 1;
|
||||
SELECT x FROM _01154_test WHERE x IN (1, 2);
|
||||
DROP TABLE _01154_test;
|
||||
|
||||
CREATE TABLE 01154_test (x Int256, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192;
|
||||
INSERT INTO 01154_test VALUES (1), (2), (3);
|
||||
SELECT x FROM 01154_test WHERE x = 1;
|
||||
SELECT x FROM 01154_test WHERE x IN (1, 2);
|
||||
DROP TABLE 01154_test;
|
||||
CREATE TABLE _01154_test (x Int256, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192;
|
||||
INSERT INTO _01154_test VALUES (1), (2), (3);
|
||||
SELECT x FROM _01154_test WHERE x = 1;
|
||||
SELECT x FROM _01154_test WHERE x IN (1, 2);
|
||||
DROP TABLE _01154_test;
|
||||
|
||||
CREATE TABLE 01154_test (x UInt256, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192;
|
||||
INSERT INTO 01154_test VALUES (1), (2), (3);
|
||||
SELECT x FROM 01154_test WHERE x = 1;
|
||||
SELECT x FROM 01154_test WHERE x IN (1, 2);
|
||||
DROP TABLE 01154_test;
|
||||
CREATE TABLE _01154_test (x UInt256, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192;
|
||||
INSERT INTO _01154_test VALUES (1), (2), (3);
|
||||
SELECT x FROM _01154_test WHERE x = 1;
|
||||
SELECT x FROM _01154_test WHERE x IN (1, 2);
|
||||
DROP TABLE _01154_test;
|
||||
|
||||
CREATE TABLE 01154_test (x UUID, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192;
|
||||
INSERT INTO 01154_test VALUES (toUUID('00000000-0000-0000-0000-000000000001')), (toUUID('00000000-0000-0000-0000-000000000002')), (toUUID('00000000-0000-0000-0000-000000000003'));
|
||||
SELECT x FROM 01154_test WHERE x = toUUID('00000000-0000-0000-0000-000000000001');
|
||||
SELECT x FROM 01154_test WHERE x IN (toUUID('00000000-0000-0000-0000-000000000001'), toUUID('00000000-0000-0000-0000-000000000002'));
|
||||
DROP TABLE 01154_test;
|
||||
CREATE TABLE _01154_test (x UUID, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192;
|
||||
INSERT INTO _01154_test VALUES (toUUID('00000000-0000-0000-0000-000000000001')), (toUUID('00000000-0000-0000-0000-000000000002')), (toUUID('00000000-0000-0000-0000-000000000003'));
|
||||
SELECT x FROM _01154_test WHERE x = toUUID('00000000-0000-0000-0000-000000000001');
|
||||
SELECT x FROM _01154_test WHERE x IN (toUUID('00000000-0000-0000-0000-000000000001'), toUUID('00000000-0000-0000-0000-000000000002'));
|
||||
DROP TABLE _01154_test;
|
||||
|
@ -1,9 +1,9 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01681_database_for_cache_dictionary;
|
||||
CREATE DATABASE 01681_database_for_cache_dictionary;
|
||||
DROP DATABASE IF EXISTS _01681_database_for_cache_dictionary;
|
||||
CREATE DATABASE _01681_database_for_cache_dictionary;
|
||||
|
||||
CREATE TABLE 01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table
|
||||
CREATE TABLE _01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
@ -11,11 +11,11 @@ CREATE TABLE 01681_database_for_cache_dictionary.simple_key_simple_attributes_so
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO 01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
INSERT INTO _01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO _01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO _01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes
|
||||
CREATE DICTIONARY _01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -28,26 +28,26 @@ LAYOUT(CACHE(SIZE_IN_CELLS 10));
|
||||
|
||||
SELECT 'Dictionary cache_dictionary_simple_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes ORDER BY id;
|
||||
SELECT * FROM _01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes;
|
||||
DROP TABLE 01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table;
|
||||
DROP DICTIONARY _01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes;
|
||||
DROP TABLE _01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table
|
||||
CREATE TABLE _01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
@ -55,11 +55,11 @@ CREATE TABLE 01681_database_for_cache_dictionary.simple_key_complex_attributes_s
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO 01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
INSERT INTO _01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO _01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO _01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes
|
||||
CREATE DICTIONARY _01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -72,37 +72,37 @@ LAYOUT(CACHE(SIZE_IN_CELLS 10));
|
||||
|
||||
SELECT 'Dictionary cache_dictionary_simple_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes ORDER BY id;
|
||||
SELECT * FROM _01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes;
|
||||
DROP TABLE 01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table;
|
||||
DROP DICTIONARY _01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes;
|
||||
DROP TABLE _01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01681_database_for_cache_dictionary.simple_key_hierarchy_table
|
||||
CREATE TABLE _01681_database_for_cache_dictionary.simple_key_hierarchy_table
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO 01681_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO 01681_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO 01681_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO 01681_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (4, 2);
|
||||
INSERT INTO _01681_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO _01681_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO _01681_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO _01681_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (4, 2);
|
||||
|
||||
CREATE DICTIONARY 01681_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy
|
||||
CREATE DICTIONARY _01681_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
@ -114,12 +114,12 @@ LAYOUT(CACHE(SIZE_IN_CELLS 10));
|
||||
|
||||
SELECT 'Dictionary cache_dictionary_simple_key_hierarchy';
|
||||
SELECT 'dictGet';
|
||||
SELECT dictGet('01681_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT dictGet('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT 'dictGetHierarchy';
|
||||
SELECT dictGetHierarchy('01681_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('01681_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
SELECT dictGetHierarchy('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('_01681_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
|
||||
DROP DICTIONARY 01681_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy;
|
||||
DROP TABLE 01681_database_for_cache_dictionary.simple_key_hierarchy_table;
|
||||
DROP DICTIONARY _01681_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy;
|
||||
DROP TABLE _01681_database_for_cache_dictionary.simple_key_hierarchy_table;
|
||||
|
||||
DROP DATABASE 01681_database_for_cache_dictionary;
|
||||
DROP DATABASE _01681_database_for_cache_dictionary;
|
||||
|
@ -1,9 +1,9 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01682_database_for_cache_dictionary;
|
||||
CREATE DATABASE 01682_database_for_cache_dictionary;
|
||||
DROP DATABASE IF EXISTS _01682_database_for_cache_dictionary;
|
||||
CREATE DATABASE _01682_database_for_cache_dictionary;
|
||||
|
||||
CREATE TABLE 01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table
|
||||
CREATE TABLE _01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -12,11 +12,11 @@ CREATE TABLE 01682_database_for_cache_dictionary.complex_key_simple_attributes_s
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO 01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
|
||||
INSERT INTO 01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
INSERT INTO _01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO _01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
|
||||
INSERT INTO _01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes
|
||||
CREATE DICTIONARY _01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -24,32 +24,32 @@ CREATE DICTIONARY 01682_database_for_cache_dictionary.cache_dictionary_complex_k
|
||||
value_second String DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '01682_database_for_cache_dictionary'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '_01682_database_for_cache_dictionary'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10));
|
||||
|
||||
SELECT 'Dictionary cache_dictionary_complex_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes ORDER BY id;
|
||||
SELECT * FROM _01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes;
|
||||
DROP TABLE 01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table;
|
||||
DROP DICTIONARY _01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes;
|
||||
DROP TABLE _01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table
|
||||
CREATE TABLE _01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -58,11 +58,11 @@ CREATE TABLE 01682_database_for_cache_dictionary.complex_key_complex_attributes_
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO 01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
|
||||
INSERT INTO 01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
INSERT INTO _01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO _01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
|
||||
INSERT INTO _01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes
|
||||
CREATE DICTIONARY _01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -71,29 +71,29 @@ CREATE DICTIONARY 01682_database_for_cache_dictionary.cache_dictionary_complex_k
|
||||
value_second Nullable(String) DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '01682_database_for_cache_dictionary'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '_01682_database_for_cache_dictionary'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10));
|
||||
|
||||
SELECT 'Dictionary cache_dictionary_complex_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes ORDER BY id;
|
||||
SELECT * FROM _01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes;
|
||||
DROP TABLE 01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table;
|
||||
DROP DICTIONARY _01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes;
|
||||
DROP TABLE _01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table;
|
||||
|
||||
DROP DATABASE 01682_database_for_cache_dictionary;
|
||||
DROP DATABASE _01682_database_for_cache_dictionary;
|
||||
|
@ -1,9 +1,9 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01681_database_for_flat_dictionary;
|
||||
CREATE DATABASE 01681_database_for_flat_dictionary;
|
||||
DROP DATABASE IF EXISTS _01681_database_for_flat_dictionary;
|
||||
CREATE DATABASE _01681_database_for_flat_dictionary;
|
||||
|
||||
CREATE TABLE 01681_database_for_flat_dictionary.simple_key_simple_attributes_source_table
|
||||
CREATE TABLE _01681_database_for_flat_dictionary.simple_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
@ -11,11 +11,11 @@ CREATE TABLE 01681_database_for_flat_dictionary.simple_key_simple_attributes_sou
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01681_database_for_flat_dictionary.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01681_database_for_flat_dictionary.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO 01681_database_for_flat_dictionary.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
INSERT INTO _01681_database_for_flat_dictionary.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO _01681_database_for_flat_dictionary.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO _01681_database_for_flat_dictionary.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes
|
||||
CREATE DICTIONARY _01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -28,24 +28,24 @@ LAYOUT(FLAT());
|
||||
|
||||
SELECT 'Dictionary flat_dictionary_simple_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
|
||||
DROP DICTIONARY 01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes;
|
||||
DROP TABLE 01681_database_for_flat_dictionary.simple_key_simple_attributes_source_table;
|
||||
DROP DICTIONARY _01681_database_for_flat_dictionary.flat_dictionary_simple_key_simple_attributes;
|
||||
DROP TABLE _01681_database_for_flat_dictionary.simple_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01681_database_for_flat_dictionary.simple_key_complex_attributes_source_table
|
||||
CREATE TABLE _01681_database_for_flat_dictionary.simple_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
@ -53,11 +53,11 @@ CREATE TABLE 01681_database_for_flat_dictionary.simple_key_complex_attributes_so
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01681_database_for_flat_dictionary.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01681_database_for_flat_dictionary.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO 01681_database_for_flat_dictionary.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
INSERT INTO _01681_database_for_flat_dictionary.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO _01681_database_for_flat_dictionary.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO _01681_database_for_flat_dictionary.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes
|
||||
CREATE DICTIONARY _01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -70,35 +70,35 @@ LAYOUT(FLAT());
|
||||
|
||||
SELECT 'Dictionary flat_dictionary_simple_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
|
||||
DROP DICTIONARY 01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes;
|
||||
DROP TABLE 01681_database_for_flat_dictionary.simple_key_complex_attributes_source_table;
|
||||
DROP DICTIONARY _01681_database_for_flat_dictionary.flat_dictionary_simple_key_complex_attributes;
|
||||
DROP TABLE _01681_database_for_flat_dictionary.simple_key_complex_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01681_database_for_flat_dictionary.simple_key_hierarchy_table
|
||||
CREATE TABLE _01681_database_for_flat_dictionary.simple_key_hierarchy_table
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO 01681_database_for_flat_dictionary.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO 01681_database_for_flat_dictionary.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO 01681_database_for_flat_dictionary.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO 01681_database_for_flat_dictionary.simple_key_hierarchy_table VALUES (4, 2);
|
||||
INSERT INTO _01681_database_for_flat_dictionary.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO _01681_database_for_flat_dictionary.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO _01681_database_for_flat_dictionary.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO _01681_database_for_flat_dictionary.simple_key_hierarchy_table VALUES (4, 2);
|
||||
|
||||
CREATE DICTIONARY 01681_database_for_flat_dictionary.flat_dictionary_simple_key_hierarchy
|
||||
CREATE DICTIONARY _01681_database_for_flat_dictionary.flat_dictionary_simple_key_hierarchy
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
@ -110,12 +110,12 @@ LAYOUT(FLAT());
|
||||
|
||||
SELECT 'Dictionary flat_dictionary_simple_key_hierarchy';
|
||||
SELECT 'dictGet';
|
||||
SELECT dictGet('01681_database_for_flat_dictionary.flat_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT dictGet('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT 'dictGetHierarchy';
|
||||
SELECT dictGetHierarchy('01681_database_for_flat_dictionary.flat_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('01681_database_for_flat_dictionary.flat_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
SELECT dictGetHierarchy('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('_01681_database_for_flat_dictionary.flat_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
|
||||
DROP DICTIONARY 01681_database_for_flat_dictionary.flat_dictionary_simple_key_hierarchy;
|
||||
DROP TABLE 01681_database_for_flat_dictionary.simple_key_hierarchy_table;
|
||||
DROP DICTIONARY _01681_database_for_flat_dictionary.flat_dictionary_simple_key_hierarchy;
|
||||
DROP TABLE _01681_database_for_flat_dictionary.simple_key_hierarchy_table;
|
||||
|
||||
DROP DATABASE 01681_database_for_flat_dictionary;
|
||||
DROP DATABASE _01681_database_for_flat_dictionary;
|
||||
|
@ -8,10 +8,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
DROP DATABASE IF EXISTS 01684_database_for_cache_dictionary;
|
||||
CREATE DATABASE 01684_database_for_cache_dictionary;
|
||||
DROP DATABASE IF EXISTS _01684_database_for_cache_dictionary;
|
||||
CREATE DATABASE _01684_database_for_cache_dictionary;
|
||||
|
||||
CREATE TABLE 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table
|
||||
CREATE TABLE _01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
@ -19,11 +19,11 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
INSERT INTO _01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO _01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO _01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes
|
||||
CREATE DICTIONARY _01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -36,26 +36,26 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
|
||||
SELECT 'Dictionary cache_dictionary_simple_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes ORDER BY id;
|
||||
SELECT * FROM _01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes;
|
||||
DROP TABLE 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table;
|
||||
DROP DICTIONARY _01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes;
|
||||
DROP TABLE _01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table
|
||||
CREATE TABLE _01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
@ -63,11 +63,11 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
INSERT INTO _01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO _01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO _01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes
|
||||
CREATE DICTIONARY _01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -80,37 +80,37 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
|
||||
SELECT 'Dictionary cache_dictionary_simple_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes ORDER BY id;
|
||||
SELECT * FROM _01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes;
|
||||
DROP TABLE 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table;
|
||||
DROP DICTIONARY _01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes;
|
||||
DROP TABLE _01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01684_database_for_cache_dictionary.simple_key_hierarchy_table
|
||||
CREATE TABLE _01684_database_for_cache_dictionary.simple_key_hierarchy_table
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (4, 2);
|
||||
INSERT INTO _01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO _01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO _01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO _01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (4, 2);
|
||||
|
||||
CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy
|
||||
CREATE DICTIONARY _01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
@ -122,12 +122,12 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
|
||||
SELECT 'Dictionary cache_dictionary_simple_key_hierarchy';
|
||||
SELECT 'dictGet';
|
||||
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT dictGet('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT 'dictGetHierarchy';
|
||||
SELECT dictGetHierarchy('01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
SELECT dictGetHierarchy('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('_01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
|
||||
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy;
|
||||
DROP TABLE 01684_database_for_cache_dictionary.simple_key_hierarchy_table;
|
||||
DROP DICTIONARY _01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy;
|
||||
DROP TABLE _01684_database_for_cache_dictionary.simple_key_hierarchy_table;
|
||||
|
||||
DROP DATABASE 01684_database_for_cache_dictionary;"
|
||||
DROP DATABASE _01684_database_for_cache_dictionary;"
|
||||
|
@ -8,10 +8,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
DROP DATABASE IF EXISTS 01685_database_for_cache_dictionary;
|
||||
CREATE DATABASE 01685_database_for_cache_dictionary;
|
||||
DROP DATABASE IF EXISTS _01685_database_for_cache_dictionary;
|
||||
CREATE DATABASE _01685_database_for_cache_dictionary;
|
||||
|
||||
CREATE TABLE 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table
|
||||
CREATE TABLE _01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -20,11 +20,11 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
|
||||
INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
INSERT INTO _01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO _01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
|
||||
INSERT INTO _01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes
|
||||
CREATE DICTIONARY _01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -32,32 +32,32 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
value_second String DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '01685_database_for_cache_dictionary'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '_01685_database_for_cache_dictionary'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '$USER_FILES_PATH/0d'));
|
||||
|
||||
SELECT 'Dictionary cache_dictionary_complex_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes ORDER BY id;
|
||||
SELECT * FROM _01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes;
|
||||
DROP TABLE 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table;
|
||||
DROP DICTIONARY _01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes;
|
||||
DROP TABLE _01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table
|
||||
CREATE TABLE _01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -66,11 +66,11 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
|
||||
INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
INSERT INTO _01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO _01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
|
||||
INSERT INTO _01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes
|
||||
CREATE DICTIONARY _01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -79,29 +79,29 @@ $CLICKHOUSE_CLIENT -n --query="
|
||||
value_second Nullable(String) DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '01685_database_for_cache_dictionary'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '_01685_database_for_cache_dictionary'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '$USER_FILES_PATH/1d'));
|
||||
|
||||
SELECT 'Dictionary cache_dictionary_complex_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes ORDER BY id;
|
||||
SELECT * FROM _01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes;
|
||||
DROP TABLE 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table;
|
||||
DROP DICTIONARY _01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes;
|
||||
DROP TABLE _01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table;
|
||||
|
||||
DROP DATABASE 01685_database_for_cache_dictionary;"
|
||||
DROP DATABASE _01685_database_for_cache_dictionary;"
|
||||
|
@ -2,31 +2,31 @@
|
||||
-- Tag no-ordinary-database: Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database
|
||||
-- Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default
|
||||
|
||||
DROP TABLE IF EXISTS 01686_test;
|
||||
DROP TABLE IF EXISTS _01686_test;
|
||||
|
||||
CREATE TABLE 01686_test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
CREATE TABLE _01686_test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key);
|
||||
|
||||
SELECT value FROM system.rocksdb WHERE database = currentDatabase() and table = '01686_test' and name = 'number.keys.written';
|
||||
INSERT INTO 01686_test SELECT number, format('Hello, world ({})', toString(number)) FROM numbers(10000);
|
||||
SELECT value FROM system.rocksdb WHERE database = currentDatabase() and table = '01686_test' and name = 'number.keys.written';
|
||||
SELECT value FROM system.rocksdb WHERE database = currentDatabase() and table = '_01686_test' and name = 'number.keys.written';
|
||||
INSERT INTO _01686_test SELECT number, format('Hello, world ({})', toString(number)) FROM numbers(10000);
|
||||
SELECT value FROM system.rocksdb WHERE database = currentDatabase() and table = '_01686_test' and name = 'number.keys.written';
|
||||
|
||||
SELECT * FROM 01686_test WHERE key = 123;
|
||||
SELECT * FROM _01686_test WHERE key = 123;
|
||||
SELECT '--';
|
||||
SELECT * FROM 01686_test WHERE key = -123;
|
||||
SELECT * FROM _01686_test WHERE key = -123;
|
||||
SELECT '--';
|
||||
SELECT * FROM 01686_test WHERE key = 123 OR key = 4567 ORDER BY key;
|
||||
SELECT * FROM _01686_test WHERE key = 123 OR key = 4567 ORDER BY key;
|
||||
SELECT '--';
|
||||
SELECT * FROM 01686_test WHERE key = NULL;
|
||||
SELECT * FROM _01686_test WHERE key = NULL;
|
||||
SELECT '--';
|
||||
SELECT * FROM 01686_test WHERE key = NULL OR key = 0;
|
||||
SELECT * FROM _01686_test WHERE key = NULL OR key = 0;
|
||||
SELECT '--';
|
||||
SELECT * FROM 01686_test WHERE key IN (123, 456, -123) ORDER BY key;
|
||||
SELECT * FROM _01686_test WHERE key IN (123, 456, -123) ORDER BY key;
|
||||
SELECT '--';
|
||||
SELECT * FROM 01686_test WHERE key = 'Hello'; -- { serverError 53 }
|
||||
SELECT * FROM _01686_test WHERE key = 'Hello'; -- { serverError 53 }
|
||||
|
||||
DETACH TABLE 01686_test NO DELAY;
|
||||
ATTACH TABLE 01686_test;
|
||||
DETACH TABLE _01686_test NO DELAY;
|
||||
ATTACH TABLE _01686_test;
|
||||
|
||||
SELECT * FROM 01686_test WHERE key IN (99, 999, 9999, -123) ORDER BY key;
|
||||
SELECT * FROM _01686_test WHERE key IN (99, 999, 9999, -123) ORDER BY key;
|
||||
|
||||
DROP TABLE IF EXISTS 01686_test;
|
||||
DROP TABLE IF EXISTS _01686_test;
|
||||
|
@ -1,30 +1,30 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01720_dictionary_db;
|
||||
CREATE DATABASE 01720_dictionary_db;
|
||||
DROP DATABASE IF EXISTS _01720_dictionary_db;
|
||||
CREATE DATABASE _01720_dictionary_db;
|
||||
|
||||
CREATE TABLE 01720_dictionary_db.dictionary_source_table
|
||||
CREATE TABLE _01720_dictionary_db.dictionary_source_table
|
||||
(
|
||||
key UInt8,
|
||||
value String
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01720_dictionary_db.dictionary_source_table VALUES (1, 'First');
|
||||
INSERT INTO _01720_dictionary_db.dictionary_source_table VALUES (1, 'First');
|
||||
|
||||
CREATE DICTIONARY 01720_dictionary_db.dictionary
|
||||
CREATE DICTIONARY _01720_dictionary_db.dictionary
|
||||
(
|
||||
key UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key
|
||||
SOURCE(CLICKHOUSE(DB '01720_dictionary_db' TABLE 'dictionary_source_table' HOST hostName() PORT tcpPort()))
|
||||
SOURCE(CLICKHOUSE(DB '_01720_dictionary_db' TABLE 'dictionary_source_table' HOST hostName() PORT tcpPort()))
|
||||
LIFETIME(0)
|
||||
LAYOUT(FLAT());
|
||||
|
||||
SELECT * FROM 01720_dictionary_db.dictionary;
|
||||
SELECT * FROM _01720_dictionary_db.dictionary;
|
||||
|
||||
DROP DICTIONARY 01720_dictionary_db.dictionary;
|
||||
DROP TABLE 01720_dictionary_db.dictionary_source_table;
|
||||
DROP DICTIONARY _01720_dictionary_db.dictionary;
|
||||
DROP TABLE _01720_dictionary_db.dictionary_source_table;
|
||||
|
||||
DROP DATABASE 01720_dictionary_db;
|
||||
DROP DATABASE _01720_dictionary_db;
|
||||
|
@ -1,9 +1,9 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01753_dictionary_db;
|
||||
CREATE DATABASE 01753_dictionary_db;
|
||||
DROP DATABASE IF EXISTS _01753_dictionary_db;
|
||||
CREATE DATABASE _01753_dictionary_db;
|
||||
|
||||
CREATE TABLE 01753_dictionary_db.simple_key_simple_attributes_source_table
|
||||
CREATE TABLE _01753_dictionary_db.simple_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
@ -11,11 +11,11 @@ CREATE TABLE 01753_dictionary_db.simple_key_simple_attributes_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01753_dictionary_db.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01753_dictionary_db.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO 01753_dictionary_db.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
INSERT INTO _01753_dictionary_db.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO _01753_dictionary_db.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO _01753_dictionary_db.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01753_dictionary_db.direct_dictionary_simple_key_simple_attributes
|
||||
CREATE DICTIONARY _01753_dictionary_db.direct_dictionary_simple_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -27,26 +27,26 @@ LAYOUT(DIRECT());
|
||||
|
||||
SELECT 'Dictionary direct_dictionary_simple_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01753_dictionary_db.direct_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01753_dictionary_db.direct_dictionary_simple_key_simple_attributes;
|
||||
SELECT * FROM _01753_dictionary_db.direct_dictionary_simple_key_simple_attributes;
|
||||
|
||||
DROP DICTIONARY 01753_dictionary_db.direct_dictionary_simple_key_simple_attributes;
|
||||
DROP TABLE 01753_dictionary_db.simple_key_simple_attributes_source_table;
|
||||
DROP DICTIONARY _01753_dictionary_db.direct_dictionary_simple_key_simple_attributes;
|
||||
DROP TABLE _01753_dictionary_db.simple_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01753_dictionary_db.simple_key_complex_attributes_source_table
|
||||
CREATE TABLE _01753_dictionary_db.simple_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
@ -54,11 +54,11 @@ CREATE TABLE 01753_dictionary_db.simple_key_complex_attributes_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01753_dictionary_db.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01753_dictionary_db.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO 01753_dictionary_db.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
INSERT INTO _01753_dictionary_db.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO _01753_dictionary_db.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO _01753_dictionary_db.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01753_dictionary_db.direct_dictionary_simple_key_complex_attributes
|
||||
CREATE DICTIONARY _01753_dictionary_db.direct_dictionary_simple_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -70,37 +70,37 @@ LAYOUT(DIRECT());
|
||||
|
||||
SELECT 'Dictionary direct_dictionary_simple_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01753_dictionary_db.direct_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01753_dictionary_db.direct_dictionary_simple_key_complex_attributes;
|
||||
SELECT * FROM _01753_dictionary_db.direct_dictionary_simple_key_complex_attributes;
|
||||
|
||||
DROP DICTIONARY 01753_dictionary_db.direct_dictionary_simple_key_complex_attributes;
|
||||
DROP TABLE 01753_dictionary_db.simple_key_complex_attributes_source_table;
|
||||
DROP DICTIONARY _01753_dictionary_db.direct_dictionary_simple_key_complex_attributes;
|
||||
DROP TABLE _01753_dictionary_db.simple_key_complex_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01753_dictionary_db.simple_key_hierarchy_table
|
||||
CREATE TABLE _01753_dictionary_db.simple_key_hierarchy_table
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO 01753_dictionary_db.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO 01753_dictionary_db.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO 01753_dictionary_db.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO 01753_dictionary_db.simple_key_hierarchy_table VALUES (4, 2);
|
||||
INSERT INTO _01753_dictionary_db.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO _01753_dictionary_db.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO _01753_dictionary_db.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO _01753_dictionary_db.simple_key_hierarchy_table VALUES (4, 2);
|
||||
|
||||
CREATE DICTIONARY 01753_dictionary_db.direct_dictionary_simple_key_hierarchy
|
||||
CREATE DICTIONARY _01753_dictionary_db.direct_dictionary_simple_key_hierarchy
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
@ -111,12 +111,12 @@ LAYOUT(DIRECT());
|
||||
|
||||
SELECT 'Dictionary direct_dictionary_simple_key_hierarchy';
|
||||
SELECT 'dictGet';
|
||||
SELECT dictGet('01753_dictionary_db.direct_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT dictGet('_01753_dictionary_db.direct_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT 'dictGetHierarchy';
|
||||
SELECT dictGetHierarchy('01753_dictionary_db.direct_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('01753_dictionary_db.direct_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
SELECT dictGetHierarchy('_01753_dictionary_db.direct_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('_01753_dictionary_db.direct_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
|
||||
DROP DICTIONARY 01753_dictionary_db.direct_dictionary_simple_key_hierarchy;
|
||||
DROP TABLE 01753_dictionary_db.simple_key_hierarchy_table;
|
||||
DROP DICTIONARY _01753_dictionary_db.direct_dictionary_simple_key_hierarchy;
|
||||
DROP TABLE _01753_dictionary_db.simple_key_hierarchy_table;
|
||||
|
||||
DROP DATABASE 01753_dictionary_db;
|
||||
DROP DATABASE _01753_dictionary_db;
|
||||
|
@ -1,9 +1,9 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01754_dictionary_db;
|
||||
CREATE DATABASE 01754_dictionary_db;
|
||||
DROP DATABASE IF EXISTS _01754_dictionary_db;
|
||||
CREATE DATABASE _01754_dictionary_db;
|
||||
|
||||
CREATE TABLE 01754_dictionary_db.complex_key_simple_attributes_source_table
|
||||
CREATE TABLE _01754_dictionary_db.complex_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -12,11 +12,11 @@ CREATE TABLE 01754_dictionary_db.complex_key_simple_attributes_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01754_dictionary_db.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO 01754_dictionary_db.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
|
||||
INSERT INTO 01754_dictionary_db.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
INSERT INTO _01754_dictionary_db.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO _01754_dictionary_db.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
|
||||
INSERT INTO _01754_dictionary_db.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01754_dictionary_db.direct_dictionary_complex_key_simple_attributes
|
||||
CREATE DICTIONARY _01754_dictionary_db.direct_dictionary_complex_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
id_key String DEFAULT 'test_default_id_key',
|
||||
@ -29,26 +29,26 @@ LAYOUT(COMPLEX_KEY_DIRECT());
|
||||
|
||||
SELECT 'Dictionary direct_dictionary_complex_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01754_dictionary_db.direct_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01754_dictionary_db.direct_dictionary_complex_key_simple_attributes;
|
||||
SELECT * FROM _01754_dictionary_db.direct_dictionary_complex_key_simple_attributes;
|
||||
|
||||
DROP DICTIONARY 01754_dictionary_db.direct_dictionary_complex_key_simple_attributes;
|
||||
DROP TABLE 01754_dictionary_db.complex_key_simple_attributes_source_table;
|
||||
DROP DICTIONARY _01754_dictionary_db.direct_dictionary_complex_key_simple_attributes;
|
||||
DROP TABLE _01754_dictionary_db.complex_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01754_dictionary_db.complex_key_complex_attributes_source_table
|
||||
CREATE TABLE _01754_dictionary_db.complex_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -57,11 +57,11 @@ CREATE TABLE 01754_dictionary_db.complex_key_complex_attributes_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01754_dictionary_db.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO 01754_dictionary_db.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
|
||||
INSERT INTO 01754_dictionary_db.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
INSERT INTO _01754_dictionary_db.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO _01754_dictionary_db.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
|
||||
INSERT INTO _01754_dictionary_db.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01754_dictionary_db.direct_dictionary_complex_key_complex_attributes
|
||||
CREATE DICTIONARY _01754_dictionary_db.direct_dictionary_complex_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -75,23 +75,23 @@ LAYOUT(COMPLEX_KEY_DIRECT());
|
||||
|
||||
SELECT 'Dictionary direct_dictionary_complex_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01754_dictionary_db.direct_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01754_dictionary_db.direct_dictionary_complex_key_complex_attributes;
|
||||
SELECT * FROM _01754_dictionary_db.direct_dictionary_complex_key_complex_attributes;
|
||||
|
||||
DROP DICTIONARY 01754_dictionary_db.direct_dictionary_complex_key_complex_attributes;
|
||||
DROP TABLE 01754_dictionary_db.complex_key_complex_attributes_source_table;
|
||||
DROP DICTIONARY _01754_dictionary_db.direct_dictionary_complex_key_complex_attributes;
|
||||
DROP TABLE _01754_dictionary_db.complex_key_complex_attributes_source_table;
|
||||
|
||||
DROP DATABASE 01754_dictionary_db;
|
||||
DROP DATABASE _01754_dictionary_db;
|
||||
|
@ -1,9 +1,9 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01759_db;
|
||||
CREATE DATABASE 01759_db;
|
||||
DROP DATABASE IF EXISTS _01759_db;
|
||||
CREATE DATABASE _01759_db;
|
||||
|
||||
CREATE TABLE 01759_db.dictionary_source_table
|
||||
CREATE TABLE _01759_db.dictionary_source_table
|
||||
(
|
||||
key UInt64,
|
||||
value1 UInt64,
|
||||
@ -11,19 +11,19 @@ CREATE TABLE 01759_db.dictionary_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01759_db.dictionary_source_table VALUES (0, 2, 3), (1, 5, 6), (2, 8, 9);
|
||||
INSERT INTO _01759_db.dictionary_source_table VALUES (0, 2, 3), (1, 5, 6), (2, 8, 9);
|
||||
|
||||
CREATE DICTIONARY 01759_db.test_dictionary(key UInt64, value1 UInt64, value1 UInt64)
|
||||
CREATE DICTIONARY _01759_db.test_dictionary(key UInt64, value1 UInt64, value1 UInt64)
|
||||
PRIMARY KEY key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table' DB '01759_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table' DB '_01759_db'))
|
||||
LAYOUT(COMPLEX_KEY_DIRECT()); -- {serverError 36}
|
||||
|
||||
CREATE DICTIONARY 01759_db.test_dictionary(key UInt64, value1 UInt64, value2 UInt64)
|
||||
CREATE DICTIONARY _01759_db.test_dictionary(key UInt64, value1 UInt64, value2 UInt64)
|
||||
PRIMARY KEY key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table' DB '01759_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dictionary_source_table' DB '_01759_db'))
|
||||
LAYOUT(COMPLEX_KEY_DIRECT());
|
||||
|
||||
SELECT number, dictGet('01759_db.test_dictionary', 'value1', tuple(number)) as value1,
|
||||
dictGet('01759_db.test_dictionary', 'value2', tuple(number)) as value2 FROM system.numbers LIMIT 3;
|
||||
SELECT number, dictGet('_01759_db.test_dictionary', 'value1', tuple(number)) as value1,
|
||||
dictGet('_01759_db.test_dictionary', 'value2', tuple(number)) as value2 FROM system.numbers LIMIT 3;
|
||||
|
||||
DROP DATABASE 01759_db;
|
||||
DROP DATABASE _01759_db;
|
||||
|
@ -1,25 +1,25 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01760_db;
|
||||
CREATE DATABASE 01760_db;
|
||||
DROP DATABASE IF EXISTS _01760_db;
|
||||
CREATE DATABASE _01760_db;
|
||||
|
||||
DROP TABLE IF EXISTS 01760_db.polygons;
|
||||
CREATE TABLE 01760_db.polygons (key Array(Array(Array(Tuple(Float64, Float64)))), name String, value UInt64, value_nullable Nullable(UInt64)) ENGINE = Memory;
|
||||
INSERT INTO 01760_db.polygons VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Click East', 421, 421);
|
||||
INSERT INTO 01760_db.polygons VALUES ([[[(-1, 1), (1, 1), (1, 3), (-1, 3)]]], 'Click North', 422, NULL);
|
||||
INSERT INTO 01760_db.polygons VALUES ([[[(-3, 1), (-3, -1), (0, -1), (0, 1)]]], 'Click South', 423, 423);
|
||||
INSERT INTO 01760_db.polygons VALUES ([[[(-1, -1), (1, -1), (1, -3), (-1, -3)]]], 'Click West', 424, NULL);
|
||||
DROP TABLE IF EXISTS _01760_db.polygons;
|
||||
CREATE TABLE _01760_db.polygons (key Array(Array(Array(Tuple(Float64, Float64)))), name String, value UInt64, value_nullable Nullable(UInt64)) ENGINE = Memory;
|
||||
INSERT INTO _01760_db.polygons VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Click East', 421, 421);
|
||||
INSERT INTO _01760_db.polygons VALUES ([[[(-1, 1), (1, 1), (1, 3), (-1, 3)]]], 'Click North', 422, NULL);
|
||||
INSERT INTO _01760_db.polygons VALUES ([[[(-3, 1), (-3, -1), (0, -1), (0, 1)]]], 'Click South', 423, 423);
|
||||
INSERT INTO _01760_db.polygons VALUES ([[[(-1, -1), (1, -1), (1, -3), (-1, -3)]]], 'Click West', 424, NULL);
|
||||
|
||||
DROP TABLE IF EXISTS 01760_db.points;
|
||||
CREATE TABLE 01760_db.points (x Float64, y Float64, def_i UInt64, def_s String) ENGINE = Memory;
|
||||
INSERT INTO 01760_db.points VALUES (0.1, 0.0, 112, 'aax');
|
||||
INSERT INTO 01760_db.points VALUES (-0.1, 0.0, 113, 'aay');
|
||||
INSERT INTO 01760_db.points VALUES (0.0, 1.1, 114, 'aaz');
|
||||
INSERT INTO 01760_db.points VALUES (0.0, -1.1, 115, 'aat');
|
||||
INSERT INTO 01760_db.points VALUES (3.0, 3.0, 22, 'bb');
|
||||
DROP TABLE IF EXISTS _01760_db.points;
|
||||
CREATE TABLE _01760_db.points (x Float64, y Float64, def_i UInt64, def_s String) ENGINE = Memory;
|
||||
INSERT INTO _01760_db.points VALUES (0.1, 0.0, 112, 'aax');
|
||||
INSERT INTO _01760_db.points VALUES (-0.1, 0.0, 113, 'aay');
|
||||
INSERT INTO _01760_db.points VALUES (0.0, 1.1, 114, 'aaz');
|
||||
INSERT INTO _01760_db.points VALUES (0.0, -1.1, 115, 'aat');
|
||||
INSERT INTO _01760_db.points VALUES (3.0, 3.0, 22, 'bb');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01760_db.dict_array;
|
||||
CREATE DICTIONARY 01760_db.dict_array
|
||||
DROP DICTIONARY IF EXISTS _01760_db.dict_array;
|
||||
CREATE DICTIONARY _01760_db.dict_array
|
||||
(
|
||||
key Array(Array(Array(Tuple(Float64, Float64)))),
|
||||
name String DEFAULT 'qqq',
|
||||
@ -27,44 +27,44 @@ CREATE DICTIONARY 01760_db.dict_array
|
||||
value_nullable Nullable(UInt64) DEFAULT 20
|
||||
)
|
||||
PRIMARY KEY key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons' DB '01760_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons' DB '_01760_db'))
|
||||
LIFETIME(0)
|
||||
LAYOUT(POLYGON());
|
||||
|
||||
SELECT 'dictGet';
|
||||
|
||||
SELECT tuple(x, y) as key,
|
||||
dictGet('01760_db.dict_array', 'name', key),
|
||||
dictGet('01760_db.dict_array', 'value', key),
|
||||
dictGet('01760_db.dict_array', 'value_nullable', key)
|
||||
FROM 01760_db.points
|
||||
dictGet('_01760_db.dict_array', 'name', key),
|
||||
dictGet('_01760_db.dict_array', 'value', key),
|
||||
dictGet('_01760_db.dict_array', 'value_nullable', key)
|
||||
FROM _01760_db.points
|
||||
ORDER BY x, y;
|
||||
|
||||
SELECT 'dictGetOrDefault';
|
||||
|
||||
SELECT tuple(x, y) as key,
|
||||
dictGetOrDefault('01760_db.dict_array', 'name', key, 'DefaultName'),
|
||||
dictGetOrDefault('01760_db.dict_array', 'value', key, 30),
|
||||
dictGetOrDefault('01760_db.dict_array', 'value_nullable', key, 40)
|
||||
FROM 01760_db.points
|
||||
dictGetOrDefault('_01760_db.dict_array', 'name', key, 'DefaultName'),
|
||||
dictGetOrDefault('_01760_db.dict_array', 'value', key, 30),
|
||||
dictGetOrDefault('_01760_db.dict_array', 'value_nullable', key, 40)
|
||||
FROM _01760_db.points
|
||||
ORDER BY x, y;
|
||||
|
||||
SELECT 'dictHas';
|
||||
|
||||
SELECT tuple(x, y) as key,
|
||||
dictHas('01760_db.dict_array', key),
|
||||
dictHas('01760_db.dict_array', key),
|
||||
dictHas('01760_db.dict_array', key)
|
||||
FROM 01760_db.points
|
||||
dictHas('_01760_db.dict_array', key),
|
||||
dictHas('_01760_db.dict_array', key),
|
||||
dictHas('_01760_db.dict_array', key)
|
||||
FROM _01760_db.points
|
||||
ORDER BY x, y;
|
||||
|
||||
SELECT 'check NaN or infinite point input';
|
||||
SELECT tuple(nan, inf) as key, dictGet('01760_db.dict_array', 'name', key); --{serverError 36}
|
||||
SELECT tuple(nan, nan) as key, dictGet('01760_db.dict_array', 'name', key); --{serverError 36}
|
||||
SELECT tuple(inf, nan) as key, dictGet('01760_db.dict_array', 'name', key); --{serverError 36}
|
||||
SELECT tuple(inf, inf) as key, dictGet('01760_db.dict_array', 'name', key); --{serverError 36}
|
||||
SELECT tuple(nan, inf) as key, dictGet('_01760_db.dict_array', 'name', key); --{serverError 36}
|
||||
SELECT tuple(nan, nan) as key, dictGet('_01760_db.dict_array', 'name', key); --{serverError 36}
|
||||
SELECT tuple(inf, nan) as key, dictGet('_01760_db.dict_array', 'name', key); --{serverError 36}
|
||||
SELECT tuple(inf, inf) as key, dictGet('_01760_db.dict_array', 'name', key); --{serverError 36}
|
||||
|
||||
DROP DICTIONARY 01760_db.dict_array;
|
||||
DROP TABLE 01760_db.points;
|
||||
DROP TABLE 01760_db.polygons;
|
||||
DROP DATABASE 01760_db;
|
||||
DROP DICTIONARY _01760_db.dict_array;
|
||||
DROP TABLE _01760_db.points;
|
||||
DROP TABLE _01760_db.polygons;
|
||||
DROP DATABASE _01760_db;
|
||||
|
@ -1,59 +1,59 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01760_db;
|
||||
CREATE DATABASE 01760_db;
|
||||
DROP DATABASE IF EXISTS _01760_db;
|
||||
CREATE DATABASE _01760_db;
|
||||
|
||||
DROP TABLE IF EXISTS 01760_db.example_simple_key_source;
|
||||
CREATE TABLE 01760_db.example_simple_key_source (id UInt64, value UInt64) ENGINE=TinyLog;
|
||||
INSERT INTO 01760_db.example_simple_key_source VALUES (0, 0), (1, 1), (2, 2);
|
||||
DROP TABLE IF EXISTS _01760_db.example_simple_key_source;
|
||||
CREATE TABLE _01760_db.example_simple_key_source (id UInt64, value UInt64) ENGINE=TinyLog;
|
||||
INSERT INTO _01760_db.example_simple_key_source VALUES (0, 0), (1, 1), (2, 2);
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01760_db.example_simple_key_dictionary;
|
||||
CREATE DICTIONARY 01760_db.example_simple_key_dictionary (
|
||||
DROP DICTIONARY IF EXISTS _01760_db.example_simple_key_dictionary;
|
||||
CREATE DICTIONARY _01760_db.example_simple_key_dictionary (
|
||||
id UInt64,
|
||||
value UInt64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'example_simple_key_source' DATABASE '01760_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'example_simple_key_source' DATABASE '_01760_db'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT 'simple key';
|
||||
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='01760_db';
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='01760_db';
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='_01760_db';
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='_01760_db';
|
||||
|
||||
SELECT * FROM 01760_db.example_simple_key_dictionary;
|
||||
SELECT * FROM _01760_db.example_simple_key_dictionary;
|
||||
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='01760_db';
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='_01760_db';
|
||||
|
||||
DROP TABLE 01760_db.example_simple_key_source;
|
||||
DROP DICTIONARY 01760_db.example_simple_key_dictionary;
|
||||
DROP TABLE _01760_db.example_simple_key_source;
|
||||
DROP DICTIONARY _01760_db.example_simple_key_dictionary;
|
||||
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='01760_db';
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='_01760_db';
|
||||
|
||||
DROP TABLE IF EXISTS 01760_db.example_complex_key_source;
|
||||
CREATE TABLE 01760_db.example_complex_key_source (id UInt64, id_key String, value UInt64) ENGINE=TinyLog;
|
||||
INSERT INTO 01760_db.example_complex_key_source VALUES (0, '0_key', 0), (1, '1_key', 1), (2, '2_key', 2);
|
||||
DROP TABLE IF EXISTS _01760_db.example_complex_key_source;
|
||||
CREATE TABLE _01760_db.example_complex_key_source (id UInt64, id_key String, value UInt64) ENGINE=TinyLog;
|
||||
INSERT INTO _01760_db.example_complex_key_source VALUES (0, '0_key', 0), (1, '1_key', 1), (2, '2_key', 2);
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01760_db.example_complex_key_dictionary;
|
||||
CREATE DICTIONARY 01760_db.example_complex_key_dictionary (
|
||||
DROP DICTIONARY IF EXISTS _01760_db.example_complex_key_dictionary;
|
||||
CREATE DICTIONARY _01760_db.example_complex_key_dictionary (
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value UInt64
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'example_complex_key_source' DATABASE '01760_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'example_complex_key_source' DATABASE '_01760_db'))
|
||||
LAYOUT(COMPLEX_KEY_DIRECT());
|
||||
|
||||
SELECT 'complex key';
|
||||
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='01760_db';
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='01760_db';
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='_01760_db';
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='_01760_db';
|
||||
|
||||
SELECT * FROM 01760_db.example_complex_key_dictionary;
|
||||
SELECT * FROM _01760_db.example_complex_key_dictionary;
|
||||
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='01760_db';
|
||||
SELECT name, database, key.names, key.types, attribute.names, attribute.types, status FROM system.dictionaries WHERE database='_01760_db';
|
||||
|
||||
DROP TABLE 01760_db.example_complex_key_source;
|
||||
DROP DICTIONARY 01760_db.example_complex_key_dictionary;
|
||||
DROP TABLE _01760_db.example_complex_key_source;
|
||||
DROP DICTIONARY _01760_db.example_complex_key_dictionary;
|
||||
|
||||
DROP DATABASE 01760_db;
|
||||
DROP DATABASE _01760_db;
|
||||
|
@ -1,9 +1,9 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01765_db;
|
||||
CREATE DATABASE 01765_db;
|
||||
DROP DATABASE IF EXISTS _01765_db;
|
||||
CREATE DATABASE _01765_db;
|
||||
|
||||
CREATE TABLE 01765_db.simple_key_simple_attributes_source_table
|
||||
CREATE TABLE _01765_db.simple_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
@ -11,11 +11,11 @@ CREATE TABLE 01765_db.simple_key_simple_attributes_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01765_db.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01765_db.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO 01765_db.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
INSERT INTO _01765_db.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO _01765_db.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
|
||||
INSERT INTO _01765_db.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01765_db.hashed_dictionary_simple_key_simple_attributes
|
||||
CREATE DICTIONARY _01765_db.hashed_dictionary_simple_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -28,25 +28,25 @@ LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Dictionary hashed_dictionary_simple_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01765_db.hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01765_db.hashed_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01765_db.hashed_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01765_db.hashed_dictionary_simple_key_simple_attributes ORDER BY id;
|
||||
SELECT * FROM _01765_db.hashed_dictionary_simple_key_simple_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01765_db.hashed_dictionary_simple_key_simple_attributes;
|
||||
DROP DICTIONARY _01765_db.hashed_dictionary_simple_key_simple_attributes;
|
||||
|
||||
CREATE DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_simple_attributes
|
||||
CREATE DICTIONARY _01765_db.sparse_hashed_dictionary_simple_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -59,27 +59,27 @@ LAYOUT(SPARSE_HASHED());
|
||||
|
||||
SELECT 'Dictionary sparse_hashed_dictionary_simple_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01765_db.sparse_hashed_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01765_db.sparse_hashed_dictionary_simple_key_simple_attributes ORDER BY id;
|
||||
SELECT * FROM _01765_db.sparse_hashed_dictionary_simple_key_simple_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_simple_attributes;
|
||||
DROP DICTIONARY _01765_db.sparse_hashed_dictionary_simple_key_simple_attributes;
|
||||
|
||||
DROP TABLE 01765_db.simple_key_simple_attributes_source_table;
|
||||
DROP TABLE _01765_db.simple_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01765_db.simple_key_complex_attributes_source_table
|
||||
CREATE TABLE _01765_db.simple_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_first String,
|
||||
@ -87,11 +87,11 @@ CREATE TABLE 01765_db.simple_key_complex_attributes_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01765_db.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO 01765_db.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO 01765_db.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
INSERT INTO _01765_db.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
|
||||
INSERT INTO _01765_db.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
|
||||
INSERT INTO _01765_db.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01765_db.hashed_dictionary_simple_key_complex_attributes
|
||||
CREATE DICTIONARY _01765_db.hashed_dictionary_simple_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -104,25 +104,25 @@ LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Dictionary hashed_dictionary_simple_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01765_db.hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01765_db.hashed_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01765_db.hashed_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01765_db.hashed_dictionary_simple_key_complex_attributes ORDER BY id;
|
||||
SELECT * FROM _01765_db.hashed_dictionary_simple_key_complex_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01765_db.hashed_dictionary_simple_key_complex_attributes;
|
||||
DROP DICTIONARY _01765_db.hashed_dictionary_simple_key_complex_attributes;
|
||||
|
||||
CREATE DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_complex_attributes
|
||||
CREATE DICTIONARY _01765_db.sparse_hashed_dictionary_simple_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
value_first String DEFAULT 'value_first_default',
|
||||
@ -135,38 +135,38 @@ LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Dictionary sparse_hashed_dictionary_simple_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
|
||||
dictGet('_01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
|
||||
dictGetOrDefault('_01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01765_db.sparse_hashed_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01765_db.sparse_hashed_dictionary_simple_key_complex_attributes ORDER BY id;
|
||||
SELECT * FROM _01765_db.sparse_hashed_dictionary_simple_key_complex_attributes ORDER BY id;
|
||||
|
||||
DROP DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_complex_attributes;
|
||||
DROP DICTIONARY _01765_db.sparse_hashed_dictionary_simple_key_complex_attributes;
|
||||
|
||||
DROP TABLE 01765_db.simple_key_complex_attributes_source_table;
|
||||
DROP TABLE _01765_db.simple_key_complex_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01765_db.simple_key_hierarchy_table
|
||||
CREATE TABLE _01765_db.simple_key_hierarchy_table
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO 01765_db.simple_key_hierarchy_table VALUES (4, 2);
|
||||
INSERT INTO _01765_db.simple_key_hierarchy_table VALUES (1, 0);
|
||||
INSERT INTO _01765_db.simple_key_hierarchy_table VALUES (2, 1);
|
||||
INSERT INTO _01765_db.simple_key_hierarchy_table VALUES (3, 1);
|
||||
INSERT INTO _01765_db.simple_key_hierarchy_table VALUES (4, 2);
|
||||
|
||||
CREATE DICTIONARY 01765_db.hashed_dictionary_simple_key_hierarchy
|
||||
CREATE DICTIONARY _01765_db.hashed_dictionary_simple_key_hierarchy
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
@ -178,14 +178,14 @@ LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Dictionary hashed_dictionary_simple_key_hierarchy';
|
||||
SELECT 'dictGet';
|
||||
SELECT dictGet('01765_db.hashed_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT dictGet('_01765_db.hashed_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT 'dictGetHierarchy';
|
||||
SELECT dictGetHierarchy('01765_db.hashed_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('01765_db.hashed_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
SELECT dictGetHierarchy('_01765_db.hashed_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('_01765_db.hashed_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
|
||||
DROP DICTIONARY 01765_db.hashed_dictionary_simple_key_hierarchy;
|
||||
DROP DICTIONARY _01765_db.hashed_dictionary_simple_key_hierarchy;
|
||||
|
||||
CREATE DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_hierarchy
|
||||
CREATE DICTIONARY _01765_db.sparse_hashed_dictionary_simple_key_hierarchy
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
@ -197,13 +197,13 @@ LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Dictionary sparse_hashed_dictionary_simple_key_hierarchy';
|
||||
SELECT 'dictGet';
|
||||
SELECT dictGet('01765_db.sparse_hashed_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT dictGet('_01765_db.sparse_hashed_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
|
||||
SELECT 'dictGetHierarchy';
|
||||
SELECT dictGetHierarchy('01765_db.sparse_hashed_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('01765_db.sparse_hashed_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
SELECT dictGetHierarchy('_01765_db.sparse_hashed_dictionary_simple_key_hierarchy', toUInt64(1));
|
||||
SELECT dictGetHierarchy('_01765_db.sparse_hashed_dictionary_simple_key_hierarchy', toUInt64(4));
|
||||
|
||||
DROP DICTIONARY 01765_db.sparse_hashed_dictionary_simple_key_hierarchy;
|
||||
DROP DICTIONARY _01765_db.sparse_hashed_dictionary_simple_key_hierarchy;
|
||||
|
||||
DROP TABLE 01765_db.simple_key_hierarchy_table;
|
||||
DROP TABLE _01765_db.simple_key_hierarchy_table;
|
||||
|
||||
DROP DATABASE 01765_db;
|
||||
DROP DATABASE _01765_db;
|
||||
|
@ -1,9 +1,9 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01766_db;
|
||||
CREATE DATABASE 01766_db;
|
||||
DROP DATABASE IF EXISTS _01766_db;
|
||||
CREATE DATABASE _01766_db;
|
||||
|
||||
CREATE TABLE 01766_db.complex_key_simple_attributes_source_table
|
||||
CREATE TABLE _01766_db.complex_key_simple_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -12,11 +12,11 @@ CREATE TABLE 01766_db.complex_key_simple_attributes_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01766_db.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO 01766_db.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
|
||||
INSERT INTO 01766_db.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
INSERT INTO _01766_db.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO _01766_db.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
|
||||
INSERT INTO _01766_db.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01766_db.hashed_dictionary_complex_key_simple_attributes
|
||||
CREATE DICTIONARY _01766_db.hashed_dictionary_complex_key_simple_attributes
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -24,33 +24,33 @@ CREATE DICTIONARY 01766_db.hashed_dictionary_complex_key_simple_attributes
|
||||
value_second String DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '01766_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '_01766_db'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(COMPLEX_KEY_HASHED());
|
||||
|
||||
SELECT 'Dictionary hashed_dictionary_complex_key_simple_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01766_db.hashed_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01766_db.hashed_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01766_db.hashed_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01766_db.hashed_dictionary_complex_key_simple_attributes ORDER BY (id, id_key);
|
||||
SELECT * FROM _01766_db.hashed_dictionary_complex_key_simple_attributes ORDER BY (id, id_key);
|
||||
|
||||
DROP DICTIONARY 01766_db.hashed_dictionary_complex_key_simple_attributes;
|
||||
DROP DICTIONARY _01766_db.hashed_dictionary_complex_key_simple_attributes;
|
||||
|
||||
DROP TABLE 01766_db.complex_key_simple_attributes_source_table;
|
||||
DROP TABLE _01766_db.complex_key_simple_attributes_source_table;
|
||||
|
||||
CREATE TABLE 01766_db.complex_key_complex_attributes_source_table
|
||||
CREATE TABLE _01766_db.complex_key_complex_attributes_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -59,11 +59,11 @@ CREATE TABLE 01766_db.complex_key_complex_attributes_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01766_db.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO 01766_db.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
|
||||
INSERT INTO 01766_db.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
INSERT INTO _01766_db.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
|
||||
INSERT INTO _01766_db.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
|
||||
INSERT INTO _01766_db.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
|
||||
|
||||
CREATE DICTIONARY 01766_db.hashed_dictionary_complex_key_complex_attributes
|
||||
CREATE DICTIONARY _01766_db.hashed_dictionary_complex_key_complex_attributes
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
@ -72,29 +72,29 @@ CREATE DICTIONARY 01766_db.hashed_dictionary_complex_key_complex_attributes
|
||||
value_second Nullable(String) DEFAULT 'value_second_default'
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '01766_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '_01766_db'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(COMPLEX_KEY_HASHED());
|
||||
|
||||
SELECT 'Dictionary hashed_dictionary_complex_key_complex_attributes';
|
||||
SELECT 'dictGet existing value';
|
||||
SELECT dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGet('_01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGet with non existing value';
|
||||
SELECT dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGet('_01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
|
||||
dictGet('_01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictGetOrDefault existing value';
|
||||
SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT dictGetOrDefault('_01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
|
||||
SELECT 'dictGetOrDefault non existing value';
|
||||
SELECT dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT dictGetOrDefault('_01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
|
||||
dictGetOrDefault('_01766_db.hashed_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
|
||||
SELECT 'dictHas';
|
||||
SELECT dictHas('01766_db.hashed_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT dictHas('_01766_db.hashed_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
|
||||
SELECT 'select all values as input stream';
|
||||
SELECT * FROM 01766_db.hashed_dictionary_complex_key_complex_attributes ORDER BY (id, id_key);
|
||||
SELECT * FROM _01766_db.hashed_dictionary_complex_key_complex_attributes ORDER BY (id, id_key);
|
||||
|
||||
DROP DICTIONARY 01766_db.hashed_dictionary_complex_key_complex_attributes;
|
||||
DROP TABLE 01766_db.complex_key_complex_attributes_source_table;
|
||||
DROP DICTIONARY _01766_db.hashed_dictionary_complex_key_complex_attributes;
|
||||
DROP TABLE _01766_db.complex_key_complex_attributes_source_table;
|
||||
|
||||
DROP DATABASE 01766_db;
|
||||
DROP DATABASE _01766_db;
|
||||
|
@ -1,97 +1,97 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01778_db;
|
||||
CREATE DATABASE 01778_db;
|
||||
DROP DATABASE IF EXISTS _01778_db;
|
||||
CREATE DATABASE _01778_db;
|
||||
|
||||
CREATE TABLE 01778_db.hierarchy_source_table (id UInt64, parent_id UInt64) ENGINE = TinyLog;
|
||||
INSERT INTO 01778_db.hierarchy_source_table VALUES (1, 0), (2, 1), (3, 1), (4, 2);
|
||||
CREATE TABLE _01778_db.hierarchy_source_table (id UInt64, parent_id UInt64) ENGINE = TinyLog;
|
||||
INSERT INTO _01778_db.hierarchy_source_table VALUES (1, 0), (2, 1), (3, 1), (4, 2);
|
||||
|
||||
CREATE DICTIONARY 01778_db.hierarchy_flat_dictionary
|
||||
CREATE DICTIONARY _01778_db.hierarchy_flat_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '_01778_db'))
|
||||
LAYOUT(FLAT())
|
||||
LIFETIME(MIN 1 MAX 1000);
|
||||
|
||||
SELECT 'Flat dictionary';
|
||||
|
||||
SELECT 'Get hierarchy';
|
||||
SELECT dictGetHierarchy('01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictGetHierarchy('_01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get is in hierarchy';
|
||||
SELECT dictIsIn('01778_db.hierarchy_flat_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictIsIn('_01778_db.hierarchy_flat_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get children';
|
||||
SELECT dictGetChildren('01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictGetChildren('_01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get all descendants';
|
||||
SELECT dictGetDescendants('01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictGetDescendants('_01778_db.hierarchy_flat_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get descendants at first level';
|
||||
SELECT dictGetDescendants('01778_db.hierarchy_flat_dictionary', number, 1) FROM system.numbers LIMIT 6;
|
||||
SELECT dictGetDescendants('_01778_db.hierarchy_flat_dictionary', number, 1) FROM system.numbers LIMIT 6;
|
||||
|
||||
DROP DICTIONARY 01778_db.hierarchy_flat_dictionary;
|
||||
DROP DICTIONARY _01778_db.hierarchy_flat_dictionary;
|
||||
|
||||
CREATE DICTIONARY 01778_db.hierarchy_hashed_dictionary
|
||||
CREATE DICTIONARY _01778_db.hierarchy_hashed_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '_01778_db'))
|
||||
LAYOUT(HASHED())
|
||||
LIFETIME(MIN 1 MAX 1000);
|
||||
|
||||
SELECT 'Hashed dictionary';
|
||||
|
||||
SELECT 'Get hierarchy';
|
||||
SELECT dictGetHierarchy('01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictGetHierarchy('_01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get is in hierarchy';
|
||||
SELECT dictIsIn('01778_db.hierarchy_hashed_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictIsIn('_01778_db.hierarchy_hashed_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get children';
|
||||
SELECT dictGetChildren('01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictGetChildren('_01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get all descendants';
|
||||
SELECT dictGetDescendants('01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictGetDescendants('_01778_db.hierarchy_hashed_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get descendants at first level';
|
||||
SELECT dictGetDescendants('01778_db.hierarchy_hashed_dictionary', number, 1) FROM system.numbers LIMIT 6;
|
||||
SELECT dictGetDescendants('_01778_db.hierarchy_hashed_dictionary', number, 1) FROM system.numbers LIMIT 6;
|
||||
|
||||
DROP DICTIONARY 01778_db.hierarchy_hashed_dictionary;
|
||||
DROP DICTIONARY _01778_db.hierarchy_hashed_dictionary;
|
||||
|
||||
CREATE DICTIONARY 01778_db.hierarchy_cache_dictionary
|
||||
CREATE DICTIONARY _01778_db.hierarchy_cache_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '_01778_db'))
|
||||
LAYOUT(CACHE(SIZE_IN_CELLS 10))
|
||||
LIFETIME(MIN 1 MAX 1000);
|
||||
|
||||
SELECT 'Cache dictionary';
|
||||
|
||||
SELECT 'Get hierarchy';
|
||||
SELECT dictGetHierarchy('01778_db.hierarchy_cache_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictGetHierarchy('_01778_db.hierarchy_cache_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get is in hierarchy';
|
||||
SELECT dictIsIn('01778_db.hierarchy_cache_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictIsIn('_01778_db.hierarchy_cache_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
|
||||
DROP DICTIONARY 01778_db.hierarchy_cache_dictionary;
|
||||
DROP DICTIONARY _01778_db.hierarchy_cache_dictionary;
|
||||
|
||||
CREATE DICTIONARY 01778_db.hierarchy_direct_dictionary
|
||||
CREATE DICTIONARY _01778_db.hierarchy_direct_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
parent_id UInt64 HIERARCHICAL
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '01778_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'hierarchy_source_table' DB '_01778_db'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT 'Direct dictionary';
|
||||
|
||||
SELECT 'Get hierarchy';
|
||||
SELECT dictGetHierarchy('01778_db.hierarchy_direct_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictGetHierarchy('_01778_db.hierarchy_direct_dictionary', number) FROM system.numbers LIMIT 6;
|
||||
SELECT 'Get is in hierarchy';
|
||||
SELECT dictIsIn('01778_db.hierarchy_direct_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
SELECT dictIsIn('_01778_db.hierarchy_direct_dictionary', number, number) FROM system.numbers LIMIT 6;
|
||||
|
||||
DROP DICTIONARY 01778_db.hierarchy_direct_dictionary;
|
||||
DROP DICTIONARY _01778_db.hierarchy_direct_dictionary;
|
||||
|
||||
DROP TABLE 01778_db.hierarchy_source_table;
|
||||
DROP DATABASE 01778_db;
|
||||
DROP TABLE _01778_db.hierarchy_source_table;
|
||||
DROP DATABASE _01778_db;
|
||||
|
@ -1,7 +1,7 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01780_db;
|
||||
CREATE DATABASE 01780_db;
|
||||
DROP DATABASE IF EXISTS _01780_db;
|
||||
CREATE DATABASE _01780_db;
|
||||
|
||||
DROP DICTIONARY IF EXISTS dict1;
|
||||
CREATE DICTIONARY dict1
|
||||
@ -18,38 +18,38 @@ SELECT * FROM dict1; --{serverError 36}
|
||||
DROP DICTIONARY dict1;
|
||||
|
||||
DROP DICTIONARY IF EXISTS dict2;
|
||||
CREATE DICTIONARY 01780_db.dict2
|
||||
CREATE DICTIONARY _01780_db.dict2
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DATABASE '01780_db' TABLE 'dict2'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DATABASE '_01780_db' TABLE 'dict2'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT * FROM 01780_db.dict2; --{serverError 36}
|
||||
DROP DICTIONARY 01780_db.dict2;
|
||||
SELECT * FROM _01780_db.dict2; --{serverError 36}
|
||||
DROP DICTIONARY _01780_db.dict2;
|
||||
|
||||
DROP TABLE IF EXISTS 01780_db.dict3_source;
|
||||
CREATE TABLE 01780_db.dict3_source
|
||||
DROP TABLE IF EXISTS _01780_db.dict3_source;
|
||||
CREATE TABLE _01780_db.dict3_source
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01780_db.dict3_source VALUES (1, '1'), (2, '2'), (3, '3');
|
||||
INSERT INTO _01780_db.dict3_source VALUES (1, '1'), (2, '2'), (3, '3');
|
||||
|
||||
CREATE DICTIONARY 01780_db.dict3
|
||||
CREATE DICTIONARY _01780_db.dict3
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 TABLE 'dict3_source' DATABASE '01780_db'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 TABLE 'dict3_source' DATABASE '_01780_db'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT * FROM 01780_db.dict3;
|
||||
SELECT * FROM _01780_db.dict3;
|
||||
|
||||
DROP DICTIONARY 01780_db.dict3;
|
||||
DROP DICTIONARY _01780_db.dict3;
|
||||
|
||||
DROP DATABASE 01780_db;
|
||||
DROP DATABASE _01780_db;
|
||||
|
@ -1,93 +1,93 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01785_db;
|
||||
CREATE DATABASE 01785_db;
|
||||
DROP DATABASE IF EXISTS _01785_db;
|
||||
CREATE DATABASE _01785_db;
|
||||
|
||||
DROP TABLE IF EXISTS 01785_db.simple_key_source_table;
|
||||
CREATE TABLE 01785_db.simple_key_source_table
|
||||
DROP TABLE IF EXISTS _01785_db.simple_key_source_table;
|
||||
CREATE TABLE _01785_db.simple_key_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO 01785_db.simple_key_source_table VALUES (1, 'First');
|
||||
INSERT INTO 01785_db.simple_key_source_table VALUES (1, 'First');
|
||||
INSERT INTO _01785_db.simple_key_source_table VALUES (1, 'First');
|
||||
INSERT INTO _01785_db.simple_key_source_table VALUES (1, 'First');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01785_db.simple_key_flat_dictionary;
|
||||
CREATE DICTIONARY 01785_db.simple_key_flat_dictionary
|
||||
DROP DICTIONARY IF EXISTS _01785_db.simple_key_flat_dictionary;
|
||||
CREATE DICTIONARY _01785_db.simple_key_flat_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB '01785_db' TABLE 'simple_key_source_table'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB '_01785_db' TABLE 'simple_key_source_table'))
|
||||
LAYOUT(FLAT())
|
||||
LIFETIME(MIN 0 MAX 1000);
|
||||
|
||||
SELECT * FROM 01785_db.simple_key_flat_dictionary;
|
||||
SELECT name, database, element_count FROM system.dictionaries WHERE database = '01785_db' AND name = 'simple_key_flat_dictionary';
|
||||
SELECT * FROM _01785_db.simple_key_flat_dictionary;
|
||||
SELECT name, database, element_count FROM system.dictionaries WHERE database = '_01785_db' AND name = 'simple_key_flat_dictionary';
|
||||
|
||||
DROP DICTIONARY 01785_db.simple_key_flat_dictionary;
|
||||
DROP DICTIONARY _01785_db.simple_key_flat_dictionary;
|
||||
|
||||
CREATE DICTIONARY 01785_db.simple_key_hashed_dictionary
|
||||
CREATE DICTIONARY _01785_db.simple_key_hashed_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB '01785_db' TABLE 'simple_key_source_table'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB '_01785_db' TABLE 'simple_key_source_table'))
|
||||
LAYOUT(HASHED())
|
||||
LIFETIME(MIN 0 MAX 1000);
|
||||
|
||||
SELECT * FROM 01785_db.simple_key_hashed_dictionary;
|
||||
SELECT name, database, element_count FROM system.dictionaries WHERE database = '01785_db' AND name = 'simple_key_hashed_dictionary';
|
||||
SELECT * FROM _01785_db.simple_key_hashed_dictionary;
|
||||
SELECT name, database, element_count FROM system.dictionaries WHERE database = '_01785_db' AND name = 'simple_key_hashed_dictionary';
|
||||
|
||||
DROP DICTIONARY 01785_db.simple_key_hashed_dictionary;
|
||||
DROP DICTIONARY _01785_db.simple_key_hashed_dictionary;
|
||||
|
||||
CREATE DICTIONARY 01785_db.simple_key_cache_dictionary
|
||||
CREATE DICTIONARY _01785_db.simple_key_cache_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB '01785_db' TABLE 'simple_key_source_table'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB '_01785_db' TABLE 'simple_key_source_table'))
|
||||
LAYOUT(CACHE(SIZE_IN_CELLS 100000))
|
||||
LIFETIME(MIN 0 MAX 1000);
|
||||
|
||||
SELECT toUInt64(1) as key, dictGet('01785_db.simple_key_cache_dictionary', 'value', key);
|
||||
SELECT name, database, element_count FROM system.dictionaries WHERE database = '01785_db' AND name = 'simple_key_cache_dictionary';
|
||||
SELECT toUInt64(1) as key, dictGet('_01785_db.simple_key_cache_dictionary', 'value', key);
|
||||
SELECT name, database, element_count FROM system.dictionaries WHERE database = '_01785_db' AND name = 'simple_key_cache_dictionary';
|
||||
|
||||
DROP DICTIONARY 01785_db.simple_key_cache_dictionary;
|
||||
DROP DICTIONARY _01785_db.simple_key_cache_dictionary;
|
||||
|
||||
DROP TABLE 01785_db.simple_key_source_table;
|
||||
DROP TABLE _01785_db.simple_key_source_table;
|
||||
|
||||
DROP TABLE IF EXISTS 01785_db.complex_key_source_table;
|
||||
CREATE TABLE 01785_db.complex_key_source_table
|
||||
DROP TABLE IF EXISTS _01785_db.complex_key_source_table;
|
||||
CREATE TABLE _01785_db.complex_key_source_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value String
|
||||
) ENGINE = TinyLog();
|
||||
|
||||
INSERT INTO 01785_db.complex_key_source_table VALUES (1, 'FirstKey', 'First');
|
||||
INSERT INTO 01785_db.complex_key_source_table VALUES (1, 'FirstKey', 'First');
|
||||
INSERT INTO _01785_db.complex_key_source_table VALUES (1, 'FirstKey', 'First');
|
||||
INSERT INTO _01785_db.complex_key_source_table VALUES (1, 'FirstKey', 'First');
|
||||
|
||||
CREATE DICTIONARY 01785_db.complex_key_hashed_dictionary
|
||||
CREATE DICTIONARY _01785_db.complex_key_hashed_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB '01785_db' TABLE 'complex_key_source_table'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB '_01785_db' TABLE 'complex_key_source_table'))
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
LIFETIME(MIN 0 MAX 1000);
|
||||
|
||||
SELECT * FROM 01785_db.complex_key_hashed_dictionary;
|
||||
SELECT name, database, element_count FROM system.dictionaries WHERE database = '01785_db' AND name = 'complex_key_hashed_dictionary';
|
||||
SELECT * FROM _01785_db.complex_key_hashed_dictionary;
|
||||
SELECT name, database, element_count FROM system.dictionaries WHERE database = '_01785_db' AND name = 'complex_key_hashed_dictionary';
|
||||
|
||||
DROP DICTIONARY 01785_db.complex_key_hashed_dictionary;
|
||||
DROP DICTIONARY _01785_db.complex_key_hashed_dictionary;
|
||||
|
||||
DROP TABLE 01785_db.complex_key_source_table;
|
||||
DROP TABLE _01785_db.complex_key_source_table;
|
||||
|
||||
DROP DATABASE 01785_db;
|
||||
DROP DATABASE _01785_db;
|
||||
|
@ -1,6 +1,6 @@
|
||||
DROP TABLE IF EXISTS 01802_empsalary;
|
||||
DROP TABLE IF EXISTS _01802_empsalary;
|
||||
|
||||
CREATE TABLE 01802_empsalary
|
||||
CREATE TABLE _01802_empsalary
|
||||
(
|
||||
`depname` LowCardinality(String),
|
||||
`empno` UInt64,
|
||||
@ -11,7 +11,7 @@ ENGINE = MergeTree
|
||||
ORDER BY enroll_date
|
||||
SETTINGS index_granularity = 8192;
|
||||
|
||||
INSERT INTO 01802_empsalary VALUES ('sales', 1, 5000, '2006-10-01'), ('develop', 8, 6000, '2006-10-01'), ('personnel', 2, 3900, '2006-12-23'), ('develop', 10, 5200, '2007-08-01'), ('sales', 3, 4800, '2007-08-01'), ('sales', 4, 4801, '2007-08-08'), ('develop', 11, 5200, '2007-08-15'), ('personnel', 5, 3500, '2007-12-10'), ('develop', 7, 4200, '2008-01-01'), ('develop', 9, 4500, '2008-01-01');
|
||||
INSERT INTO _01802_empsalary VALUES ('sales', 1, 5000, '2006-10-01'), ('develop', 8, 6000, '2006-10-01'), ('personnel', 2, 3900, '2006-12-23'), ('develop', 10, 5200, '2007-08-01'), ('sales', 3, 4800, '2007-08-01'), ('sales', 4, 4801, '2007-08-08'), ('develop', 11, 5200, '2007-08-15'), ('personnel', 5, 3500, '2007-12-10'), ('develop', 7, 4200, '2008-01-01'), ('develop', 9, 4500, '2008-01-01');
|
||||
|
||||
SELECT mannWhitneyUTest(salary, salary) OVER (ORDER BY salary ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) AS func FROM 01802_empsalary; -- {serverError 36}
|
||||
|
||||
|
@ -1,32 +1,32 @@
|
||||
-- Tags: no-parallel, no-fasttest
|
||||
|
||||
DROP DATABASE IF EXISTS 01837_db;
|
||||
CREATE DATABASE 01837_db ENGINE = Memory;
|
||||
DROP DATABASE IF EXISTS _01837_db;
|
||||
CREATE DATABASE _01837_db ENGINE = Memory;
|
||||
|
||||
DROP TABLE IF EXISTS 01837_db.simple_key_dictionary_source;
|
||||
CREATE TABLE 01837_db.simple_key_dictionary_source
|
||||
DROP TABLE IF EXISTS _01837_db.simple_key_dictionary_source;
|
||||
CREATE TABLE _01837_db.simple_key_dictionary_source
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01837_db.simple_key_dictionary_source VALUES (1, 'First');
|
||||
INSERT INTO 01837_db.simple_key_dictionary_source VALUES (2, 'Second');
|
||||
INSERT INTO 01837_db.simple_key_dictionary_source VALUES (3, 'Third');
|
||||
INSERT INTO _01837_db.simple_key_dictionary_source VALUES (1, 'First');
|
||||
INSERT INTO _01837_db.simple_key_dictionary_source VALUES (2, 'Second');
|
||||
INSERT INTO _01837_db.simple_key_dictionary_source VALUES (3, 'Third');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01837_db.simple_key_direct_dictionary;
|
||||
CREATE DICTIONARY 01837_db.simple_key_direct_dictionary
|
||||
DROP DICTIONARY IF EXISTS _01837_db.simple_key_direct_dictionary;
|
||||
CREATE DICTIONARY _01837_db.simple_key_direct_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB '01837_db' TABLE 'simple_key_dictionary_source'))
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB '_01837_db' TABLE 'simple_key_dictionary_source'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT * FROM 01837_db.simple_key_direct_dictionary;
|
||||
SELECT * FROM _01837_db.simple_key_direct_dictionary;
|
||||
|
||||
DROP DICTIONARY 01837_db.simple_key_direct_dictionary;
|
||||
DROP TABLE 01837_db.simple_key_dictionary_source;
|
||||
DROP DICTIONARY _01837_db.simple_key_direct_dictionary;
|
||||
DROP TABLE _01837_db.simple_key_dictionary_source;
|
||||
|
||||
DROP DATABASE 01837_db;
|
||||
DROP DATABASE _01837_db;
|
||||
|
@ -89,17 +89,17 @@ SELECT JSON_EXISTS('{"a":[[{"b":1}, {"g":1}],[{"h":1},{"y":1}]]}', '$.a[*][0].h'
|
||||
1
|
||||
SELECT '--MANY ROWS--';
|
||||
--MANY ROWS--
|
||||
DROP TABLE IF EXISTS 01889_sql_json;
|
||||
CREATE TABLE 01889_sql_json (id UInt8, json String) ENGINE = MergeTree ORDER BY id;
|
||||
INSERT INTO 01889_sql_json(id, json) VALUES(0, '{"name":"Ivan","surname":"Ivanov","friends":["Vasily","Kostya","Artyom"]}');
|
||||
INSERT INTO 01889_sql_json(id, json) VALUES(1, '{"name":"Katya","surname":"Baltica","friends":["Tihon","Ernest","Innokentiy"]}');
|
||||
INSERT INTO 01889_sql_json(id, json) VALUES(2, '{"name":"Vitali","surname":"Brown","friends":["Katya","Anatoliy","Ivan","Oleg"]}');
|
||||
SELECT id, JSON_QUERY(json, '$.friends[0 to 2]') FROM 01889_sql_json ORDER BY id;
|
||||
DROP TABLE IF EXISTS _01889_sql_json;
|
||||
CREATE TABLE _01889_sql_json (id UInt8, json String) ENGINE = MergeTree ORDER BY id;
|
||||
INSERT INTO _01889_sql_json(id, json) VALUES(0, '{"name":"Ivan","surname":"Ivanov","friends":["Vasily","Kostya","Artyom"]}');
|
||||
INSERT INTO _01889_sql_json(id, json) VALUES(1, '{"name":"Katya","surname":"Baltica","friends":["Tihon","Ernest","Innokentiy"]}');
|
||||
INSERT INTO _01889_sql_json(id, json) VALUES(2, '{"name":"Vitali","surname":"Brown","friends":["Katya","Anatoliy","Ivan","Oleg"]}');
|
||||
SELECT id, JSON_QUERY(json, '$.friends[0 to 2]') FROM _01889_sql_json ORDER BY id;
|
||||
0 ["Vasily", "Kostya"]
|
||||
1 ["Tihon", "Ernest"]
|
||||
2 ["Katya", "Anatoliy"]
|
||||
SELECT id, JSON_VALUE(json, '$.friends[0]') FROM 01889_sql_json ORDER BY id;
|
||||
SELECT id, JSON_VALUE(json, '$.friends[0]') FROM _01889_sql_json ORDER BY id;
|
||||
0 Vasily
|
||||
1 Tihon
|
||||
2 Katya
|
||||
DROP TABLE 01889_sql_json;
|
||||
DROP TABLE _01889_sql_json;
|
||||
|
@ -49,11 +49,11 @@ SELECT JSON_EXISTS('{"a":[{"b":1},{"c":2}]}', '$.a[*].f');
|
||||
SELECT JSON_EXISTS('{"a":[[{"b":1}, {"g":1}],[{"h":1},{"y":1}]]}', '$.a[*][0].h');
|
||||
|
||||
SELECT '--MANY ROWS--';
|
||||
DROP TABLE IF EXISTS 01889_sql_json;
|
||||
CREATE TABLE 01889_sql_json (id UInt8, json String) ENGINE = MergeTree ORDER BY id;
|
||||
INSERT INTO 01889_sql_json(id, json) VALUES(0, '{"name":"Ivan","surname":"Ivanov","friends":["Vasily","Kostya","Artyom"]}');
|
||||
INSERT INTO 01889_sql_json(id, json) VALUES(1, '{"name":"Katya","surname":"Baltica","friends":["Tihon","Ernest","Innokentiy"]}');
|
||||
INSERT INTO 01889_sql_json(id, json) VALUES(2, '{"name":"Vitali","surname":"Brown","friends":["Katya","Anatoliy","Ivan","Oleg"]}');
|
||||
SELECT id, JSON_QUERY(json, '$.friends[0 to 2]') FROM 01889_sql_json ORDER BY id;
|
||||
SELECT id, JSON_VALUE(json, '$.friends[0]') FROM 01889_sql_json ORDER BY id;
|
||||
DROP TABLE 01889_sql_json;
|
||||
DROP TABLE IF EXISTS _01889_sql_json;
|
||||
CREATE TABLE _01889_sql_json (id UInt8, json String) ENGINE = MergeTree ORDER BY id;
|
||||
INSERT INTO _01889_sql_json(id, json) VALUES(0, '{"name":"Ivan","surname":"Ivanov","friends":["Vasily","Kostya","Artyom"]}');
|
||||
INSERT INTO _01889_sql_json(id, json) VALUES(1, '{"name":"Katya","surname":"Baltica","friends":["Tihon","Ernest","Innokentiy"]}');
|
||||
INSERT INTO _01889_sql_json(id, json) VALUES(2, '{"name":"Vitali","surname":"Brown","friends":["Katya","Anatoliy","Ivan","Oleg"]}');
|
||||
SELECT id, JSON_QUERY(json, '$.friends[0 to 2]') FROM _01889_sql_json ORDER BY id;
|
||||
SELECT id, JSON_VALUE(json, '$.friends[0]') FROM _01889_sql_json ORDER BY id;
|
||||
DROP TABLE _01889_sql_json;
|
||||
|
@ -1,195 +1,195 @@
|
||||
CREATE TABLE t_merge as 01902_db.t ENGINE=Merge(REGEXP(^01902_db), ^t)
|
||||
SELECT _database, _table, n FROM 01902_db.t_merge ORDER BY _database, _table, n
|
||||
01902_db t 0
|
||||
01902_db t 1
|
||||
01902_db t 2
|
||||
01902_db t 3
|
||||
01902_db t 4
|
||||
01902_db t 5
|
||||
01902_db t 6
|
||||
01902_db t 7
|
||||
01902_db t 8
|
||||
01902_db t 9
|
||||
01902_db1 t1 0
|
||||
01902_db1 t1 1
|
||||
01902_db1 t1 2
|
||||
01902_db1 t1 3
|
||||
01902_db1 t1 4
|
||||
01902_db1 t1 5
|
||||
01902_db1 t1 6
|
||||
01902_db1 t1 7
|
||||
01902_db1 t1 8
|
||||
01902_db1 t1 9
|
||||
01902_db2 t2 0
|
||||
01902_db2 t2 1
|
||||
01902_db2 t2 2
|
||||
01902_db2 t2 3
|
||||
01902_db2 t2 4
|
||||
01902_db2 t2 5
|
||||
01902_db2 t2 6
|
||||
01902_db2 t2 7
|
||||
01902_db2 t2 8
|
||||
01902_db2 t2 9
|
||||
01902_db3 t3 0
|
||||
01902_db3 t3 1
|
||||
01902_db3 t3 2
|
||||
01902_db3 t3 3
|
||||
01902_db3 t3 4
|
||||
01902_db3 t3 5
|
||||
01902_db3 t3 6
|
||||
01902_db3 t3 7
|
||||
01902_db3 t3 8
|
||||
01902_db3 t3 9
|
||||
SHOW CREATE TABLE 01902_db.t_merge
|
||||
CREATE TABLE `01902_db`.t_merge\n(\n `n` Int8\n)\nENGINE = Merge(REGEXP(\'^01902_db\'), \'^t\')
|
||||
SELECT _database, _table, n FROM merge(REGEXP(^01902_db), ^t) ORDER BY _database, _table, n
|
||||
01902_db t 0
|
||||
01902_db t 1
|
||||
01902_db t 2
|
||||
01902_db t 3
|
||||
01902_db t 4
|
||||
01902_db t 5
|
||||
01902_db t 6
|
||||
01902_db t 7
|
||||
01902_db t 8
|
||||
01902_db t 9
|
||||
01902_db t_merge 0
|
||||
01902_db t_merge 0
|
||||
01902_db t_merge 0
|
||||
01902_db t_merge 0
|
||||
01902_db t_merge 1
|
||||
01902_db t_merge 1
|
||||
01902_db t_merge 1
|
||||
01902_db t_merge 1
|
||||
01902_db t_merge 2
|
||||
01902_db t_merge 2
|
||||
01902_db t_merge 2
|
||||
01902_db t_merge 2
|
||||
01902_db t_merge 3
|
||||
01902_db t_merge 3
|
||||
01902_db t_merge 3
|
||||
01902_db t_merge 3
|
||||
01902_db t_merge 4
|
||||
01902_db t_merge 4
|
||||
01902_db t_merge 4
|
||||
01902_db t_merge 4
|
||||
01902_db t_merge 5
|
||||
01902_db t_merge 5
|
||||
01902_db t_merge 5
|
||||
01902_db t_merge 5
|
||||
01902_db t_merge 6
|
||||
01902_db t_merge 6
|
||||
01902_db t_merge 6
|
||||
01902_db t_merge 6
|
||||
01902_db t_merge 7
|
||||
01902_db t_merge 7
|
||||
01902_db t_merge 7
|
||||
01902_db t_merge 7
|
||||
01902_db t_merge 8
|
||||
01902_db t_merge 8
|
||||
01902_db t_merge 8
|
||||
01902_db t_merge 8
|
||||
01902_db t_merge 9
|
||||
01902_db t_merge 9
|
||||
01902_db t_merge 9
|
||||
01902_db t_merge 9
|
||||
01902_db1 t1 0
|
||||
01902_db1 t1 1
|
||||
01902_db1 t1 2
|
||||
01902_db1 t1 3
|
||||
01902_db1 t1 4
|
||||
01902_db1 t1 5
|
||||
01902_db1 t1 6
|
||||
01902_db1 t1 7
|
||||
01902_db1 t1 8
|
||||
01902_db1 t1 9
|
||||
01902_db2 t2 0
|
||||
01902_db2 t2 1
|
||||
01902_db2 t2 2
|
||||
01902_db2 t2 3
|
||||
01902_db2 t2 4
|
||||
01902_db2 t2 5
|
||||
01902_db2 t2 6
|
||||
01902_db2 t2 7
|
||||
01902_db2 t2 8
|
||||
01902_db2 t2 9
|
||||
01902_db3 t3 0
|
||||
01902_db3 t3 1
|
||||
01902_db3 t3 2
|
||||
01902_db3 t3 3
|
||||
01902_db3 t3 4
|
||||
01902_db3 t3 5
|
||||
01902_db3 t3 6
|
||||
01902_db3 t3 7
|
||||
01902_db3 t3 8
|
||||
01902_db3 t3 9
|
||||
SELECT _database, _table, n FROM 01902_db.t_merge WHERE _database = 01902_db1 ORDER BY _database, _table, n
|
||||
01902_db1 t1 0
|
||||
01902_db1 t1 1
|
||||
01902_db1 t1 2
|
||||
01902_db1 t1 3
|
||||
01902_db1 t1 4
|
||||
01902_db1 t1 5
|
||||
01902_db1 t1 6
|
||||
01902_db1 t1 7
|
||||
01902_db1 t1 8
|
||||
01902_db1 t1 9
|
||||
SELECT _database, _table, n FROM 01902_db.t_merge WHERE _table = t1 ORDER BY _database, _table, n
|
||||
01902_db1 t1 0
|
||||
01902_db1 t1 1
|
||||
01902_db1 t1 2
|
||||
01902_db1 t1 3
|
||||
01902_db1 t1 4
|
||||
01902_db1 t1 5
|
||||
01902_db1 t1 6
|
||||
01902_db1 t1 7
|
||||
01902_db1 t1 8
|
||||
01902_db1 t1 9
|
||||
CREATE TABLE t_merge1 as 01902_db.t ENGINE=Merge(01902_db, ^t$)
|
||||
SELECT _database, _table, n FROM 01902_db.t_merge1 ORDER BY _database, _table, n
|
||||
01902_db t 0
|
||||
01902_db t 1
|
||||
01902_db t 2
|
||||
01902_db t 3
|
||||
01902_db t 4
|
||||
01902_db t 5
|
||||
01902_db t 6
|
||||
01902_db t 7
|
||||
01902_db t 8
|
||||
01902_db t 9
|
||||
SELECT _database, _table, n FROM merge(01902_db, ^t$) ORDER BY _database, _table, n
|
||||
01902_db t 0
|
||||
01902_db t 1
|
||||
01902_db t 2
|
||||
01902_db t 3
|
||||
01902_db t 4
|
||||
01902_db t 5
|
||||
01902_db t 6
|
||||
01902_db t 7
|
||||
01902_db t 8
|
||||
01902_db t 9
|
||||
CREATE TABLE t_merge_1 as 01902_db.t ENGINE=Merge(currentDatabase(), ^t)
|
||||
SELECT _database, _table, n FROM 01902_db.t_merge_1 ORDER BY _database, _table, n
|
||||
01902_db1 t1 0
|
||||
01902_db1 t1 1
|
||||
01902_db1 t1 2
|
||||
01902_db1 t1 3
|
||||
01902_db1 t1 4
|
||||
01902_db1 t1 5
|
||||
01902_db1 t1 6
|
||||
01902_db1 t1 7
|
||||
01902_db1 t1 8
|
||||
01902_db1 t1 9
|
||||
SHOW CREATE TABLE 01902_db.t_merge_1
|
||||
CREATE TABLE `01902_db`.t_merge_1\n(\n `n` Int8\n)\nENGINE = Merge(\'01902_db1\', \'^t\')
|
||||
CREATE TABLE t_merge as _01902_db.t ENGINE=Merge(REGEXP(^_01902_db), ^t)
|
||||
SELECT _database, _table, n FROM _01902_db.t_merge ORDER BY _database, _table, n
|
||||
_01902_db t 0
|
||||
_01902_db t 1
|
||||
_01902_db t 2
|
||||
_01902_db t 3
|
||||
_01902_db t 4
|
||||
_01902_db t 5
|
||||
_01902_db t 6
|
||||
_01902_db t 7
|
||||
_01902_db t 8
|
||||
_01902_db t 9
|
||||
_01902_db1 t1 0
|
||||
_01902_db1 t1 1
|
||||
_01902_db1 t1 2
|
||||
_01902_db1 t1 3
|
||||
_01902_db1 t1 4
|
||||
_01902_db1 t1 5
|
||||
_01902_db1 t1 6
|
||||
_01902_db1 t1 7
|
||||
_01902_db1 t1 8
|
||||
_01902_db1 t1 9
|
||||
_01902_db2 t2 0
|
||||
_01902_db2 t2 1
|
||||
_01902_db2 t2 2
|
||||
_01902_db2 t2 3
|
||||
_01902_db2 t2 4
|
||||
_01902_db2 t2 5
|
||||
_01902_db2 t2 6
|
||||
_01902_db2 t2 7
|
||||
_01902_db2 t2 8
|
||||
_01902_db2 t2 9
|
||||
_01902_db3 t3 0
|
||||
_01902_db3 t3 1
|
||||
_01902_db3 t3 2
|
||||
_01902_db3 t3 3
|
||||
_01902_db3 t3 4
|
||||
_01902_db3 t3 5
|
||||
_01902_db3 t3 6
|
||||
_01902_db3 t3 7
|
||||
_01902_db3 t3 8
|
||||
_01902_db3 t3 9
|
||||
SHOW CREATE TABLE _01902_db.t_merge
|
||||
CREATE TABLE `_01902_db`.t_merge\n(\n `n` Int8\n)\nENGINE = Merge(REGEXP(\'^_01902_db\'), \'^t\')
|
||||
SELECT _database, _table, n FROM merge(REGEXP(^_01902_db), ^t) ORDER BY _database, _table, n
|
||||
_01902_db t 0
|
||||
_01902_db t 1
|
||||
_01902_db t 2
|
||||
_01902_db t 3
|
||||
_01902_db t 4
|
||||
_01902_db t 5
|
||||
_01902_db t 6
|
||||
_01902_db t 7
|
||||
_01902_db t 8
|
||||
_01902_db t 9
|
||||
_01902_db t_merge 0
|
||||
_01902_db t_merge 0
|
||||
_01902_db t_merge 0
|
||||
_01902_db t_merge 0
|
||||
_01902_db t_merge 1
|
||||
_01902_db t_merge 1
|
||||
_01902_db t_merge 1
|
||||
_01902_db t_merge 1
|
||||
_01902_db t_merge 2
|
||||
_01902_db t_merge 2
|
||||
_01902_db t_merge 2
|
||||
_01902_db t_merge 2
|
||||
_01902_db t_merge 3
|
||||
_01902_db t_merge 3
|
||||
_01902_db t_merge 3
|
||||
_01902_db t_merge 3
|
||||
_01902_db t_merge 4
|
||||
_01902_db t_merge 4
|
||||
_01902_db t_merge 4
|
||||
_01902_db t_merge 4
|
||||
_01902_db t_merge 5
|
||||
_01902_db t_merge 5
|
||||
_01902_db t_merge 5
|
||||
_01902_db t_merge 5
|
||||
_01902_db t_merge 6
|
||||
_01902_db t_merge 6
|
||||
_01902_db t_merge 6
|
||||
_01902_db t_merge 6
|
||||
_01902_db t_merge 7
|
||||
_01902_db t_merge 7
|
||||
_01902_db t_merge 7
|
||||
_01902_db t_merge 7
|
||||
_01902_db t_merge 8
|
||||
_01902_db t_merge 8
|
||||
_01902_db t_merge 8
|
||||
_01902_db t_merge 8
|
||||
_01902_db t_merge 9
|
||||
_01902_db t_merge 9
|
||||
_01902_db t_merge 9
|
||||
_01902_db t_merge 9
|
||||
_01902_db1 t1 0
|
||||
_01902_db1 t1 1
|
||||
_01902_db1 t1 2
|
||||
_01902_db1 t1 3
|
||||
_01902_db1 t1 4
|
||||
_01902_db1 t1 5
|
||||
_01902_db1 t1 6
|
||||
_01902_db1 t1 7
|
||||
_01902_db1 t1 8
|
||||
_01902_db1 t1 9
|
||||
_01902_db2 t2 0
|
||||
_01902_db2 t2 1
|
||||
_01902_db2 t2 2
|
||||
_01902_db2 t2 3
|
||||
_01902_db2 t2 4
|
||||
_01902_db2 t2 5
|
||||
_01902_db2 t2 6
|
||||
_01902_db2 t2 7
|
||||
_01902_db2 t2 8
|
||||
_01902_db2 t2 9
|
||||
_01902_db3 t3 0
|
||||
_01902_db3 t3 1
|
||||
_01902_db3 t3 2
|
||||
_01902_db3 t3 3
|
||||
_01902_db3 t3 4
|
||||
_01902_db3 t3 5
|
||||
_01902_db3 t3 6
|
||||
_01902_db3 t3 7
|
||||
_01902_db3 t3 8
|
||||
_01902_db3 t3 9
|
||||
SELECT _database, _table, n FROM _01902_db.t_merge WHERE _database = _01902_db1 ORDER BY _database, _table, n
|
||||
_01902_db1 t1 0
|
||||
_01902_db1 t1 1
|
||||
_01902_db1 t1 2
|
||||
_01902_db1 t1 3
|
||||
_01902_db1 t1 4
|
||||
_01902_db1 t1 5
|
||||
_01902_db1 t1 6
|
||||
_01902_db1 t1 7
|
||||
_01902_db1 t1 8
|
||||
_01902_db1 t1 9
|
||||
SELECT _database, _table, n FROM _01902_db.t_merge WHERE _table = t1 ORDER BY _database, _table, n
|
||||
_01902_db1 t1 0
|
||||
_01902_db1 t1 1
|
||||
_01902_db1 t1 2
|
||||
_01902_db1 t1 3
|
||||
_01902_db1 t1 4
|
||||
_01902_db1 t1 5
|
||||
_01902_db1 t1 6
|
||||
_01902_db1 t1 7
|
||||
_01902_db1 t1 8
|
||||
_01902_db1 t1 9
|
||||
CREATE TABLE t_merge1 as _01902_db.t ENGINE=Merge(_01902_db, ^t$)
|
||||
SELECT _database, _table, n FROM _01902_db.t_merge1 ORDER BY _database, _table, n
|
||||
_01902_db t 0
|
||||
_01902_db t 1
|
||||
_01902_db t 2
|
||||
_01902_db t 3
|
||||
_01902_db t 4
|
||||
_01902_db t 5
|
||||
_01902_db t 6
|
||||
_01902_db t 7
|
||||
_01902_db t 8
|
||||
_01902_db t 9
|
||||
SELECT _database, _table, n FROM merge(_01902_db, ^t$) ORDER BY _database, _table, n
|
||||
_01902_db t 0
|
||||
_01902_db t 1
|
||||
_01902_db t 2
|
||||
_01902_db t 3
|
||||
_01902_db t 4
|
||||
_01902_db t 5
|
||||
_01902_db t 6
|
||||
_01902_db t 7
|
||||
_01902_db t 8
|
||||
_01902_db t 9
|
||||
CREATE TABLE t_merge_1 as _01902_db.t ENGINE=Merge(currentDatabase(), ^t)
|
||||
SELECT _database, _table, n FROM _01902_db.t_merge_1 ORDER BY _database, _table, n
|
||||
_01902_db1 t1 0
|
||||
_01902_db1 t1 1
|
||||
_01902_db1 t1 2
|
||||
_01902_db1 t1 3
|
||||
_01902_db1 t1 4
|
||||
_01902_db1 t1 5
|
||||
_01902_db1 t1 6
|
||||
_01902_db1 t1 7
|
||||
_01902_db1 t1 8
|
||||
_01902_db1 t1 9
|
||||
SHOW CREATE TABLE _01902_db.t_merge_1
|
||||
CREATE TABLE `_01902_db`.t_merge_1\n(\n `n` Int8\n)\nENGINE = Merge(\'_01902_db1\', \'^t\')
|
||||
SELECT _database, _table, n FROM merge(currentDatabase(), ^t) ORDER BY _database, _table, n
|
||||
01902_db1 t1 0
|
||||
01902_db1 t1 1
|
||||
01902_db1 t1 2
|
||||
01902_db1 t1 3
|
||||
01902_db1 t1 4
|
||||
01902_db1 t1 5
|
||||
01902_db1 t1 6
|
||||
01902_db1 t1 7
|
||||
01902_db1 t1 8
|
||||
01902_db1 t1 9
|
||||
_01902_db1 t1 0
|
||||
_01902_db1 t1 1
|
||||
_01902_db1 t1 2
|
||||
_01902_db1 t1 3
|
||||
_01902_db1 t1 4
|
||||
_01902_db1 t1 5
|
||||
_01902_db1 t1 6
|
||||
_01902_db1 t1 7
|
||||
_01902_db1 t1 8
|
||||
_01902_db1 t1 9
|
||||
|
@ -1,68 +1,68 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01902_db;
|
||||
DROP DATABASE IF EXISTS 01902_db1;
|
||||
DROP DATABASE IF EXISTS 01902_db2;
|
||||
DROP DATABASE IF EXISTS 01902_db3;
|
||||
DROP DATABASE IF EXISTS _01902_db;
|
||||
DROP DATABASE IF EXISTS _01902_db1;
|
||||
DROP DATABASE IF EXISTS _01902_db2;
|
||||
DROP DATABASE IF EXISTS _01902_db3;
|
||||
|
||||
CREATE DATABASE 01902_db;
|
||||
CREATE DATABASE 01902_db1;
|
||||
CREATE DATABASE 01902_db2;
|
||||
CREATE DATABASE 01902_db3;
|
||||
CREATE DATABASE _01902_db;
|
||||
CREATE DATABASE _01902_db1;
|
||||
CREATE DATABASE _01902_db2;
|
||||
CREATE DATABASE _01902_db3;
|
||||
|
||||
CREATE TABLE 01902_db.t (n Int8) ENGINE=MergeTree ORDER BY n;
|
||||
CREATE TABLE 01902_db1.t1 (n Int8) ENGINE=MergeTree ORDER BY n;
|
||||
CREATE TABLE 01902_db2.t2 (n Int8) ENGINE=MergeTree ORDER BY n;
|
||||
CREATE TABLE 01902_db3.t3 (n Int8) ENGINE=MergeTree ORDER BY n;
|
||||
CREATE TABLE _01902_db.t (n Int8) ENGINE=MergeTree ORDER BY n;
|
||||
CREATE TABLE _01902_db1.t1 (n Int8) ENGINE=MergeTree ORDER BY n;
|
||||
CREATE TABLE _01902_db2.t2 (n Int8) ENGINE=MergeTree ORDER BY n;
|
||||
CREATE TABLE _01902_db3.t3 (n Int8) ENGINE=MergeTree ORDER BY n;
|
||||
|
||||
INSERT INTO 01902_db.t SELECT * FROM numbers(10);
|
||||
INSERT INTO 01902_db1.t1 SELECT * FROM numbers(10);
|
||||
INSERT INTO 01902_db2.t2 SELECT * FROM numbers(10);
|
||||
INSERT INTO 01902_db3.t3 SELECT * FROM numbers(10);
|
||||
INSERT INTO _01902_db.t SELECT * FROM numbers(10);
|
||||
INSERT INTO _01902_db1.t1 SELECT * FROM numbers(10);
|
||||
INSERT INTO _01902_db2.t2 SELECT * FROM numbers(10);
|
||||
INSERT INTO _01902_db3.t3 SELECT * FROM numbers(10);
|
||||
|
||||
SELECT 'CREATE TABLE t_merge as 01902_db.t ENGINE=Merge(REGEXP(^01902_db), ^t)';
|
||||
CREATE TABLE 01902_db.t_merge as 01902_db.t ENGINE=Merge(REGEXP('^01902_db'), '^t');
|
||||
SELECT 'CREATE TABLE t_merge as _01902_db.t ENGINE=Merge(REGEXP(^_01902_db), ^t)';
|
||||
CREATE TABLE _01902_db.t_merge as _01902_db.t ENGINE=Merge(REGEXP('^_01902_db'), '^t');
|
||||
|
||||
SELECT 'SELECT _database, _table, n FROM 01902_db.t_merge ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM 01902_db.t_merge ORDER BY _database, _table, n;
|
||||
SELECT 'SELECT _database, _table, n FROM _01902_db.t_merge ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM _01902_db.t_merge ORDER BY _database, _table, n;
|
||||
|
||||
SELECT 'SHOW CREATE TABLE 01902_db.t_merge';
|
||||
SHOW CREATE TABLE 01902_db.t_merge;
|
||||
SELECT 'SHOW CREATE TABLE _01902_db.t_merge';
|
||||
SHOW CREATE TABLE _01902_db.t_merge;
|
||||
|
||||
SELECT 'SELECT _database, _table, n FROM merge(REGEXP(^01902_db), ^t) ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM merge(REGEXP('^01902_db'), '^t') ORDER BY _database, _table, n;
|
||||
SELECT 'SELECT _database, _table, n FROM merge(REGEXP(^_01902_db), ^t) ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM merge(REGEXP('^_01902_db'), '^t') ORDER BY _database, _table, n;
|
||||
|
||||
SELECT 'SELECT _database, _table, n FROM 01902_db.t_merge WHERE _database = 01902_db1 ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM 01902_db.t_merge WHERE _database = '01902_db1' ORDER BY _database, _table, n;
|
||||
SELECT 'SELECT _database, _table, n FROM _01902_db.t_merge WHERE _database = _01902_db1 ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM _01902_db.t_merge WHERE _database = '_01902_db1' ORDER BY _database, _table, n;
|
||||
|
||||
SELECT 'SELECT _database, _table, n FROM 01902_db.t_merge WHERE _table = t1 ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM 01902_db.t_merge WHERE _table = 't1' ORDER BY _database, _table, n;
|
||||
SELECT 'SELECT _database, _table, n FROM _01902_db.t_merge WHERE _table = t1 ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM _01902_db.t_merge WHERE _table = 't1' ORDER BY _database, _table, n;
|
||||
|
||||
-- not regexp
|
||||
SELECT 'CREATE TABLE t_merge1 as 01902_db.t ENGINE=Merge(01902_db, ^t$)';
|
||||
CREATE TABLE 01902_db.t_merge1 as 01902_db.t ENGINE=Merge('01902_db', '^t$');
|
||||
SELECT 'CREATE TABLE t_merge1 as _01902_db.t ENGINE=Merge(_01902_db, ^t$)';
|
||||
CREATE TABLE _01902_db.t_merge1 as _01902_db.t ENGINE=Merge('_01902_db', '^t$');
|
||||
|
||||
SELECT 'SELECT _database, _table, n FROM 01902_db.t_merge1 ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM 01902_db.t_merge1 ORDER BY _database, _table, n;
|
||||
SELECT 'SELECT _database, _table, n FROM _01902_db.t_merge1 ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM _01902_db.t_merge1 ORDER BY _database, _table, n;
|
||||
|
||||
SELECT 'SELECT _database, _table, n FROM merge(01902_db, ^t$) ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM merge('01902_db', '^t$') ORDER BY _database, _table, n;
|
||||
SELECT 'SELECT _database, _table, n FROM merge(_01902_db, ^t$) ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM merge('_01902_db', '^t$') ORDER BY _database, _table, n;
|
||||
|
||||
USE 01902_db1;
|
||||
USE _01902_db1;
|
||||
|
||||
SELECT 'CREATE TABLE t_merge_1 as 01902_db.t ENGINE=Merge(currentDatabase(), ^t)';
|
||||
CREATE TABLE 01902_db.t_merge_1 as 01902_db.t ENGINE=Merge(currentDatabase(), '^t');
|
||||
SELECT 'CREATE TABLE t_merge_1 as _01902_db.t ENGINE=Merge(currentDatabase(), ^t)';
|
||||
CREATE TABLE _01902_db.t_merge_1 as _01902_db.t ENGINE=Merge(currentDatabase(), '^t');
|
||||
|
||||
SELECT 'SELECT _database, _table, n FROM 01902_db.t_merge_1 ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM 01902_db.t_merge_1 ORDER BY _database, _table, n;
|
||||
SELECT 'SELECT _database, _table, n FROM _01902_db.t_merge_1 ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM _01902_db.t_merge_1 ORDER BY _database, _table, n;
|
||||
|
||||
SELECT 'SHOW CREATE TABLE 01902_db.t_merge_1';
|
||||
SHOW CREATE TABLE 01902_db.t_merge_1;
|
||||
SELECT 'SHOW CREATE TABLE _01902_db.t_merge_1';
|
||||
SHOW CREATE TABLE _01902_db.t_merge_1;
|
||||
|
||||
SELECT 'SELECT _database, _table, n FROM merge(currentDatabase(), ^t) ORDER BY _database, _table, n';
|
||||
SELECT _database, _table, n FROM merge(currentDatabase(), '^t') ORDER BY _database, _table, n;
|
||||
|
||||
DROP DATABASE 01902_db;
|
||||
DROP DATABASE 01902_db1;
|
||||
DROP DATABASE 01902_db2;
|
||||
DROP DATABASE 01902_db3;
|
||||
DROP DATABASE _01902_db;
|
||||
DROP DATABASE _01902_db1;
|
||||
DROP DATABASE _01902_db2;
|
||||
DROP DATABASE _01902_db3;
|
||||
|
@ -1,53 +1,53 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 01913_db;
|
||||
CREATE DATABASE 01913_db ENGINE=Atomic;
|
||||
DROP DATABASE IF EXISTS _01913_db;
|
||||
CREATE DATABASE _01913_db ENGINE=Atomic;
|
||||
|
||||
DROP TABLE IF EXISTS 01913_db.test_source_table_1;
|
||||
CREATE TABLE 01913_db.test_source_table_1
|
||||
DROP TABLE IF EXISTS _01913_db.test_source_table_1;
|
||||
CREATE TABLE _01913_db.test_source_table_1
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
) ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 01913_db.test_source_table_1 VALUES (0, 'Value0');
|
||||
INSERT INTO _01913_db.test_source_table_1 VALUES (0, 'Value0');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01913_db.test_dictionary;
|
||||
CREATE DICTIONARY 01913_db.test_dictionary
|
||||
DROP DICTIONARY IF EXISTS _01913_db.test_dictionary;
|
||||
CREATE DICTIONARY _01913_db.test_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(DIRECT())
|
||||
SOURCE(CLICKHOUSE(DB '01913_db' TABLE 'test_source_table_1'));
|
||||
SOURCE(CLICKHOUSE(DB '_01913_db' TABLE 'test_source_table_1'));
|
||||
|
||||
SELECT * FROM 01913_db.test_dictionary;
|
||||
SELECT * FROM _01913_db.test_dictionary;
|
||||
|
||||
DROP TABLE IF EXISTS 01913_db.test_source_table_2;
|
||||
CREATE TABLE 01913_db.test_source_table_2
|
||||
DROP TABLE IF EXISTS _01913_db.test_source_table_2;
|
||||
CREATE TABLE _01913_db.test_source_table_2
|
||||
(
|
||||
id UInt64,
|
||||
value_1 String
|
||||
) ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 01913_db.test_source_table_2 VALUES (0, 'Value1');
|
||||
INSERT INTO _01913_db.test_source_table_2 VALUES (0, 'Value1');
|
||||
|
||||
REPLACE DICTIONARY 01913_db.test_dictionary
|
||||
REPLACE DICTIONARY _01913_db.test_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value_1 String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(HASHED())
|
||||
SOURCE(CLICKHOUSE(DB '01913_db' TABLE 'test_source_table_2'))
|
||||
SOURCE(CLICKHOUSE(DB '_01913_db' TABLE 'test_source_table_2'))
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT * FROM 01913_db.test_dictionary;
|
||||
SELECT * FROM _01913_db.test_dictionary;
|
||||
|
||||
DROP DICTIONARY 01913_db.test_dictionary;
|
||||
DROP DICTIONARY _01913_db.test_dictionary;
|
||||
|
||||
DROP TABLE 01913_db.test_source_table_1;
|
||||
DROP TABLE 01913_db.test_source_table_2;
|
||||
DROP TABLE _01913_db.test_source_table_1;
|
||||
DROP TABLE _01913_db.test_source_table_2;
|
||||
|
||||
DROP DATABASE 01913_db;
|
||||
DROP DATABASE _01913_db;
|
||||
|
@ -1,42 +1,42 @@
|
||||
-- Tags: no-ordinary-database, no-parallel
|
||||
-- Tag no-ordinary-database: Requires Atomic database
|
||||
|
||||
DROP DATABASE IF EXISTS 01914_db;
|
||||
CREATE DATABASE 01914_db ENGINE=Atomic;
|
||||
DROP DATABASE IF EXISTS _01914_db;
|
||||
CREATE DATABASE _01914_db ENGINE=Atomic;
|
||||
|
||||
DROP TABLE IF EXISTS 01914_db.table_1;
|
||||
CREATE TABLE 01914_db.table_1 (id UInt64, value String) ENGINE=TinyLog;
|
||||
DROP TABLE IF EXISTS _01914_db.table_1;
|
||||
CREATE TABLE _01914_db.table_1 (id UInt64, value String) ENGINE=TinyLog;
|
||||
|
||||
DROP TABLE IF EXISTS 01914_db.table_2;
|
||||
CREATE TABLE 01914_db.table_2 (id UInt64, value String) ENGINE=TinyLog;
|
||||
DROP TABLE IF EXISTS _01914_db.table_2;
|
||||
CREATE TABLE _01914_db.table_2 (id UInt64, value String) ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 01914_db.table_1 VALUES (1, 'Table1');
|
||||
INSERT INTO 01914_db.table_2 VALUES (2, 'Table2');
|
||||
INSERT INTO _01914_db.table_1 VALUES (1, 'Table1');
|
||||
INSERT INTO _01914_db.table_2 VALUES (2, 'Table2');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01914_db.dictionary_1;
|
||||
CREATE DICTIONARY 01914_db.dictionary_1 (id UInt64, value String)
|
||||
DROP DICTIONARY IF EXISTS _01914_db.dictionary_1;
|
||||
CREATE DICTIONARY _01914_db.dictionary_1 (id UInt64, value String)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(DIRECT())
|
||||
SOURCE(CLICKHOUSE(DB '01914_db' TABLE 'table_1'));
|
||||
SOURCE(CLICKHOUSE(DB '_01914_db' TABLE 'table_1'));
|
||||
|
||||
DROP DICTIONARY IF EXISTS 01914_db.dictionary_2;
|
||||
CREATE DICTIONARY 01914_db.dictionary_2 (id UInt64, value String)
|
||||
DROP DICTIONARY IF EXISTS _01914_db.dictionary_2;
|
||||
CREATE DICTIONARY _01914_db.dictionary_2 (id UInt64, value String)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(DIRECT())
|
||||
SOURCE(CLICKHOUSE(DB '01914_db' TABLE 'table_2'));
|
||||
SOURCE(CLICKHOUSE(DB '_01914_db' TABLE 'table_2'));
|
||||
|
||||
SELECT * FROM 01914_db.dictionary_1;
|
||||
SELECT * FROM 01914_db.dictionary_2;
|
||||
SELECT * FROM _01914_db.dictionary_1;
|
||||
SELECT * FROM _01914_db.dictionary_2;
|
||||
|
||||
EXCHANGE DICTIONARIES 01914_db.dictionary_1 AND 01914_db.dictionary_2;
|
||||
EXCHANGE DICTIONARIES _01914_db.dictionary_1 AND _01914_db.dictionary_2;
|
||||
|
||||
SELECT * FROM 01914_db.dictionary_1;
|
||||
SELECT * FROM 01914_db.dictionary_2;
|
||||
SELECT * FROM _01914_db.dictionary_1;
|
||||
SELECT * FROM _01914_db.dictionary_2;
|
||||
|
||||
DROP DICTIONARY 01914_db.dictionary_1;
|
||||
DROP DICTIONARY 01914_db.dictionary_2;
|
||||
DROP DICTIONARY _01914_db.dictionary_1;
|
||||
DROP DICTIONARY _01914_db.dictionary_2;
|
||||
|
||||
DROP TABLE 01914_db.table_1;
|
||||
DROP TABLE 01914_db.table_2;
|
||||
DROP TABLE _01914_db.table_1;
|
||||
DROP TABLE _01914_db.table_2;
|
||||
|
||||
DROP DATABASE 01914_db;
|
||||
DROP DATABASE _01914_db;
|
||||
|
@ -1,9 +1,9 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 02015_db;
|
||||
CREATE DATABASE 02015_db;
|
||||
DROP DATABASE IF EXISTS _02015_db;
|
||||
CREATE DATABASE _02015_db;
|
||||
|
||||
CREATE TABLE 02015_db.test_table
|
||||
CREATE TABLE _02015_db.test_table
|
||||
(
|
||||
key_column UInt64,
|
||||
data_column_1 UInt64,
|
||||
@ -12,9 +12,9 @@ CREATE TABLE 02015_db.test_table
|
||||
ENGINE = MergeTree
|
||||
ORDER BY key_column;
|
||||
|
||||
INSERT INTO 02015_db.test_table VALUES (0, 0, 0);
|
||||
INSERT INTO _02015_db.test_table VALUES (0, 0, 0);
|
||||
|
||||
CREATE DICTIONARY 02015_db.test_dictionary
|
||||
CREATE DICTIONARY _02015_db.test_dictionary
|
||||
(
|
||||
key_column UInt64 DEFAULT 0,
|
||||
data_column_1 UInt64 DEFAULT 1,
|
||||
@ -22,20 +22,20 @@ CREATE DICTIONARY 02015_db.test_dictionary
|
||||
)
|
||||
PRIMARY KEY key_column
|
||||
LAYOUT(DIRECT())
|
||||
SOURCE(CLICKHOUSE(DB '02015_db' TABLE 'test_table'));
|
||||
SOURCE(CLICKHOUSE(DB '_02015_db' TABLE 'test_table'));
|
||||
|
||||
CREATE TABLE 02015_db.test_table_default
|
||||
CREATE TABLE _02015_db.test_table_default
|
||||
(
|
||||
data_1 DEFAULT dictGetUInt64('02015_db.test_dictionary', 'data_column_1', toUInt64(0)),
|
||||
data_2 DEFAULT dictGet(02015_db.test_dictionary, 'data_column_2', toUInt64(0))
|
||||
data_1 DEFAULT dictGetUInt64('_02015_db.test_dictionary', 'data_column_1', toUInt64(0)),
|
||||
data_2 DEFAULT dictGet(_02015_db.test_dictionary, 'data_column_2', toUInt64(0))
|
||||
)
|
||||
ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 02015_db.test_table_default(data_1) VALUES (5);
|
||||
SELECT * FROM 02015_db.test_table_default;
|
||||
INSERT INTO _02015_db.test_table_default(data_1) VALUES (5);
|
||||
SELECT * FROM _02015_db.test_table_default;
|
||||
|
||||
DROP TABLE 02015_db.test_table_default;
|
||||
DROP DICTIONARY 02015_db.test_dictionary;
|
||||
DROP TABLE 02015_db.test_table;
|
||||
DROP TABLE _02015_db.test_table_default;
|
||||
DROP DICTIONARY _02015_db.test_dictionary;
|
||||
DROP TABLE _02015_db.test_table;
|
||||
|
||||
DROP DATABASE 02015_db;
|
||||
DROP DATABASE _02015_db;
|
||||
|
@ -1,22 +1,22 @@
|
||||
DROP TABLE IF EXISTS 02125_test_table;
|
||||
CREATE TABLE 02125_test_table
|
||||
DROP TABLE IF EXISTS _02125_test_table;
|
||||
CREATE TABLE _02125_test_table
|
||||
(
|
||||
id UInt64,
|
||||
value Nullable(String)
|
||||
)
|
||||
ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 02125_test_table VALUES (0, 'Value');
|
||||
INSERT INTO _02125_test_table VALUES (0, 'Value');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02125_test_dictionary;
|
||||
CREATE DICTIONARY 02125_test_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02125_test_dictionary;
|
||||
CREATE DICTIONARY _02125_test_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value Nullable(String)
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02125_test_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02125_test_table'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT dictGet('02125_test_dictionary', 'value', toUInt64(0));
|
||||
SELECT dictGetString('02125_test_dictionary', 'value', toUInt64(0)); --{serverError 53}
|
||||
SELECT dictGet('_02125_test_dictionary', 'value', toUInt64(0));
|
||||
SELECT dictGetString('_02125_test_dictionary', 'value', toUInt64(0)); --{serverError 53}
|
||||
|
@ -1,6 +1,6 @@
|
||||
DROP TABLE IF EXISTS 02127_join_settings_with_persistency_1;
|
||||
CREATE TABLE 02127_join_settings_with_persistency_1 (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=1, join_any_take_last_row=0;
|
||||
SHOW CREATE TABLE 02127_join_settings_with_persistency_1;
|
||||
DROP TABLE IF EXISTS 02127_join_settings_with_persistency_0;
|
||||
CREATE TABLE 02127_join_settings_with_persistency_0 (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=0, join_any_take_last_row=0;
|
||||
SHOW CREATE TABLE 02127_join_settings_with_persistency_0;
|
||||
DROP TABLE IF EXISTS _02127_join_settings_with_persistency_1;
|
||||
CREATE TABLE _02127_join_settings_with_persistency_1 (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=1, join_any_take_last_row=0;
|
||||
SHOW CREATE TABLE _02127_join_settings_with_persistency_1;
|
||||
DROP TABLE IF EXISTS _02127_join_settings_with_persistency_0;
|
||||
CREATE TABLE _02127_join_settings_with_persistency_0 (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=0, join_any_take_last_row=0;
|
||||
SHOW CREATE TABLE _02127_join_settings_with_persistency_0;
|
||||
|
@ -1,54 +1,54 @@
|
||||
DROP TABLE IF EXISTS 02131_rptable;
|
||||
CREATE TABLE 02131_rptable (x UInt8) ENGINE = MergeTree ORDER BY x;
|
||||
INSERT INTO 02131_rptable VALUES (1), (2), (3), (4);
|
||||
DROP TABLE IF EXISTS _02131_rptable;
|
||||
CREATE TABLE _02131_rptable (x UInt8) ENGINE = MergeTree ORDER BY x;
|
||||
INSERT INTO _02131_rptable VALUES (1), (2), (3), (4);
|
||||
|
||||
DROP ROW POLICY IF EXISTS 02131_filter_1 ON 02131_rptable;
|
||||
DROP ROW POLICY IF EXISTS 02131_filter_2 ON 02131_rptable;
|
||||
DROP ROW POLICY IF EXISTS 02131_filter_3 ON 02131_rptable;
|
||||
DROP ROW POLICY IF EXISTS 02131_filter_4 ON 02131_rptable;
|
||||
DROP ROW POLICY IF EXISTS 02131_filter_5 ON 02131_rptable;
|
||||
DROP ROW POLICY IF EXISTS _02131_filter_1 ON _02131_rptable;
|
||||
DROP ROW POLICY IF EXISTS _02131_filter_2 ON _02131_rptable;
|
||||
DROP ROW POLICY IF EXISTS _02131_filter_3 ON _02131_rptable;
|
||||
DROP ROW POLICY IF EXISTS _02131_filter_4 ON _02131_rptable;
|
||||
DROP ROW POLICY IF EXISTS _02131_filter_5 ON _02131_rptable;
|
||||
|
||||
SELECT 'None';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
CREATE ROW POLICY 02131_filter_1 ON 02131_rptable USING x=1 AS permissive TO ALL;
|
||||
CREATE ROW POLICY _02131_filter_1 ON _02131_rptable USING x=1 AS permissive TO ALL;
|
||||
SELECT 'R1: x == 1';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
CREATE ROW POLICY 02131_filter_2 ON 02131_rptable USING x=2 AS permissive TO ALL;
|
||||
CREATE ROW POLICY _02131_filter_2 ON _02131_rptable USING x=2 AS permissive TO ALL;
|
||||
SELECT 'R1, R2: (x == 1) OR (x == 2)';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
CREATE ROW POLICY 02131_filter_3 ON 02131_rptable USING x=3 AS permissive TO ALL;
|
||||
CREATE ROW POLICY _02131_filter_3 ON _02131_rptable USING x=3 AS permissive TO ALL;
|
||||
SELECT 'R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
CREATE ROW POLICY 02131_filter_4 ON 02131_rptable USING x<=2 AS restrictive TO ALL;
|
||||
CREATE ROW POLICY _02131_filter_4 ON _02131_rptable USING x<=2 AS restrictive TO ALL;
|
||||
SELECT 'R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
CREATE ROW POLICY 02131_filter_5 ON 02131_rptable USING x>=2 AS restrictive TO ALL;
|
||||
CREATE ROW POLICY _02131_filter_5 ON _02131_rptable USING x>=2 AS restrictive TO ALL;
|
||||
SELECT 'R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
DROP ROW POLICY 02131_filter_1 ON 02131_rptable;
|
||||
DROP ROW POLICY _02131_filter_1 ON _02131_rptable;
|
||||
SELECT 'R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
DROP ROW POLICY 02131_filter_2 ON 02131_rptable;
|
||||
DROP ROW POLICY _02131_filter_2 ON _02131_rptable;
|
||||
SELECT 'R3, R4, R5: (x == 3) AND (x <= 2) AND (x >= 2)';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
DROP ROW POLICY 02131_filter_3 ON 02131_rptable;
|
||||
DROP ROW POLICY _02131_filter_3 ON _02131_rptable;
|
||||
SELECT 'R4, R5: (x <= 2) AND (x >= 2)';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
DROP ROW POLICY 02131_filter_4 ON 02131_rptable;
|
||||
DROP ROW POLICY _02131_filter_4 ON _02131_rptable;
|
||||
SELECT 'R5: (x >= 2)';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
DROP ROW POLICY 02131_filter_5 ON 02131_rptable;
|
||||
DROP ROW POLICY _02131_filter_5 ON _02131_rptable;
|
||||
SELECT 'None';
|
||||
SELECT * FROM 02131_rptable;
|
||||
SELECT * FROM _02131_rptable;
|
||||
|
||||
DROP TABLE 02131_rptable;
|
||||
DROP TABLE _02131_rptable;
|
||||
|
@ -1,35 +1,35 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP FUNCTION IF EXISTS 02148_test_function;
|
||||
CREATE FUNCTION 02148_test_function AS () -> (SELECT 1);
|
||||
DROP FUNCTION IF EXISTS _02148_test_function;
|
||||
CREATE FUNCTION _02148_test_function AS () -> (SELECT 1);
|
||||
|
||||
SELECT 02148_test_function();
|
||||
SELECT _02148_test_function();
|
||||
|
||||
CREATE OR REPLACE FUNCTION 02148_test_function AS () -> (SELECT 2);
|
||||
CREATE OR REPLACE FUNCTION _02148_test_function AS () -> (SELECT 2);
|
||||
|
||||
SELECT 02148_test_function();
|
||||
SELECT _02148_test_function();
|
||||
|
||||
DROP FUNCTION 02148_test_function;
|
||||
DROP FUNCTION _02148_test_function;
|
||||
|
||||
CREATE FUNCTION 02148_test_function AS (x) -> (SELECT x + 1);
|
||||
SELECT 02148_test_function(1);
|
||||
CREATE FUNCTION _02148_test_function AS (x) -> (SELECT x + 1);
|
||||
SELECT _02148_test_function(1);
|
||||
|
||||
DROP FUNCTION IF EXISTS 02148_test_function_nested;
|
||||
CREATE FUNCTION 02148_test_function_nested AS (x) -> 02148_test_function(x + 2);
|
||||
SELECT 02148_test_function_nested(1);
|
||||
DROP FUNCTION IF EXISTS _02148_test_function_nested;
|
||||
CREATE FUNCTION _02148_test_function_nested AS (x) -> _02148_test_function(x + 2);
|
||||
SELECT _02148_test_function_nested(1);
|
||||
|
||||
DROP FUNCTION 02148_test_function;
|
||||
DROP FUNCTION 02148_test_function_nested;
|
||||
DROP FUNCTION _02148_test_function;
|
||||
DROP FUNCTION _02148_test_function_nested;
|
||||
|
||||
DROP TABLE IF EXISTS 02148_test_table;
|
||||
CREATE TABLE 02148_test_table (id UInt64, value String) ENGINE=TinyLog;
|
||||
INSERT INTO 02148_test_table VALUES (0, 'Value');
|
||||
DROP TABLE IF EXISTS _02148_test_table;
|
||||
CREATE TABLE _02148_test_table (id UInt64, value String) ENGINE=TinyLog;
|
||||
INSERT INTO _02148_test_table VALUES (0, 'Value');
|
||||
|
||||
CREATE FUNCTION 02148_test_function AS () -> (SELECT * FROM 02148_test_table LIMIT 1);
|
||||
SELECT 02148_test_function();
|
||||
CREATE FUNCTION _02148_test_function AS () -> (SELECT * FROM _02148_test_table LIMIT 1);
|
||||
SELECT _02148_test_function();
|
||||
|
||||
CREATE OR REPLACE FUNCTION 02148_test_function AS () -> (SELECT value FROM 02148_test_table LIMIT 1);
|
||||
SELECT 02148_test_function();
|
||||
CREATE OR REPLACE FUNCTION _02148_test_function AS () -> (SELECT value FROM _02148_test_table LIMIT 1);
|
||||
SELECT _02148_test_function();
|
||||
|
||||
DROP FUNCTION 02148_test_function;
|
||||
DROP TABLE 02148_test_table;
|
||||
DROP FUNCTION _02148_test_function;
|
||||
DROP TABLE _02148_test_table;
|
||||
|
@ -4,22 +4,22 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02154_test_source_table"
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS _02154_test_source_table"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q """
|
||||
CREATE TABLE 02154_test_source_table
|
||||
CREATE TABLE _02154_test_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
) ENGINE=TinyLog;
|
||||
"""
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO 02154_test_source_table VALUES (0, 'Value')"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM 02154_test_source_table"
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO _02154_test_source_table VALUES (0, 'Value')"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM _02154_test_source_table"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP DICTIONARY IF EXISTS 02154_test_dictionary"
|
||||
$CLICKHOUSE_CLIENT -q "DROP DICTIONARY IF EXISTS _02154_test_dictionary"
|
||||
$CLICKHOUSE_CLIENT -q """
|
||||
CREATE DICTIONARY 02154_test_dictionary
|
||||
CREATE DICTIONARY _02154_test_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
@ -27,13 +27,13 @@ $CLICKHOUSE_CLIENT -q """
|
||||
PRIMARY KEY id
|
||||
LAYOUT(HASHED())
|
||||
LIFETIME(0)
|
||||
SOURCE(CLICKHOUSE(TABLE '02154_test_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02154_test_source_table'))
|
||||
"""
|
||||
|
||||
echo """
|
||||
SELECT dictGet(02154_test_dictionary, 'value', toUInt64(0)), dictGet(02154_test_dictionary, 'value', toUInt64(1))
|
||||
SELECT dictGet(_02154_test_dictionary, 'value', toUInt64(0)), dictGet(_02154_test_dictionary, 'value', toUInt64(1))
|
||||
FORMAT JSON
|
||||
""" | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&wait_end_of_query=1&output_format_write_statistics=0" -d @-
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP DICTIONARY 02154_test_dictionary"
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE 02154_test_source_table"
|
||||
$CLICKHOUSE_CLIENT -q "DROP DICTIONARY _02154_test_dictionary"
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE _02154_test_source_table"
|
||||
|
@ -1,53 +1,53 @@
|
||||
DROP TABLE IF EXISTS 02155_test_table;
|
||||
CREATE TABLE 02155_test_table
|
||||
DROP TABLE IF EXISTS _02155_test_table;
|
||||
CREATE TABLE _02155_test_table
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
) ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 02155_test_table VALUES (0, 'Value');
|
||||
INSERT INTO _02155_test_table VALUES (0, 'Value');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02155_test_dictionary;
|
||||
CREATE DICTIONARY 02155_test_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02155_test_dictionary;
|
||||
CREATE DICTIONARY _02155_test_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02155_test_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02155_test_table'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary' AND database == currentDatabase();
|
||||
SELECT name, comment FROM system.dictionaries WHERE name == '_02155_test_dictionary' AND database == currentDatabase();
|
||||
|
||||
ALTER TABLE 02155_test_dictionary COMMENT COLUMN value 'value_column'; --{serverError 48}
|
||||
ALTER TABLE _02155_test_dictionary COMMENT COLUMN value 'value_column'; --{serverError 48}
|
||||
|
||||
ALTER TABLE 02155_test_dictionary MODIFY COMMENT '02155_test_dictionary_comment_0';
|
||||
SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary' AND database == currentDatabase();
|
||||
SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary' AND database == currentDatabase();
|
||||
ALTER TABLE _02155_test_dictionary MODIFY COMMENT '_02155_test_dictionary_comment_0';
|
||||
SELECT name, comment FROM system.dictionaries WHERE name == '_02155_test_dictionary' AND database == currentDatabase();
|
||||
SELECT name, comment FROM system.tables WHERE name == '_02155_test_dictionary' AND database == currentDatabase();
|
||||
|
||||
SELECT * FROM 02155_test_dictionary;
|
||||
SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary' AND database == currentDatabase();
|
||||
SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary' AND database == currentDatabase();
|
||||
SELECT * FROM _02155_test_dictionary;
|
||||
SELECT name, comment FROM system.dictionaries WHERE name == '_02155_test_dictionary' AND database == currentDatabase();
|
||||
SELECT name, comment FROM system.tables WHERE name == '_02155_test_dictionary' AND database == currentDatabase();
|
||||
|
||||
ALTER TABLE 02155_test_dictionary MODIFY COMMENT '02155_test_dictionary_comment_1';
|
||||
SELECT name, comment FROM system.dictionaries WHERE name == '02155_test_dictionary' AND database == currentDatabase();
|
||||
SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary' AND database == currentDatabase();
|
||||
ALTER TABLE _02155_test_dictionary MODIFY COMMENT '_02155_test_dictionary_comment_1';
|
||||
SELECT name, comment FROM system.dictionaries WHERE name == '_02155_test_dictionary' AND database == currentDatabase();
|
||||
SELECT name, comment FROM system.tables WHERE name == '_02155_test_dictionary' AND database == currentDatabase();
|
||||
|
||||
DROP TABLE IF EXISTS 02155_test_dictionary_view;
|
||||
CREATE TABLE 02155_test_dictionary_view
|
||||
DROP TABLE IF EXISTS _02155_test_dictionary_view;
|
||||
CREATE TABLE _02155_test_dictionary_view
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
) ENGINE=Dictionary(concat(currentDatabase(), '.02155_test_dictionary'));
|
||||
) ENGINE=Dictionary(concat(currentDatabase(), '._02155_test_dictionary'));
|
||||
|
||||
SELECT * FROM 02155_test_dictionary_view;
|
||||
SELECT * FROM _02155_test_dictionary_view;
|
||||
|
||||
ALTER TABLE 02155_test_dictionary_view COMMENT COLUMN value 'value_column'; --{serverError 48}
|
||||
ALTER TABLE _02155_test_dictionary_view COMMENT COLUMN value 'value_column'; --{serverError 48}
|
||||
|
||||
ALTER TABLE 02155_test_dictionary_view MODIFY COMMENT '02155_test_dictionary_view_comment_0';
|
||||
SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary_view' AND database == currentDatabase();
|
||||
SELECT name, comment FROM system.tables WHERE name == '02155_test_dictionary_view' AND database == currentDatabase();
|
||||
ALTER TABLE _02155_test_dictionary_view MODIFY COMMENT '_02155_test_dictionary_view_comment_0';
|
||||
SELECT name, comment FROM system.tables WHERE name == '_02155_test_dictionary_view' AND database == currentDatabase();
|
||||
SELECT name, comment FROM system.tables WHERE name == '_02155_test_dictionary_view' AND database == currentDatabase();
|
||||
|
||||
DROP TABLE 02155_test_dictionary_view;
|
||||
DROP TABLE 02155_test_table;
|
||||
DROP DICTIONARY 02155_test_dictionary;
|
||||
DROP TABLE _02155_test_dictionary_view;
|
||||
DROP TABLE _02155_test_table;
|
||||
DROP DICTIONARY _02155_test_dictionary;
|
||||
|
@ -1,15 +1,15 @@
|
||||
DROP TABLE IF EXISTS 02162_test_table;
|
||||
CREATE TABLE 02162_test_table
|
||||
DROP TABLE IF EXISTS _02162_table;
|
||||
CREATE TABLE _02162_table
|
||||
(
|
||||
id UInt64,
|
||||
value String,
|
||||
range_value UInt64
|
||||
) ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 02162_test_table VALUES (0, 'Value', 1);
|
||||
INSERT INTO _02162_table VALUES (0, 'Value', 1);
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02162_test_dictionary;
|
||||
CREATE DICTIONARY 02162_test_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02162_dictionary;
|
||||
CREATE DICTIONARY _02162_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String,
|
||||
@ -18,12 +18,12 @@ CREATE DICTIONARY 02162_test_dictionary
|
||||
end UInt64 EXPRESSION range_value
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02162_test_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02162_table'))
|
||||
LAYOUT(RANGE_HASHED())
|
||||
RANGE(MIN start MAX end)
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT * FROM 02162_test_dictionary;
|
||||
SELECT * FROM _02162_dictionary;
|
||||
|
||||
DROP DICTIONARY 02162_test_dictionary;
|
||||
DROP TABLE 02162_test_table;
|
||||
DROP DICTIONARY _02162_dictionary;
|
||||
DROP TABLE _02162_table;
|
||||
|
@ -14,54 +14,54 @@ set -e
|
||||
[ -e "${CLICKHOUSE_TMP}"/example.com. ] && rm "${CLICKHOUSE_TMP}"/example.com.
|
||||
[ -e "${CLICKHOUSE_TMP}"/museum...protobuf ] && rm "${CLICKHOUSE_TMP}"/museum...protobuf
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS 02165_out_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE 02165_out_tb (id UInt64, name String) Engine=Memory;"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO 02165_out_tb Values(1, 'one'), (2, 'tow');"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS _02165_out_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE _02165_out_tb (id UInt64, name String) Engine=Memory;"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO _02165_out_tb Values(1, 'one'), (2, 'tow');"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE 02165_in_tb (id UInt64, name String) Engine=Memory;"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS _02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE _02165_in_tb (id UInt64, name String) Engine=Memory;"
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/hello.csv';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE 02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/hello.csv' FORMAT CSV;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/hello.csv';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE _02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/hello.csv' FORMAT CSV;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE _02165_in_tb;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/world.csv.gz';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE 02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/world.csv.gz' COMPRESSION 'gz' FORMAT CSV;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/world.csv.gz';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE _02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/world.csv.gz' COMPRESSION 'gz' FORMAT CSV;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE _02165_in_tb;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/hello.world.csv';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE 02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/hello.world.csv' FORMAT CSV;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/hello.world.csv';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE _02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/hello.world.csv' FORMAT CSV;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE _02165_in_tb;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/hello.world.csv.xz';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE 02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/hello.world.csv.xz' COMPRESSION 'xz' FORMAT CSV;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/hello.world.csv.xz';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE _02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/hello.world.csv.xz' COMPRESSION 'xz' FORMAT CSV;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE _02165_in_tb;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/example.com.';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE 02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/example.com.' FORMAT TabSeparated;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/example.com.';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE _02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/example.com.' FORMAT TabSeparated;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE _02165_in_tb;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/museum...JSONEachRow';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE 02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/museum...JSONEachRow';"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/museum...JSONEachRow';"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE _02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/museum...JSONEachRow';"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE _02165_in_tb;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE 02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/world.csv.gz';"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE _02165_in_tb FROM INFILE '${CLICKHOUSE_TMP}/world.csv.gz';"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE _02165_in_tb;"
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM 02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/.htaccess.json';"
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM _02165_out_tb INTO OUTFILE '${CLICKHOUSE_TMP}/.htaccess.json';"
|
||||
head -n 26 ${CLICKHOUSE_TMP}/.htaccess.json
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS 02165_out_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS 02165_in_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS _02165_out_tb;"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS _02165_in_tb;"
|
||||
|
||||
rm "${CLICKHOUSE_TMP}"/hello.csv
|
||||
rm "${CLICKHOUSE_TMP}"/world.csv.gz
|
||||
|
@ -1,67 +1,67 @@
|
||||
DROP TABLE IF EXISTS 02176_test_simple_key_table;
|
||||
CREATE TABLE 02176_test_simple_key_table
|
||||
DROP TABLE IF EXISTS _02176_test_simple_key_table;
|
||||
CREATE TABLE _02176_test_simple_key_table
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
) ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 02176_test_simple_key_table VALUES (0, 'Value');
|
||||
INSERT INTO _02176_test_simple_key_table VALUES (0, 'Value');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02176_test_simple_key_dictionary;
|
||||
CREATE DICTIONARY 02176_test_simple_key_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02176_test_simple_key_dictionary;
|
||||
CREATE DICTIONARY _02176_test_simple_key_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02176_test_simple_key_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02176_test_simple_key_table'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT dictGet('02176_test_simple_key_dictionary', 'value', toUInt64(0));
|
||||
SELECT dictGet('02176_test_simple_key_dictionary', 'value', toUInt8(0));
|
||||
SELECT dictGet('02176_test_simple_key_dictionary', 'value', '0');
|
||||
SELECT dictGet('02176_test_simple_key_dictionary', 'value', [0]); --{serverError 43}
|
||||
SELECT dictGet('_02176_test_simple_key_dictionary', 'value', toUInt64(0));
|
||||
SELECT dictGet('_02176_test_simple_key_dictionary', 'value', toUInt8(0));
|
||||
SELECT dictGet('_02176_test_simple_key_dictionary', 'value', '0');
|
||||
SELECT dictGet('_02176_test_simple_key_dictionary', 'value', [0]); --{serverError 43}
|
||||
|
||||
SELECT dictHas('02176_test_simple_key_dictionary', toUInt64(0));
|
||||
SELECT dictHas('02176_test_simple_key_dictionary', toUInt8(0));
|
||||
SELECT dictHas('02176_test_simple_key_dictionary', '0');
|
||||
SELECT dictHas('02176_test_simple_key_dictionary', [0]); --{serverError 43}
|
||||
SELECT dictHas('_02176_test_simple_key_dictionary', toUInt64(0));
|
||||
SELECT dictHas('_02176_test_simple_key_dictionary', toUInt8(0));
|
||||
SELECT dictHas('_02176_test_simple_key_dictionary', '0');
|
||||
SELECT dictHas('_02176_test_simple_key_dictionary', [0]); --{serverError 43}
|
||||
|
||||
DROP DICTIONARY 02176_test_simple_key_dictionary;
|
||||
DROP TABLE 02176_test_simple_key_table;
|
||||
DROP DICTIONARY _02176_test_simple_key_dictionary;
|
||||
DROP TABLE _02176_test_simple_key_table;
|
||||
|
||||
DROP TABLE IF EXISTS 02176_test_complex_key_table;
|
||||
CREATE TABLE 02176_test_complex_key_table
|
||||
DROP TABLE IF EXISTS _02176_test_complex_key_table;
|
||||
CREATE TABLE _02176_test_complex_key_table
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value String
|
||||
) ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 02176_test_complex_key_table VALUES (0, '0', 'Value');
|
||||
INSERT INTO _02176_test_complex_key_table VALUES (0, '0', 'Value');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02176_test_complex_key_dictionary;
|
||||
CREATE DICTIONARY 02176_test_complex_key_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02176_test_complex_key_dictionary;
|
||||
CREATE DICTIONARY _02176_test_complex_key_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
id_key String,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id, id_key
|
||||
SOURCE(CLICKHOUSE(TABLE '02176_test_complex_key_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02176_test_complex_key_table'))
|
||||
LAYOUT(COMPLEX_KEY_DIRECT());
|
||||
|
||||
SELECT dictGet('02176_test_complex_key_dictionary', 'value', tuple(toUInt64(0), '0'));
|
||||
SELECT dictGet('02176_test_complex_key_dictionary', 'value', tuple(toUInt8(0), '0'));
|
||||
SELECT dictGet('02176_test_complex_key_dictionary', 'value', tuple('0', '0'));
|
||||
SELECT dictGet('02176_test_complex_key_dictionary', 'value', tuple([0], '0')); --{serverError 43}
|
||||
SELECT dictGet('02176_test_complex_key_dictionary', 'value', tuple(toUInt64(0), 0));
|
||||
SELECT dictGet('_02176_test_complex_key_dictionary', 'value', tuple(toUInt64(0), '0'));
|
||||
SELECT dictGet('_02176_test_complex_key_dictionary', 'value', tuple(toUInt8(0), '0'));
|
||||
SELECT dictGet('_02176_test_complex_key_dictionary', 'value', tuple('0', '0'));
|
||||
SELECT dictGet('_02176_test_complex_key_dictionary', 'value', tuple([0], '0')); --{serverError 43}
|
||||
SELECT dictGet('_02176_test_complex_key_dictionary', 'value', tuple(toUInt64(0), 0));
|
||||
|
||||
SELECT dictHas('02176_test_complex_key_dictionary', tuple(toUInt64(0), '0'));
|
||||
SELECT dictHas('02176_test_complex_key_dictionary', tuple(toUInt8(0), '0'));
|
||||
SELECT dictHas('02176_test_complex_key_dictionary', tuple('0', '0'));
|
||||
SELECT dictHas('02176_test_complex_key_dictionary', tuple([0], '0')); --{serverError 43}
|
||||
SELECT dictHas('02176_test_complex_key_dictionary', tuple(toUInt64(0), 0));
|
||||
SELECT dictHas('_02176_test_complex_key_dictionary', tuple(toUInt64(0), '0'));
|
||||
SELECT dictHas('_02176_test_complex_key_dictionary', tuple(toUInt8(0), '0'));
|
||||
SELECT dictHas('_02176_test_complex_key_dictionary', tuple('0', '0'));
|
||||
SELECT dictHas('_02176_test_complex_key_dictionary', tuple([0], '0')); --{serverError 43}
|
||||
SELECT dictHas('_02176_test_complex_key_dictionary', tuple(toUInt64(0), 0));
|
||||
|
||||
DROP DICTIONARY 02176_test_complex_key_dictionary;
|
||||
DROP TABLE 02176_test_complex_key_table;
|
||||
DROP DICTIONARY _02176_test_complex_key_dictionary;
|
||||
DROP TABLE _02176_test_complex_key_table;
|
||||
|
@ -1,5 +1,5 @@
|
||||
DROP TABLE IF EXISTS 02179_test_table;
|
||||
CREATE TABLE 02179_test_table
|
||||
DROP TABLE IF EXISTS _02179_test_table;
|
||||
CREATE TABLE _02179_test_table
|
||||
(
|
||||
id UInt64,
|
||||
value String,
|
||||
@ -7,12 +7,12 @@ CREATE TABLE 02179_test_table
|
||||
end Int64
|
||||
) Engine = TinyLog;
|
||||
|
||||
INSERT INTO 02179_test_table VALUES (0, 'Value', 10, 0);
|
||||
INSERT INTO 02179_test_table VALUES (0, 'Value', 15, 10);
|
||||
INSERT INTO 02179_test_table VALUES (0, 'Value', 15, 20);
|
||||
INSERT INTO _02179_test_table VALUES (0, 'Value', 10, 0);
|
||||
INSERT INTO _02179_test_table VALUES (0, 'Value', 15, 10);
|
||||
INSERT INTO _02179_test_table VALUES (0, 'Value', 15, 20);
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02179_test_dictionary;
|
||||
CREATE DICTIONARY 02179_test_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02179_test_dictionary;
|
||||
CREATE DICTIONARY _02179_test_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String DEFAULT 'DefaultValue',
|
||||
@ -20,17 +20,17 @@ CREATE DICTIONARY 02179_test_dictionary
|
||||
end Int64
|
||||
) PRIMARY KEY id
|
||||
LAYOUT(RANGE_HASHED())
|
||||
SOURCE(CLICKHOUSE(TABLE '02179_test_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02179_test_table'))
|
||||
RANGE(MIN start MAX end)
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT dictGet('02179_test_dictionary', 'value', 0, 15);
|
||||
SELECT dictGet('02179_test_dictionary', 'value', 0, 5);
|
||||
SELECT dictGet('_02179_test_dictionary', 'value', 0, 15);
|
||||
SELECT dictGet('_02179_test_dictionary', 'value', 0, 5);
|
||||
|
||||
SELECT dictHas('02179_test_dictionary', 0, 15);
|
||||
SELECT dictHas('02179_test_dictionary', 0, 5);
|
||||
SELECT dictHas('_02179_test_dictionary', 0, 15);
|
||||
SELECT dictHas('_02179_test_dictionary', 0, 5);
|
||||
|
||||
SELECT * FROM 02179_test_dictionary;
|
||||
SELECT * FROM _02179_test_dictionary;
|
||||
|
||||
DROP DICTIONARY 02179_test_dictionary;
|
||||
DROP TABLE 02179_test_table;
|
||||
DROP DICTIONARY _02179_test_dictionary;
|
||||
DROP TABLE _02179_test_table;
|
||||
|
@ -1,39 +1,39 @@
|
||||
DROP TABLE IF EXISTS 02181_test_table;
|
||||
CREATE TABLE 02181_test_table
|
||||
DROP TABLE IF EXISTS _02181_test_table;
|
||||
CREATE TABLE _02181_test_table
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 02181_test_table VALUES (0, 'Value');
|
||||
INSERT INTO _02181_test_table VALUES (0, 'Value');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02181_test_dictionary;
|
||||
CREATE DICTIONARY 02181_test_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02181_test_dictionary;
|
||||
CREATE DICTIONARY _02181_test_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02181_test_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02181_test_table'))
|
||||
LAYOUT(HASHED())
|
||||
LIFETIME(0);
|
||||
|
||||
DETACH TABLE 02181_test_dictionary; --{serverError 520}
|
||||
ATTACH TABLE 02181_test_dictionary; --{serverError 80}
|
||||
DETACH TABLE _02181_test_dictionary; --{serverError 520}
|
||||
ATTACH TABLE _02181_test_dictionary; --{serverError 80}
|
||||
|
||||
DETACH DICTIONARY 02181_test_dictionary;
|
||||
ATTACH DICTIONARY 02181_test_dictionary;
|
||||
DETACH DICTIONARY _02181_test_dictionary;
|
||||
ATTACH DICTIONARY _02181_test_dictionary;
|
||||
|
||||
SELECT * FROM 02181_test_dictionary;
|
||||
SELECT * FROM _02181_test_dictionary;
|
||||
|
||||
DETACH DICTIONARY 02181_test_dictionary;
|
||||
ATTACH DICTIONARY 02181_test_dictionary;
|
||||
DETACH DICTIONARY _02181_test_dictionary;
|
||||
ATTACH DICTIONARY _02181_test_dictionary;
|
||||
|
||||
SELECT * FROM 02181_test_dictionary;
|
||||
SELECT * FROM _02181_test_dictionary;
|
||||
|
||||
DETACH DICTIONARY 02181_test_dictionary;
|
||||
ATTACH DICTIONARY 02181_test_dictionary;
|
||||
DETACH DICTIONARY _02181_test_dictionary;
|
||||
ATTACH DICTIONARY _02181_test_dictionary;
|
||||
|
||||
DROP DICTIONARY 02181_test_dictionary;
|
||||
DROP TABLE 02181_test_table;
|
||||
DROP DICTIONARY _02181_test_dictionary;
|
||||
DROP TABLE _02181_test_table;
|
||||
|
@ -1,5 +1,5 @@
|
||||
DROP TABLE IF EXISTS 02183_dictionary_source_table;
|
||||
CREATE TABLE 02183_dictionary_source_table
|
||||
DROP TABLE IF EXISTS _02183_dictionary_source_table;
|
||||
CREATE TABLE _02183_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
value_date Date,
|
||||
@ -8,12 +8,12 @@ CREATE TABLE 02183_dictionary_source_table
|
||||
value_date_time_64 DateTime64
|
||||
) ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 02183_dictionary_source_table VALUES (0, '2019-05-05', '2019-05-05', '2019-05-05', '2019-05-05');
|
||||
INSERT INTO _02183_dictionary_source_table VALUES (0, '2019-05-05', '2019-05-05', '2019-05-05', '2019-05-05');
|
||||
|
||||
SELECT * FROM 02183_dictionary_source_table;
|
||||
SELECT * FROM _02183_dictionary_source_table;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_flat_dictionary;
|
||||
CREATE DICTIONARY 02183_flat_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_flat_dictionary;
|
||||
CREATE DICTIONARY _02183_flat_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value_date Date,
|
||||
@ -22,17 +22,17 @@ CREATE DICTIONARY 02183_flat_dictionary
|
||||
value_date_time_64 DateTime64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_dictionary_source_table'))
|
||||
LIFETIME(0)
|
||||
LAYOUT(FLAT());
|
||||
|
||||
SELECT 'Flat dictionary';
|
||||
SELECT * FROM 02183_flat_dictionary;
|
||||
SELECT * FROM _02183_flat_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_flat_dictionary;
|
||||
DROP DICTIONARY _02183_flat_dictionary;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_hashed_dictionary;
|
||||
CREATE DICTIONARY 02183_hashed_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_hashed_dictionary;
|
||||
CREATE DICTIONARY _02183_hashed_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value_date Date,
|
||||
@ -41,17 +41,17 @@ CREATE DICTIONARY 02183_hashed_dictionary
|
||||
value_date_time_64 DateTime64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_dictionary_source_table'))
|
||||
LIFETIME(0)
|
||||
LAYOUT(HASHED());
|
||||
|
||||
SELECT 'Hashed dictionary';
|
||||
SELECT * FROM 02183_hashed_dictionary;
|
||||
SELECT * FROM _02183_hashed_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_hashed_dictionary;
|
||||
DROP DICTIONARY _02183_hashed_dictionary;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_hashed_array_dictionary;
|
||||
CREATE DICTIONARY 02183_hashed_array_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_hashed_array_dictionary;
|
||||
CREATE DICTIONARY _02183_hashed_array_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value_date Date,
|
||||
@ -60,17 +60,17 @@ CREATE DICTIONARY 02183_hashed_array_dictionary
|
||||
value_date_time_64 DateTime64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_dictionary_source_table'))
|
||||
LIFETIME(0)
|
||||
LAYOUT(HASHED_ARRAY());
|
||||
|
||||
SELECT 'Hashed array dictionary';
|
||||
SELECT * FROM 02183_hashed_array_dictionary;
|
||||
SELECT * FROM _02183_hashed_array_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_hashed_array_dictionary;
|
||||
DROP DICTIONARY _02183_hashed_array_dictionary;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_cache_dictionary;
|
||||
CREATE DICTIONARY 02183_cache_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_cache_dictionary;
|
||||
CREATE DICTIONARY _02183_cache_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value_date Date,
|
||||
@ -79,18 +79,18 @@ CREATE DICTIONARY 02183_cache_dictionary
|
||||
value_date_time_64 DateTime64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_dictionary_source_table'))
|
||||
LIFETIME(0)
|
||||
LAYOUT(CACHE(SIZE_IN_CELLS 10));
|
||||
|
||||
SELECT 'Cache dictionary';
|
||||
SELECT dictGet('02183_cache_dictionary', 'value_date', 0);
|
||||
SELECT * FROM 02183_cache_dictionary;
|
||||
SELECT dictGet('_02183_cache_dictionary', 'value_date', 0);
|
||||
SELECT * FROM _02183_cache_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_cache_dictionary;
|
||||
DROP DICTIONARY _02183_cache_dictionary;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_direct_dictionary;
|
||||
CREATE DICTIONARY 02183_direct_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_direct_dictionary;
|
||||
CREATE DICTIONARY _02183_direct_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
value_date Date,
|
||||
@ -99,17 +99,17 @@ CREATE DICTIONARY 02183_direct_dictionary
|
||||
value_date_time_64 DateTime64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_dictionary_source_table'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT 'Direct dictionary';
|
||||
SELECT * FROM 02183_direct_dictionary;
|
||||
SELECT * FROM _02183_direct_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_direct_dictionary;
|
||||
DROP TABLE 02183_dictionary_source_table;
|
||||
DROP DICTIONARY _02183_direct_dictionary;
|
||||
DROP TABLE _02183_dictionary_source_table;
|
||||
|
||||
DROP TABLE IF EXISTS 02183_ip_trie_dictionary_source_table;
|
||||
CREATE TABLE 02183_ip_trie_dictionary_source_table
|
||||
DROP TABLE IF EXISTS _02183_ip_trie_dictionary_source_table;
|
||||
CREATE TABLE _02183_ip_trie_dictionary_source_table
|
||||
(
|
||||
prefix String,
|
||||
value_date Date,
|
||||
@ -118,11 +118,11 @@ CREATE TABLE 02183_ip_trie_dictionary_source_table
|
||||
value_date_time_64 DateTime64
|
||||
) ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 02183_ip_trie_dictionary_source_table VALUES ('127.0.0.1', '2019-05-05', '2019-05-05', '2019-05-05', '2019-05-05');
|
||||
SELECT * FROM 02183_ip_trie_dictionary_source_table;
|
||||
INSERT INTO _02183_ip_trie_dictionary_source_table VALUES ('127.0.0.1', '2019-05-05', '2019-05-05', '2019-05-05', '2019-05-05');
|
||||
SELECT * FROM _02183_ip_trie_dictionary_source_table;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_ip_trie_dictionary;
|
||||
CREATE DICTIONARY 02183_ip_trie_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_ip_trie_dictionary;
|
||||
CREATE DICTIONARY _02183_ip_trie_dictionary
|
||||
(
|
||||
prefix String,
|
||||
value_date Date,
|
||||
@ -131,18 +131,18 @@ CREATE DICTIONARY 02183_ip_trie_dictionary
|
||||
value_date_time_64 DateTime64
|
||||
)
|
||||
PRIMARY KEY prefix
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_ip_trie_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_ip_trie_dictionary_source_table'))
|
||||
LAYOUT(IP_TRIE(access_to_key_from_attributes 1))
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'IPTrie dictionary';
|
||||
SELECT * FROM 02183_ip_trie_dictionary;
|
||||
SELECT * FROM _02183_ip_trie_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_ip_trie_dictionary;
|
||||
DROP TABLE 02183_ip_trie_dictionary_source_table;
|
||||
DROP DICTIONARY _02183_ip_trie_dictionary;
|
||||
DROP TABLE _02183_ip_trie_dictionary_source_table;
|
||||
|
||||
DROP TABLE IF EXISTS 02183_polygon_dictionary_source_table;
|
||||
CREATE TABLE 02183_polygon_dictionary_source_table
|
||||
DROP TABLE IF EXISTS _02183_polygon_dictionary_source_table;
|
||||
CREATE TABLE _02183_polygon_dictionary_source_table
|
||||
(
|
||||
key Array(Array(Array(Tuple(Float64, Float64)))),
|
||||
value_date Date,
|
||||
@ -151,10 +151,10 @@ CREATE TABLE 02183_polygon_dictionary_source_table
|
||||
value_date_time_64 DateTime64
|
||||
) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 02183_polygon_dictionary_source_table VALUES ([[[(0, 0), (0, 1), (1, 1), (1, 0)]]], '2019-05-05', '2019-05-05', '2019-05-05', '2019-05-05');
|
||||
INSERT INTO _02183_polygon_dictionary_source_table VALUES ([[[(0, 0), (0, 1), (1, 1), (1, 0)]]], '2019-05-05', '2019-05-05', '2019-05-05', '2019-05-05');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_polygon_dictionary;
|
||||
CREATE DICTIONARY 02183_polygon_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_polygon_dictionary;
|
||||
CREATE DICTIONARY _02183_polygon_dictionary
|
||||
(
|
||||
key Array(Array(Array(Tuple(Float64, Float64)))),
|
||||
value_date Date,
|
||||
@ -163,18 +163,18 @@ CREATE DICTIONARY 02183_polygon_dictionary
|
||||
value_date_time_64 DateTime64
|
||||
)
|
||||
PRIMARY KEY key
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_polygon_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_polygon_dictionary_source_table'))
|
||||
LAYOUT(POLYGON(store_polygon_key_column 1))
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'Polygon dictionary';
|
||||
SELECT * FROM 02183_polygon_dictionary;
|
||||
SELECT * FROM _02183_polygon_dictionary;
|
||||
|
||||
DROP TABLE 02183_polygon_dictionary_source_table;
|
||||
DROP DICTIONARY 02183_polygon_dictionary;
|
||||
DROP TABLE _02183_polygon_dictionary_source_table;
|
||||
DROP DICTIONARY _02183_polygon_dictionary;
|
||||
|
||||
DROP TABLE IF EXISTS 02183_range_dictionary_source_table;
|
||||
CREATE TABLE 02183_range_dictionary_source_table
|
||||
DROP TABLE IF EXISTS _02183_range_dictionary_source_table;
|
||||
CREATE TABLE _02183_range_dictionary_source_table
|
||||
(
|
||||
key UInt64,
|
||||
start UInt64,
|
||||
@ -185,10 +185,10 @@ CREATE TABLE 02183_range_dictionary_source_table
|
||||
value_date_time_64 DateTime64
|
||||
) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 02183_range_dictionary_source_table VALUES(0, 0, 1, '2019-05-05', '2019-05-05', '2019-05-05', '2019-05-05');
|
||||
SELECT * FROM 02183_range_dictionary_source_table;
|
||||
INSERT INTO _02183_range_dictionary_source_table VALUES(0, 0, 1, '2019-05-05', '2019-05-05', '2019-05-05', '2019-05-05');
|
||||
SELECT * FROM _02183_range_dictionary_source_table;
|
||||
|
||||
CREATE DICTIONARY 02183_range_dictionary
|
||||
CREATE DICTIONARY _02183_range_dictionary
|
||||
(
|
||||
key UInt64,
|
||||
start UInt64,
|
||||
@ -199,13 +199,13 @@ CREATE DICTIONARY 02183_range_dictionary
|
||||
value_date_time_64 DateTime64
|
||||
)
|
||||
PRIMARY KEY key
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_range_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_range_dictionary_source_table'))
|
||||
LAYOUT(RANGE_HASHED())
|
||||
RANGE(MIN start MAX end)
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'Range dictionary';
|
||||
SELECT * FROM 02183_range_dictionary;
|
||||
SELECT * FROM _02183_range_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_range_dictionary;
|
||||
DROP TABLE 02183_range_dictionary_source_table;
|
||||
DROP DICTIONARY _02183_range_dictionary;
|
||||
DROP TABLE _02183_range_dictionary_source_table;
|
||||
|
@ -1,111 +1,111 @@
|
||||
DROP TABLE IF EXISTS 02183_dictionary_test_table;
|
||||
CREATE TABLE 02183_dictionary_test_table (id UInt64) ENGINE=TinyLog;
|
||||
INSERT INTO 02183_dictionary_test_table VALUES (0), (1);
|
||||
DROP TABLE IF EXISTS _02183_dictionary_test_table;
|
||||
CREATE TABLE _02183_dictionary_test_table (id UInt64) ENGINE=TinyLog;
|
||||
INSERT INTO _02183_dictionary_test_table VALUES (0), (1);
|
||||
|
||||
SELECT * FROM 02183_dictionary_test_table;
|
||||
SELECT * FROM _02183_dictionary_test_table;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_flat_dictionary;
|
||||
CREATE DICTIONARY 02183_flat_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_flat_dictionary;
|
||||
CREATE DICTIONARY _02183_flat_dictionary
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(FLAT())
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_dictionary_test_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_dictionary_test_table'))
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'FlatDictionary';
|
||||
|
||||
SELECT dictGet('02183_flat_dictionary', 'value', 0); -- {serverError 36}
|
||||
SELECT dictHas('02183_flat_dictionary', 0);
|
||||
SELECT dictHas('02183_flat_dictionary', 1);
|
||||
SELECT dictHas('02183_flat_dictionary', 2);
|
||||
SELECT dictGet('_02183_flat_dictionary', 'value', 0); -- {serverError 36}
|
||||
SELECT dictHas('_02183_flat_dictionary', 0);
|
||||
SELECT dictHas('_02183_flat_dictionary', 1);
|
||||
SELECT dictHas('_02183_flat_dictionary', 2);
|
||||
|
||||
SELECT * FROM 02183_flat_dictionary;
|
||||
SELECT * FROM _02183_flat_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_flat_dictionary;
|
||||
DROP DICTIONARY _02183_flat_dictionary;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_hashed_dictionary;
|
||||
CREATE DICTIONARY 02183_hashed_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_hashed_dictionary;
|
||||
CREATE DICTIONARY _02183_hashed_dictionary
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(HASHED())
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_dictionary_test_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_dictionary_test_table'))
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'HashedDictionary';
|
||||
|
||||
SELECT dictHas('02183_hashed_dictionary', 0);
|
||||
SELECT dictHas('02183_hashed_dictionary', 1);
|
||||
SELECT dictHas('02183_hashed_dictionary', 2);
|
||||
SELECT dictHas('_02183_hashed_dictionary', 0);
|
||||
SELECT dictHas('_02183_hashed_dictionary', 1);
|
||||
SELECT dictHas('_02183_hashed_dictionary', 2);
|
||||
|
||||
SELECT * FROM 02183_hashed_dictionary;
|
||||
SELECT * FROM _02183_hashed_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_hashed_dictionary;
|
||||
DROP DICTIONARY _02183_hashed_dictionary;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_hashed_array_dictionary;
|
||||
CREATE DICTIONARY 02183_hashed_array_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_hashed_array_dictionary;
|
||||
CREATE DICTIONARY _02183_hashed_array_dictionary
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(HASHED_ARRAY())
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_dictionary_test_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_dictionary_test_table'))
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'HashedArrayDictionary';
|
||||
|
||||
SELECT dictHas('02183_hashed_array_dictionary', 0);
|
||||
SELECT dictHas('02183_hashed_array_dictionary', 1);
|
||||
SELECT dictHas('02183_hashed_array_dictionary', 2);
|
||||
SELECT dictHas('_02183_hashed_array_dictionary', 0);
|
||||
SELECT dictHas('_02183_hashed_array_dictionary', 1);
|
||||
SELECT dictHas('_02183_hashed_array_dictionary', 2);
|
||||
|
||||
SELECT * FROM 02183_hashed_array_dictionary;
|
||||
SELECT * FROM _02183_hashed_array_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_hashed_array_dictionary;
|
||||
DROP DICTIONARY _02183_hashed_array_dictionary;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_cache_dictionary;
|
||||
CREATE DICTIONARY 02183_cache_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_cache_dictionary;
|
||||
CREATE DICTIONARY _02183_cache_dictionary
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(CACHE(SIZE_IN_CELLS 10))
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_dictionary_test_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_dictionary_test_table'))
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'CacheDictionary';
|
||||
|
||||
SELECT dictHas('02183_cache_dictionary', 0);
|
||||
SELECT dictHas('02183_cache_dictionary', 1);
|
||||
SELECT dictHas('02183_cache_dictionary', 2);
|
||||
SELECT dictHas('_02183_cache_dictionary', 0);
|
||||
SELECT dictHas('_02183_cache_dictionary', 1);
|
||||
SELECT dictHas('_02183_cache_dictionary', 2);
|
||||
|
||||
SELECT * FROM 02183_cache_dictionary;
|
||||
SELECT * FROM _02183_cache_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_cache_dictionary;
|
||||
DROP DICTIONARY _02183_cache_dictionary;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_direct_dictionary;
|
||||
CREATE DICTIONARY 02183_direct_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_direct_dictionary;
|
||||
CREATE DICTIONARY _02183_direct_dictionary
|
||||
(
|
||||
id UInt64
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(HASHED())
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_dictionary_test_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_dictionary_test_table'))
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'DirectDictionary';
|
||||
|
||||
SELECT dictHas('02183_direct_dictionary', 0);
|
||||
SELECT dictHas('02183_direct_dictionary', 1);
|
||||
SELECT dictHas('02183_direct_dictionary', 2);
|
||||
SELECT dictHas('_02183_direct_dictionary', 0);
|
||||
SELECT dictHas('_02183_direct_dictionary', 1);
|
||||
SELECT dictHas('_02183_direct_dictionary', 2);
|
||||
|
||||
SELECT * FROM 02183_direct_dictionary;
|
||||
SELECT * FROM _02183_direct_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_direct_dictionary;
|
||||
DROP DICTIONARY _02183_direct_dictionary;
|
||||
|
||||
DROP TABLE 02183_dictionary_test_table;
|
||||
DROP TABLE _02183_dictionary_test_table;
|
||||
|
||||
DROP TABLE IF EXISTS ip_trie_dictionary_source_table;
|
||||
CREATE TABLE ip_trie_dictionary_source_table
|
||||
@ -115,8 +115,8 @@ CREATE TABLE ip_trie_dictionary_source_table
|
||||
|
||||
INSERT INTO ip_trie_dictionary_source_table VALUES ('127.0.0.0');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_ip_trie_dictionary;
|
||||
CREATE DICTIONARY 02183_ip_trie_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_ip_trie_dictionary;
|
||||
CREATE DICTIONARY _02183_ip_trie_dictionary
|
||||
(
|
||||
prefix String
|
||||
)
|
||||
@ -127,42 +127,42 @@ LIFETIME(0);
|
||||
|
||||
SELECT 'IPTrieDictionary';
|
||||
|
||||
SELECT dictHas('02183_ip_trie_dictionary', tuple(IPv4StringToNum('127.0.0.0')));
|
||||
SELECT dictHas('02183_ip_trie_dictionary', tuple(IPv4StringToNum('127.0.0.1')));
|
||||
SELECT * FROM 02183_ip_trie_dictionary;
|
||||
SELECT dictHas('_02183_ip_trie_dictionary', tuple(IPv4StringToNum('127.0.0.0')));
|
||||
SELECT dictHas('_02183_ip_trie_dictionary', tuple(IPv4StringToNum('127.0.0.1')));
|
||||
SELECT * FROM _02183_ip_trie_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_ip_trie_dictionary;
|
||||
DROP DICTIONARY _02183_ip_trie_dictionary;
|
||||
DROP TABLE ip_trie_dictionary_source_table;
|
||||
|
||||
DROP TABLE IF EXISTS 02183_polygon_dictionary_source_table;
|
||||
CREATE TABLE 02183_polygon_dictionary_source_table
|
||||
DROP TABLE IF EXISTS _02183_polygon_dictionary_source_table;
|
||||
CREATE TABLE _02183_polygon_dictionary_source_table
|
||||
(
|
||||
key Array(Array(Array(Tuple(Float64, Float64))))
|
||||
) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 02183_polygon_dictionary_source_table VALUES ([[[(0, 0), (0, 1), (1, 1), (1, 0)]]]);
|
||||
INSERT INTO _02183_polygon_dictionary_source_table VALUES ([[[(0, 0), (0, 1), (1, 1), (1, 0)]]]);
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_polygon_dictionary;
|
||||
CREATE DICTIONARY 02183_polygon_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_polygon_dictionary;
|
||||
CREATE DICTIONARY _02183_polygon_dictionary
|
||||
(
|
||||
key Array(Array(Array(Tuple(Float64, Float64))))
|
||||
)
|
||||
PRIMARY KEY key
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_polygon_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_polygon_dictionary_source_table'))
|
||||
LAYOUT(POLYGON(store_polygon_key_column 1))
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'PolygonDictionary';
|
||||
|
||||
SELECT dictHas('02183_polygon_dictionary', tuple(0.5, 0.5));
|
||||
SELECT dictHas('02183_polygon_dictionary', tuple(1.5, 1.5));
|
||||
SELECT * FROM 02183_polygon_dictionary;
|
||||
SELECT dictHas('_02183_polygon_dictionary', tuple(0.5, 0.5));
|
||||
SELECT dictHas('_02183_polygon_dictionary', tuple(1.5, 1.5));
|
||||
SELECT * FROM _02183_polygon_dictionary;
|
||||
|
||||
DROP DICTIONARY 02183_polygon_dictionary;
|
||||
DROP TABLE 02183_polygon_dictionary_source_table;
|
||||
DROP DICTIONARY _02183_polygon_dictionary;
|
||||
DROP TABLE _02183_polygon_dictionary_source_table;
|
||||
|
||||
DROP TABLE IF EXISTS 02183_range_dictionary_source_table;
|
||||
CREATE TABLE 02183_range_dictionary_source_table
|
||||
DROP TABLE IF EXISTS _02183_range_dictionary_source_table;
|
||||
CREATE TABLE _02183_range_dictionary_source_table
|
||||
(
|
||||
key UInt64,
|
||||
start UInt64,
|
||||
@ -170,25 +170,25 @@ CREATE TABLE 02183_range_dictionary_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 02183_range_dictionary_source_table VALUES(0, 0, 1);
|
||||
INSERT INTO _02183_range_dictionary_source_table VALUES(0, 0, 1);
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02183_range_dictionary;
|
||||
CREATE DICTIONARY 02183_range_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02183_range_dictionary;
|
||||
CREATE DICTIONARY _02183_range_dictionary
|
||||
(
|
||||
key UInt64,
|
||||
start UInt64,
|
||||
end UInt64
|
||||
)
|
||||
PRIMARY KEY key
|
||||
SOURCE(CLICKHOUSE(TABLE '02183_range_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02183_range_dictionary_source_table'))
|
||||
LAYOUT(RANGE_HASHED())
|
||||
RANGE(MIN start MAX end)
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'RangeHashedDictionary';
|
||||
SELECT * FROM 02183_range_dictionary;
|
||||
SELECT dictHas('02183_range_dictionary', 0, 0);
|
||||
SELECT dictHas('02183_range_dictionary', 0, 2);
|
||||
SELECT * FROM _02183_range_dictionary;
|
||||
SELECT dictHas('_02183_range_dictionary', 0, 0);
|
||||
SELECT dictHas('_02183_range_dictionary', 0, 2);
|
||||
|
||||
DROP DICTIONARY 02183_range_dictionary;
|
||||
DROP TABLE 02183_range_dictionary_source_table;
|
||||
DROP DICTIONARY _02183_range_dictionary;
|
||||
DROP TABLE _02183_range_dictionary_source_table;
|
||||
|
@ -1,5 +1,5 @@
|
||||
DROP TABLE IF EXISTS 02184_range_dictionary_source_table;
|
||||
CREATE TABLE 02184_range_dictionary_source_table
|
||||
DROP TABLE IF EXISTS _02184_range_dictionary_source_table;
|
||||
CREATE TABLE _02184_range_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
start UInt64,
|
||||
@ -10,10 +10,10 @@ CREATE TABLE 02184_range_dictionary_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 02184_range_dictionary_source_table VALUES (1, 0, 18446744073709551615, 'value0', 'value1', 'value2');
|
||||
INSERT INTO _02184_range_dictionary_source_table VALUES (1, 0, 18446744073709551615, 'value0', 'value1', 'value2');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02184_range_dictionary;
|
||||
CREATE DICTIONARY 02184_range_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02184_range_dictionary;
|
||||
CREATE DICTIONARY _02184_range_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
start UInt64,
|
||||
@ -23,14 +23,14 @@ CREATE DICTIONARY 02184_range_dictionary
|
||||
value_2 String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02184_range_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02184_range_dictionary_source_table'))
|
||||
LAYOUT(RANGE_HASHED())
|
||||
RANGE(MIN start MAX end)
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT * FROM 02184_range_dictionary;
|
||||
SELECT dictGet('02184_range_dictionary', ('value_0', 'value_1', 'value_2'), 1, 18446744073709551615);
|
||||
SELECT dictHas('02184_range_dictionary', 1, 18446744073709551615);
|
||||
SELECT * FROM _02184_range_dictionary;
|
||||
SELECT dictGet('_02184_range_dictionary', ('value_0', 'value_1', 'value_2'), 1, 18446744073709551615);
|
||||
SELECT dictHas('_02184_range_dictionary', 1, 18446744073709551615);
|
||||
|
||||
DROP DICTIONARY 02184_range_dictionary;
|
||||
DROP TABLE 02184_range_dictionary_source_table;
|
||||
DROP DICTIONARY _02184_range_dictionary;
|
||||
DROP TABLE _02184_range_dictionary_source_table;
|
||||
|
@ -1,5 +1,5 @@
|
||||
DROP TABLE IF EXISTS 02185_range_dictionary_source_table;
|
||||
CREATE TABLE 02185_range_dictionary_source_table
|
||||
DROP TABLE IF EXISTS _02185_range_dictionary_source_table;
|
||||
CREATE TABLE _02185_range_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
start Nullable(UInt64),
|
||||
@ -8,13 +8,13 @@ CREATE TABLE 02185_range_dictionary_source_table
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 02185_range_dictionary_source_table VALUES (0, NULL, 5000, 'Value0'), (0, 5001, 10000, 'Value1'), (0, 10001, NULL, 'Value2');
|
||||
INSERT INTO _02185_range_dictionary_source_table VALUES (0, NULL, 5000, 'Value0'), (0, 5001, 10000, 'Value1'), (0, 10001, NULL, 'Value2');
|
||||
|
||||
SELECT 'Source table';
|
||||
SELECT * FROM 02185_range_dictionary_source_table;
|
||||
SELECT * FROM _02185_range_dictionary_source_table;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02185_range_dictionary;
|
||||
CREATE DICTIONARY 02185_range_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02185_range_dictionary;
|
||||
CREATE DICTIONARY _02185_range_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
start UInt64,
|
||||
@ -22,23 +22,23 @@ CREATE DICTIONARY 02185_range_dictionary
|
||||
value String DEFAULT 'DefaultValue'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02185_range_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02185_range_dictionary_source_table'))
|
||||
LAYOUT(RANGE_HASHED(convert_null_range_bound_to_open 1))
|
||||
RANGE(MIN start MAX end)
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'Dictionary convert_null_range_bound_to_open = 1';
|
||||
SELECT * FROM 02185_range_dictionary;
|
||||
SELECT dictGet('02185_range_dictionary', 'value', 0, 0);
|
||||
SELECT dictGet('02185_range_dictionary', 'value', 0, 5001);
|
||||
SELECT dictGet('02185_range_dictionary', 'value', 0, 10001);
|
||||
SELECT dictHas('02185_range_dictionary', 0, 0);
|
||||
SELECT dictHas('02185_range_dictionary', 0, 5001);
|
||||
SELECT dictHas('02185_range_dictionary', 0, 10001);
|
||||
SELECT * FROM _02185_range_dictionary;
|
||||
SELECT dictGet('_02185_range_dictionary', 'value', 0, 0);
|
||||
SELECT dictGet('_02185_range_dictionary', 'value', 0, 5001);
|
||||
SELECT dictGet('_02185_range_dictionary', 'value', 0, 10001);
|
||||
SELECT dictHas('_02185_range_dictionary', 0, 0);
|
||||
SELECT dictHas('_02185_range_dictionary', 0, 5001);
|
||||
SELECT dictHas('_02185_range_dictionary', 0, 10001);
|
||||
|
||||
DROP DICTIONARY 02185_range_dictionary;
|
||||
DROP DICTIONARY _02185_range_dictionary;
|
||||
|
||||
CREATE DICTIONARY 02185_range_dictionary
|
||||
CREATE DICTIONARY _02185_range_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
start UInt64,
|
||||
@ -46,18 +46,18 @@ CREATE DICTIONARY 02185_range_dictionary
|
||||
value String DEFAULT 'DefaultValue'
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02185_range_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02185_range_dictionary_source_table'))
|
||||
LAYOUT(RANGE_HASHED(convert_null_range_bound_to_open 0))
|
||||
RANGE(MIN start MAX end)
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'Dictionary convert_null_range_bound_to_open = 0';
|
||||
SELECT * FROM 02185_range_dictionary;
|
||||
SELECT dictGet('02185_range_dictionary', 'value', 0, 0);
|
||||
SELECT dictGet('02185_range_dictionary', 'value', 0, 5001);
|
||||
SELECT dictGet('02185_range_dictionary', 'value', 0, 10001);
|
||||
SELECT dictHas('02185_range_dictionary', 0, 0);
|
||||
SELECT dictHas('02185_range_dictionary', 0, 5001);
|
||||
SELECT dictHas('02185_range_dictionary', 0, 10001);
|
||||
SELECT * FROM _02185_range_dictionary;
|
||||
SELECT dictGet('_02185_range_dictionary', 'value', 0, 0);
|
||||
SELECT dictGet('_02185_range_dictionary', 'value', 0, 5001);
|
||||
SELECT dictGet('_02185_range_dictionary', 'value', 0, 10001);
|
||||
SELECT dictHas('_02185_range_dictionary', 0, 0);
|
||||
SELECT dictHas('_02185_range_dictionary', 0, 5001);
|
||||
SELECT dictHas('_02185_range_dictionary', 0, 10001);
|
||||
|
||||
DROP TABLE 02185_range_dictionary_source_table;
|
||||
DROP TABLE _02185_range_dictionary_source_table;
|
||||
|
@ -1,5 +1,5 @@
|
||||
DROP TABLE IF EXISTS 02186_range_dictionary_source_table;
|
||||
CREATE TABLE 02186_range_dictionary_source_table
|
||||
DROP TABLE IF EXISTS _02186_range_dictionary_source_table;
|
||||
CREATE TABLE _02186_range_dictionary_source_table
|
||||
(
|
||||
id UInt64,
|
||||
start Date,
|
||||
@ -8,15 +8,15 @@ CREATE TABLE 02186_range_dictionary_source_table
|
||||
)
|
||||
Engine = TinyLog;
|
||||
|
||||
INSERT INTO 02186_range_dictionary_source_table VALUES (1, '2020-01-01', '2100-01-01', 'Value0');
|
||||
INSERT INTO 02186_range_dictionary_source_table VALUES (1, '2020-01-02', '2100-01-01', 'Value1');
|
||||
INSERT INTO 02186_range_dictionary_source_table VALUES (1, '2020-01-03', '2100-01-01', 'Value2');
|
||||
INSERT INTO _02186_range_dictionary_source_table VALUES (1, '2020-01-01', '2100-01-01', 'Value0');
|
||||
INSERT INTO _02186_range_dictionary_source_table VALUES (1, '2020-01-02', '2100-01-01', 'Value1');
|
||||
INSERT INTO _02186_range_dictionary_source_table VALUES (1, '2020-01-03', '2100-01-01', 'Value2');
|
||||
|
||||
SELECT 'Source table';
|
||||
SELECT * FROM 02186_range_dictionary_source_table;
|
||||
SELECT * FROM _02186_range_dictionary_source_table;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02186_range_dictionary;
|
||||
CREATE DICTIONARY 02186_range_dictionary
|
||||
DROP DICTIONARY IF EXISTS _02186_range_dictionary;
|
||||
CREATE DICTIONARY _02186_range_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
start Date,
|
||||
@ -24,22 +24,22 @@ CREATE DICTIONARY 02186_range_dictionary
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02186_range_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02186_range_dictionary_source_table'))
|
||||
LAYOUT(RANGE_HASHED(range_lookup_strategy 'min'))
|
||||
RANGE(MIN start MAX end)
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'Dictionary .range_lookup_strategy = min';
|
||||
|
||||
SELECT * FROM 02186_range_dictionary;
|
||||
SELECT * FROM _02186_range_dictionary;
|
||||
|
||||
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-01'));
|
||||
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-02'));
|
||||
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-03'));
|
||||
select dictGet('_02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-01'));
|
||||
select dictGet('_02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-02'));
|
||||
select dictGet('_02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-03'));
|
||||
|
||||
DROP DICTIONARY 02186_range_dictionary;
|
||||
DROP DICTIONARY _02186_range_dictionary;
|
||||
|
||||
CREATE DICTIONARY 02186_range_dictionary
|
||||
CREATE DICTIONARY _02186_range_dictionary
|
||||
(
|
||||
id UInt64,
|
||||
start Date,
|
||||
@ -47,18 +47,18 @@ CREATE DICTIONARY 02186_range_dictionary
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02186_range_dictionary_source_table'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02186_range_dictionary_source_table'))
|
||||
LAYOUT(RANGE_HASHED(range_lookup_strategy 'max'))
|
||||
RANGE(MIN start MAX end)
|
||||
LIFETIME(0);
|
||||
|
||||
SELECT 'Dictionary .range_lookup_strategy = max';
|
||||
|
||||
SELECT * FROM 02186_range_dictionary;
|
||||
SELECT * FROM _02186_range_dictionary;
|
||||
|
||||
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-01'));
|
||||
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-02'));
|
||||
select dictGet('02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-03'));
|
||||
select dictGet('_02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-01'));
|
||||
select dictGet('_02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-02'));
|
||||
select dictGet('_02186_range_dictionary', 'value', toUInt64(1), toDate('2020-01-03'));
|
||||
|
||||
DROP DICTIONARY 02186_range_dictionary;
|
||||
DROP TABLE 02186_range_dictionary_source_table;
|
||||
DROP DICTIONARY _02186_range_dictionary;
|
||||
DROP TABLE _02186_range_dictionary_source_table;
|
||||
|
@ -1,65 +1,65 @@
|
||||
DROP TABLE IF EXISTS 02188_test_dictionary_source;
|
||||
CREATE TABLE 02188_test_dictionary_source
|
||||
DROP TABLE IF EXISTS _02188_test_dictionary_source;
|
||||
CREATE TABLE _02188_test_dictionary_source
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO 02188_test_dictionary_source VALUES (0, 'Value');
|
||||
INSERT INTO _02188_test_dictionary_source VALUES (0, 'Value');
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02188_test_dictionary_simple_primary_key;
|
||||
CREATE DICTIONARY 02188_test_dictionary_simple_primary_key
|
||||
DROP DICTIONARY IF EXISTS _02188_test_dictionary_simple_primary_key;
|
||||
CREATE DICTIONARY _02188_test_dictionary_simple_primary_key
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(TABLE '02188_test_dictionary_source'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02188_test_dictionary_source'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT 'Dictionary output';
|
||||
SELECT * FROM 02188_test_dictionary_simple_primary_key;
|
||||
DROP DICTIONARY 02188_test_dictionary_simple_primary_key;
|
||||
SELECT * FROM _02188_test_dictionary_simple_primary_key;
|
||||
DROP DICTIONARY _02188_test_dictionary_simple_primary_key;
|
||||
|
||||
CREATE DICTIONARY 02188_test_dictionary_simple_primary_key
|
||||
CREATE DICTIONARY _02188_test_dictionary_simple_primary_key
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY (id)
|
||||
SOURCE(CLICKHOUSE(TABLE '02188_test_dictionary_source'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02188_test_dictionary_source'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT 'Dictionary output';
|
||||
SELECT * FROM 02188_test_dictionary_simple_primary_key;
|
||||
DROP DICTIONARY 02188_test_dictionary_simple_primary_key;
|
||||
SELECT * FROM _02188_test_dictionary_simple_primary_key;
|
||||
DROP DICTIONARY _02188_test_dictionary_simple_primary_key;
|
||||
|
||||
DROP DICTIONARY IF EXISTS 02188_test_dictionary_complex_primary_key;
|
||||
CREATE DICTIONARY 02188_test_dictionary_complex_primary_key
|
||||
DROP DICTIONARY IF EXISTS _02188_test_dictionary_complex_primary_key;
|
||||
CREATE DICTIONARY _02188_test_dictionary_complex_primary_key
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id, value
|
||||
SOURCE(CLICKHOUSE(TABLE '02188_test_dictionary_source'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02188_test_dictionary_source'))
|
||||
LAYOUT(COMPLEX_KEY_DIRECT());
|
||||
|
||||
SELECT 'Dictionary output';
|
||||
SELECT * FROM 02188_test_dictionary_complex_primary_key;
|
||||
DROP DICTIONARY 02188_test_dictionary_complex_primary_key;
|
||||
SELECT * FROM _02188_test_dictionary_complex_primary_key;
|
||||
DROP DICTIONARY _02188_test_dictionary_complex_primary_key;
|
||||
|
||||
CREATE DICTIONARY 02188_test_dictionary_complex_primary_key
|
||||
CREATE DICTIONARY _02188_test_dictionary_complex_primary_key
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY (id, value)
|
||||
SOURCE(CLICKHOUSE(TABLE '02188_test_dictionary_source'))
|
||||
SOURCE(CLICKHOUSE(TABLE '_02188_test_dictionary_source'))
|
||||
LAYOUT(COMPLEX_KEY_DIRECT());
|
||||
|
||||
SELECT 'Dictionary output';
|
||||
SELECT * FROM 02188_test_dictionary_complex_primary_key;
|
||||
DROP DICTIONARY 02188_test_dictionary_complex_primary_key;
|
||||
SELECT * FROM _02188_test_dictionary_complex_primary_key;
|
||||
DROP DICTIONARY _02188_test_dictionary_complex_primary_key;
|
||||
|
||||
DROP TABLE 02188_test_dictionary_source;
|
||||
DROP TABLE _02188_test_dictionary_source;
|
||||
|
@ -1,5 +1,5 @@
|
||||
DROP TABLE IF EXISTS 02005_test_table;
|
||||
CREATE TABLE 02005_test_table
|
||||
DROP TABLE IF EXISTS _02005_test_table;
|
||||
CREATE TABLE _02005_test_table
|
||||
(
|
||||
value Map(Int64, Int64)
|
||||
)
|
||||
@ -9,45 +9,45 @@ SELECT 'mapPopulateSeries with map';
|
||||
|
||||
SELECT 'Without max key';
|
||||
|
||||
SELECT mapPopulateSeries(value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
SELECT mapPopulateSeries(value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES (map(0, 5));
|
||||
SELECT mapPopulateSeries(value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES (map(0, 5));
|
||||
SELECT mapPopulateSeries(value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES (map(0, 5, 5, 10));
|
||||
SELECT mapPopulateSeries(value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES (map(0, 5, 5, 10));
|
||||
SELECT mapPopulateSeries(value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10));
|
||||
SELECT mapPopulateSeries(value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10));
|
||||
SELECT mapPopulateSeries(value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10, 10, 15));
|
||||
SELECT mapPopulateSeries(value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10, 10, 15));
|
||||
SELECT mapPopulateSeries(value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
SELECT 'With max key';
|
||||
|
||||
SELECT mapPopulateSeries(value, materialize(20)) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
SELECT mapPopulateSeries(value, materialize(20)) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES (map(0, 5));
|
||||
SELECT mapPopulateSeries(value, materialize(20)) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES (map(0, 5));
|
||||
SELECT mapPopulateSeries(value, materialize(20)) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES (map(0, 5, 5, 10));
|
||||
SELECT mapPopulateSeries(value, materialize(20)) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES (map(0, 5, 5, 10));
|
||||
SELECT mapPopulateSeries(value, materialize(20)) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10));
|
||||
SELECT mapPopulateSeries(value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10));
|
||||
SELECT mapPopulateSeries(value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10, 10, 15));
|
||||
SELECT mapPopulateSeries(value, materialize(20)) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES (map(-5, -5, 0, 5, 5, 10, 10, 15));
|
||||
SELECT mapPopulateSeries(value, materialize(20)) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
SELECT 'Possible verflow';
|
||||
|
||||
@ -59,10 +59,10 @@ SELECT 'Duplicate keys';
|
||||
SELECT mapPopulateSeries(map(1, 4, 1, 5, 5, 6));
|
||||
SELECT mapPopulateSeries(map(1, 4, 1, 5, 5, 6), materialize(10));
|
||||
|
||||
DROP TABLE 02005_test_table;
|
||||
DROP TABLE _02005_test_table;
|
||||
|
||||
DROP TABLE IF EXISTS 02005_test_table;
|
||||
CREATE TABLE 02005_test_table
|
||||
DROP TABLE IF EXISTS _02005_test_table;
|
||||
CREATE TABLE _02005_test_table
|
||||
(
|
||||
key Array(Int64),
|
||||
value Array(Int64)
|
||||
@ -72,45 +72,45 @@ ENGINE = TinyLog;
|
||||
SELECT 'mapPopulateSeries with two arrays';
|
||||
SELECT 'Without max key';
|
||||
|
||||
SELECT mapPopulateSeries(key, value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
SELECT mapPopulateSeries(key, value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES ([0], [5]);
|
||||
SELECT mapPopulateSeries(key, value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES ([0], [5]);
|
||||
SELECT mapPopulateSeries(key, value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES ([0, 5], [5, 10]);
|
||||
SELECT mapPopulateSeries(key, value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES ([0, 5], [5, 10]);
|
||||
SELECT mapPopulateSeries(key, value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES ([-5, 0, 5], [-5, 5, 10]);
|
||||
SELECT mapPopulateSeries(key, value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES ([-5, 0, 5], [-5, 5, 10]);
|
||||
SELECT mapPopulateSeries(key, value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES ([-5, 0, 5, 10], [-5, 5, 10, 15]);
|
||||
SELECT mapPopulateSeries(key, value) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES ([-5, 0, 5, 10], [-5, 5, 10, 15]);
|
||||
SELECT mapPopulateSeries(key, value) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
SELECT 'With max key';
|
||||
|
||||
SELECT mapPopulateSeries(key, value, materialize(20)) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
SELECT mapPopulateSeries(key, value, materialize(20)) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES ([0], [5]);
|
||||
SELECT mapPopulateSeries(key, value, materialize(20)) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES ([0], [5]);
|
||||
SELECT mapPopulateSeries(key, value, materialize(20)) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES ([0, 5], [5, 10]);
|
||||
SELECT mapPopulateSeries(key, value, materialize(20)) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES ([0, 5], [5, 10]);
|
||||
SELECT mapPopulateSeries(key, value, materialize(20)) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES ([-5, 0, 5], [-5, 5, 10]);
|
||||
SELECT mapPopulateSeries(key, value, materialize(20)) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES ([-5, 0, 5], [-5, 5, 10]);
|
||||
SELECT mapPopulateSeries(key, value, materialize(20)) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
INSERT INTO 02005_test_table VALUES ([-5, 0, 5, 10], [-5, 5, 10, 15]);
|
||||
SELECT mapPopulateSeries(key, value, materialize(20)) FROM 02005_test_table;
|
||||
TRUNCATE TABLE 02005_test_table;
|
||||
INSERT INTO _02005_test_table VALUES ([-5, 0, 5, 10], [-5, 5, 10, 15]);
|
||||
SELECT mapPopulateSeries(key, value, materialize(20)) FROM _02005_test_table;
|
||||
TRUNCATE TABLE _02005_test_table;
|
||||
|
||||
SELECT 'Possible verflow';
|
||||
|
||||
@ -122,4 +122,4 @@ SELECT 'Duplicate keys';
|
||||
SELECT mapPopulateSeries([1, 1, 5], [4, 5, 6]);
|
||||
SELECT mapPopulateSeries([1, 1, 5], [4, 5, 6], materialize(10));
|
||||
|
||||
DROP TABLE 02005_test_table;
|
||||
DROP TABLE _02005_test_table;
|
||||
|
@ -1,18 +1,18 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
set allow_deprecated_database_ordinary=1;
|
||||
DROP DATABASE IF EXISTS 02265_atomic_db;
|
||||
DROP DATABASE IF EXISTS 02265_ordinary_db;
|
||||
DROP DATABASE IF EXISTS _02265_atomic_db;
|
||||
DROP DATABASE IF EXISTS _02265_ordinary_db;
|
||||
|
||||
CREATE DATABASE 02265_atomic_db ENGINE = Atomic;
|
||||
CREATE DATABASE 02265_ordinary_db ENGINE = Ordinary;
|
||||
CREATE DATABASE _02265_atomic_db ENGINE = Atomic;
|
||||
CREATE DATABASE _02265_ordinary_db ENGINE = Ordinary;
|
||||
|
||||
CREATE TABLE 02265_ordinary_db.join_table ( `a` Int64 ) ENGINE = Join(`ALL`, LEFT, a);
|
||||
INSERT INTO 02265_ordinary_db.join_table VALUES (111);
|
||||
CREATE TABLE _02265_ordinary_db.join_table ( `a` Int64 ) ENGINE = Join(`ALL`, LEFT, a);
|
||||
INSERT INTO _02265_ordinary_db.join_table VALUES (111);
|
||||
|
||||
RENAME TABLE 02265_ordinary_db.join_table TO 02265_atomic_db.join_table;
|
||||
RENAME TABLE _02265_ordinary_db.join_table TO _02265_atomic_db.join_table;
|
||||
|
||||
SELECT * FROM 02265_atomic_db.join_table;
|
||||
SELECT * FROM _02265_atomic_db.join_table;
|
||||
|
||||
DROP DATABASE IF EXISTS 02265_atomic_db;
|
||||
DROP DATABASE IF EXISTS 02265_ordinary_db;
|
||||
DROP DATABASE IF EXISTS _02265_atomic_db;
|
||||
DROP DATABASE IF EXISTS _02265_ordinary_db;
|
||||
|
@ -1,7 +1,7 @@
|
||||
SET allow_suspicious_low_cardinality_types = 1;
|
||||
DROP TABLE IF EXISTS 02266_auto_add_nullable;
|
||||
DROP TABLE IF EXISTS _02266_auto_add_nullable;
|
||||
|
||||
CREATE TABLE 02266_auto_add_nullable
|
||||
CREATE TABLE _02266_auto_add_nullable
|
||||
(
|
||||
val0 Int8 DEFAULT NULL,
|
||||
val1 Nullable(Int8) DEFAULT NULL,
|
||||
@ -12,6 +12,6 @@ CREATE TABLE 02266_auto_add_nullable
|
||||
)
|
||||
ENGINE = MergeTree order by tuple();
|
||||
|
||||
DESCRIBE TABLE 02266_auto_add_nullable;
|
||||
DESCRIBE TABLE _02266_auto_add_nullable;
|
||||
|
||||
DROP TABLE IF EXISTS 02266_auto_add_nullable;
|
||||
DROP TABLE IF EXISTS _02266_auto_add_nullable;
|
@ -1,14 +1,14 @@
|
||||
SELECT * FROM numbers(10) FORMAT JSONLines;
|
||||
SELECT * FROM numbers(10) FORMAT NDJSON;
|
||||
|
||||
DROP TABLE IF EXISTS 02267_t;
|
||||
DROP TABLE IF EXISTS _02267_t;
|
||||
|
||||
CREATE TABLE 02267_t (n1 UInt32, n2 UInt32) ENGINE = Memory;
|
||||
CREATE TABLE _02267_t (n1 UInt32, n2 UInt32) ENGINE = Memory;
|
||||
|
||||
INSERT INTO 02267_t FORMAT JSONLines {"n1": 1, "n2": 2} {"n1": 3, "n2": 4} {"n1": 5, "n2": 6};
|
||||
INSERT INTO 02267_t FORMAT NDJSON {"n1": 1, "n2": 2} {"n1": 3, "n2": 4} {"n1": 5, "n2": 6};
|
||||
INSERT INTO _02267_t FORMAT JSONLines {"n1": 1, "n2": 2} {"n1": 3, "n2": 4} {"n1": 5, "n2": 6};
|
||||
INSERT INTO _02267_t FORMAT NDJSON {"n1": 1, "n2": 2} {"n1": 3, "n2": 4} {"n1": 5, "n2": 6};
|
||||
|
||||
SELECT * FROM 02267_t ORDER BY n1, n2 FORMAT JSONLines;
|
||||
SELECT * FROM 02267_t ORDER BY n1, n2 FORMAT NDJSON;
|
||||
SELECT * FROM _02267_t ORDER BY n1, n2 FORMAT JSONLines;
|
||||
SELECT * FROM _02267_t ORDER BY n1, n2 FORMAT NDJSON;
|
||||
|
||||
DROP TABLE 02267_t;
|
||||
DROP TABLE _02267_t;
|
||||
|
@ -10,11 +10,11 @@ FORMATS=('Parquet' 'Arrow' 'ORC')
|
||||
for format in "${FORMATS[@]}"
|
||||
do
|
||||
echo $format
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 00163_column_oriented SYNC"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE 00163_column_oriented(ClientEventTime DateTime('Asia/Dubai'), MobilePhoneModel String, ClientIP6 FixedString(16)) ENGINE=File($format)"
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO 00163_column_oriented SELECT ClientEventTime, MobilePhoneModel, ClientIP6 FROM test.hits ORDER BY ClientEventTime, MobilePhoneModel, ClientIP6 LIMIT 100"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT ClientEventTime from 00163_column_oriented" | md5sum
|
||||
$CLICKHOUSE_CLIENT -q "SELECT MobilePhoneModel from 00163_column_oriented" | md5sum
|
||||
$CLICKHOUSE_CLIENT -q "SELECT ClientIP6 from 00163_column_oriented" | md5sum
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 00163_column_oriented SYNC"
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS _00163_column_oriented SYNC"
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE _00163_column_oriented(ClientEventTime DateTime('Asia/Dubai'), MobilePhoneModel String, ClientIP6 FixedString(16)) ENGINE=File($format)"
|
||||
$CLICKHOUSE_CLIENT -q "INSERT INTO _00163_column_oriented SELECT ClientEventTime, MobilePhoneModel, ClientIP6 FROM test.hits ORDER BY ClientEventTime, MobilePhoneModel, ClientIP6 LIMIT 100"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT ClientEventTime from _00163_column_oriented" | md5sum
|
||||
$CLICKHOUSE_CLIENT -q "SELECT MobilePhoneModel from _00163_column_oriented" | md5sum
|
||||
$CLICKHOUSE_CLIENT -q "SELECT ClientIP6 from _00163_column_oriented" | md5sum
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS _00163_column_oriented SYNC"
|
||||
done
|
||||
|
Loading…
Reference in New Issue
Block a user