mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Fix some tests
This commit is contained in:
parent
a7957914cf
commit
a8fee10059
@ -2,19 +2,19 @@ SET max_rows_to_group_by = 100000;
|
||||
SET max_block_size = 100001;
|
||||
SET group_by_overflow_mode = 'any';
|
||||
|
||||
DROP TABLE IF EXISTS test.numbers500k;
|
||||
CREATE VIEW test.numbers500k AS SELECT number FROM system.numbers LIMIT 500000;
|
||||
DROP TABLE IF EXISTS numbers500k;
|
||||
CREATE VIEW numbers500k AS SELECT number FROM system.numbers LIMIT 500000;
|
||||
|
||||
SET totals_mode = 'after_having_auto';
|
||||
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', test, numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
|
||||
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
|
||||
|
||||
SET totals_mode = 'after_having_inclusive';
|
||||
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', test, numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
|
||||
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
|
||||
|
||||
SET totals_mode = 'after_having_exclusive';
|
||||
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', test, numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
|
||||
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
|
||||
|
||||
SET totals_mode = 'before_having';
|
||||
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', test, numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
|
||||
SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10;
|
||||
|
||||
DROP TABLE test.numbers500k;
|
||||
DROP TABLE numbers500k;
|
||||
|
@ -1,9 +1,7 @@
|
||||
DROP TABLE IF EXISTS test.set;
|
||||
DROP TABLE IF EXISTS test.set2;
|
||||
DROP TABLE IF EXISTS set;
|
||||
DROP TABLE IF EXISTS set2;
|
||||
|
||||
CREATE TABLE test.set (x String) ENGINE = Set;
|
||||
|
||||
USE test;
|
||||
CREATE TABLE set (x String) ENGINE = Set;
|
||||
|
||||
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set;
|
||||
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s NOT IN set;
|
||||
@ -14,10 +12,10 @@ SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s
|
||||
RENAME TABLE set TO set2;
|
||||
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
|
||||
|
||||
INSERT INTO test.set2 VALUES ('Hello'), ('World');
|
||||
INSERT INTO set2 VALUES ('Hello'), ('World');
|
||||
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
|
||||
|
||||
INSERT INTO test.set2 VALUES ('abc'), ('World');
|
||||
INSERT INTO set2 VALUES ('abc'), ('World');
|
||||
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set2;
|
||||
|
||||
DETACH TABLE set2;
|
||||
@ -28,6 +26,4 @@ SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s
|
||||
RENAME TABLE set2 TO set;
|
||||
SELECT arrayJoin(['Hello', 'test', 'World', 'world', 'abc', 'xyz']) AS s WHERE s IN set;
|
||||
|
||||
USE default;
|
||||
|
||||
DROP TABLE test.set;
|
||||
DROP TABLE set;
|
||||
|
@ -1,11 +1,10 @@
|
||||
SET max_parallel_replicas = 2;
|
||||
|
||||
DROP TABLE IF EXISTS test.report;
|
||||
DROP TABLE IF EXISTS report;
|
||||
|
||||
CREATE TABLE test.report(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);
|
||||
CREATE TABLE report(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192);
|
||||
|
||||
INSERT INTO test.report(id,event_date,priority,description) VALUES (1, '2015-01-01', 1, 'foo')(2, '2015-02-01', 2, 'bar')(3, '2015-03-01', 3, 'foo')(4, '2015-04-01', 4, 'bar')(5, '2015-05-01', 5, 'foo');
|
||||
SELECT * FROM (SELECT id, event_date, priority, description FROM remote('127.0.0.{2|3}', test, report)) ORDER BY id ASC;
|
||||
|
||||
DROP TABLE test.report;
|
||||
INSERT INTO report(id,event_date,priority,description) VALUES (1, '2015-01-01', 1, 'foo')(2, '2015-02-01', 2, 'bar')(3, '2015-03-01', 3, 'foo')(4, '2015-04-01', 4, 'bar')(5, '2015-05-01', 5, 'foo');
|
||||
SELECT * FROM (SELECT id, event_date, priority, description FROM remote('127.0.0.{2|3}', currentDatabase(), report)) ORDER BY id ASC;
|
||||
|
||||
DROP TABLE report;
|
||||
|
@ -5,4 +5,4 @@ insert into tab_00577 values ('2018-01-01', 0, 0);
|
||||
select * from tab_00577 order by version;
|
||||
OPTIMIZE TABLE tab_00577;
|
||||
select * from tab_00577;
|
||||
|
||||
drop table tab_00577;
|
||||
|
@ -1,38 +1,38 @@
|
||||
DROP TABLE IF EXISTS test.t1;
|
||||
DROP TABLE IF EXISTS test.t2;
|
||||
DROP TABLE IF EXISTS test.t3;
|
||||
DROP TABLE IF EXISTS test.t4;
|
||||
DROP TABLE IF EXISTS t1;
|
||||
DROP TABLE IF EXISTS t2;
|
||||
DROP TABLE IF EXISTS t3;
|
||||
DROP TABLE IF EXISTS t4;
|
||||
|
||||
CREATE TABLE test.t1(x UInt32, y UInt32) ENGINE TinyLog;
|
||||
CREATE TABLE test.t2(x UInt32, y UInt32 DEFAULT x + 1) ENGINE TinyLog;
|
||||
CREATE TABLE test.t3(x UInt32, y UInt32 MATERIALIZED x + 1) ENGINE TinyLog;
|
||||
CREATE TABLE test.t4(x UInt32, y UInt32 ALIAS x + 1) ENGINE TinyLog;
|
||||
CREATE TABLE t1(x UInt32, y UInt32) ENGINE TinyLog;
|
||||
CREATE TABLE t2(x UInt32, y UInt32 DEFAULT x + 1) ENGINE TinyLog;
|
||||
CREATE TABLE t3(x UInt32, y UInt32 MATERIALIZED x + 1) ENGINE TinyLog;
|
||||
CREATE TABLE t4(x UInt32, y UInt32 ALIAS x + 1) ENGINE TinyLog;
|
||||
|
||||
INSERT INTO test.t1 VALUES (1, 1);
|
||||
INSERT INTO test.t2 VALUES (1, 1);
|
||||
INSERT INTO test.t3 VALUES (1);
|
||||
INSERT INTO test.t4 VALUES (1);
|
||||
INSERT INTO t1 VALUES (1, 1);
|
||||
INSERT INTO t2 VALUES (1, 1);
|
||||
INSERT INTO t3 VALUES (1);
|
||||
INSERT INTO t4 VALUES (1);
|
||||
|
||||
INSERT INTO FUNCTION remote('127.0.0.2', test.t1) VALUES (2, 2);
|
||||
INSERT INTO FUNCTION remote('127.0.0.2', test.t2) VALUES (2, 2);
|
||||
INSERT INTO FUNCTION remote('127.0.0.2', currentDatabase(), t1) VALUES (2, 2);
|
||||
INSERT INTO FUNCTION remote('127.0.0.2', currentDatabase(), t2) VALUES (2, 2);
|
||||
--TODO: INSERT into remote tables with MATERIALIZED columns.
|
||||
--INSERT INTO FUNCTION remote('127.0.0.2', test.t3) VALUES (2);
|
||||
INSERT INTO FUNCTION remote('127.0.0.2', test.t4) VALUES (2);
|
||||
--INSERT INTO FUNCTION remote('127.0.0.2', currentDatabase(), t3) VALUES (2);
|
||||
INSERT INTO FUNCTION remote('127.0.0.2', currentDatabase(), t4) VALUES (2);
|
||||
|
||||
SELECT * FROM remote('127.0.0.2', test.t1) ORDER BY x;
|
||||
SELECT * FROM remote('127.0.0.2', currentDatabase(), t1) ORDER BY x;
|
||||
|
||||
SELECT '*** With a DEFAULT column ***';
|
||||
SELECT * FROM remote('127.0.0.2', test.t2) ORDER BY x;
|
||||
SELECT * FROM remote('127.0.0.2', currentDatabase(), t2) ORDER BY x;
|
||||
|
||||
SELECT '*** With a MATERIALIZED column ***';
|
||||
SELECT * FROM remote('127.0.0.2', test.t3) ORDER BY x;
|
||||
SELECT x, y FROM remote('127.0.0.2', test.t3) ORDER BY x;
|
||||
SELECT * FROM remote('127.0.0.2', currentDatabase(), t3) ORDER BY x;
|
||||
SELECT x, y FROM remote('127.0.0.2', currentDatabase(), t3) ORDER BY x;
|
||||
|
||||
SELECT '*** With an ALIAS column ***';
|
||||
SELECT * FROM remote('127.0.0.2', test.t4) ORDER BY x;
|
||||
SELECT x, y FROM remote('127.0.0.2', test.t4) ORDER BY x;
|
||||
SELECT * FROM remote('127.0.0.2', currentDatabase(), t4) ORDER BY x;
|
||||
SELECT x, y FROM remote('127.0.0.2', currentDatabase(), t4) ORDER BY x;
|
||||
|
||||
DROP TABLE test.t1;
|
||||
DROP TABLE test.t2;
|
||||
DROP TABLE test.t3;
|
||||
DROP TABLE test.t4;
|
||||
DROP TABLE t1;
|
||||
DROP TABLE t2;
|
||||
DROP TABLE t3;
|
||||
DROP TABLE t4;
|
||||
|
@ -1,9 +1,9 @@
|
||||
DROP TABLE IF EXISTS test.remote_test;
|
||||
CREATE TABLE test.remote_test(a1 UInt8) ENGINE=Memory;
|
||||
INSERT INTO FUNCTION remote('127.0.0.1', test.remote_test) VALUES(1);
|
||||
INSERT INTO FUNCTION remote('127.0.0.1', test.remote_test) VALUES(2);
|
||||
INSERT INTO FUNCTION remote('127.0.0.1', test.remote_test) VALUES(3);
|
||||
INSERT INTO FUNCTION remote('127.0.0.1', test.remote_test) VALUES(4);
|
||||
SELECT COUNT(*) FROM remote('127.0.0.1', test.remote_test);
|
||||
SELECT count(*) FROM remote('127.0.0.{1,2}', merge(test, '^remote_test'));
|
||||
DROP TABLE test.remote_test;
|
||||
DROP TABLE IF EXISTS remote_test;
|
||||
CREATE TABLE remote_test(a1 UInt8) ENGINE=Memory;
|
||||
INSERT INTO FUNCTION remote('127.0.0.1', currentDatabase(), remote_test) VALUES(1);
|
||||
INSERT INTO FUNCTION remote('127.0.0.1', currentDatabase(), remote_test) VALUES(2);
|
||||
INSERT INTO FUNCTION remote('127.0.0.1', currentDatabase(), remote_test) VALUES(3);
|
||||
INSERT INTO FUNCTION remote('127.0.0.1', currentDatabase(), remote_test) VALUES(4);
|
||||
SELECT COUNT(*) FROM remote('127.0.0.1', currentDatabase(), remote_test);
|
||||
SELECT count(*) FROM remote('127.0.0.{1,2}', merge(currentDatabase(), '^remote_test'));
|
||||
DROP TABLE remote_test;
|
||||
|
@ -2,4 +2,4 @@ drop table if exists tab_00712_1;
|
||||
create table tab_00712_1 (a UInt32, b UInt32 alias a + 1, c UInt32) engine = MergeTree order by tuple();
|
||||
insert into tab_00712_1 values (1, 2);
|
||||
select ignore(_part) from tab_00712_1 prewhere b = 2;
|
||||
|
||||
drop table tab_00712_1;
|
||||
|
@ -17,3 +17,5 @@ SELECT a, b, sum(s), count() from rollup GROUP BY CUBE(a, b) WITH TOTALS ORDER B
|
||||
SELECT a, b, sum(s), count() from rollup GROUP BY a, b WITH CUBE ORDER BY a;
|
||||
|
||||
SELECT a, b, sum(s), count() from rollup GROUP BY a, b WITH CUBE WITH TOTALS ORDER BY a;
|
||||
|
||||
DROP TABLE rollup;
|
||||
|
@ -1,23 +1,23 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP TABLE IF EXISTS test.quorum1;
|
||||
DROP TABLE IF EXISTS test.quorum2;
|
||||
DROP TABLE IF EXISTS quorum1;
|
||||
DROP TABLE IF EXISTS quorum2;
|
||||
|
||||
CREATE TABLE test.quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '1') ORDER BY x PARTITION BY y;
|
||||
CREATE TABLE test.quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '2') ORDER BY x PARTITION BY y;
|
||||
CREATE TABLE quorum1(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '1') ORDER BY x PARTITION BY y;
|
||||
CREATE TABLE quorum2(x UInt32, y Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/quorum', '2') ORDER BY x PARTITION BY y;
|
||||
|
||||
SET insert_quorum=2;
|
||||
SET select_sequential_consistency=1;
|
||||
|
||||
INSERT INTO test.quorum1 VALUES (1, '2018-11-15');
|
||||
INSERT INTO test.quorum1 VALUES (2, '2018-11-15');
|
||||
INSERT INTO quorum1 VALUES (1, '2018-11-15');
|
||||
INSERT INTO quorum1 VALUES (2, '2018-11-15');
|
||||
|
||||
SELECT x FROM test.quorum1 ORDER BY x;
|
||||
SELECT x FROM test.quorum2 ORDER BY x;
|
||||
SELECT x FROM quorum1 ORDER BY x;
|
||||
SELECT x FROM quorum2 ORDER BY x;
|
||||
|
||||
OPTIMIZE TABLE test.quorum1 PARTITION '2018-11-15' FINAL;
|
||||
OPTIMIZE TABLE quorum1 PARTITION '2018-11-15' FINAL;
|
||||
|
||||
SELECT count(*) FROM system.parts WHERE active AND database = 'test' AND table='quorum1';
|
||||
SELECT count(*) FROM system.parts WHERE active AND database = currentDatabase() AND table='quorum1';
|
||||
|
||||
DROP TABLE IF EXISTS test.quorum1;
|
||||
DROP TABLE IF EXISTS test.quorum2;
|
||||
DROP TABLE IF EXISTS quorum1;
|
||||
DROP TABLE IF EXISTS quorum2;
|
||||
|
@ -1,4 +1,3 @@
|
||||
USE test;
|
||||
SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5]));
|
||||
SELECT bitmapToArray(bitmapAnd(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])));
|
||||
SELECT bitmapToArray(bitmapOr(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])));
|
||||
|
@ -1,7 +1,7 @@
|
||||
----- Group of very similar simple tests ------
|
||||
DROP TABLE IF EXISTS test.zero_rows_per_granule;
|
||||
DROP TABLE IF EXISTS zero_rows_per_granule;
|
||||
|
||||
CREATE TABLE test.zero_rows_per_granule (
|
||||
CREATE TABLE zero_rows_per_granule (
|
||||
p Date,
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
@ -13,26 +13,26 @@ CREATE TABLE test.zero_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0;
|
||||
|
||||
INSERT INTO test.zero_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1);
|
||||
INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1);
|
||||
|
||||
SELECT COUNT(*) FROM test.zero_rows_per_granule;
|
||||
SELECT COUNT(*) FROM zero_rows_per_granule;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule' and database=currentDatabase() and active=1;
|
||||
|
||||
INSERT INTO test.zero_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 5, 1000, 2000, 1), ('2018-05-16', 6, 3000, 4000, 1), ('2018-05-17', 7, 5000, 6000, 1), ('2018-05-19', 8, 7000, 8000, 1);
|
||||
INSERT INTO zero_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 5, 1000, 2000, 1), ('2018-05-16', 6, 3000, 4000, 1), ('2018-05-17', 7, 5000, 6000, 1), ('2018-05-19', 8, 7000, 8000, 1);
|
||||
|
||||
OPTIMIZE TABLE test.zero_rows_per_granule FINAL;
|
||||
OPTIMIZE TABLE zero_rows_per_granule FINAL;
|
||||
|
||||
SELECT COUNT(*) FROM test.zero_rows_per_granule FINAL;
|
||||
SELECT COUNT(*) FROM zero_rows_per_granule FINAL;
|
||||
|
||||
SELECT sum(marks) from system.parts WHERE table = 'zero_rows_per_granule' and database='test' and active=1;
|
||||
SELECT sum(marks) from system.parts WHERE table = 'zero_rows_per_granule' and database=currentDatabase() and active=1;
|
||||
|
||||
DROP TABLE IF EXISTS test.zero_rows_per_granule;
|
||||
DROP TABLE IF EXISTS zero_rows_per_granule;
|
||||
|
||||
SELECT '-----';
|
||||
DROP TABLE IF EXISTS test.four_rows_per_granule;
|
||||
DROP TABLE IF EXISTS four_rows_per_granule;
|
||||
|
||||
CREATE TABLE test.four_rows_per_granule (
|
||||
CREATE TABLE four_rows_per_granule (
|
||||
p Date,
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
@ -44,30 +44,30 @@ CREATE TABLE test.four_rows_per_granule (
|
||||
vertical_merge_algorithm_min_rows_to_activate=0,
|
||||
vertical_merge_algorithm_min_columns_to_activate=0;
|
||||
|
||||
INSERT INTO test.four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1);
|
||||
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1);
|
||||
|
||||
SELECT COUNT(*) FROM test.four_rows_per_granule;
|
||||
SELECT COUNT(*) FROM four_rows_per_granule;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database=currentDatabase() and active=1;
|
||||
|
||||
INSERT INTO test.four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, -1), ('2018-05-16', 2, 3000, 4000, -1), ('2018-05-17', 3, 5000, 6000, -1), ('2018-05-18', 4, 7000, 8000, -1);
|
||||
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, -1), ('2018-05-16', 2, 3000, 4000, -1), ('2018-05-17', 3, 5000, 6000, -1), ('2018-05-18', 4, 7000, 8000, -1);
|
||||
|
||||
OPTIMIZE TABLE test.four_rows_per_granule FINAL;
|
||||
OPTIMIZE TABLE four_rows_per_granule FINAL;
|
||||
|
||||
SELECT COUNT(*) FROM test.four_rows_per_granule;
|
||||
SELECT COUNT(*) FROM four_rows_per_granule;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database=currentDatabase() and active=1;
|
||||
|
||||
INSERT INTO test.four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1);
|
||||
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 1, 1000, 2000, 1), ('2018-05-16', 2, 3000, 4000, 1), ('2018-05-17', 3, 5000, 6000, 1), ('2018-05-18', 4, 7000, 8000, 1);
|
||||
|
||||
INSERT INTO test.four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 5, 1000, 2000, 1), ('2018-05-16', 6, 3000, 4000, 1), ('2018-05-17', 7, 5000, 6000, 1), ('2018-05-18', 8, 7000, 8000, 1);
|
||||
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 5, 1000, 2000, 1), ('2018-05-16', 6, 3000, 4000, 1), ('2018-05-17', 7, 5000, 6000, 1), ('2018-05-18', 8, 7000, 8000, 1);
|
||||
|
||||
INSERT INTO test.four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 5, 1000, 2000, -1), ('2018-05-17', 7, 5000, 6000, -1);
|
||||
INSERT INTO four_rows_per_granule (p, k, v1, v2, Sign) VALUES ('2018-05-15', 5, 1000, 2000, -1), ('2018-05-17', 7, 5000, 6000, -1);
|
||||
|
||||
OPTIMIZE TABLE test.four_rows_per_granule FINAL;
|
||||
OPTIMIZE TABLE four_rows_per_granule FINAL;
|
||||
|
||||
SELECT COUNT(*) FROM test.four_rows_per_granule;
|
||||
SELECT COUNT(*) FROM four_rows_per_granule;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule' and database=currentDatabase() and active=1;
|
||||
|
||||
DROP TABLE IF EXISTS test.four_rows_per_granule;
|
||||
DROP TABLE IF EXISTS four_rows_per_granule;
|
||||
|
@ -1,4 +1,4 @@
|
||||
DROP TABLE IF EXISTS test.join_test;
|
||||
CREATE TABLE test.join_test (number UInt8, value Float32) Engine = Join(ANY, LEFT, number);
|
||||
TRUNCATE TABLE test.join_test;
|
||||
DROP TABLE IF EXISTS test.join_test;
|
||||
DROP TABLE IF EXISTS join_test;
|
||||
CREATE TABLE join_test (number UInt8, value Float32) Engine = Join(ANY, LEFT, number);
|
||||
TRUNCATE TABLE join_test;
|
||||
DROP TABLE IF EXISTS join_test;
|
||||
|
@ -10,3 +10,6 @@ INSERT INTO distributed_00952 VALUES ('2018-08-01');
|
||||
SELECT * FROM distributed_00952;
|
||||
SELECT * FROM local_00952;
|
||||
SELECT date, value FROM local_00952;
|
||||
|
||||
DROP TABLE distributed_00952;
|
||||
DROP TABLE local_00952;
|
||||
|
Loading…
Reference in New Issue
Block a user