Tests: Allow use any test database (remove test. hardcode where it possible) (#5008)

This commit is contained in:
proller 2019-04-16 17:13:13 +03:00 committed by GitHub
parent 6ccb720659
commit f1cc83ca9e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
416 changed files with 7660 additions and 7701 deletions

View File

@ -25,7 +25,20 @@ if (ENABLE_TESTS)
# maybe add --no-long ?
# if you want disable some tests: env TEST_OPT0='--skip compile'
add_test(NAME with_server COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server")
if(TEST_CMAKE_PARALLEL)
# NUMBER_OF_LOGICAL_CORES
if (TEST_CMAKE_PARALLEL GREATER 1)
set(TOTAL_TESTS ${TEST_CMAKE_PARALLEL})
else()
set(TOTAL_TESTS ${NUMBER_OF_LOGICAL_CORES})
endif()
foreach(proc RANGE 1 ${TOTAL_TESTS})
add_test(NAME with_server${proc} COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} TEST_OPT0=--parallel=${proc}/${TOTAL_TESTS} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server")
endforeach()
else()
add_test(NAME with_server COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server")
endif()
endif ()
if (ENABLE_TEST_INTEGRATION)

View File

@ -167,6 +167,9 @@ def main(args):
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS " + args.database)
if args.database != "test":
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS test")
def is_test_from_dir(suite_dir, case):
case_file = os.path.join(suite_dir, case)

View File

@ -141,7 +141,26 @@ else
done
$CLICKHOUSE_PERFORMANCE_TEST &
fi
( [ "$TEST_RUN" ] && $CLICKHOUSE_TEST ) || ${TEST_TRUE:=false}
if [ "${TEST_RUN_PARALLEL}" ]; then
# Running test in parallel will fail some results (tests can create/fill/drop same tables)
TEST_NPROC=${TEST_NPROC:=$(( `nproc || sysctl -n hw.ncpu || echo 2` * 2))}
for i in `seq 1 ${TEST_NPROC}`; do
$CLICKHOUSE_TEST --testname --tmp=$DATA_DIR/tmp/tmp${i} --database=test${i} --parallel=${i}/${TEST_NPROC} &
done
for job in `jobs -p`; do
#echo wait $job
wait $job || let "FAIL+=1"
done
#echo $FAIL
if [ "$FAIL" != "0" ]; then
return $FAIL
fi
else
( [ "$TEST_RUN" ] && $CLICKHOUSE_TEST ) || ${TEST_TRUE:=false}
fi
( [ "$TEST_PERF" ] && $CLICKHOUSE_PERFORMANCE_TEST $* ) || true
#( [ "$TEST_DICT" ] && mkdir -p $DATA_DIR/etc/dictionaries/ && cd $CUR_DIR/external_dictionaries && python generate_and_test.py --port=$CLICKHOUSE_PORT_TCP --client=$CLICKHOUSE_CLIENT --source=$CUR_DIR/external_dictionaries/source.tsv --reference=$CUR_DIR/external_dictionaries/reference --generated=$DATA_DIR/etc/dictionaries/ --no_mysql --no_mongo ) || true
$CLICKHOUSE_CLIENT_QUERY "SELECT event, value FROM system.events; SELECT metric, value FROM system.metrics; SELECT metric, value FROM system.asynchronous_metrics;"

View File

@ -1,37 +1,37 @@
DROP TABLE IF EXISTS test.alter_test;
DROP TABLE IF EXISTS alter_test;
CREATE TABLE test.alter_test (CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192);
CREATE TABLE alter_test (CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192);
INSERT INTO test.alter_test VALUES (1, '2014-01-01', 2, 3, [1,2,3], ['a','b','c'], 4);
INSERT INTO alter_test VALUES (1, '2014-01-01', 2, 3, [1,2,3], ['a','b','c'], 4);
ALTER TABLE test.alter_test ADD COLUMN Added0 UInt32;
ALTER TABLE test.alter_test ADD COLUMN Added2 UInt32;
ALTER TABLE test.alter_test ADD COLUMN Added1 UInt32 AFTER Added0;
ALTER TABLE alter_test ADD COLUMN Added0 UInt32;
ALTER TABLE alter_test ADD COLUMN Added2 UInt32;
ALTER TABLE alter_test ADD COLUMN Added1 UInt32 AFTER Added0;
ALTER TABLE test.alter_test ADD COLUMN AddedNested1 Nested(A UInt32, B UInt64) AFTER Added2;
ALTER TABLE test.alter_test ADD COLUMN AddedNested1.C Array(String) AFTER AddedNested1.B;
ALTER TABLE test.alter_test ADD COLUMN AddedNested2 Nested(A UInt32, B UInt64) AFTER AddedNested1;
ALTER TABLE alter_test ADD COLUMN AddedNested1 Nested(A UInt32, B UInt64) AFTER Added2;
ALTER TABLE alter_test ADD COLUMN AddedNested1.C Array(String) AFTER AddedNested1.B;
ALTER TABLE alter_test ADD COLUMN AddedNested2 Nested(A UInt32, B UInt64) AFTER AddedNested1;
DESC TABLE test.alter_test;
DESC TABLE alter_test;
ALTER TABLE test.alter_test DROP COLUMN ToDrop;
ALTER TABLE alter_test DROP COLUMN ToDrop;
ALTER TABLE test.alter_test MODIFY COLUMN Added0 String;
ALTER TABLE alter_test MODIFY COLUMN Added0 String;
ALTER TABLE test.alter_test DROP COLUMN NestedColumn.A;
ALTER TABLE test.alter_test DROP COLUMN NestedColumn.S;
ALTER TABLE alter_test DROP COLUMN NestedColumn.A;
ALTER TABLE alter_test DROP COLUMN NestedColumn.S;
ALTER TABLE test.alter_test DROP COLUMN AddedNested1.B;
ALTER TABLE alter_test DROP COLUMN AddedNested1.B;
ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS Added0 UInt32;
ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS AddedNested1 Nested(A UInt32, B UInt64);
ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS AddedNested1.C Array(String);
ALTER TABLE test.alter_test MODIFY COLUMN IF EXISTS ToDrop UInt64;
ALTER TABLE test.alter_test DROP COLUMN IF EXISTS ToDrop;
ALTER TABLE test.alter_test COMMENT COLUMN IF EXISTS ToDrop 'new comment';
ALTER TABLE alter_test ADD COLUMN IF NOT EXISTS Added0 UInt32;
ALTER TABLE alter_test ADD COLUMN IF NOT EXISTS AddedNested1 Nested(A UInt32, B UInt64);
ALTER TABLE alter_test ADD COLUMN IF NOT EXISTS AddedNested1.C Array(String);
ALTER TABLE alter_test MODIFY COLUMN IF EXISTS ToDrop UInt64;
ALTER TABLE alter_test DROP COLUMN IF EXISTS ToDrop;
ALTER TABLE alter_test COMMENT COLUMN IF EXISTS ToDrop 'new comment';
DESC TABLE test.alter_test;
DESC TABLE alter_test;
SELECT * FROM test.alter_test;
SELECT * FROM alter_test;
DROP TABLE test.alter_test;
DROP TABLE alter_test;

View File

@ -3,10 +3,10 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
echo 'DROP TABLE IF EXISTS test.long_insert' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'CREATE TABLE test.long_insert (a String) ENGINE = Memory' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'DROP TABLE IF EXISTS long_insert' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'CREATE TABLE long_insert (a String) ENGINE = Memory' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
for string_size in 1 10 100 1000 10000 100000 1000000; do
# LC_ALL=C is needed because otherwise Perl will bark on bad tuned environment.
LC_ALL=C perl -we 'for my $letter ("a" .. "z") { print(($letter x '$string_size') . "\n") }' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}?query=INSERT+INTO+test.long_insert+FORMAT+TabSeparated" --data-binary @-
echo 'SELECT substring(a, 1, 1) AS c, length(a) AS l FROM test.long_insert ORDER BY c, l' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
LC_ALL=C perl -we 'for my $letter ("a" .. "z") { print(($letter x '$string_size') . "\n") }' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}?query=INSERT+INTO+long_insert+FORMAT+TabSeparated" --data-binary @-
echo 'SELECT substring(a, 1, 1) AS c, length(a) AS l FROM long_insert ORDER BY c, l' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
done

View File

@ -1,15 +1,15 @@
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.big_array;
CREATE TABLE test.big_array (x Array(UInt8)) ENGINE=TinyLog;
DROP TABLE IF EXISTS big_array;
CREATE TABLE big_array (x Array(UInt8)) ENGINE=TinyLog;
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
INSERT INTO test.big_array SELECT groupArray(number % 255) AS x FROM (SELECT * FROM system.numbers LIMIT 1000000);
INSERT INTO big_array SELECT groupArray(number % 255) AS x FROM (SELECT * FROM system.numbers LIMIT 1000000);
SELECT count() FROM test.big_array ARRAY JOIN x;
SELECT count() FROM test.big_array ARRAY JOIN x AS y;
SELECT countIf(has(x, 10)), sum(y) FROM test.big_array ARRAY JOIN x AS y;
SELECT countIf(has(x, 10)) FROM test.big_array ARRAY JOIN x AS y;
SELECT countIf(has(x, 10)), sum(y) FROM test.big_array ARRAY JOIN x AS y WHERE 1;
SELECT countIf(has(x, 10)) FROM test.big_array ARRAY JOIN x AS y WHERE 1;
SELECT countIf(has(x, 10)), sum(y) FROM test.big_array ARRAY JOIN x AS y WHERE has(x,15);
SELECT count() FROM big_array ARRAY JOIN x;
SELECT count() FROM big_array ARRAY JOIN x AS y;
SELECT countIf(has(x, 10)), sum(y) FROM big_array ARRAY JOIN x AS y;
SELECT countIf(has(x, 10)) FROM big_array ARRAY JOIN x AS y;
SELECT countIf(has(x, 10)), sum(y) FROM big_array ARRAY JOIN x AS y WHERE 1;
SELECT countIf(has(x, 10)) FROM big_array ARRAY JOIN x AS y WHERE 1;
SELECT countIf(has(x, 10)), sum(y) FROM big_array ARRAY JOIN x AS y WHERE has(x,15);
DROP TABLE test.big_array;
DROP TABLE big_array;

View File

@ -1,18 +1,18 @@
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.empty_summing;
CREATE TABLE test.empty_summing (d Date, k UInt64, v Int8) ENGINE=SummingMergeTree(d, k, 8192);
DROP TABLE IF EXISTS empty_summing;
CREATE TABLE empty_summing (d Date, k UInt64, v Int8) ENGINE=SummingMergeTree(d, k, 8192);
INSERT INTO test.empty_summing VALUES ('2015-01-01', 1, 10);
INSERT INTO test.empty_summing VALUES ('2015-01-01', 1, -10);
INSERT INTO empty_summing VALUES ('2015-01-01', 1, 10);
INSERT INTO empty_summing VALUES ('2015-01-01', 1, -10);
OPTIMIZE TABLE test.empty_summing;
SELECT * FROM test.empty_summing;
OPTIMIZE TABLE empty_summing;
SELECT * FROM empty_summing;
INSERT INTO test.empty_summing VALUES ('2015-01-01', 1, 4),('2015-01-01', 2, -9),('2015-01-01', 3, -14);
INSERT INTO test.empty_summing VALUES ('2015-01-01', 1, -2),('2015-01-01', 1, -2),('2015-01-01', 3, 14);
INSERT INTO test.empty_summing VALUES ('2015-01-01', 1, 0),('2015-01-01', 3, 0);
INSERT INTO empty_summing VALUES ('2015-01-01', 1, 4),('2015-01-01', 2, -9),('2015-01-01', 3, -14);
INSERT INTO empty_summing VALUES ('2015-01-01', 1, -2),('2015-01-01', 1, -2),('2015-01-01', 3, 14);
INSERT INTO empty_summing VALUES ('2015-01-01', 1, 0),('2015-01-01', 3, 0);
OPTIMIZE TABLE test.empty_summing;
SELECT * FROM test.empty_summing;
OPTIMIZE TABLE empty_summing;
SELECT * FROM empty_summing;
DROP TABLE test.empty_summing;
DROP TABLE empty_summing;

View File

@ -1,13 +1,13 @@
DROP TABLE IF EXISTS test.stored_aggregates;
DROP TABLE IF EXISTS stored_aggregates;
CREATE TABLE test.stored_aggregates
CREATE TABLE stored_aggregates
(
d Date,
Uniq AggregateFunction(uniq, UInt64)
)
ENGINE = AggregatingMergeTree(d, d, 8192);
INSERT INTO test.stored_aggregates
INSERT INTO stored_aggregates
SELECT
toDate('2014-06-01') AS d,
uniqState(number) AS Uniq
@ -16,6 +16,6 @@ FROM
SELECT * FROM system.numbers LIMIT 1000
);
SELECT uniqMerge(Uniq) FROM test.stored_aggregates;
SELECT uniqMerge(Uniq) FROM stored_aggregates;
DROP TABLE test.stored_aggregates;
DROP TABLE stored_aggregates;

View File

@ -1,6 +1,6 @@
DROP TABLE IF EXISTS test.stored_aggregates;
DROP TABLE IF EXISTS stored_aggregates;
CREATE TABLE test.stored_aggregates
CREATE TABLE stored_aggregates
(
d Date,
k1 UInt64,
@ -15,7 +15,7 @@ CREATE TABLE test.stored_aggregates
)
ENGINE = AggregatingMergeTree(d, (d, k1, k2), 8192);
INSERT INTO test.stored_aggregates
INSERT INTO stored_aggregates
SELECT
toDate('2014-06-01') AS d,
intDiv(number, 100) AS k1,
@ -39,7 +39,7 @@ SELECT d, k1, k2,
anyMerge(Any), anyIfMerge(AnyIf),
quantilesMerge(0.5, 0.9)(Quantiles),
groupArrayMerge(GroupArray)
FROM test.stored_aggregates
FROM stored_aggregates
GROUP BY d, k1, k2
ORDER BY d, k1, k2;
@ -48,7 +48,7 @@ SELECT d, k1,
anyMerge(Any), anyIfMerge(AnyIf),
quantilesMerge(0.5, 0.9)(Quantiles),
groupArrayMerge(GroupArray)
FROM test.stored_aggregates
FROM stored_aggregates
GROUP BY d, k1
ORDER BY d, k1;
@ -57,8 +57,8 @@ SELECT d,
anyMerge(Any), anyIfMerge(AnyIf),
quantilesMerge(0.5, 0.9)(Quantiles),
groupArrayMerge(GroupArray)
FROM test.stored_aggregates
FROM stored_aggregates
GROUP BY d
ORDER BY d;
DROP TABLE test.stored_aggregates;
DROP TABLE stored_aggregates;

View File

@ -1,13 +1,13 @@
DROP TABLE IF EXISTS test.stored_aggregates;
DROP TABLE IF EXISTS stored_aggregates;
CREATE TABLE test.stored_aggregates
CREATE TABLE stored_aggregates
(
d Date,
Uniq AggregateFunction(uniq, UInt64)
)
ENGINE = AggregatingMergeTree(d, d, 8192);
INSERT INTO test.stored_aggregates
INSERT INTO stored_aggregates
SELECT
toDate(toUInt16(toDate('2014-06-01')) + intDiv(number, 100)) AS d,
uniqState(intDiv(number, 10)) AS Uniq
@ -17,11 +17,11 @@ FROM
)
GROUP BY d;
SELECT uniqMerge(Uniq) FROM test.stored_aggregates;
SELECT uniqMerge(Uniq) FROM stored_aggregates;
SELECT d, uniqMerge(Uniq) FROM test.stored_aggregates GROUP BY d ORDER BY d;
SELECT d, uniqMerge(Uniq) FROM stored_aggregates GROUP BY d ORDER BY d;
INSERT INTO test.stored_aggregates
INSERT INTO stored_aggregates
SELECT
toDate(toUInt16(toDate('2014-06-01')) + intDiv(number, 100)) AS d,
uniqState(intDiv(number + 50, 10)) AS Uniq
@ -31,15 +31,15 @@ FROM
)
GROUP BY d;
SELECT uniqMerge(Uniq) FROM test.stored_aggregates;
SELECT uniqMerge(Uniq) FROM stored_aggregates;
SELECT d, uniqMerge(Uniq) FROM test.stored_aggregates GROUP BY d ORDER BY d;
SELECT d, uniqMerge(Uniq) FROM stored_aggregates GROUP BY d ORDER BY d;
OPTIMIZE TABLE test.stored_aggregates;
OPTIMIZE TABLE stored_aggregates;
SELECT uniqMerge(Uniq) FROM test.stored_aggregates;
SELECT uniqMerge(Uniq) FROM stored_aggregates;
SELECT d, uniqMerge(Uniq) FROM test.stored_aggregates GROUP BY d ORDER BY d;
SELECT d, uniqMerge(Uniq) FROM stored_aggregates GROUP BY d ORDER BY d;
DROP TABLE test.stored_aggregates;
DROP TABLE stored_aggregates;

View File

@ -1,13 +1,13 @@
DROP TABLE IF EXISTS test.stored_aggregates;
DROP TABLE IF EXISTS stored_aggregates;
CREATE TABLE test.stored_aggregates
CREATE TABLE stored_aggregates
(
d Date,
Uniq AggregateFunction(uniq, UInt64)
)
ENGINE = AggregatingMergeTree(d, d, 8192);
INSERT INTO test.stored_aggregates
INSERT INTO stored_aggregates
SELECT
toDate(toUInt16(toDate('2014-06-01')) + intDiv(number, 100)) AS d,
uniqState(intDiv(number, 10)) AS Uniq
@ -17,11 +17,11 @@ FROM
)
GROUP BY d;
SELECT uniqMerge(Uniq) FROM test.stored_aggregates;
SELECT uniqMerge(Uniq) FROM stored_aggregates;
SELECT d, uniqMerge(Uniq) FROM test.stored_aggregates GROUP BY d ORDER BY d;
SELECT d, uniqMerge(Uniq) FROM stored_aggregates GROUP BY d ORDER BY d;
INSERT INTO test.stored_aggregates
INSERT INTO stored_aggregates
SELECT
toDate(toUInt16(toDate('2014-06-01')) + intDiv(number, 100)) AS d,
uniqState(intDiv(number + 50, 10)) AS Uniq
@ -31,15 +31,15 @@ FROM
)
GROUP BY d;
SELECT uniqMerge(Uniq) FROM test.stored_aggregates;
SELECT uniqMerge(Uniq) FROM stored_aggregates;
SELECT d, uniqMerge(Uniq) FROM test.stored_aggregates GROUP BY d ORDER BY d;
SELECT d, uniqMerge(Uniq) FROM stored_aggregates GROUP BY d ORDER BY d;
OPTIMIZE TABLE test.stored_aggregates;
OPTIMIZE TABLE stored_aggregates;
SELECT uniqMerge(Uniq) FROM test.stored_aggregates;
SELECT uniqMerge(Uniq) FROM stored_aggregates;
SELECT d, uniqMerge(Uniq) FROM test.stored_aggregates GROUP BY d ORDER BY d;
SELECT d, uniqMerge(Uniq) FROM stored_aggregates GROUP BY d ORDER BY d;
DROP TABLE test.stored_aggregates;
DROP TABLE stored_aggregates;

View File

@ -1,107 +1,107 @@
DROP TABLE IF EXISTS test.replicated_alter1;
DROP TABLE IF EXISTS test.replicated_alter2;
CREATE TABLE test.replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test/alter', 'r1', d, k, 8192);
CREATE TABLE test.replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test/alter', 'r2', d, k, 8192);
DROP TABLE IF EXISTS replicated_alter1;
DROP TABLE IF EXISTS replicated_alter2;
CREATE TABLE replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test/alter', 'r1', d, k, 8192);
CREATE TABLE replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test/alter', 'r2', d, k, 8192);
INSERT INTO test.replicated_alter1 VALUES ('2015-01-01', 10, 42);
INSERT INTO replicated_alter1 VALUES ('2015-01-01', 10, 42);
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 ADD COLUMN dt DateTime;
INSERT INTO test.replicated_alter1 VALUES ('2015-01-01', 9, 41, '1992-01-01 08:00:00');
ALTER TABLE replicated_alter1 ADD COLUMN dt DateTime;
INSERT INTO replicated_alter1 VALUES ('2015-01-01', 9, 41, '1992-01-01 08:00:00');
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 ADD COLUMN n Nested(ui8 UInt8, s String);
INSERT INTO test.replicated_alter1 VALUES ('2015-01-01', 8, 40, '2012-12-12 12:12:12', [1,2,3], ['12','13','14']);
ALTER TABLE replicated_alter1 ADD COLUMN n Nested(ui8 UInt8, s String);
INSERT INTO replicated_alter1 VALUES ('2015-01-01', 8, 40, '2012-12-12 12:12:12', [1,2,3], ['12','13','14']);
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 ADD COLUMN `n.d` Array(Date);
INSERT INTO test.replicated_alter1 VALUES ('2015-01-01', 7, 39, '2014-07-14 13:26:50', [10,20,30], ['120','130','140'],['2000-01-01','2000-01-01','2000-01-03']);
ALTER TABLE replicated_alter1 ADD COLUMN `n.d` Array(Date);
INSERT INTO replicated_alter1 VALUES ('2015-01-01', 7, 39, '2014-07-14 13:26:50', [10,20,30], ['120','130','140'],['2000-01-01','2000-01-01','2000-01-03']);
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 ADD COLUMN s String DEFAULT '0';
INSERT INTO test.replicated_alter1 VALUES ('2015-01-01', 6,38,'2014-07-15 13:26:50',[10,20,30],['asd','qwe','qwe'],['2000-01-01','2000-01-01','2000-01-03'],'100500');
ALTER TABLE replicated_alter1 ADD COLUMN s String DEFAULT '0';
INSERT INTO replicated_alter1 VALUES ('2015-01-01', 6,38,'2014-07-15 13:26:50',[10,20,30],['asd','qwe','qwe'],['2000-01-01','2000-01-01','2000-01-03'],'100500');
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 DROP COLUMN `n.d`, MODIFY COLUMN s Int64;
ALTER TABLE replicated_alter1 DROP COLUMN `n.d`, MODIFY COLUMN s Int64;
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 ADD COLUMN `n.d` Array(Date), MODIFY COLUMN s UInt32;
ALTER TABLE replicated_alter1 ADD COLUMN `n.d` Array(Date), MODIFY COLUMN s UInt32;
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 DROP COLUMN n.ui8, DROP COLUMN n.d;
ALTER TABLE replicated_alter1 DROP COLUMN n.ui8, DROP COLUMN n.d;
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 DROP COLUMN n.s;
ALTER TABLE replicated_alter1 DROP COLUMN n.s;
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 ADD COLUMN n.s Array(String), ADD COLUMN n.d Array(Date);
ALTER TABLE replicated_alter1 ADD COLUMN n.s Array(String), ADD COLUMN n.d Array(Date);
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 DROP COLUMN n;
ALTER TABLE replicated_alter1 DROP COLUMN n;
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
ALTER TABLE test.replicated_alter1 MODIFY COLUMN dt Date, MODIFY COLUMN s DateTime;
ALTER TABLE replicated_alter1 MODIFY COLUMN dt Date, MODIFY COLUMN s DateTime;
DESC TABLE test.replicated_alter1;
SHOW CREATE TABLE test.replicated_alter1;
DESC TABLE test.replicated_alter2;
SHOW CREATE TABLE test.replicated_alter2;
SELECT * FROM test.replicated_alter1 ORDER BY k;
DESC TABLE replicated_alter1;
SHOW CREATE TABLE replicated_alter1;
DESC TABLE replicated_alter2;
SHOW CREATE TABLE replicated_alter2;
SELECT * FROM replicated_alter1 ORDER BY k;
DROP TABLE test.replicated_alter1;
DROP TABLE test.replicated_alter2;
DROP TABLE replicated_alter1;
DROP TABLE replicated_alter2;

View File

@ -1,5 +1,5 @@
CREATE TABLE IF NOT EXISTS test.empty_tiny_log(A UInt8) Engine = TinyLog;
CREATE TABLE IF NOT EXISTS empty_tiny_log(A UInt8) Engine = TinyLog;
SELECT A FROM test.empty_tiny_log;
SELECT A FROM empty_tiny_log;
DROP TABLE test.empty_tiny_log;
DROP TABLE empty_tiny_log;

View File

@ -4,7 +4,7 @@ set -e
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
echo 'DROP TABLE IF EXISTS test.insert_fewer_columns' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'CREATE TABLE test.insert_fewer_columns (a UInt8, b UInt8) ENGINE = Memory' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'INSERT INTO test.insert_fewer_columns (a) VALUES (1), (2)' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'SELECT * FROM test.insert_fewer_columns' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'DROP TABLE IF EXISTS insert_fewer_columns' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'CREATE TABLE insert_fewer_columns (a UInt8, b UInt8) ENGINE = Memory' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'INSERT INTO insert_fewer_columns (a) VALUES (1), (2)' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'SELECT * FROM insert_fewer_columns' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-

View File

@ -1,14 +1,14 @@
DROP TABLE IF EXISTS test.insert_fewer_columns;
CREATE TABLE test.insert_fewer_columns (a UInt8, b UInt8) ENGINE = Memory;
INSERT INTO test.insert_fewer_columns (a) VALUES (1), (2);
SELECT * FROM test.insert_fewer_columns;
DROP TABLE IF EXISTS insert_fewer_columns;
CREATE TABLE insert_fewer_columns (a UInt8, b UInt8) ENGINE = Memory;
INSERT INTO insert_fewer_columns (a) VALUES (1), (2);
SELECT * FROM insert_fewer_columns;
-- Test position arguments in insert.
DROP TABLE IF EXISTS test.insert_fewer_columns_2;
CREATE TABLE test.insert_fewer_columns_2 (b UInt8, a UInt8) ENGINE = Memory;
INSERT INTO test.insert_fewer_columns_2 SELECT * FROM test.insert_fewer_columns;
SELECT a, b FROM test.insert_fewer_columns;
SELECT a, b FROM test.insert_fewer_columns_2;
DROP TABLE IF EXISTS insert_fewer_columns_2;
CREATE TABLE insert_fewer_columns_2 (b UInt8, a UInt8) ENGINE = Memory;
INSERT INTO insert_fewer_columns_2 SELECT * FROM insert_fewer_columns;
SELECT a, b FROM insert_fewer_columns;
SELECT a, b FROM insert_fewer_columns_2;
DROP TABLE IF EXISTS test.insert_fewer_columns_2;
DROP TABLE test.insert_fewer_columns;
DROP TABLE IF EXISTS insert_fewer_columns_2;
DROP TABLE insert_fewer_columns;

View File

@ -26,10 +26,10 @@ SELECT hex(IPv4ToIPv6(1297626935));
/* Тест с таблицей */
DROP TABLE IF EXISTS test.addresses;
CREATE TABLE test.addresses(addr UInt32) ENGINE = Memory;
INSERT INTO test.addresses(addr) VALUES (1297626935), (2130706433), (3254522122);
SELECT hex(IPv4ToIPv6(addr)) FROM test.addresses ORDER BY addr ASC;
DROP TABLE IF EXISTS addresses;
CREATE TABLE addresses(addr UInt32) ENGINE = Memory;
INSERT INTO addresses(addr) VALUES (1297626935), (2130706433), (3254522122);
SELECT hex(IPv4ToIPv6(addr)) FROM addresses ORDER BY addr ASC;
/* cutIPv6 */
@ -113,15 +113,15 @@ SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0,
/* Реальные IPv6-адреса */
DROP TABLE IF EXISTS test.addresses;
CREATE TABLE test.addresses(addr String) ENGINE = Memory;
INSERT INTO test.addresses(addr) VALUES ('20010DB8AC10FE01FEEDBABECAFEF00D'), ('20010DB8AC10FE01DEADC0DECAFED00D'), ('20010DB8AC10FE01ABADBABEFACEB00C');
SELECT cutIPv6(toFixedString(unhex(addr), 16), 3, 0) FROM test.addresses ORDER BY addr ASC;
DROP TABLE IF EXISTS addresses;
CREATE TABLE addresses(addr String) ENGINE = Memory;
INSERT INTO addresses(addr) VALUES ('20010DB8AC10FE01FEEDBABECAFEF00D'), ('20010DB8AC10FE01DEADC0DECAFED00D'), ('20010DB8AC10FE01ABADBABEFACEB00C');
SELECT cutIPv6(toFixedString(unhex(addr), 16), 3, 0) FROM addresses ORDER BY addr ASC;
/* IPv4-mapped IPv6-адреса */
DROP TABLE IF EXISTS test.addresses;
CREATE TABLE test.addresses(addr String) ENGINE = Memory;
INSERT INTO test.addresses(addr) VALUES ('00000000000000000000FFFFC1FC110A'), ('00000000000000000000FFFF4D583737'), ('00000000000000000000FFFF7F000001');
SELECT cutIPv6(toFixedString(unhex(addr), 16), 0, 3) FROM test.addresses ORDER BY addr ASC;
DROP TABLE IF EXISTS addresses;
CREATE TABLE addresses(addr String) ENGINE = Memory;
INSERT INTO addresses(addr) VALUES ('00000000000000000000FFFFC1FC110A'), ('00000000000000000000FFFF4D583737'), ('00000000000000000000FFFF7F000001');
SELECT cutIPv6(toFixedString(unhex(addr), 16), 0, 3) FROM addresses ORDER BY addr ASC;

View File

@ -1,40 +1,40 @@
drop table if exists test.defaulted;
drop table if exists defaulted;
create table test.defaulted (col1 default 0) engine=Memory;
desc table test.defaulted;
drop table test.defaulted;
create table defaulted (col1 default 0) engine=Memory;
desc table defaulted;
drop table defaulted;
create table test.defaulted (col1 UInt32, col2 default col1 + 1, col3 materialized col1 + 2, col4 alias col1 + 3) engine=Memory;
desc table test.defaulted;
insert into test.defaulted (col1) values (10);
select * from test.defaulted;
select col3, col4 from test.defaulted;
drop table test.defaulted;
create table defaulted (col1 UInt32, col2 default col1 + 1, col3 materialized col1 + 2, col4 alias col1 + 3) engine=Memory;
desc table defaulted;
insert into defaulted (col1) values (10);
select * from defaulted;
select col3, col4 from defaulted;
drop table defaulted;
create table test.defaulted (col1 Int8, col2 UInt64 default (SELECT dummy+99 from system.one)) engine=Memory;
insert into test.defaulted (col1) values (0);
select col2 from test.defaulted;
drop table test.defaulted;
create table defaulted (col1 Int8, col2 UInt64 default (SELECT dummy+99 from system.one)) engine=Memory;
insert into defaulted (col1) values (0);
select col2 from defaulted;
drop table defaulted;
create table test.defaulted (payload String, date materialized today(), key materialized 0 * rand()) engine=MergeTree(date, key, 8192);
desc table test.defaulted;
insert into test.defaulted (payload) values ('hello clickhouse');
select * from test.defaulted;
alter table test.defaulted add column payload_length materialized length(payload);
desc table test.defaulted;
select *, payload_length from test.defaulted;
insert into test.defaulted (payload) values ('some string');
select *, payload_length from test.defaulted order by payload;
select *, payload_length from test.defaulted order by payload;
alter table test.defaulted modify column payload_length default length(payload);
desc table test.defaulted;
select * from test.defaulted order by payload;
alter table test.defaulted modify column payload_length default length(payload) % 65535;
desc table test.defaulted;
select * from test.defaulted order by payload;
alter table test.defaulted modify column payload_length UInt16 default length(payload);
desc table test.defaulted;
alter table test.defaulted drop column payload_length;
desc table test.defaulted;
select * from test.defaulted order by payload;
drop table test.defaulted;
create table defaulted (payload String, date materialized today(), key materialized 0 * rand()) engine=MergeTree(date, key, 8192);
desc table defaulted;
insert into defaulted (payload) values ('hello clickhouse');
select * from defaulted;
alter table defaulted add column payload_length materialized length(payload);
desc table defaulted;
select *, payload_length from defaulted;
insert into defaulted (payload) values ('some string');
select *, payload_length from defaulted order by payload;
select *, payload_length from defaulted order by payload;
alter table defaulted modify column payload_length default length(payload);
desc table defaulted;
select * from defaulted order by payload;
alter table defaulted modify column payload_length default length(payload) % 65535;
desc table defaulted;
select * from defaulted order by payload;
alter table defaulted modify column payload_length UInt16 default length(payload);
desc table defaulted;
alter table defaulted drop column payload_length;
desc table defaulted;
select * from defaulted order by payload;
drop table defaulted;

View File

@ -1,138 +1,138 @@
DROP TABLE IF EXISTS test.merge_tree;
DROP TABLE IF EXISTS test.collapsing_merge_tree;
DROP TABLE IF EXISTS test.versioned_collapsing_merge_tree;
DROP TABLE IF EXISTS test.summing_merge_tree;
DROP TABLE IF EXISTS test.summing_merge_tree_with_list_of_columns_to_sum;
DROP TABLE IF EXISTS test.aggregating_merge_tree;
DROP TABLE IF EXISTS merge_tree;
DROP TABLE IF EXISTS collapsing_merge_tree;
DROP TABLE IF EXISTS versioned_collapsing_merge_tree;
DROP TABLE IF EXISTS summing_merge_tree;
DROP TABLE IF EXISTS summing_merge_tree_with_list_of_columns_to_sum;
DROP TABLE IF EXISTS aggregating_merge_tree;
DROP TABLE IF EXISTS test.merge_tree_with_sampling;
DROP TABLE IF EXISTS test.collapsing_merge_tree_with_sampling;
DROP TABLE IF EXISTS test.versioned_collapsing_merge_tree_with_sampling;
DROP TABLE IF EXISTS test.summing_merge_tree_with_sampling;
DROP TABLE IF EXISTS test.summing_merge_tree_with_sampling_with_list_of_columns_to_sum;
DROP TABLE IF EXISTS test.aggregating_merge_tree_with_sampling;
DROP TABLE IF EXISTS merge_tree_with_sampling;
DROP TABLE IF EXISTS collapsing_merge_tree_with_sampling;
DROP TABLE IF EXISTS versioned_collapsing_merge_tree_with_sampling;
DROP TABLE IF EXISTS summing_merge_tree_with_sampling;
DROP TABLE IF EXISTS summing_merge_tree_with_sampling_with_list_of_columns_to_sum;
DROP TABLE IF EXISTS aggregating_merge_tree_with_sampling;
DROP TABLE IF EXISTS test.replicated_merge_tree;
DROP TABLE IF EXISTS test.replicated_collapsing_merge_tree;
DROP TABLE IF EXISTS test.replicated_versioned_collapsing_merge_tree;
DROP TABLE IF EXISTS test.replicated_summing_merge_tree;
DROP TABLE IF EXISTS test.replicated_summing_merge_tree_with_list_of_columns_to_sum;
DROP TABLE IF EXISTS test.replicated_aggregating_merge_tree;
DROP TABLE IF EXISTS replicated_merge_tree;
DROP TABLE IF EXISTS replicated_collapsing_merge_tree;
DROP TABLE IF EXISTS replicated_versioned_collapsing_merge_tree;
DROP TABLE IF EXISTS replicated_summing_merge_tree;
DROP TABLE IF EXISTS replicated_summing_merge_tree_with_list_of_columns_to_sum;
DROP TABLE IF EXISTS replicated_aggregating_merge_tree;
DROP TABLE IF EXISTS test.replicated_merge_tree_with_sampling;
DROP TABLE IF EXISTS test.replicated_collapsing_merge_tree_with_sampling;
DROP TABLE IF EXISTS test.replicated_versioned_collapsing_merge_tree_with_sampling;
DROP TABLE IF EXISTS test.replicated_summing_merge_tree_with_sampling;
DROP TABLE IF EXISTS test.replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum;
DROP TABLE IF EXISTS test.replicated_aggregating_merge_tree_with_sampling;
DROP TABLE IF EXISTS replicated_merge_tree_with_sampling;
DROP TABLE IF EXISTS replicated_collapsing_merge_tree_with_sampling;
DROP TABLE IF EXISTS replicated_versioned_collapsing_merge_tree_with_sampling;
DROP TABLE IF EXISTS replicated_summing_merge_tree_with_sampling;
DROP TABLE IF EXISTS replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum;
DROP TABLE IF EXISTS replicated_aggregating_merge_tree_with_sampling;
CREATE TABLE test.merge_tree
CREATE TABLE merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111);
CREATE TABLE test.collapsing_merge_tree
CREATE TABLE collapsing_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = CollapsingMergeTree(d, (a, b), 111, y);
CREATE TABLE test.versioned_collapsing_merge_tree
CREATE TABLE versioned_collapsing_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = VersionedCollapsingMergeTree(d, (a, b), 111, y, b);
CREATE TABLE test.summing_merge_tree
CREATE TABLE summing_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = SummingMergeTree(d, (a, b), 111);
CREATE TABLE test.summing_merge_tree_with_list_of_columns_to_sum
CREATE TABLE summing_merge_tree_with_list_of_columns_to_sum
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = SummingMergeTree(d, (a, b), 111, (y, z));
CREATE TABLE test.aggregating_merge_tree
CREATE TABLE aggregating_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = AggregatingMergeTree(d, (a, b), 111);
CREATE TABLE test.merge_tree_with_sampling
CREATE TABLE merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
CREATE TABLE test.collapsing_merge_tree_with_sampling
CREATE TABLE collapsing_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = CollapsingMergeTree(d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, y);
CREATE TABLE test.versioned_collapsing_merge_tree_with_sampling
CREATE TABLE versioned_collapsing_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = VersionedCollapsingMergeTree(d, sipHash64(a) + b, (a, sipHash64(a) + b, b), 111, y, b);
CREATE TABLE test.summing_merge_tree_with_sampling
CREATE TABLE summing_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = SummingMergeTree(d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
CREATE TABLE test.summing_merge_tree_with_sampling_with_list_of_columns_to_sum
CREATE TABLE summing_merge_tree_with_sampling_with_list_of_columns_to_sum
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = SummingMergeTree(d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, (y, z));
CREATE TABLE test.aggregating_merge_tree_with_sampling
CREATE TABLE aggregating_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = AggregatingMergeTree(d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
CREATE TABLE test.replicated_merge_tree
CREATE TABLE replicated_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01/replicated_merge_tree/', 'r1', d, (a, b), 111);
CREATE TABLE test.replicated_collapsing_merge_tree
CREATE TABLE replicated_collapsing_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test/01/replicated_collapsing_merge_tree/', 'r1', d, (a, b), 111, y);
CREATE TABLE test.replicated_versioned_collapsing_merge_tree
CREATE TABLE replicated_versioned_collapsing_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/test/01/replicated_versioned_collapsing_merge_tree/', 'r1', d, (a, b), 111, y, b);
CREATE TABLE test.replicated_summing_merge_tree
CREATE TABLE replicated_summing_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test/01/replicated_summing_merge_tree/', 'r1', d, (a, b), 111);
CREATE TABLE test.replicated_summing_merge_tree_with_list_of_columns_to_sum
CREATE TABLE replicated_summing_merge_tree_with_list_of_columns_to_sum
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test/01/replicated_summing_merge_tree_with_list_of_columns_to_sum/', 'r1', d, (a, b), 111, (y, z));
CREATE TABLE test.replicated_aggregating_merge_tree
CREATE TABLE replicated_aggregating_merge_tree
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test/01/replicated_aggregating_merge_tree/', 'r1', d, (a, b), 111);
CREATE TABLE test.replicated_merge_tree_with_sampling
CREATE TABLE replicated_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01/replicated_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
CREATE TABLE test.replicated_collapsing_merge_tree_with_sampling
CREATE TABLE replicated_collapsing_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/test/01/replicated_collapsing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, y);
CREATE TABLE test.replicated_versioned_collapsing_merge_tree_with_sampling
CREATE TABLE replicated_versioned_collapsing_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/test/01/replicated_versioned_collapsing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b, b), 111, y, b);
CREATE TABLE test.replicated_summing_merge_tree_with_sampling
CREATE TABLE replicated_summing_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test/01/replicated_summing_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
CREATE TABLE test.replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum
CREATE TABLE replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/test/01/replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111, (y, z));
CREATE TABLE test.replicated_aggregating_merge_tree_with_sampling
CREATE TABLE replicated_aggregating_merge_tree_with_sampling
(d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test/01/replicated_aggregating_merge_tree_with_sampling/', 'r1', d, sipHash64(a) + b, (a, sipHash64(a) + b), 111);
INSERT INTO test.merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.collapsing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.versioned_collapsing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.summing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.summing_merge_tree_with_list_of_columns_to_sum VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.aggregating_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO collapsing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO versioned_collapsing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO summing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO summing_merge_tree_with_list_of_columns_to_sum VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO aggregating_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.collapsing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.versioned_collapsing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.summing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.summing_merge_tree_with_sampling_with_list_of_columns_to_sum VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.aggregating_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO collapsing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO versioned_collapsing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO summing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO summing_merge_tree_with_sampling_with_list_of_columns_to_sum VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO aggregating_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_collapsing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_versioned_collapsing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_summing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_summing_merge_tree_with_list_of_columns_to_sum VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_aggregating_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_collapsing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_versioned_collapsing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_summing_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_summing_merge_tree_with_list_of_columns_to_sum VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_aggregating_merge_tree VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_collapsing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_versioned_collapsing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_summing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO test.replicated_aggregating_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_collapsing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_versioned_collapsing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_summing_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
INSERT INTO replicated_aggregating_merge_tree_with_sampling VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);
DROP TABLE test.merge_tree;
DROP TABLE test.collapsing_merge_tree;
DROP TABLE test.versioned_collapsing_merge_tree;
DROP TABLE test.summing_merge_tree;
DROP TABLE test.summing_merge_tree_with_list_of_columns_to_sum;
DROP TABLE test.aggregating_merge_tree;
DROP TABLE merge_tree;
DROP TABLE collapsing_merge_tree;
DROP TABLE versioned_collapsing_merge_tree;
DROP TABLE summing_merge_tree;
DROP TABLE summing_merge_tree_with_list_of_columns_to_sum;
DROP TABLE aggregating_merge_tree;
DROP TABLE test.merge_tree_with_sampling;
DROP TABLE test.collapsing_merge_tree_with_sampling;
DROP TABLE test.versioned_collapsing_merge_tree_with_sampling;
DROP TABLE test.summing_merge_tree_with_sampling;
DROP TABLE test.summing_merge_tree_with_sampling_with_list_of_columns_to_sum;
DROP TABLE test.aggregating_merge_tree_with_sampling;
DROP TABLE merge_tree_with_sampling;
DROP TABLE collapsing_merge_tree_with_sampling;
DROP TABLE versioned_collapsing_merge_tree_with_sampling;
DROP TABLE summing_merge_tree_with_sampling;
DROP TABLE summing_merge_tree_with_sampling_with_list_of_columns_to_sum;
DROP TABLE aggregating_merge_tree_with_sampling;
DROP TABLE test.replicated_merge_tree;
DROP TABLE test.replicated_collapsing_merge_tree;
DROP TABLE test.replicated_versioned_collapsing_merge_tree;
DROP TABLE test.replicated_summing_merge_tree;
DROP TABLE test.replicated_summing_merge_tree_with_list_of_columns_to_sum;
DROP TABLE test.replicated_aggregating_merge_tree;
DROP TABLE replicated_merge_tree;
DROP TABLE replicated_collapsing_merge_tree;
DROP TABLE replicated_versioned_collapsing_merge_tree;
DROP TABLE replicated_summing_merge_tree;
DROP TABLE replicated_summing_merge_tree_with_list_of_columns_to_sum;
DROP TABLE replicated_aggregating_merge_tree;
DROP TABLE test.replicated_merge_tree_with_sampling;
DROP TABLE test.replicated_collapsing_merge_tree_with_sampling;
DROP TABLE test.replicated_versioned_collapsing_merge_tree_with_sampling;
DROP TABLE test.replicated_summing_merge_tree_with_sampling;
DROP TABLE test.replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum;
DROP TABLE test.replicated_aggregating_merge_tree_with_sampling;
DROP TABLE replicated_merge_tree_with_sampling;
DROP TABLE replicated_collapsing_merge_tree_with_sampling;
DROP TABLE replicated_versioned_collapsing_merge_tree_with_sampling;
DROP TABLE replicated_summing_merge_tree_with_sampling;
DROP TABLE replicated_summing_merge_tree_with_sampling_with_list_of_columns_to_sum;
DROP TABLE replicated_aggregating_merge_tree_with_sampling;

View File

@ -1,41 +1,41 @@
DROP TABLE IF EXISTS test.summing_merge_tree;
DROP TABLE IF EXISTS summing_merge_tree;
CREATE TABLE test.summing_merge_tree (d Date, a String, x UInt32, y UInt64, z Float64) ENGINE = SummingMergeTree(d, a, 8192);
CREATE TABLE summing_merge_tree (d Date, a String, x UInt32, y UInt64, z Float64) ENGINE = SummingMergeTree(d, a, 8192);
INSERT INTO test.summing_merge_tree VALUES ('2000-01-01', 'Hello', 1, 2, 3);
INSERT INTO test.summing_merge_tree VALUES ('2000-01-01', 'Hello', 4, 5, 6);
INSERT INTO test.summing_merge_tree VALUES ('2000-01-01', 'Goodbye', 1, 2, 3);
INSERT INTO summing_merge_tree VALUES ('2000-01-01', 'Hello', 1, 2, 3);
INSERT INTO summing_merge_tree VALUES ('2000-01-01', 'Hello', 4, 5, 6);
INSERT INTO summing_merge_tree VALUES ('2000-01-01', 'Goodbye', 1, 2, 3);
OPTIMIZE TABLE test.summing_merge_tree;
OPTIMIZE TABLE test.summing_merge_tree;
OPTIMIZE TABLE test.summing_merge_tree;
OPTIMIZE TABLE summing_merge_tree;
OPTIMIZE TABLE summing_merge_tree;
OPTIMIZE TABLE summing_merge_tree;
SELECT * FROM test.summing_merge_tree ORDER BY d, a, x, y, z;
SELECT * FROM summing_merge_tree ORDER BY d, a, x, y, z;
DROP TABLE test.summing_merge_tree;
DROP TABLE summing_merge_tree;
CREATE TABLE test.summing_merge_tree (d Date, a String, x UInt32, y UInt64, z Float64) ENGINE = SummingMergeTree(d, a, 8192, (y, z));
CREATE TABLE summing_merge_tree (d Date, a String, x UInt32, y UInt64, z Float64) ENGINE = SummingMergeTree(d, a, 8192, (y, z));
INSERT INTO test.summing_merge_tree VALUES ('2000-01-01', 'Hello', 1, 2, 3);
INSERT INTO test.summing_merge_tree VALUES ('2000-01-01', 'Hello', 4, 5, 6);
INSERT INTO test.summing_merge_tree VALUES ('2000-01-01', 'Goodbye', 1, 2, 3);
INSERT INTO summing_merge_tree VALUES ('2000-01-01', 'Hello', 1, 2, 3);
INSERT INTO summing_merge_tree VALUES ('2000-01-01', 'Hello', 4, 5, 6);
INSERT INTO summing_merge_tree VALUES ('2000-01-01', 'Goodbye', 1, 2, 3);
OPTIMIZE TABLE test.summing_merge_tree;
OPTIMIZE TABLE test.summing_merge_tree;
OPTIMIZE TABLE test.summing_merge_tree;
OPTIMIZE TABLE summing_merge_tree;
OPTIMIZE TABLE summing_merge_tree;
OPTIMIZE TABLE summing_merge_tree;
SELECT * FROM test.summing_merge_tree ORDER BY d, a, x, y, z;
SELECT * FROM summing_merge_tree ORDER BY d, a, x, y, z;
DROP TABLE test.summing_merge_tree;
DROP TABLE summing_merge_tree;
--
DROP TABLE IF EXISTS test.summing;
CREATE TABLE test.summing (p Date, k UInt64, s UInt64) ENGINE = SummingMergeTree(p, k, 1);
DROP TABLE IF EXISTS summing;
CREATE TABLE summing (p Date, k UInt64, s UInt64) ENGINE = SummingMergeTree(p, k, 1);
INSERT INTO test.summing (k, s) VALUES (0, 1);
INSERT INTO test.summing (k, s) VALUES (0, 1), (666, 1), (666, 0);
OPTIMIZE TABLE test.summing PARTITION 197001;
INSERT INTO summing (k, s) VALUES (0, 1);
INSERT INTO summing (k, s) VALUES (0, 1), (666, 1), (666, 0);
OPTIMIZE TABLE summing PARTITION 197001;
SELECT k, s FROM test.summing ORDER BY k;
SELECT k, s FROM summing ORDER BY k;

View File

@ -7,12 +7,12 @@ set -o errexit
set -o pipefail
echo "
DROP TABLE IF EXISTS test.two_blocks;
CREATE TABLE test.two_blocks (d Date) ENGINE = MergeTree(d, d, 1);
INSERT INTO test.two_blocks VALUES ('2000-01-01');
INSERT INTO test.two_blocks VALUES ('2000-01-02');
DROP TABLE IF EXISTS two_blocks;
CREATE TABLE two_blocks (d Date) ENGINE = MergeTree(d, d, 1);
INSERT INTO two_blocks VALUES ('2000-01-01');
INSERT INTO two_blocks VALUES ('2000-01-02');
" | $CLICKHOUSE_CLIENT -n
for i in {1..10}; do seq 1 100 | sed 's/.*/SELECT count() FROM (SELECT * FROM test.two_blocks);/' | $CLICKHOUSE_CLIENT -n --receive_timeout=1 | grep -vE '^2$' && echo 'Fail!' && break; echo -n '.'; done; echo
for i in {1..10}; do seq 1 100 | sed 's/.*/SELECT count() FROM (SELECT * FROM two_blocks);/' | $CLICKHOUSE_CLIENT -n --receive_timeout=1 | grep -vE '^2$' && echo 'Fail!' && break; echo -n '.'; done; echo
echo "DROP TABLE test.two_blocks;" | $CLICKHOUSE_CLIENT -n
echo "DROP TABLE two_blocks;" | $CLICKHOUSE_CLIENT -n

View File

@ -1,250 +1,250 @@
DROP TABLE IF EXISTS test.min_if;
CREATE TABLE test.min_if (arr Array(UInt8), str String, int Int32) ENGINE = Memory;
INSERT INTO test.min_if SELECT emptyArrayUInt8() AS arr, '' AS str, toInt32(0) AS int FROM system.numbers LIMIT 100000;
INSERT INTO test.min_if SELECT [1] AS arr, '2' AS str, toInt32(3) AS int;
INSERT INTO test.min_if SELECT emptyArrayUInt8() AS arr, '' AS str, toInt32(0) AS int FROM system.numbers LIMIT 100000;
DROP TABLE IF EXISTS min_if;
CREATE TABLE min_if (arr Array(UInt8), str String, int Int32) ENGINE = Memory;
INSERT INTO min_if SELECT emptyArrayUInt8() AS arr, '' AS str, toInt32(0) AS int FROM system.numbers LIMIT 100000;
INSERT INTO min_if SELECT [1] AS arr, '2' AS str, toInt32(3) AS int;
INSERT INTO min_if SELECT emptyArrayUInt8() AS arr, '' AS str, toInt32(0) AS int FROM system.numbers LIMIT 100000;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM test.min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(arr, notEmpty(arr)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM test.min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(str, notEmpty(str)) FROM min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM test.min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
SELECT minIf(int, int != 0) FROM min_if;
DROP TABLE test.min_if;
DROP TABLE min_if;

View File

@ -2,8 +2,8 @@ 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;
@ -17,4 +17,4 @@ SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM re
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;
DROP TABLE test.numbers500k;
DROP TABLE numbers500k;

View File

@ -1,9 +1,9 @@
SET max_memory_usage = 100000000;
SET max_bytes_before_external_sort = 20000000;
DROP TABLE IF EXISTS test.numbers10m;
CREATE VIEW test.numbers10m AS SELECT number FROM system.numbers LIMIT 10000000;
DROP TABLE IF EXISTS numbers10m;
CREATE VIEW numbers10m AS SELECT number FROM system.numbers LIMIT 10000000;
SELECT number FROM remote('127.0.0.{2,3}', test, numbers10m) ORDER BY number * 1234567890123456789 LIMIT 19999980, 20;
DROP TABLE test.numbers10m;
DROP TABLE numbers10m;

View File

@ -7,12 +7,12 @@ set -o errexit
set -o pipefail
$CLICKHOUSE_CLIENT -n --query="
DROP TABLE IF EXISTS test.users;
CREATE TABLE test.users (UserID UInt64) ENGINE = Log;
INSERT INTO test.users VALUES (1468013291393583084);
INSERT INTO test.users VALUES (1321770221388956068);
DROP TABLE IF EXISTS users;
CREATE TABLE users (UserID UInt64) ENGINE = Log;
INSERT INTO users VALUES (1468013291393583084);
INSERT INTO users VALUES (1321770221388956068);
";
for i in {1..10}; do seq 1 10 | sed "s/.*/SELECT count() FROM (SELECT * FROM remote('127.0.0.{2,3}', test, users) WHERE UserID IN (SELECT arrayJoin([1468013291393583084, 1321770221388956068])));/" | $CLICKHOUSE_CLIENT -n | grep -vE '^4$' && echo 'Fail!' && break; echo -n '.'; done; echo
for i in {1..10}; do seq 1 10 | sed "s/.*/SELECT count() FROM (SELECT * FROM remote('127.0.0.{2,3}', ${CLICKHOUSE_DATABASE}, users) WHERE UserID IN (SELECT arrayJoin([1468013291393583084, 1321770221388956068])));/" | $CLICKHOUSE_CLIENT -n | grep -vE '^4$' && echo 'Fail!' && break; echo -n '.'; done; echo
$CLICKHOUSE_CLIENT --query="DROP TABLE test.users;";
$CLICKHOUSE_CLIENT --query="DROP TABLE users;";

View File

@ -1,10 +1,10 @@
DROP TABLE IF EXISTS test.null;
CREATE TABLE test.null (a Array(UInt64), b Array(String), c Array(Array(Date))) ENGINE = Memory;
DROP TABLE IF EXISTS null;
CREATE TABLE null (a Array(UInt64), b Array(String), c Array(Array(Date))) ENGINE = Memory;
INSERT INTO test.null (a) VALUES ([1,2]), ([3, 4]), ([ 5 ,6]), ([ 7 , 8 ]), ([]), ([ ]);
INSERT INTO test.null (b) VALUES ([ 'Hello' , 'World' ]);
INSERT INTO test.null (c) VALUES ([ ]), ([ [ ] ]), ([[],[]]), ([['2015-01-01', '2015-01-02'], ['2015-01-03', '2015-01-04']]);
INSERT INTO null (a) VALUES ([1,2]), ([3, 4]), ([ 5 ,6]), ([ 7 , 8 ]), ([]), ([ ]);
INSERT INTO null (b) VALUES ([ 'Hello' , 'World' ]);
INSERT INTO null (c) VALUES ([ ]), ([ [ ] ]), ([[],[]]), ([['2015-01-01', '2015-01-02'], ['2015-01-03', '2015-01-04']]);
SELECT a, b, c FROM test.null ORDER BY a, b, c;
SELECT a, b, c FROM null ORDER BY a, b, c;
DROP TABLE test.null;
DROP TABLE null;

View File

@ -1,18 +1,14 @@
DROP TABLE IF EXISTS test.join;
DROP TABLE IF EXISTS join;
CREATE TABLE test.join (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, LEFT, k);
CREATE TABLE join (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, LEFT, k);
USE test;
INSERT INTO test.join VALUES ('abc', [0], 1), ('def', [1, 2], 2);
INSERT INTO test.join (k, s) VALUES (3, 'ghi');
INSERT INTO test.join (x, k) VALUES ([3, 4, 5], 4);
INSERT INTO join VALUES ('abc', [0], 1), ('def', [1, 2], 2);
INSERT INTO join (k, s) VALUES (3, 'ghi');
INSERT INTO join (x, k) VALUES ([3, 4, 5], 4);
SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
SELECT s, x FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
SELECT x, s, k FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
SELECT 1, x, 2, s, 3, k, 4 FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
USE default;
DROP TABLE test.join;
DROP TABLE join;

View File

@ -1,27 +1,27 @@
DROP TABLE IF EXISTS test.alter;
CREATE TABLE test.alter (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter', 'r1', d, (d), 8192);
DROP TABLE IF EXISTS alter;
CREATE TABLE alter (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter', 'r1', d, (d), 8192);
INSERT INTO test.alter VALUES ('2014-01-01', 1);
ALTER TABLE test.alter DROP COLUMN x;
INSERT INTO alter VALUES ('2014-01-01', 1);
ALTER TABLE alter DROP COLUMN x;
SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test/alter/replicas/r1/parts/20140101_20140101_0_0_0' AND name = 'columns' FORMAT TabSeparatedRaw;
DROP TABLE test.alter;
DROP TABLE alter;
CREATE TABLE test.alter (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter', 'r1', d, (d), 8192);
CREATE TABLE alter (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/alter', 'r1', d, (d), 8192);
INSERT INTO test.alter VALUES ('2014-01-01');
SELECT * FROM test.alter ORDER BY d;
INSERT INTO alter VALUES ('2014-01-01');
SELECT * FROM alter ORDER BY d;
ALTER TABLE test.alter ADD COLUMN x UInt8;
ALTER TABLE alter ADD COLUMN x UInt8;
INSERT INTO test.alter VALUES ('2014-02-01', 1);
SELECT * FROM test.alter ORDER BY d;
INSERT INTO alter VALUES ('2014-02-01', 1);
SELECT * FROM alter ORDER BY d;
ALTER TABLE test.alter DROP COLUMN x;
SELECT * FROM test.alter ORDER BY d;
ALTER TABLE alter DROP COLUMN x;
SELECT * FROM alter ORDER BY d;
SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test/alter/replicas/r1/parts/20140201_20140201_0_0_0' AND name = 'columns' FORMAT TabSeparatedRaw;
DROP TABLE test.alter;
DROP TABLE alter;

View File

@ -4,8 +4,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT -n --query="
DROP TABLE IF EXISTS test.numbers_100k;
CREATE VIEW test.numbers_100k AS SELECT * FROM system.numbers LIMIT 100000;
DROP TABLE IF EXISTS numbers_100k;
CREATE VIEW numbers_100k AS SELECT * FROM system.numbers LIMIT 100000;
";
STEP_MULTIPLIER=25
@ -15,10 +15,10 @@ fi
for i in $(seq 1000000 $((20000 * $STEP_MULTIPLIER)) 10000000 && seq 10100000 $((100000 * $STEP_MULTIPLIER)) 50000000); do
$CLICKHOUSE_CLIENT --max_memory_usage=$i --query="
SELECT intDiv(number, 5) AS k, max(toString(number)) FROM remote('127.0.0.{2,3}', test.numbers_100k) GROUP BY k ORDER BY k LIMIT 1;
SELECT intDiv(number, 5) AS k, max(toString(number)) FROM remote('127.0.0.{2,3}', ${CLICKHOUSE_DATABASE}.numbers_100k) GROUP BY k ORDER BY k LIMIT 1;
" 2> /dev/null;
CODE=$?;
[ "$CODE" -ne "241" ] && [ "$CODE" -ne "0" ] && echo "Fail" && break;
done | uniq
$CLICKHOUSE_CLIENT --query="DROP TABLE test.numbers_100k;";
$CLICKHOUSE_CLIENT --query="DROP TABLE numbers_100k;";

View File

@ -1,11 +1,11 @@
DROP TABLE IF EXISTS test.prewhere;
DROP TABLE IF EXISTS prewhere;
CREATE TABLE test.prewhere (d Date, a String, b String) ENGINE = MergeTree(d, d, 8192);
INSERT INTO test.prewhere VALUES ('2015-01-01', 'hello', 'world');
CREATE TABLE prewhere (d Date, a String, b String) ENGINE = MergeTree(d, d, 8192);
INSERT INTO prewhere VALUES ('2015-01-01', 'hello', 'world');
ALTER TABLE test.prewhere ADD COLUMN a1 String AFTER a;
INSERT INTO test.prewhere VALUES ('2015-01-01', 'hello1', 'xxx', 'world1');
ALTER TABLE prewhere ADD COLUMN a1 String AFTER a;
INSERT INTO prewhere VALUES ('2015-01-01', 'hello1', 'xxx', 'world1');
SELECT d, a, a1, b FROM test.prewhere PREWHERE a LIKE 'hello%' ORDER BY a1;
SELECT d, a, a1, b FROM prewhere PREWHERE a LIKE 'hello%' ORDER BY a1;
DROP TABLE test.prewhere;
DROP TABLE prewhere;

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS test.default;
DROP TABLE IF EXISTS default;
CREATE TABLE test.default (d Date DEFAULT toDate(t), t DateTime) ENGINE = MergeTree(d, t, 8192);
INSERT INTO test.default (t) VALUES ('1234567890');
SELECT toStartOfMonth(d), toUInt32(t) FROM test.default;
CREATE TABLE default (d Date DEFAULT toDate(t), t DateTime) ENGINE = MergeTree(d, t, 8192);
INSERT INTO default (t) VALUES ('1234567890');
SELECT toStartOfMonth(d), toUInt32(t) FROM default;
DROP TABLE test.default;
DROP TABLE default;

View File

@ -1,21 +1,21 @@
drop table if exists test.nested_map;
drop table if exists nested_map;
create table test.nested_map (d default today(), k UInt64, payload default rand(), SomeMap Nested(ID UInt32, Num Int64)) engine=SummingMergeTree(d, k, 8192);
create table nested_map (d default today(), k UInt64, payload default rand(), SomeMap Nested(ID UInt32, Num Int64)) engine=SummingMergeTree(d, k, 8192);
insert into test.nested_map (k, `SomeMap.ID`, `SomeMap.Num`) values (0,[1],[100]),(1,[1],[100]),(2,[1],[100]),(3,[1,2],[100,150]);
insert into test.nested_map (k, `SomeMap.ID`, `SomeMap.Num`) values (0,[2],[150]),(1,[1],[150]),(2,[1,2],[150,150]),(3,[1],[-100]);
optimize table test.nested_map;
select `SomeMap.ID`, `SomeMap.Num` from test.nested_map;
insert into nested_map (k, `SomeMap.ID`, `SomeMap.Num`) values (0,[1],[100]),(1,[1],[100]),(2,[1],[100]),(3,[1,2],[100,150]);
insert into nested_map (k, `SomeMap.ID`, `SomeMap.Num`) values (0,[2],[150]),(1,[1],[150]),(2,[1,2],[150,150]),(3,[1],[-100]);
optimize table nested_map;
select `SomeMap.ID`, `SomeMap.Num` from nested_map;
drop table test.nested_map;
drop table nested_map;
drop table if exists test.nested_map_explicit;
drop table if exists nested_map_explicit;
create table test.nested_map_explicit (d default today(), k UInt64, SomeIntExcluded UInt32, SomeMap Nested(ID UInt32, Num Int64)) engine=SummingMergeTree(d, k, 8192, (SomeMap));
create table nested_map_explicit (d default today(), k UInt64, SomeIntExcluded UInt32, SomeMap Nested(ID UInt32, Num Int64)) engine=SummingMergeTree(d, k, 8192, (SomeMap));
insert into test.nested_map_explicit (k, `SomeIntExcluded`, `SomeMap.ID`, `SomeMap.Num`) values (0, 20, [1],[100]),(1, 20, [1],[100]),(2, 20, [1],[100]),(3, 20, [1,2],[100,150]);
insert into test.nested_map_explicit (k, `SomeIntExcluded`, `SomeMap.ID`, `SomeMap.Num`) values (0, 20, [2],[150]),(1, 20, [1],[150]),(2, 20, [1,2],[150,150]),(3, 20, [1],[-100]);
optimize table test.nested_map_explicit;
select `SomeIntExcluded`, `SomeMap.ID`, `SomeMap.Num` from test.nested_map_explicit;
insert into nested_map_explicit (k, `SomeIntExcluded`, `SomeMap.ID`, `SomeMap.Num`) values (0, 20, [1],[100]),(1, 20, [1],[100]),(2, 20, [1],[100]),(3, 20, [1,2],[100,150]);
insert into nested_map_explicit (k, `SomeIntExcluded`, `SomeMap.ID`, `SomeMap.Num`) values (0, 20, [2],[150]),(1, 20, [1],[150]),(2, 20, [1,2],[150,150]),(3, 20, [1],[-100]);
optimize table nested_map_explicit;
select `SomeIntExcluded`, `SomeMap.ID`, `SomeMap.Num` from nested_map_explicit;
drop table test.nested_map_explicit;
drop table nested_map_explicit;

View File

@ -1,29 +1,29 @@
DROP TABLE IF EXISTS test.alter;
DROP TABLE IF EXISTS alter;
CREATE TABLE test.alter (d Date DEFAULT toDate('2015-01-01'), n Nested(x String)) ENGINE = MergeTree(d, d, 8192);
CREATE TABLE alter (d Date DEFAULT toDate('2015-01-01'), n Nested(x String)) ENGINE = MergeTree(d, d, 8192);
INSERT INTO test.alter (`n.x`) VALUES (['Hello', 'World']);
INSERT INTO alter (`n.x`) VALUES (['Hello', 'World']);
SELECT * FROM test.alter;
SELECT * FROM test.alter ARRAY JOIN n;
SELECT * FROM test.alter ARRAY JOIN n WHERE n.x LIKE '%Hello%';
SELECT * FROM alter;
SELECT * FROM alter ARRAY JOIN n;
SELECT * FROM alter ARRAY JOIN n WHERE n.x LIKE '%Hello%';
ALTER TABLE test.alter ADD COLUMN n.y Array(UInt64);
ALTER TABLE alter ADD COLUMN n.y Array(UInt64);
SELECT * FROM test.alter;
SELECT * FROM test.alter ARRAY JOIN n;
SELECT * FROM test.alter ARRAY JOIN n WHERE n.x LIKE '%Hello%';
SELECT * FROM alter;
SELECT * FROM alter ARRAY JOIN n;
SELECT * FROM alter ARRAY JOIN n WHERE n.x LIKE '%Hello%';
INSERT INTO test.alter (`n.x`) VALUES (['Hello2', 'World2']);
INSERT INTO alter (`n.x`) VALUES (['Hello2', 'World2']);
SELECT * FROM test.alter ORDER BY n.x;
SELECT * FROM test.alter ARRAY JOIN n ORDER BY n.x;
SELECT * FROM test.alter ARRAY JOIN n WHERE n.x LIKE '%Hello%' ORDER BY n.x;
SELECT * FROM alter ORDER BY n.x;
SELECT * FROM alter ARRAY JOIN n ORDER BY n.x;
SELECT * FROM alter ARRAY JOIN n WHERE n.x LIKE '%Hello%' ORDER BY n.x;
OPTIMIZE TABLE test.alter;
OPTIMIZE TABLE alter;
SELECT * FROM test.alter;
SELECT * FROM test.alter ARRAY JOIN n;
SELECT * FROM test.alter ARRAY JOIN n WHERE n.x LIKE '%Hello%';
SELECT * FROM alter;
SELECT * FROM alter ARRAY JOIN n;
SELECT * FROM alter ARRAY JOIN n WHERE n.x LIKE '%Hello%';
DROP TABLE test.alter;
DROP TABLE alter;

View File

@ -1,35 +1,35 @@
drop table if exists test.summing_merge_tree_aggregate_function;
drop table if exists test.summing_merge_tree_null;
drop table if exists summing_merge_tree_aggregate_function;
drop table if exists summing_merge_tree_null;
---- partition merge
create table test.summing_merge_tree_aggregate_function (
create table summing_merge_tree_aggregate_function (
d Date,
k UInt64,
u AggregateFunction(uniq, UInt64)
) engine=SummingMergeTree(d, k, 1);
insert into test.summing_merge_tree_aggregate_function
insert into summing_merge_tree_aggregate_function
select today() as d,
number as k,
uniqState(toUInt64(number % 500))
from numbers(5000)
group by d, k;
insert into test.summing_merge_tree_aggregate_function
insert into summing_merge_tree_aggregate_function
select today() as d,
number + 5000 as k,
uniqState(toUInt64(number % 500))
from numbers(5000)
group by d, k;
select count() from test.summing_merge_tree_aggregate_function;
optimize table test.summing_merge_tree_aggregate_function;
select count() from test.summing_merge_tree_aggregate_function;
select count() from summing_merge_tree_aggregate_function;
optimize table summing_merge_tree_aggregate_function;
select count() from summing_merge_tree_aggregate_function;
drop table test.summing_merge_tree_aggregate_function;
drop table summing_merge_tree_aggregate_function;
---- sum + uniq + uniqExact
create table test.summing_merge_tree_aggregate_function (
create table summing_merge_tree_aggregate_function (
d materialized today(),
k UInt64,
c UInt64,
@ -37,121 +37,121 @@ create table test.summing_merge_tree_aggregate_function (
ue AggregateFunction(uniqExact, UInt8)
) engine=SummingMergeTree(d, k, 8192);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(1), uniqExactState(1);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(2), uniqExactState(2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(3), uniqExactState(2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(1), uniqExactState(1);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(2), uniqExactState(2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, uniqState(3), uniqExactState(3);
insert into summing_merge_tree_aggregate_function select 1, 1, uniqState(1), uniqExactState(1);
insert into summing_merge_tree_aggregate_function select 1, 1, uniqState(2), uniqExactState(2);
insert into summing_merge_tree_aggregate_function select 1, 1, uniqState(3), uniqExactState(2);
insert into summing_merge_tree_aggregate_function select 1, 1, uniqState(1), uniqExactState(1);
insert into summing_merge_tree_aggregate_function select 1, 1, uniqState(2), uniqExactState(2);
insert into summing_merge_tree_aggregate_function select 1, 1, uniqState(3), uniqExactState(3);
select
k, sum(c),
uniqMerge(u), uniqExactMerge(ue)
from test.summing_merge_tree_aggregate_function group by k;
from summing_merge_tree_aggregate_function group by k;
optimize table test.summing_merge_tree_aggregate_function;
optimize table summing_merge_tree_aggregate_function;
select
k, sum(c),
uniqMerge(u), uniqExactMerge(ue)
from test.summing_merge_tree_aggregate_function group by k;
from summing_merge_tree_aggregate_function group by k;
drop table test.summing_merge_tree_aggregate_function;
drop table summing_merge_tree_aggregate_function;
---- sum + topK
create table test.summing_merge_tree_aggregate_function (d materialized today(), k UInt64, c UInt64, x AggregateFunction(topK(2), UInt8)) engine=SummingMergeTree(d, k, 8192);
create table summing_merge_tree_aggregate_function (d materialized today(), k UInt64, c UInt64, x AggregateFunction(topK(2), UInt8)) engine=SummingMergeTree(d, k, 8192);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(1);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(3);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(3);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKState(2)(3);
select k, sum(c), topKMerge(2)(x) from test.summing_merge_tree_aggregate_function group by k;
optimize table test.summing_merge_tree_aggregate_function;
select k, sum(c), topKMerge(2)(x) from test.summing_merge_tree_aggregate_function group by k;
insert into summing_merge_tree_aggregate_function select 1, 1, topKState(2)(1);
insert into summing_merge_tree_aggregate_function select 1, 1, topKState(2)(2);
insert into summing_merge_tree_aggregate_function select 1, 1, topKState(2)(2);
insert into summing_merge_tree_aggregate_function select 1, 1, topKState(2)(3);
insert into summing_merge_tree_aggregate_function select 1, 1, topKState(2)(3);
insert into summing_merge_tree_aggregate_function select 1, 1, topKState(2)(3);
select k, sum(c), topKMerge(2)(x) from summing_merge_tree_aggregate_function group by k;
optimize table summing_merge_tree_aggregate_function;
select k, sum(c), topKMerge(2)(x) from summing_merge_tree_aggregate_function group by k;
drop table test.summing_merge_tree_aggregate_function;
drop table summing_merge_tree_aggregate_function;
---- sum + topKWeighted
create table test.summing_merge_tree_aggregate_function (d materialized today(), k UInt64, c UInt64, x AggregateFunction(topKWeighted(2), UInt8, UInt8)) engine=SummingMergeTree(d, k, 8192);
create table summing_merge_tree_aggregate_function (d materialized today(), k UInt64, c UInt64, x AggregateFunction(topKWeighted(2), UInt8, UInt8)) engine=SummingMergeTree(d, k, 8192);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(1, 1);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(1, 1);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(1, 1);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(2, 2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(2, 2);
insert into test.summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(3, 5);
select k, sum(c), topKWeightedMerge(2)(x) from test.summing_merge_tree_aggregate_function group by k;
optimize table test.summing_merge_tree_aggregate_function;
select k, sum(c), topKWeightedMerge(2)(x) from test.summing_merge_tree_aggregate_function group by k;
insert into summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(1, 1);
insert into summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(1, 1);
insert into summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(1, 1);
insert into summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(2, 2);
insert into summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(2, 2);
insert into summing_merge_tree_aggregate_function select 1, 1, topKWeightedState(2)(3, 5);
select k, sum(c), topKWeightedMerge(2)(x) from summing_merge_tree_aggregate_function group by k;
optimize table summing_merge_tree_aggregate_function;
select k, sum(c), topKWeightedMerge(2)(x) from summing_merge_tree_aggregate_function group by k;
drop table test.summing_merge_tree_aggregate_function;
drop table summing_merge_tree_aggregate_function;
---- avg
create table test.summing_merge_tree_aggregate_function (d materialized today(), k UInt64, x AggregateFunction(avg, Float64)) engine=SummingMergeTree(d, k, 8192);
create table summing_merge_tree_aggregate_function (d materialized today(), k UInt64, x AggregateFunction(avg, Float64)) engine=SummingMergeTree(d, k, 8192);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.0);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.125);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.25);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.375);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.4375);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.5);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.5625);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.625);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.75);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(0.875);
insert into test.summing_merge_tree_aggregate_function select 1, avgState(1.0);
select k, avgMerge(x) from test.summing_merge_tree_aggregate_function group by k;
optimize table test.summing_merge_tree_aggregate_function;
select k, avgMerge(x) from test.summing_merge_tree_aggregate_function group by k;
insert into summing_merge_tree_aggregate_function select 1, avgState(0.0);
insert into summing_merge_tree_aggregate_function select 1, avgState(0.125);
insert into summing_merge_tree_aggregate_function select 1, avgState(0.25);
insert into summing_merge_tree_aggregate_function select 1, avgState(0.375);
insert into summing_merge_tree_aggregate_function select 1, avgState(0.4375);
insert into summing_merge_tree_aggregate_function select 1, avgState(0.5);
insert into summing_merge_tree_aggregate_function select 1, avgState(0.5625);
insert into summing_merge_tree_aggregate_function select 1, avgState(0.625);
insert into summing_merge_tree_aggregate_function select 1, avgState(0.75);
insert into summing_merge_tree_aggregate_function select 1, avgState(0.875);
insert into summing_merge_tree_aggregate_function select 1, avgState(1.0);
select k, avgMerge(x) from summing_merge_tree_aggregate_function group by k;
optimize table summing_merge_tree_aggregate_function;
select k, avgMerge(x) from summing_merge_tree_aggregate_function group by k;
drop table test.summing_merge_tree_aggregate_function;
drop table summing_merge_tree_aggregate_function;
---- quantile
create table test.summing_merge_tree_aggregate_function (d materialized today(), k UInt64, x AggregateFunction(quantile(0.1), Float64)) engine=SummingMergeTree(d, k, 8192);
create table summing_merge_tree_aggregate_function (d materialized today(), k UInt64, x AggregateFunction(quantile(0.1), Float64)) engine=SummingMergeTree(d, k, 8192);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.0);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.1);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.2);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.3);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.4);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.5);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.6);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.7);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.8);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.9);
insert into test.summing_merge_tree_aggregate_function select 1, quantileState(0.1)(1.0);
select k, round(quantileMerge(0.1)(x), 1) from test.summing_merge_tree_aggregate_function group by k;
optimize table test.summing_merge_tree_aggregate_function;
select k, round(quantileMerge(0.1)(x), 1) from test.summing_merge_tree_aggregate_function group by k;
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.0);
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.1);
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.2);
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.3);
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.4);
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.5);
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.6);
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.7);
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.8);
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(0.9);
insert into summing_merge_tree_aggregate_function select 1, quantileState(0.1)(1.0);
select k, round(quantileMerge(0.1)(x), 1) from summing_merge_tree_aggregate_function group by k;
optimize table summing_merge_tree_aggregate_function;
select k, round(quantileMerge(0.1)(x), 1) from summing_merge_tree_aggregate_function group by k;
drop table test.summing_merge_tree_aggregate_function;
drop table summing_merge_tree_aggregate_function;
---- sum + uniq with more data
create table test.summing_merge_tree_null (
create table summing_merge_tree_null (
d materialized today(),
k UInt64,
c UInt64,
u UInt64
) engine=Null;
create materialized view test.summing_merge_tree_aggregate_function (
create materialized view summing_merge_tree_aggregate_function (
d materialized today(),
k UInt64,
c UInt64,
u AggregateFunction(uniq, UInt64)
) engine=SummingMergeTree(d, k, 8192)
as select d, k, sum(c) as c, uniqState(u) as u
from test.summing_merge_tree_null
from summing_merge_tree_null
group by d, k;
-- prime number 53 to avoid resonanse between %3 and %53
insert into test.summing_merge_tree_null select number % 3, 1, number % 53 from numbers(999999);
insert into summing_merge_tree_null select number % 3, 1, number % 53 from numbers(999999);
select k, sum(c), uniqMerge(u) from test.summing_merge_tree_aggregate_function group by k order by k;
optimize table test.summing_merge_tree_aggregate_function;
select k, sum(c), uniqMerge(u) from test.summing_merge_tree_aggregate_function group by k order by k;
select k, sum(c), uniqMerge(u) from summing_merge_tree_aggregate_function group by k order by k;
optimize table summing_merge_tree_aggregate_function;
select k, sum(c), uniqMerge(u) from summing_merge_tree_aggregate_function group by k order by k;
drop table test.summing_merge_tree_aggregate_function;
drop table test.summing_merge_tree_null;
drop table summing_merge_tree_aggregate_function;
drop table summing_merge_tree_null;

View File

@ -1,20 +1,20 @@
drop table if exists test.nested_map_multiple_values;
drop table if exists nested_map_multiple_values;
create table test.nested_map_multiple_values (d materialized today(), k UInt64, payload materialized rand(), SomeMap Nested(ID UInt32, Num1 Int64, Num2 Float64)) engine=SummingMergeTree(d, k, 8192);
create table nested_map_multiple_values (d materialized today(), k UInt64, payload materialized rand(), SomeMap Nested(ID UInt32, Num1 Int64, Num2 Float64)) engine=SummingMergeTree(d, k, 8192);
insert into test.nested_map_multiple_values values (0,[1],[100],[1.0]),(1,[1],[100],[1.0]),(2,[1],[100],[1.0]),(3,[1,2],[100,150],[1.0,1.5]);
insert into test.nested_map_multiple_values values (0,[2],[150],[-2.5]),(1,[1],[150],[-1.0]),(2,[1,2],[150,150],[2.5,3.5]),(3,[1],[-100],[-1]);
optimize table test.nested_map_multiple_values;
select * from test.nested_map_multiple_values;
insert into nested_map_multiple_values values (0,[1],[100],[1.0]),(1,[1],[100],[1.0]),(2,[1],[100],[1.0]),(3,[1,2],[100,150],[1.0,1.5]);
insert into nested_map_multiple_values values (0,[2],[150],[-2.5]),(1,[1],[150],[-1.0]),(2,[1,2],[150,150],[2.5,3.5]),(3,[1],[-100],[-1]);
optimize table nested_map_multiple_values;
select * from nested_map_multiple_values;
drop table test.nested_map_multiple_values;
drop table nested_map_multiple_values;
drop table if exists test.nested_not_a_map;
create table test.nested_not_a_map (d materialized today(), k UInt64, payload materialized rand(), OnlyOneColumnMap Nested(ID UInt32), NonArithmeticValueMap Nested(ID UInt32, Date Date), Nested_ Nested(ID UInt32, Num Int64)) engine=SummingMergeTree(d, k, 8192);
drop table if exists nested_not_a_map;
create table nested_not_a_map (d materialized today(), k UInt64, payload materialized rand(), OnlyOneColumnMap Nested(ID UInt32), NonArithmeticValueMap Nested(ID UInt32, Date Date), Nested_ Nested(ID UInt32, Num Int64)) engine=SummingMergeTree(d, k, 8192);
insert into test.nested_not_a_map values (0,[1],[1],['2015-04-09'],[1],[100]);
insert into test.nested_not_a_map values (0,[1],[1],['2015-04-08'],[1],[200]);
optimize table test.nested_not_a_map;
select * from test.nested_not_a_map;
insert into nested_not_a_map values (0,[1],[1],['2015-04-09'],[1],[100]);
insert into nested_not_a_map values (0,[1],[1],['2015-04-08'],[1],[200]);
optimize table nested_not_a_map;
select * from nested_not_a_map;
drop table test.nested_not_a_map;
drop table nested_not_a_map;

View File

@ -4,23 +4,23 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
function create {
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.summing"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.collapsing"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.aggregating"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.replacing"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS summing"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS collapsing"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS aggregating"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS replacing"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.summing (d Date DEFAULT today(), x UInt64, s UInt64 DEFAULT 1) ENGINE = SummingMergeTree(d, x, 8192)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.collapsing (d Date DEFAULT today(), x UInt64, s Int8 DEFAULT 1) ENGINE = CollapsingMergeTree(d, x, 8192, s)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.aggregating (d Date DEFAULT today(), x UInt64, s AggregateFunction(sum, UInt64)) ENGINE = AggregatingMergeTree(d, x, 8192)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.replacing (d Date DEFAULT today(), x UInt64, s Int8 DEFAULT 1, v UInt64) ENGINE = ReplacingMergeTree(d, (x), 8192, v)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE summing (d Date DEFAULT today(), x UInt64, s UInt64 DEFAULT 1) ENGINE = SummingMergeTree(d, x, 8192)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE collapsing (d Date DEFAULT today(), x UInt64, s Int8 DEFAULT 1) ENGINE = CollapsingMergeTree(d, x, 8192, s)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE aggregating (d Date DEFAULT today(), x UInt64, s AggregateFunction(sum, UInt64)) ENGINE = AggregatingMergeTree(d, x, 8192)"
$CLICKHOUSE_CLIENT --query="CREATE TABLE replacing (d Date DEFAULT today(), x UInt64, s Int8 DEFAULT 1, v UInt64) ENGINE = ReplacingMergeTree(d, (x), 8192, v)"
}
function cleanup {
$CLICKHOUSE_CLIENT --query="DROP TABLE test.summing"
$CLICKHOUSE_CLIENT --query="DROP TABLE test.collapsing"
$CLICKHOUSE_CLIENT --query="DROP TABLE test.aggregating"
$CLICKHOUSE_CLIENT --query="DROP TABLE test.replacing"
$CLICKHOUSE_CLIENT --query="DROP TABLE summing"
$CLICKHOUSE_CLIENT --query="DROP TABLE collapsing"
$CLICKHOUSE_CLIENT --query="DROP TABLE aggregating"
$CLICKHOUSE_CLIENT --query="DROP TABLE replacing"
}
@ -32,36 +32,36 @@ function test {
SETTINGS="--min_insert_block_size_rows=0 --min_insert_block_size_bytes=0"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO test.summing (x) SELECT number AS x FROM system.numbers LIMIT $1"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO test.summing (x) SELECT number AS x FROM system.numbers LIMIT $2"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO summing (x) SELECT number AS x FROM system.numbers LIMIT $1"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO summing (x) SELECT number AS x FROM system.numbers LIMIT $2"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO test.collapsing (x) SELECT number AS x FROM system.numbers LIMIT $1"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO test.collapsing (x) SELECT number AS x FROM system.numbers LIMIT $2"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO collapsing (x) SELECT number AS x FROM system.numbers LIMIT $1"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO collapsing (x) SELECT number AS x FROM system.numbers LIMIT $2"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO test.aggregating (d, x, s) SELECT today() AS d, number AS x, sumState(materialize(toUInt64(1))) AS s FROM (SELECT number FROM system.numbers LIMIT $1) GROUP BY number"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO test.aggregating (d, x, s) SELECT today() AS d, number AS x, sumState(materialize(toUInt64(1))) AS s FROM (SELECT number FROM system.numbers LIMIT $2) GROUP BY number"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO aggregating (d, x, s) SELECT today() AS d, number AS x, sumState(materialize(toUInt64(1))) AS s FROM (SELECT number FROM system.numbers LIMIT $1) GROUP BY number"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO aggregating (d, x, s) SELECT today() AS d, number AS x, sumState(materialize(toUInt64(1))) AS s FROM (SELECT number FROM system.numbers LIMIT $2) GROUP BY number"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO test.replacing (x, v) SELECT number AS x, toUInt64(number % 3 == 0) FROM system.numbers LIMIT $1"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO test.replacing (x, v) SELECT number AS x, toUInt64(number % 3 == 1) FROM system.numbers LIMIT $2"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO replacing (x, v) SELECT number AS x, toUInt64(number % 3 == 0) FROM system.numbers LIMIT $1"
$CLICKHOUSE_CLIENT $SETTINGS --query="INSERT INTO replacing (x, v) SELECT number AS x, toUInt64(number % 3 == 1) FROM system.numbers LIMIT $2"
$CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sum(s) = $SUM FROM test.summing"
$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test.summing"
$CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sum(s) = $SUM FROM test.summing"
$CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sum(s) = $SUM FROM summing"
$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE summing"
$CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sum(s) = $SUM FROM summing"
echo
$CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sum(s) = $SUM FROM test.collapsing"
$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test.collapsing" --server_logs_file='/dev/null';
$CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sum(s) = $MAX FROM test.collapsing"
$CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sum(s) = $SUM FROM collapsing"
$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE collapsing" --server_logs_file='/dev/null';
$CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sum(s) = $MAX FROM collapsing"
echo
$CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sumMerge(s) = $SUM FROM test.aggregating"
$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test.aggregating"
$CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sumMerge(s) = $SUM FROM test.aggregating"
$CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sumMerge(s) = $SUM FROM aggregating"
$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE aggregating"
$CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sumMerge(s) = $SUM FROM aggregating"
echo
$CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sum(s) = $SUM FROM test.replacing"
$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test.replacing"
$CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sum(s) = $MAX FROM test.replacing"
$CLICKHOUSE_CLIENT --query="SELECT count() = sum(v) FROM test.replacing where x % 3 == 0 and x < $1"
$CLICKHOUSE_CLIENT --query="SELECT count() = sum(v) FROM test.replacing where x % 3 == 1 and x < $2"
$CLICKHOUSE_CLIENT --query="SELECT sum(v) = 0 FROM test.replacing where x % 3 == 2"
$CLICKHOUSE_CLIENT --query="SELECT count() = $SUM, sum(s) = $SUM FROM replacing"
$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE replacing"
$CLICKHOUSE_CLIENT --query="SELECT count() = $MAX, sum(s) = $MAX FROM replacing"
$CLICKHOUSE_CLIENT --query="SELECT count() = sum(v) FROM replacing where x % 3 == 0 and x < $1"
$CLICKHOUSE_CLIENT --query="SELECT count() = sum(v) FROM replacing where x % 3 == 1 and x < $2"
$CLICKHOUSE_CLIENT --query="SELECT sum(v) = 0 FROM replacing where x % 3 == 2"
echo
echo
}

View File

@ -1,7 +1,9 @@
CREATE DATABASE IF NOT EXISTS test2;
DROP TABLE IF EXISTS test2.mt_buffer;
CREATE TABLE test2.mt_buffer (d Date DEFAULT today(), x UInt64) ENGINE = Buffer(test2, mt, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000);
CREATE DATABASE IF NOT EXISTS test2_00158;
DROP TABLE IF EXISTS test2_00158.mt_buffer_00158;
DROP TABLE IF EXISTS test2_00158.mt_00158;
CREATE TABLE test2_00158.mt_buffer_00158 (d Date DEFAULT today(), x UInt64) ENGINE = Buffer(test2_00158, mt_00158, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000);
SET send_logs_level = 'none'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded."
INSERT INTO test2.mt_buffer (x) SELECT number AS x FROM system.numbers LIMIT 100000;
INSERT INTO test2.mt_buffer (x) SELECT number AS x FROM system.numbers LIMIT 1000000;
DROP DATABASE test2;
INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 100000;
INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 1000000;
DROP TABLE IF EXISTS test2_00158.mt_buffer_00158;
DROP DATABASE test2_00158;

View File

@ -1,13 +1,13 @@
DROP TABLE IF EXISTS test.memory;
CREATE TABLE test.memory (x UInt8) ENGINE = Memory;
DROP TABLE IF EXISTS memory;
CREATE TABLE memory (x UInt8) ENGINE = Memory;
INSERT INTO test.memory VALUES (1);
INSERT INTO test.memory (x) VALUES (2);
INSERT INTO test.memory ( x) VALUES (3);
INSERT INTO test.memory (x ) VALUES (4);
INSERT INTO test.memory ( x ) VALUES (5);
INSERT INTO test.memory(x)VALUES(6);
INSERT INTO memory VALUES (1);
INSERT INTO memory (x) VALUES (2);
INSERT INTO memory ( x) VALUES (3);
INSERT INTO memory (x ) VALUES (4);
INSERT INTO memory ( x ) VALUES (5);
INSERT INTO memory(x)VALUES(6);
SELECT * FROM test.memory ORDER BY x;
SELECT * FROM memory ORDER BY x;
DROP TABLE test.memory;
DROP TABLE memory;

View File

@ -3,12 +3,12 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=DROP+TABLE" -d 'IF EXISTS test.insert'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=CREATE" -d 'TABLE test.insert (x UInt8) ENGINE = Memory'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'INSERT INTO test.insert VALUES (1),(2)'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+test.insert+VALUES" -d '(3),(4)'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+test.insert" -d 'VALUES (5),(6)'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+test.insert+VALUES+(7)" -d ',(8)'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+test.insert+VALUES+(9),(10)" -d ' '
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT x FROM test.insert ORDER BY x'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=DROP+TABLE" -d 'test.insert'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=DROP+TABLE" -d 'IF EXISTS insert'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=CREATE" -d 'TABLE insert (x UInt8) ENGINE = Memory'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'INSERT INTO insert VALUES (1),(2)'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+insert+VALUES" -d '(3),(4)'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+insert" -d 'VALUES (5),(6)'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+insert+VALUES+(7)" -d ',(8)'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+insert+VALUES+(9),(10)" -d ' '
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT x FROM insert ORDER BY x'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=DROP+TABLE" -d 'insert'

View File

@ -1,12 +1,12 @@
DROP TABLE IF EXISTS test.t;
DROP TABLE IF EXISTS test.mv;
DROP TABLE IF EXISTS test.`.inner.mv`;
DROP TABLE IF EXISTS t;
DROP TABLE IF EXISTS mv;
DROP TABLE IF EXISTS `.inner.mv`;
CREATE TABLE test.t (x UInt8) ENGINE = Null;
CREATE MATERIALIZED VIEW test.mv ENGINE = Null AS SELECT * FROM test.t;
CREATE TABLE t (x UInt8) ENGINE = Null;
CREATE MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM t;
DETACH TABLE test.mv;
ATTACH MATERIALIZED VIEW test.mv ENGINE = Null AS SELECT * FROM test.t;
DETACH TABLE mv;
ATTACH MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM t;
DROP TABLE test.t;
DROP TABLE test.mv;
DROP TABLE t;
DROP TABLE mv;

View File

@ -1,65 +1,65 @@
DROP TABLE IF EXISTS test.series;
DROP TABLE IF EXISTS series;
CREATE TABLE test.series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;
CREATE TABLE series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;
INSERT INTO test.series(i, x_value, y_value) VALUES (1, 5.6,-4.4),(2, -9.6,3),(3, -1.3,-4),(4, 5.3,9.7),(5, 4.4,0.037),(6, -8.6,-7.8),(7, 5.1,9.3),(8, 7.9,-3.6),(9, -8.2,0.62),(10, -3,7.3);
INSERT INTO series(i, x_value, y_value) VALUES (1, 5.6,-4.4),(2, -9.6,3),(3, -1.3,-4),(4, 5.3,9.7),(5, 4.4,0.037),(6, -8.6,-7.8),(7, 5.1,9.3),(8, 7.9,-3.6),(9, -8.2,0.62),(10, -3,7.3);
/* varSamp */
SELECT varSamp(x_value) FROM (SELECT x_value FROM test.series LIMIT 0);
SELECT varSamp(x_value) FROM (SELECT x_value FROM test.series LIMIT 1);
SELECT varSamp(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT varSamp(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
varSamp(x_value) AS res1,
(sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1) AS res2
FROM test.series
FROM series
);
/* stddevSamp */
SELECT stddevSamp(x_value) FROM (SELECT x_value FROM test.series LIMIT 0);
SELECT stddevSamp(x_value) FROM (SELECT x_value FROM test.series LIMIT 1);
SELECT stddevSamp(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT stddevSamp(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
stddevSamp(x_value) AS res1,
sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1)) AS res2
FROM test.series
FROM series
);
/* varPop */
SELECT varPop(x_value) FROM (SELECT x_value FROM test.series LIMIT 0);
SELECT varPop(x_value) FROM (SELECT x_value FROM test.series LIMIT 1);
SELECT varPop(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT varPop(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
varPop(x_value) AS res1,
(sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count() AS res2
FROM test.series
FROM series
);
/* stddevPop */
SELECT stddevPop(x_value) FROM (SELECT x_value FROM test.series LIMIT 0);
SELECT stddevPop(x_value) FROM (SELECT x_value FROM test.series LIMIT 1);
SELECT stddevPop(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT stddevPop(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
stddevPop(x_value) AS res1,
sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count()) AS res2
FROM test.series
FROM series
);
/* covarSamp */
SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 0);
SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 1);
SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
SELECT covarSamp(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
SELECT round(abs(COVAR1 - COVAR2), 6)
FROM
@ -67,7 +67,7 @@ FROM
SELECT
arrayJoin([1]) AS ID2,
covarSamp(x_value, y_value) AS COVAR1
FROM test.series
FROM series
) ANY INNER JOIN
(
SELECT
@ -82,22 +82,22 @@ FROM
toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID,
avg(x_value) AS AVG_X,
avg(y_value) AS AVG_Y
FROM test.series
FROM series
) ANY INNER JOIN
(
SELECT
i AS ID,
x_value AS X,
y_value AS Y
FROM test.series
FROM series
) USING ID
)
) USING ID2;
/* covarPop */
SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 0);
SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 1);
SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
SELECT covarPop(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
SELECT round(abs(COVAR1 - COVAR2), 6)
FROM
@ -105,7 +105,7 @@ FROM
SELECT
arrayJoin([1]) AS ID2,
covarPop(x_value, y_value) AS COVAR1
FROM test.series
FROM series
) ANY INNER JOIN
(
SELECT
@ -120,24 +120,24 @@ FROM
toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID,
avg(x_value) AS AVG_X,
avg(y_value) AS AVG_Y
FROM test.series
FROM series
) ANY INNER JOIN
(
SELECT
i AS ID,
x_value AS X,
y_value AS Y
FROM test.series
FROM series
) USING ID
)
) USING ID2;
/* corr */
SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 0);
SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 1);
SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
SELECT corr(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
SELECT round(abs(corr(x_value, y_value) - covarPop(x_value, y_value) / (stddevPop(x_value) * stddevPop(y_value))), 6) FROM test.series;
SELECT round(abs(corr(x_value, y_value) - covarPop(x_value, y_value) / (stddevPop(x_value) * stddevPop(y_value))), 6) FROM series;
/* quantile AND quantileExact */
SELECT '----quantile----';
@ -152,4 +152,4 @@ SELECT quantileIf(number, number > 100) FROM numbers(90);
SELECT quantileIf(toFloat32(number) , number > 100) FROM numbers(90);
SELECT quantileIf(toFloat64(number) , number > 100) FROM numbers(90);
DROP TABLE test.series;
DROP TABLE series;

View File

@ -1,65 +1,65 @@
DROP TABLE IF EXISTS test.series;
DROP TABLE IF EXISTS series;
CREATE TABLE test.series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;
CREATE TABLE series(i UInt32, x_value Float64, y_value Float64) ENGINE = Memory;
INSERT INTO test.series(i, x_value, y_value) VALUES (1, 5.6,-4.4),(2, -9.6,3),(3, -1.3,-4),(4, 5.3,9.7),(5, 4.4,0.037),(6, -8.6,-7.8),(7, 5.1,9.3),(8, 7.9,-3.6),(9, -8.2,0.62),(10, -3,7.3);
INSERT INTO series(i, x_value, y_value) VALUES (1, 5.6,-4.4),(2, -9.6,3),(3, -1.3,-4),(4, 5.3,9.7),(5, 4.4,0.037),(6, -8.6,-7.8),(7, 5.1,9.3),(8, 7.9,-3.6),(9, -8.2,0.62),(10, -3,7.3);
/* varSampStable */
SELECT varSampStable(x_value) FROM (SELECT x_value FROM test.series LIMIT 0);
SELECT varSampStable(x_value) FROM (SELECT x_value FROM test.series LIMIT 1);
SELECT varSampStable(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT varSampStable(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
varSampStable(x_value) AS res1,
(sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1) AS res2
FROM test.series
FROM series
);
/* stddevSampStable */
SELECT stddevSampStable(x_value) FROM (SELECT x_value FROM test.series LIMIT 0);
SELECT stddevSampStable(x_value) FROM (SELECT x_value FROM test.series LIMIT 1);
SELECT stddevSampStable(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT stddevSampStable(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
stddevSampStable(x_value) AS res1,
sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / (count() - 1)) AS res2
FROM test.series
FROM series
);
/* varPopStable */
SELECT varPopStable(x_value) FROM (SELECT x_value FROM test.series LIMIT 0);
SELECT varPopStable(x_value) FROM (SELECT x_value FROM test.series LIMIT 1);
SELECT varPopStable(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT varPopStable(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
varPopStable(x_value) AS res1,
(sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count() AS res2
FROM test.series
FROM series
);
/* stddevPopStable */
SELECT stddevPopStable(x_value) FROM (SELECT x_value FROM test.series LIMIT 0);
SELECT stddevPopStable(x_value) FROM (SELECT x_value FROM test.series LIMIT 1);
SELECT stddevPopStable(x_value) FROM (SELECT x_value FROM series LIMIT 0);
SELECT stddevPopStable(x_value) FROM (SELECT x_value FROM series LIMIT 1);
SELECT round(abs(res1 - res2), 6) FROM
(
SELECT
stddevPopStable(x_value) AS res1,
sqrt((sum(x_value * x_value) - ((sum(x_value) * sum(x_value)) / count())) / count()) AS res2
FROM test.series
FROM series
);
/* covarSampStable */
SELECT covarSampStable(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 0);
SELECT covarSampStable(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 1);
SELECT covarSampStable(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
SELECT covarSampStable(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
SELECT round(abs(COVAR1 - COVAR2), 6)
FROM
@ -67,7 +67,7 @@ FROM
SELECT
arrayJoin([1]) AS ID2,
covarSampStable(x_value, y_value) AS COVAR1
FROM test.series
FROM series
) ANY INNER JOIN
(
SELECT
@ -82,22 +82,22 @@ FROM
toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID,
avg(x_value) AS AVG_X,
avg(y_value) AS AVG_Y
FROM test.series
FROM series
) ANY INNER JOIN
(
SELECT
i AS ID,
x_value AS X,
y_value AS Y
FROM test.series
FROM series
) USING ID
)
) USING ID2;
/* covarPopStable */
SELECT covarPopStable(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 0);
SELECT covarPopStable(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 1);
SELECT covarPopStable(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
SELECT covarPopStable(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
SELECT round(abs(COVAR1 - COVAR2), 6)
FROM
@ -105,7 +105,7 @@ FROM
SELECT
arrayJoin([1]) AS ID2,
covarPopStable(x_value, y_value) AS COVAR1
FROM test.series
FROM series
) ANY INNER JOIN
(
SELECT
@ -120,23 +120,23 @@ FROM
toUInt32(arrayJoin([1, 2, 3, 4, 5, 6, 7, 8, 9, 10])) AS ID,
avg(x_value) AS AVG_X,
avg(y_value) AS AVG_Y
FROM test.series
FROM series
) ANY INNER JOIN
(
SELECT
i AS ID,
x_value AS X,
y_value AS Y
FROM test.series
FROM series
) USING ID
)
) USING ID2;
/* corr */
SELECT corrStable(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 0);
SELECT corrStable(x_value, y_value) FROM (SELECT x_value, y_value FROM test.series LIMIT 1);
SELECT corrStable(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 0);
SELECT corrStable(x_value, y_value) FROM (SELECT x_value, y_value FROM series LIMIT 1);
SELECT round(abs(corrStable(x_value, y_value) - covarPopStable(x_value, y_value) / (stddevPopStable(x_value) * stddevPopStable(y_value))), 6) FROM test.series;
SELECT round(abs(corrStable(x_value, y_value) - covarPopStable(x_value, y_value) / (stddevPopStable(x_value) * stddevPopStable(y_value))), 6) FROM series;
DROP TABLE test.series;
DROP TABLE series;

View File

@ -1,14 +1,14 @@
DROP TABLE IF EXISTS test.aggregating;
CREATE TABLE test.aggregating (d Date DEFAULT '2000-01-01', k UInt64, u AggregateFunction(uniq, UInt64)) ENGINE = AggregatingMergeTree(d, k, 8192);
DROP TABLE IF EXISTS aggregating;
CREATE TABLE aggregating (d Date DEFAULT '2000-01-01', k UInt64, u AggregateFunction(uniq, UInt64)) ENGINE = AggregatingMergeTree(d, k, 8192);
INSERT INTO test.aggregating (k, u) SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000) GROUP BY k;
INSERT INTO test.aggregating (k, u) SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100) + 50) AS u FROM (SELECT * FROM system.numbers LIMIT 500, 1000) GROUP BY k;
INSERT INTO aggregating (k, u) SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100)) AS u FROM (SELECT * FROM system.numbers LIMIT 1000) GROUP BY k;
INSERT INTO aggregating (k, u) SELECT intDiv(number, 100) AS k, uniqState(toUInt64(number % 100) + 50) AS u FROM (SELECT * FROM system.numbers LIMIT 500, 1000) GROUP BY k;
SELECT k, finalizeAggregation(u) FROM test.aggregating FINAL;
SELECT k, finalizeAggregation(u) FROM aggregating FINAL;
OPTIMIZE TABLE test.aggregating;
OPTIMIZE TABLE aggregating;
SELECT k, finalizeAggregation(u) FROM test.aggregating;
SELECT k, finalizeAggregation(u) FROM test.aggregating FINAL;
SELECT k, finalizeAggregation(u) FROM aggregating;
SELECT k, finalizeAggregation(u) FROM aggregating FINAL;
DROP TABLE test.aggregating;
DROP TABLE aggregating;

View File

@ -1,58 +1,58 @@
DROP TABLE IF EXISTS test.parallel_replicas;
DROP TABLE IF EXISTS test.parallel_replicas_backup;
DROP TABLE IF EXISTS parallel_replicas;
DROP TABLE IF EXISTS parallel_replicas_backup;
CREATE TABLE test.parallel_replicas (d Date DEFAULT today(), x UInt32, u UInt64, s String) ENGINE = MergeTree(d, cityHash64(u, s), (x, d, cityHash64(u, s)), 8192);
INSERT INTO test.parallel_replicas (x, u, s) VALUES (1, 2, 'A'),(3, 4, 'B'),(5, 6, 'C'),(7, 8, 'D'),(9,10,'E');
INSERT INTO test.parallel_replicas (x, u, s) VALUES (11, 12, 'F'),(13, 14, 'G'),(15, 16, 'H'),(17, 18, 'I'),(19,20,'J');
INSERT INTO test.parallel_replicas (x, u, s) VALUES (21, 22, 'K'),(23, 24, 'L'),(25, 26, 'M'),(27, 28, 'N'),(29,30,'O');
INSERT INTO test.parallel_replicas (x, u, s) VALUES (31, 32, 'P'),(33, 34, 'Q'),(35, 36, 'R'),(37, 38, 'S'),(39,40,'T');
INSERT INTO test.parallel_replicas (x, u, s) VALUES (41, 42, 'U'),(43, 44, 'V'),(45, 46, 'W'),(47, 48, 'X'),(49,50,'Y');
INSERT INTO test.parallel_replicas (x, u, s) VALUES (51, 52, 'Z');
CREATE TABLE parallel_replicas (d Date DEFAULT today(), x UInt32, u UInt64, s String) ENGINE = MergeTree(d, cityHash64(u, s), (x, d, cityHash64(u, s)), 8192);
INSERT INTO parallel_replicas (x, u, s) VALUES (1, 2, 'A'),(3, 4, 'B'),(5, 6, 'C'),(7, 8, 'D'),(9,10,'E');
INSERT INTO parallel_replicas (x, u, s) VALUES (11, 12, 'F'),(13, 14, 'G'),(15, 16, 'H'),(17, 18, 'I'),(19,20,'J');
INSERT INTO parallel_replicas (x, u, s) VALUES (21, 22, 'K'),(23, 24, 'L'),(25, 26, 'M'),(27, 28, 'N'),(29,30,'O');
INSERT INTO parallel_replicas (x, u, s) VALUES (31, 32, 'P'),(33, 34, 'Q'),(35, 36, 'R'),(37, 38, 'S'),(39,40,'T');
INSERT INTO parallel_replicas (x, u, s) VALUES (41, 42, 'U'),(43, 44, 'V'),(45, 46, 'W'),(47, 48, 'X'),(49,50,'Y');
INSERT INTO parallel_replicas (x, u, s) VALUES (51, 52, 'Z');
/*
* Проверяем, что:
* - на каждой реплике таблица не пустая;
* - объединение данных всех реплик совпадает с содержанием таблицы test.parallel_replicas.
* - объединение данных всех реплик совпадает с содержанием таблицы parallel_replicas.
*/
/* Две реплики */
CREATE TABLE test.parallel_replicas_backup(d Date DEFAULT today(), x UInt32, u UInt64, s String) ENGINE = Memory;
CREATE TABLE parallel_replicas_backup(d Date DEFAULT today(), x UInt32, u UInt64, s String) ENGINE = Memory;
SET parallel_replicas_count = 2;
SET parallel_replica_offset = 0;
INSERT INTO test.parallel_replicas_backup(d, x, u, s) SELECT d, x, u, s FROM test.parallel_replicas;
SELECT count() > 0 FROM test.parallel_replicas;
INSERT INTO parallel_replicas_backup(d, x, u, s) SELECT d, x, u, s FROM parallel_replicas;
SELECT count() > 0 FROM parallel_replicas;
SET parallel_replica_offset = 1;
INSERT INTO test.parallel_replicas_backup(d, x, u, s) SELECT d, x, u, s FROM test.parallel_replicas;
SELECT count() > 0 FROM test.parallel_replicas;
INSERT INTO parallel_replicas_backup(d, x, u, s) SELECT d, x, u, s FROM parallel_replicas;
SELECT count() > 0 FROM parallel_replicas;
SET parallel_replicas_count = 0;
SELECT x, u, s FROM test.parallel_replicas_backup ORDER BY x, u, s ASC;
SELECT x, u, s FROM parallel_replicas_backup ORDER BY x, u, s ASC;
DROP TABLE test.parallel_replicas_backup;
CREATE TABLE test.parallel_replicas_backup(d Date DEFAULT today(), x UInt32, u UInt64, s String) ENGINE = Memory;
DROP TABLE parallel_replicas_backup;
CREATE TABLE parallel_replicas_backup(d Date DEFAULT today(), x UInt32, u UInt64, s String) ENGINE = Memory;
/* Три реплики */
SET parallel_replicas_count = 3;
SET parallel_replica_offset = 0;
INSERT INTO test.parallel_replicas_backup(d, x, u, s) SELECT d, x, u, s FROM test.parallel_replicas;
SELECT count() > 0 FROM test.parallel_replicas;
INSERT INTO parallel_replicas_backup(d, x, u, s) SELECT d, x, u, s FROM parallel_replicas;
SELECT count() > 0 FROM parallel_replicas;
SET parallel_replica_offset = 1;
INSERT INTO test.parallel_replicas_backup(d, x, u, s) SELECT d, x, u, s FROM test.parallel_replicas;
SELECT count() > 0 FROM test.parallel_replicas;
INSERT INTO parallel_replicas_backup(d, x, u, s) SELECT d, x, u, s FROM parallel_replicas;
SELECT count() > 0 FROM parallel_replicas;
SET parallel_replica_offset = 2;
INSERT INTO test.parallel_replicas_backup(d, x, u, s) SELECT d, x, u, s FROM test.parallel_replicas;
SELECT count() > 0 FROM test.parallel_replicas;
INSERT INTO parallel_replicas_backup(d, x, u, s) SELECT d, x, u, s FROM parallel_replicas;
SELECT count() > 0 FROM parallel_replicas;
SET parallel_replicas_count = 0;
SELECT x, u, s FROM test.parallel_replicas_backup ORDER BY x, u, s ASC;
SELECT x, u, s FROM parallel_replicas_backup ORDER BY x, u, s ASC;
DROP TABLE test.parallel_replicas;
DROP TABLE test.parallel_replicas_backup;
DROP TABLE parallel_replicas;
DROP TABLE parallel_replicas_backup;

View File

@ -1 +1 @@
SELECT extractURLParameter('http://test.com/?testq=aaa&q=111', 'q');
SELECT extractURLParameter('http://com/?testq=aaa&q=111', 'q');

View File

@ -1,11 +1,11 @@
DROP TABLE IF EXISTS test.test;
CREATE TABLE test.test (x UInt8) ENGINE = Log;
DROP TABLE IF EXISTS test;
CREATE TABLE test (x UInt8) ENGINE = Log;
INSERT INTO test.test SELECT 1 AS x;
INSERT INTO test.test SELECT 1 AS x SETTINGS extremes = 1;
INSERT INTO test.test SELECT 1 AS x GROUP BY 1 WITH TOTALS;
INSERT INTO test.test SELECT 1 AS x GROUP BY 1 WITH TOTALS SETTINGS extremes = 1;
INSERT INTO test SELECT 1 AS x;
INSERT INTO test SELECT 1 AS x SETTINGS extremes = 1;
INSERT INTO test SELECT 1 AS x GROUP BY 1 WITH TOTALS;
INSERT INTO test SELECT 1 AS x GROUP BY 1 WITH TOTALS SETTINGS extremes = 1;
SELECT count(), min(x), max(x) FROM test.test;
SELECT count(), min(x), max(x) FROM test;
DROP TABLE test.test;
DROP TABLE test;

View File

@ -3,7 +3,7 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
${CLICKHOUSE_CURL} -sS ${CLICKHOUSE_URL}?extremes=1 -d @- <<< "DROP TABLE IF EXISTS test.test"
${CLICKHOUSE_CURL} -sS ${CLICKHOUSE_URL}?extremes=1 -d @- <<< "CREATE TABLE test.test (x UInt8) ENGINE = Log"
${CLICKHOUSE_CURL} -sS ${CLICKHOUSE_URL}?extremes=1 -d @- <<< "INSERT INTO test.test SELECT 1 AS x"
${CLICKHOUSE_CURL} -sS ${CLICKHOUSE_URL}?extremes=1 -d @- <<< "DROP TABLE test.test"
${CLICKHOUSE_CURL} -sS ${CLICKHOUSE_URL}?extremes=1 -d @- <<< "DROP TABLE IF EXISTS test"
${CLICKHOUSE_CURL} -sS ${CLICKHOUSE_URL}?extremes=1 -d @- <<< "CREATE TABLE test (x UInt8) ENGINE = Log"
${CLICKHOUSE_CURL} -sS ${CLICKHOUSE_URL}?extremes=1 -d @- <<< "INSERT INTO test SELECT 1 AS x"
${CLICKHOUSE_CURL} -sS ${CLICKHOUSE_URL}?extremes=1 -d @- <<< "DROP TABLE test"

View File

@ -1,13 +1,13 @@
DROP TABLE IF EXISTS test.primary_key;
CREATE TABLE test.primary_key (d Date DEFAULT today(), x Int8) ENGINE = MergeTree(d, -x, 1);
DROP TABLE IF EXISTS primary_key;
CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = MergeTree(d, -x, 1);
INSERT INTO test.primary_key (x) VALUES (1), (2), (3);
INSERT INTO primary_key (x) VALUES (1), (2), (3);
SELECT x FROM test.primary_key ORDER BY x;
SELECT x FROM primary_key ORDER BY x;
SELECT 'a', -x FROM test.primary_key WHERE -x < -3;
SELECT 'b', -x FROM test.primary_key WHERE -x < -2;
SELECT 'c', -x FROM test.primary_key WHERE -x < -1;
SELECT 'd', -x FROM test.primary_key WHERE -x < toInt8(0);
SELECT 'a', -x FROM primary_key WHERE -x < -3;
SELECT 'b', -x FROM primary_key WHERE -x < -2;
SELECT 'c', -x FROM primary_key WHERE -x < -1;
SELECT 'd', -x FROM primary_key WHERE -x < toInt8(0);
DROP TABLE test.primary_key;
DROP TABLE primary_key;

View File

@ -1,14 +1,14 @@
DROP TABLE IF EXISTS test.primary_key;
CREATE TABLE test.primary_key (d Date DEFAULT today(), x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/primary_key', 'r1', d, -x, 1);
DROP TABLE IF EXISTS primary_key;
CREATE TABLE primary_key (d Date DEFAULT today(), x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/primary_key', 'r1', d, -x, 1);
INSERT INTO test.primary_key (x) VALUES (1), (2), (3);
INSERT INTO test.primary_key (x) VALUES (1), (3), (2);
INSERT INTO test.primary_key (x) VALUES (2), (1), (3);
INSERT INTO test.primary_key (x) VALUES (2), (3), (1);
INSERT INTO test.primary_key (x) VALUES (3), (1), (2);
INSERT INTO test.primary_key (x) VALUES (3), (2), (1);
INSERT INTO primary_key (x) VALUES (1), (2), (3);
INSERT INTO primary_key (x) VALUES (1), (3), (2);
INSERT INTO primary_key (x) VALUES (2), (1), (3);
INSERT INTO primary_key (x) VALUES (2), (3), (1);
INSERT INTO primary_key (x) VALUES (3), (1), (2);
INSERT INTO primary_key (x) VALUES (3), (2), (1);
SELECT x FROM test.primary_key ORDER BY x;
SELECT x FROM test.primary_key WHERE -x < -1 ORDER BY x;
SELECT x FROM primary_key ORDER BY x;
SELECT x FROM primary_key WHERE -x < -1 ORDER BY x;
DROP TABLE test.primary_key;
DROP TABLE primary_key;

View File

@ -1,28 +1,28 @@
DROP TABLE IF EXISTS test.deduplication;
CREATE TABLE test.deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1);
DROP TABLE IF EXISTS deduplication;
CREATE TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO test.deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
INSERT INTO deduplication (x) VALUES (1);
SELECT * FROM test.deduplication;
SELECT * FROM deduplication;
DETACH TABLE test.deduplication;
ATTACH TABLE test.deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1);
DETACH TABLE deduplication;
ATTACH TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1);
SELECT * FROM test.deduplication;
SELECT * FROM deduplication;
DROP TABLE test.deduplication;
DROP TABLE deduplication;

View File

@ -1,29 +1,29 @@
drop table if exists test.prewhere_column_missing;
drop table if exists prewhere_column_missing;
create table test.prewhere_column_missing (d Date default '2015-01-01', x UInt64) engine=MergeTree(d, x, 1);
create table prewhere_column_missing (d Date default '2015-01-01', x UInt64) engine=MergeTree(d, x, 1);
insert into test.prewhere_column_missing (x) values (0);
select * from test.prewhere_column_missing;
insert into prewhere_column_missing (x) values (0);
select * from prewhere_column_missing;
alter table test.prewhere_column_missing add column arr Array(UInt64);
select * from test.prewhere_column_missing;
alter table prewhere_column_missing add column arr Array(UInt64);
select * from prewhere_column_missing;
select *, arraySum(arr) as s from test.prewhere_column_missing;
select *, arraySum(arr) as s from test.prewhere_column_missing where s = 0;
select *, arraySum(arr) as s from test.prewhere_column_missing prewhere s = 0;
select *, arraySum(arr) as s from prewhere_column_missing;
select *, arraySum(arr) as s from prewhere_column_missing where s = 0;
select *, arraySum(arr) as s from prewhere_column_missing prewhere s = 0;
select *, length(arr) as l from test.prewhere_column_missing;
select *, length(arr) as l from test.prewhere_column_missing where l = 0;
select *, length(arr) as l from test.prewhere_column_missing prewhere l = 0;
select *, length(arr) as l from prewhere_column_missing;
select *, length(arr) as l from prewhere_column_missing where l = 0;
select *, length(arr) as l from prewhere_column_missing prewhere l = 0;
alter table test.prewhere_column_missing add column hash_x default intHash64(x);
alter table prewhere_column_missing add column hash_x default intHash64(x);
select * from test.prewhere_column_missing;
select * from test.prewhere_column_missing where hash_x = intHash64(x);
select * from test.prewhere_column_missing prewhere hash_x = intHash64(x);
select * from test.prewhere_column_missing where hash_x = intHash64(x) and length(arr) = 0;
select * from test.prewhere_column_missing prewhere hash_x = intHash64(x) and length(arr) = 0;
select * from test.prewhere_column_missing where hash_x = intHash64(x) and length(arr) = 0 and arraySum(arr) = 0;
select * from test.prewhere_column_missing prewhere hash_x = intHash64(x) and length(arr) = 0 and arraySum(arr) = 0;
select * from prewhere_column_missing;
select * from prewhere_column_missing where hash_x = intHash64(x);
select * from prewhere_column_missing prewhere hash_x = intHash64(x);
select * from prewhere_column_missing where hash_x = intHash64(x) and length(arr) = 0;
select * from prewhere_column_missing prewhere hash_x = intHash64(x) and length(arr) = 0;
select * from prewhere_column_missing where hash_x = intHash64(x) and length(arr) = 0 and arraySum(arr) = 0;
select * from prewhere_column_missing prewhere hash_x = intHash64(x) and length(arr) = 0 and arraySum(arr) = 0;
drop table test.prewhere_column_missing;
drop table prewhere_column_missing;

View File

@ -1,4 +1,4 @@
CREATE TABLE IF NOT EXISTS test.foo(id UInt64) Engine=Memory;
INSERT INTO test.foo(id) VALUES (0),(4),(1),(1),(3),(1),(1),(2),(2),(2),(1),(2),(3),(2),(1),(1),(2),(1),(1),(1),(3),(1),(2),(2),(1),(1),(3),(1),(2),(1),(1),(3),(2),(1),(1),(4),(0);
SELECT sum(id = 3 OR id = 1 OR id = 2) AS x, sum(id = 3 OR id = 1 OR id = 2) AS x FROM test.foo;
DROP TABLE test.foo;
CREATE TABLE IF NOT EXISTS foo(id UInt64) Engine=Memory;
INSERT INTO foo(id) VALUES (0),(4),(1),(1),(3),(1),(1),(2),(2),(2),(1),(2),(3),(2),(1),(1),(2),(1),(1),(1),(3),(1),(2),(2),(1),(1),(3),(1),(2),(1),(1),(3),(2),(1),(1),(4),(0);
SELECT sum(id = 3 OR id = 1 OR id = 2) AS x, sum(id = 3 OR id = 1 OR id = 2) AS x FROM foo;
DROP TABLE foo;

View File

@ -1,18 +1,18 @@
SET replication_alter_partitions_sync = 2;
DROP TABLE IF EXISTS test.attach_r1;
DROP TABLE IF EXISTS test.attach_r2;
DROP TABLE IF EXISTS attach_r1;
DROP TABLE IF EXISTS attach_r2;
CREATE TABLE test.attach_r1 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01/attach', 'r1', d, d, 8192);
CREATE TABLE test.attach_r2 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01/attach', 'r2', d, d, 8192);
CREATE TABLE attach_r1 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01/attach', 'r1', d, d, 8192);
CREATE TABLE attach_r2 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/01/attach', 'r2', d, d, 8192);
INSERT INTO test.attach_r1 VALUES ('2014-01-01'), ('2014-02-01'), ('2014-03-01');
INSERT INTO attach_r1 VALUES ('2014-01-01'), ('2014-02-01'), ('2014-03-01');
SELECT d FROM test.attach_r1 ORDER BY d;
SELECT d FROM attach_r1 ORDER BY d;
ALTER TABLE test.attach_r2 DROP PARTITION 201402;
ALTER TABLE attach_r2 DROP PARTITION 201402;
SELECT d FROM test.attach_r1 ORDER BY d;
SELECT d FROM attach_r1 ORDER BY d;
DROP TABLE test.attach_r1;
DROP TABLE test.attach_r2;
DROP TABLE attach_r1;
DROP TABLE attach_r2;

View File

@ -1,11 +1,11 @@
DROP TABLE IF EXISTS test.defaults;
CREATE TABLE test.defaults (a UInt8, b DEFAULT 0, c DEFAULT identity(b)) ENGINE = Memory;
INSERT INTO test.defaults (a) VALUES (1);
SELECT * FROM test.defaults;
DROP TABLE test.defaults;
DROP TABLE IF EXISTS defaults;
CREATE TABLE defaults (a UInt8, b DEFAULT 0, c DEFAULT identity(b)) ENGINE = Memory;
INSERT INTO defaults (a) VALUES (1);
SELECT * FROM defaults;
DROP TABLE defaults;
DROP TABLE IF EXISTS test.elog_cut;
CREATE TABLE test.elog_cut
DROP TABLE IF EXISTS elog_cut;
CREATE TABLE elog_cut
(
date Date DEFAULT toDate(uts),
uts DateTime,
@ -18,6 +18,6 @@ CREATE TABLE test.elog_cut
sample_key UInt64 ALIAS page_session
) ENGINE = MergeTree(date, cityHash64(adf_uid, ya_uid, pr), (owner_id, date, cityHash64(adf_uid, ya_uid, pr)), 8192);
INSERT INTO test.elog_cut (uts, pr, ya_uid, adf_uid, owner_id) VALUES ('2015-01-01 01:02:03', 111, 123, 456, 789);
SELECT date, uts, pr, ya_uid, adf_uid, owner_id, eff_uid, page_session, sample_key FROM test.elog_cut;
DROP TABLE test.elog_cut;
INSERT INTO elog_cut (uts, pr, ya_uid, adf_uid, owner_id) VALUES ('2015-01-01 01:02:03', 111, 123, 456, 789);
SELECT date, uts, pr, ya_uid, adf_uid, owner_id, eff_uid, page_session, sample_key FROM elog_cut;
DROP TABLE elog_cut;

View File

@ -1,6 +1,6 @@
drop table if exists test.aliases_test;
drop table if exists aliases_test;
create table test.aliases_test (
create table aliases_test (
date Date, id UInt64,
array default ['zero','one','two'],
d1 default array,
@ -11,104 +11,104 @@ a4 alias arrayMap(x -> toString(x), range(3)), a5 alias a4, a6 alias a5,
`struct.a4` alias arrayMap(x -> toString(x), range(3)), `struct.a5` alias struct.a4, `struct.a6` alias struct.a5
) engine=MergeTree(date, id, 1);
insert into test.aliases_test (id) values (0);
insert into aliases_test (id) values (0);
select '-- Ensure ALIAS columns are not selected by asterisk';
select * from test.aliases_test;
select * from aliases_test;
select '-- select DEFAULT and ALIAS arrays';
select d1, a1, a2, a3, a4, a5, a6 from test.aliases_test;
select d1, a1, a2, a3, a4, a5, a6 from aliases_test;
select '-- select DEFAULT and ALIAS nested columns';
select struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test;
select struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test;
select d1, a1 from test.aliases_test array join d1, a1;
select d1, a1 from test.aliases_test array join d1, a1 as a2;
select d1, a1 from test.aliases_test array join d1 as d2, a1;
select d1, a1 from aliases_test array join d1, a1;
select d1, a1 from aliases_test array join d1, a1 as a2;
select d1, a1 from aliases_test array join d1 as d2, a1;
select '-- array join, but request the original columns';
select d1, a1 from test.aliases_test array join d1 as d2, a1 as a2;
select d1, a1 from aliases_test array join d1 as d2, a1 as a2;
select '-- array join, do not use the result';
select array from test.aliases_test array join d1, a1;
select array from test.aliases_test array join d1 as d2, a1 as a1;
select array from aliases_test array join d1, a1;
select array from aliases_test array join d1 as d2, a1 as a1;
select '-- select DEFAULT and ALIAS arrays, array joining one at a time';
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join d1;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a1;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a2;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a3;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a4;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a5;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a6;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join d1;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a1;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a2;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a3;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a4;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a5;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a6;
select '-- select DEFAULT and ALIAS arrays, array joining one at a time and aliasing result with original name';
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join d1 as d1;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a1 as a1;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a2 as a2;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a3 as a3;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a4 as a4;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a5 as a5;
select array, d1, a1, a2, a3, a4, a5, a6 from test.aliases_test array join a6 as a6;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join d1 as d1;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a1 as a1;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a2 as a2;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a3 as a3;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a4 as a4;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a5 as a5;
select array, d1, a1, a2, a3, a4, a5, a6 from aliases_test array join a6 as a6;
select '-- select DEFAULT and ALIAS arrays and array join result, aliased as `joined`';
select array, d1, a1, a2, a3, a4, a5, a6, joined from test.aliases_test array join d1 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from test.aliases_test array join a1 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from test.aliases_test array join a2 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from test.aliases_test array join a3 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from test.aliases_test array join a4 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from test.aliases_test array join a5 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from test.aliases_test array join a6 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from aliases_test array join d1 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from aliases_test array join a1 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from aliases_test array join a2 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from aliases_test array join a3 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from aliases_test array join a4 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from aliases_test array join a5 as joined;
select array, d1, a1, a2, a3, a4, a5, a6, joined from aliases_test array join a6 as joined;
select '-- select DEFAULT and ALIAS nested columns, array joining one at a time';
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.d1;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a1;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a2;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a3;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a4;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a5;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a6;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.d1;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a1;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a2;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a3;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a4;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a5;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a6;
select '-- select DEFAULT and ALIAS nested columns, array joining one at a time and aliasing result with original name';
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.d1 as `struct.d1`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a1 as `struct.a1`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a2 as `struct.a2`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a3 as `struct.a3`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a4 as `struct.a4`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a5 as `struct.a5`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct.a6 as `struct.a6`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.d1 as `struct.d1`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a1 as `struct.a1`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a2 as `struct.a2`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a3 as `struct.a3`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a4 as `struct.a4`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a5 as `struct.a5`;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct.a6 as `struct.a6`;
select '-- select DEFAULT and ALIAS nested columns and array join result, aliased as `joined`';
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from test.aliases_test array join struct.d1 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from test.aliases_test array join struct.a1 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from test.aliases_test array join struct.a2 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from test.aliases_test array join struct.a3 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from test.aliases_test array join struct.a4 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from test.aliases_test array join struct.a5 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from test.aliases_test array join struct.a6 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from aliases_test array join struct.d1 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from aliases_test array join struct.a1 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from aliases_test array join struct.a2 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from aliases_test array join struct.a3 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from aliases_test array join struct.a4 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from aliases_test array join struct.a5 as joined;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6, joined from aliases_test array join struct.a6 as joined;
select '-- array join whole nested table';
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct;
select '-- array join whole nested table not using the result';
select array from test.aliases_test array join struct;
select array from aliases_test array join struct;
select '-- array join whole nested table, aliasing with original name';
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct as struct;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct as struct;
select '-- array join whole nested table, aliasing with original name not using the result';
select array from test.aliases_test array join struct as struct;
select array from aliases_test array join struct as struct;
select '-- array join whole nested table, aliasing as `class`';
select array, class.d1, class.a1, class.a2, class.a3, class.a4, class.a5, class.a6 from test.aliases_test array join struct as class;
select array, class.d1, class.a1, class.a2, class.a3, class.a4, class.a5, class.a6 from aliases_test array join struct as class;
select '-- array join whole nested table, aliasing as `class` and not using the result';
select array from test.aliases_test array join struct as class;
select array from aliases_test array join struct as class;
select '-- array join whole nested table, aliasing as `class` but requesting the original columns';
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from test.aliases_test array join struct as class;
select array, struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6 from aliases_test array join struct as class;
select array,
struct.d1, struct.a1, struct.a2, struct.a3, struct.a4, struct.a5, struct.a6,
class.d1, class.a1, class.a2, class.a3, class.a4, class.a5, class.a6
from test.aliases_test array join struct as class;
from aliases_test array join struct as class;
drop table test.aliases_test;
drop table aliases_test;

View File

@ -1,24 +1,24 @@
drop table if exists test.aliases_test;
drop table if exists aliases_test;
create table test.aliases_test (date default today(), id default rand(), array default [0, 1, 2]) engine=MergeTree(date, id, 1);
create table aliases_test (date default today(), id default rand(), array default [0, 1, 2]) engine=MergeTree(date, id, 1);
insert into test.aliases_test (id) values (0);
select array from test.aliases_test;
insert into aliases_test (id) values (0);
select array from aliases_test;
alter table test.aliases_test modify column array alias [0, 1, 2];
select array from test.aliases_test;
alter table aliases_test modify column array alias [0, 1, 2];
select array from aliases_test;
alter table test.aliases_test modify column array default [0, 1, 2];
select array from test.aliases_test;
alter table aliases_test modify column array default [0, 1, 2];
select array from aliases_test;
alter table test.aliases_test add column struct.key default [0, 1, 2], add column struct.value default array;
select struct.key, struct.value from test.aliases_test;
alter table aliases_test add column struct.key default [0, 1, 2], add column struct.value default array;
select struct.key, struct.value from aliases_test;
alter table test.aliases_test modify column struct.value alias array;
select struct.key, struct.value from test.aliases_test;
alter table aliases_test modify column struct.value alias array;
select struct.key, struct.value from aliases_test;
select struct.key, struct.value from test.aliases_test array join struct;
select struct.key, struct.value from test.aliases_test array join struct as struct;
select class.key, class.value from test.aliases_test array join struct as class;
select struct.key, struct.value from aliases_test array join struct;
select struct.key, struct.value from aliases_test array join struct as struct;
select class.key, class.value from aliases_test array join struct as class;
drop table test.aliases_test;
drop table aliases_test;

View File

@ -1,12 +1,12 @@
DROP TABLE IF EXISTS test.numbers_10k_log;
DROP TABLE IF EXISTS numbers_10k_log;
SET max_block_size = 1000;
CREATE TABLE test.numbers_10k_log ENGINE = Log AS SELECT number FROM system.numbers LIMIT 10000;
CREATE TABLE numbers_10k_log ENGINE = Log AS SELECT number FROM system.numbers LIMIT 10000;
SET max_threads = 4;
SET max_rows_to_group_by = 3000, group_by_overflow_mode = 'any';
SELECT ignore(rand() AS k), ignore(max(toString(number))) FROM test.numbers_10k_log GROUP BY k LIMIT 1;
SELECT ignore(rand() AS k), ignore(max(toString(number))) FROM numbers_10k_log GROUP BY k LIMIT 1;
DROP TABLE test.numbers_10k_log;
DROP TABLE numbers_10k_log;

View File

@ -1,13 +1,13 @@
DROP TABLE IF EXISTS test.view1;
DROP TABLE IF EXISTS test.view2;
DROP TABLE IF EXISTS test.merge_view;
DROP TABLE IF EXISTS view1;
DROP TABLE IF EXISTS view2;
DROP TABLE IF EXISTS merge_view;
CREATE VIEW test.view1 AS SELECT number FROM system.numbers LIMIT 10;
CREATE VIEW test.view2 AS SELECT number FROM system.numbers LIMIT 10;
CREATE TABLE test.merge_view (number UInt64) ENGINE = Merge(test, '^view');
CREATE VIEW view1 AS SELECT number FROM system.numbers LIMIT 10;
CREATE VIEW view2 AS SELECT number FROM system.numbers LIMIT 10;
CREATE TABLE merge_view (number UInt64) ENGINE = Merge(test, '^view');
SELECT 'Hello, world!' FROM test.merge_view LIMIT 5;
SELECT 'Hello, world!' FROM merge_view LIMIT 5;
DROP TABLE test.view1;
DROP TABLE test.view2;
DROP TABLE test.merge_view;
DROP TABLE view1;
DROP TABLE view2;
DROP TABLE merge_view;

View File

@ -1,55 +1,55 @@
DROP TABLE IF EXISTS test.sample;
DROP TABLE IF EXISTS sample;
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
SET max_block_size = 10;
CREATE TABLE test.sample (d Date DEFAULT '2000-01-01', x UInt8) ENGINE = MergeTree(d, x, x, 10);
INSERT INTO test.sample (x) SELECT toUInt8(number) AS x FROM system.numbers LIMIT 256;
CREATE TABLE sample (d Date DEFAULT '2000-01-01', x UInt8) ENGINE = MergeTree(d, x, x, 10);
INSERT INTO sample (x) SELECT toUInt8(number) AS x FROM system.numbers LIMIT 256;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 0.1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/1e1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1e1/1e2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1e-1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 2e-2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/10 OFFSET 1/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/10 OFFSET 9/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/10 OFFSET 10/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/10 OFFSET 19/20;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 0.1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/1e1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1e1/1e2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1e-1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 2e-2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/10 OFFSET 1/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/10 OFFSET 9/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/10 OFFSET 10/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/10 OFFSET 19/20;
SELECT count() >= 100 FROM test.sample SAMPLE 100;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1000;
SELECT count() >= 100 FROM sample SAMPLE 100;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1000;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 OFFSET 1/2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 0;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 0;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/2 OFFSET 1/2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 0;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 0;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM
(
SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.0
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.1
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.2
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.3
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.4
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.5
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.6
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.7
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.8
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.9
SELECT x FROM sample SAMPLE 0.1 OFFSET 0.0
UNION ALL SELECT x FROM sample SAMPLE 0.1 OFFSET 0.1
UNION ALL SELECT x FROM sample SAMPLE 0.1 OFFSET 0.2
UNION ALL SELECT x FROM sample SAMPLE 0.1 OFFSET 0.3
UNION ALL SELECT x FROM sample SAMPLE 0.1 OFFSET 0.4
UNION ALL SELECT x FROM sample SAMPLE 0.1 OFFSET 0.5
UNION ALL SELECT x FROM sample SAMPLE 0.1 OFFSET 0.6
UNION ALL SELECT x FROM sample SAMPLE 0.1 OFFSET 0.7
UNION ALL SELECT x FROM sample SAMPLE 0.1 OFFSET 0.8
UNION ALL SELECT x FROM sample SAMPLE 0.1 OFFSET 0.9
);
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 0.05 OFFSET 0.35;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 0.05 OFFSET 0.4;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 0.05 OFFSET 0.35;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM sample SAMPLE 0.05 OFFSET 0.4;
SELECT count()
FROM
@ -59,118 +59,118 @@ FROM
count() AS c
FROM
(
SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.00
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.01
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.02
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.03
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.04
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.05
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.06
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.07
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.08
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.09
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.10
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.11
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.12
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.13
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.14
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.15
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.16
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.17
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.18
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.19
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.20
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.21
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.22
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.23
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.24
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.25
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.26
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.27
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.28
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.29
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.30
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.31
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.32
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.33
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.34
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.35
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.36
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.37
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.38
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.39
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.40
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.41
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.42
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.43
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.44
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.45
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.46
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.47
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.48
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.49
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.50
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.51
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.52
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.53
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.54
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.55
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.56
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.57
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.58
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.59
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.60
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.61
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.62
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.63
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.64
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.65
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.66
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.67
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.68
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.69
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.70
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.71
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.72
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.73
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.74
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.75
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.76
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.77
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.78
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.79
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.80
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.81
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.82
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.83
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.84
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.85
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.86
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.87
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.88
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.89
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.90
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.91
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.92
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.93
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.94
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.95
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.96
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.97
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.98
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.99
SELECT * FROM sample SAMPLE 0.01 OFFSET 0.00
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.01
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.02
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.03
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.04
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.05
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.06
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.07
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.08
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.09
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.10
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.11
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.12
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.13
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.14
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.15
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.16
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.17
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.18
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.19
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.20
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.21
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.22
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.23
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.24
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.25
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.26
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.27
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.28
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.29
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.30
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.31
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.32
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.33
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.34
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.35
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.36
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.37
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.38
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.39
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.40
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.41
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.42
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.43
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.44
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.45
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.46
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.47
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.48
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.49
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.50
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.51
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.52
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.53
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.54
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.55
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.56
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.57
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.58
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.59
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.60
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.61
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.62
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.63
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.64
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.65
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.66
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.67
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.68
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.69
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.70
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.71
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.72
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.73
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.74
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.75
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.76
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.77
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.78
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.79
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.80
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.81
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.82
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.83
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.84
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.85
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.86
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.87
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.88
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.89
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.90
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.91
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.92
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.93
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.94
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.95
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.96
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.97
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.98
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.99
)
GROUP BY x
HAVING c = 1
ORDER BY x ASC
);
DROP TABLE test.sample;
DROP TABLE sample;
SET max_block_size = 8192;
CREATE TABLE test.sample (d Date DEFAULT '2000-01-01', x UInt16) ENGINE = MergeTree(d, x, x, 10);
INSERT INTO test.sample (x) SELECT toUInt16(number) AS x FROM system.numbers LIMIT 65536;
CREATE TABLE sample (d Date DEFAULT '2000-01-01', x UInt16) ENGINE = MergeTree(d, x, x, 10);
INSERT INTO sample (x) SELECT toUInt16(number) AS x FROM system.numbers LIMIT 65536;
SELECT count()
FROM
@ -180,110 +180,110 @@ FROM
count() AS c
FROM
(
SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.00
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.01
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.02
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.03
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.04
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.05
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.06
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.07
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.08
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.09
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.10
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.11
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.12
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.13
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.14
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.15
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.16
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.17
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.18
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.19
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.20
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.21
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.22
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.23
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.24
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.25
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.26
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.27
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.28
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.29
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.30
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.31
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.32
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.33
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.34
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.35
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.36
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.37
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.38
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.39
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.40
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.41
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.42
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.43
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.44
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.45
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.46
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.47
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.48
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.49
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.50
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.51
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.52
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.53
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.54
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.55
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.56
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.57
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.58
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.59
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.60
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.61
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.62
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.63
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.64
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.65
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.66
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.67
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.68
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.69
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.70
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.71
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.72
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.73
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.74
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.75
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.76
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.77
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.78
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.79
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.80
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.81
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.82
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.83
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.84
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.85
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.86
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.87
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.88
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.89
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.90
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.91
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.92
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.93
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.94
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.95
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.96
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.97
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.98
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.99
SELECT * FROM sample SAMPLE 0.01 OFFSET 0.00
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.01
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.02
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.03
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.04
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.05
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.06
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.07
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.08
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.09
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.10
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.11
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.12
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.13
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.14
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.15
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.16
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.17
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.18
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.19
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.20
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.21
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.22
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.23
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.24
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.25
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.26
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.27
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.28
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.29
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.30
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.31
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.32
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.33
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.34
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.35
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.36
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.37
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.38
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.39
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.40
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.41
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.42
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.43
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.44
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.45
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.46
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.47
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.48
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.49
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.50
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.51
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.52
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.53
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.54
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.55
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.56
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.57
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.58
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.59
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.60
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.61
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.62
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.63
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.64
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.65
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.66
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.67
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.68
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.69
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.70
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.71
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.72
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.73
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.74
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.75
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.76
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.77
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.78
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.79
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.80
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.81
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.82
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.83
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.84
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.85
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.86
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.87
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.88
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.89
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.90
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.91
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.92
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.93
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.94
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.95
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.96
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.97
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.98
UNION ALL SELECT * FROM sample SAMPLE 0.01 OFFSET 0.99
)
GROUP BY x
HAVING c = 1
ORDER BY x ASC
);
DROP TABLE test.sample;
DROP TABLE sample;

View File

@ -1,16 +1,16 @@
DROP TABLE IF EXISTS test.sorted;
CREATE TABLE test.sorted (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192);
DROP TABLE IF EXISTS sorted;
CREATE TABLE sorted (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192);
INSERT INTO test.sorted (x) SELECT intDiv(number, 100000) AS x FROM system.numbers LIMIT 1000000;
INSERT INTO sorted (x) SELECT intDiv(number, 100000) AS x FROM system.numbers LIMIT 1000000;
SET max_threads = 1;
SELECT count() FROM test.sorted;
SELECT DISTINCT x FROM test.sorted;
SELECT count() FROM sorted;
SELECT DISTINCT x FROM sorted;
INSERT INTO test.sorted (x) SELECT (intHash64(number) % 1000 = 0 ? 999 : intDiv(number, 100000)) AS x FROM system.numbers LIMIT 1000000;
INSERT INTO sorted (x) SELECT (intHash64(number) % 1000 = 0 ? 999 : intDiv(number, 100000)) AS x FROM system.numbers LIMIT 1000000;
SELECT count() FROM test.sorted;
SELECT DISTINCT x FROM test.sorted;
SELECT count() FROM sorted;
SELECT DISTINCT x FROM sorted;
DROP TABLE test.sorted;
DROP TABLE sorted;

View File

@ -1,100 +1,100 @@
DROP TABLE IF EXISTS test.merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
DROP TABLE IF EXISTS merge;
CREATE TABLE IF NOT EXISTS merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
INSERT INTO test.merge (x) VALUES (1), (2), (3);
INSERT INTO test.merge (x) VALUES (4), (5), (6);
INSERT INTO merge (x) VALUES (1), (2), (3);
INSERT INTO merge (x) VALUES (4), (5), (6);
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
SELECT * FROM merge ORDER BY _part_index, x;
OPTIMIZE TABLE merge;
SELECT * FROM merge ORDER BY _part_index, x;
DROP TABLE test.merge;
DROP TABLE merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
CREATE TABLE IF NOT EXISTS merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
INSERT INTO test.merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number + 10 AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number + 10 AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
SELECT * FROM merge ORDER BY _part_index, x;
OPTIMIZE TABLE merge;
SELECT * FROM merge ORDER BY _part_index, x;
DROP TABLE test.merge;
DROP TABLE merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
CREATE TABLE IF NOT EXISTS merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
INSERT INTO test.merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
SELECT * FROM merge ORDER BY _part_index, x;
OPTIMIZE TABLE merge;
SELECT * FROM merge ORDER BY _part_index, x;
DROP TABLE test.merge;
DROP TABLE merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
CREATE TABLE IF NOT EXISTS merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
INSERT INTO test.merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number + 9 AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number + 9 AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
SELECT * FROM merge ORDER BY _part_index, x;
OPTIMIZE TABLE merge;
SELECT * FROM merge ORDER BY _part_index, x;
DROP TABLE test.merge;
DROP TABLE merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
CREATE TABLE IF NOT EXISTS merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
INSERT INTO test.merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number + 10 AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number + 10 AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
SELECT * FROM merge ORDER BY _part_index, x;
OPTIMIZE TABLE merge;
SELECT * FROM merge ORDER BY _part_index, x;
INSERT INTO test.merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
SELECT * FROM merge ORDER BY _part_index, x;
OPTIMIZE TABLE merge;
SELECT * FROM merge ORDER BY _part_index, x;
INSERT INTO test.merge (x) SELECT number + 100 AS x FROM system.numbers LIMIT 10;
INSERT INTO merge (x) SELECT number + 100 AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
SELECT * FROM merge ORDER BY _part_index, x;
OPTIMIZE TABLE merge;
SELECT * FROM merge ORDER BY _part_index, x;
DROP TABLE test.merge;
DROP TABLE merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192);
CREATE TABLE IF NOT EXISTS merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192);
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
SET max_block_size = 8200;
INSERT INTO test.merge (x) SELECT number AS x FROM (SELECT * FROM system.numbers LIMIT 8200) ORDER BY rand();
INSERT INTO test.merge (x) SELECT number AS x FROM (SELECT * FROM system.numbers LIMIT 8200) ORDER BY rand();
INSERT INTO merge (x) SELECT number AS x FROM (SELECT * FROM system.numbers LIMIT 8200) ORDER BY rand();
INSERT INTO merge (x) SELECT number AS x FROM (SELECT * FROM system.numbers LIMIT 8200) ORDER BY rand();
OPTIMIZE TABLE test.merge;
OPTIMIZE TABLE merge;
SELECT count(), uniqExact(x), min(x), max(x), sum(x), sum(cityHash64(x)) FROM test.merge;
SELECT count(), uniqExact(x), min(x), max(x), sum(x), sum(cityHash64(x)) FROM merge;
DROP TABLE test.merge;
DROP TABLE merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192);
CREATE TABLE IF NOT EXISTS merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192);
SET max_block_size = 10000;
INSERT INTO test.merge (x) SELECT number AS x FROM (SELECT number FROM system.numbers LIMIT 10000);
INSERT INTO test.merge (x) SELECT number AS x FROM (SELECT number + 5000 AS number FROM system.numbers LIMIT 10000);
INSERT INTO merge (x) SELECT number AS x FROM (SELECT number FROM system.numbers LIMIT 10000);
INSERT INTO merge (x) SELECT number AS x FROM (SELECT number + 5000 AS number FROM system.numbers LIMIT 10000);
OPTIMIZE TABLE test.merge;
OPTIMIZE TABLE merge;
SELECT count(), uniqExact(x), min(x), max(x), sum(x), sum(cityHash64(x)) FROM test.merge;
SELECT count(), uniqExact(x), min(x), max(x), sum(x), sum(cityHash64(x)) FROM merge;
DROP TABLE test.merge;
DROP TABLE merge;

View File

@ -1,8 +1,8 @@
DROP TABLE IF EXISTS test.stripelog;
CREATE TABLE test.stripelog (x UInt8) ENGINE = StripeLog;
DROP TABLE IF EXISTS stripelog;
CREATE TABLE stripelog (x UInt8) ENGINE = StripeLog;
SELECT * FROM test.stripelog ORDER BY x;
INSERT INTO test.stripelog VALUES (1), (2);
SELECT * FROM test.stripelog ORDER BY x;
SELECT * FROM stripelog ORDER BY x;
INSERT INTO stripelog VALUES (1), (2);
SELECT * FROM stripelog ORDER BY x;
DROP TABLE test.stripelog;
DROP TABLE stripelog;

View File

@ -1,170 +1,170 @@
DROP TABLE IF EXISTS test.r1;
DROP TABLE IF EXISTS test.r2;
DROP TABLE IF EXISTS r1;
DROP TABLE IF EXISTS r2;
CREATE TABLE test.r1 (d Date DEFAULT '2016-01-01', x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/01/r/', 'r1', d, x, 111);
CREATE TABLE r1 (d Date DEFAULT '2016-01-01', x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/01/r/', 'r1', d, x, 111);
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
SET max_block_size = 1;
INSERT INTO test.r1 (x) SELECT number + 1000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 2000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 3000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 4000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 5000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 6000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 7000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 8000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 9000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 10000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 1000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 2000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 3000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 4000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 5000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 6000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 7000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 8000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 9000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 10000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 11000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 12000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 13000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 14000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 15000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 16000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 17000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 18000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 19000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 20000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 11000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 12000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 13000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 14000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 15000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 16000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 17000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 18000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 19000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 20000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 21000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 22000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 23000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 24000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 25000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 26000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 27000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 28000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 29000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 30000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 21000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 22000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 23000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 24000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 25000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 26000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 27000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 28000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 29000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 30000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 31000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 32000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 33000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 34000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 35000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 36000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 37000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 38000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 39000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 40000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 31000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 32000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 33000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 34000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 35000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 36000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 37000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 38000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 39000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 40000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 41000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 42000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 43000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 44000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 45000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 46000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 47000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 48000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 49000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 50000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 41000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 42000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 43000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 44000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 45000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 46000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 47000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 48000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 49000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 50000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 51000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 52000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 53000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 54000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 55000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 56000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 57000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 58000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 59000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 60000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 51000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 52000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 53000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 54000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 55000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 56000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 57000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 58000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 59000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 60000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 61000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 62000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 63000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 64000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 65000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 66000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 67000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 68000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 69000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO test.r1 (x) SELECT number + 70000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE test.r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 61000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 62000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 63000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 64000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 65000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 66000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 67000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 68000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 69000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
INSERT INTO r1 (x) SELECT number + 70000 AS x FROM system.numbers LIMIT 10;
ALTER TABLE r1 DETACH PARTITION 201601;
SELECT count() FROM test.r1;
SELECT count() FROM r1;
CREATE TABLE test.r2 (d Date DEFAULT '2016-01-01', x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/01/r/', 'r2', d, x, 111);
CREATE TABLE r2 (d Date DEFAULT '2016-01-01', x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/01/r/', 'r2', d, x, 111);
SELECT count() FROM test.r2;
SELECT count() FROM r2;
SET replication_alter_partitions_sync = 2;
ALTER TABLE test.r1 ATTACH PARTITION 201601;
ALTER TABLE r1 ATTACH PARTITION 201601;
SELECT count() FROM test.r1;
SELECT count() FROM test.r2;
SELECT count() FROM r1;
SELECT count() FROM r2;
DROP TABLE test.r1;
DROP TABLE test.r2;
DROP TABLE r1;
DROP TABLE r2;

View File

@ -1,12 +1,12 @@
DROP TABLE IF EXISTS test.enum;
DROP TABLE IF EXISTS enum;
CREATE TABLE test.enum (x Enum8('Hello' = -100, '\\' = 0, '\t\\t' = 111), y UInt8) ENGINE = TinyLog;
INSERT INTO test.enum (y) VALUES (0);
SELECT * FROM test.enum ORDER BY x, y FORMAT PrettyCompact;
INSERT INTO test.enum (x) VALUES ('\\');
SELECT * FROM test.enum ORDER BY x, y FORMAT PrettyCompact;
INSERT INTO test.enum (x) VALUES ('\t\\t');
SELECT * FROM test.enum ORDER BY x, y FORMAT PrettyCompact;
SELECT x, y, toInt8(x), toString(x) AS s, CAST(s AS Enum8('Hello' = -100, '\\' = 0, '\t\\t' = 111)) AS casted FROM test.enum ORDER BY x, y FORMAT PrettyCompact;
CREATE TABLE enum (x Enum8('Hello' = -100, '\\' = 0, '\t\\t' = 111), y UInt8) ENGINE = TinyLog;
INSERT INTO enum (y) VALUES (0);
SELECT * FROM enum ORDER BY x, y FORMAT PrettyCompact;
INSERT INTO enum (x) VALUES ('\\');
SELECT * FROM enum ORDER BY x, y FORMAT PrettyCompact;
INSERT INTO enum (x) VALUES ('\t\\t');
SELECT * FROM enum ORDER BY x, y FORMAT PrettyCompact;
SELECT x, y, toInt8(x), toString(x) AS s, CAST(s AS Enum8('Hello' = -100, '\\' = 0, '\t\\t' = 111)) AS casted FROM enum ORDER BY x, y FORMAT PrettyCompact;
DROP TABLE test.enum;
DROP TABLE enum;

View File

@ -1,37 +1,37 @@
DROP TABLE IF EXISTS test.log;
DROP TABLE IF EXISTS log;
CREATE TABLE test.log (x UInt8) ENGINE = StripeLog;
CREATE TABLE log (x UInt8) ENGINE = StripeLog;
SELECT * FROM test.log ORDER BY x;
INSERT INTO test.log VALUES (0);
SELECT * FROM test.log ORDER BY x;
INSERT INTO test.log VALUES (1);
SELECT * FROM test.log ORDER BY x;
INSERT INTO test.log VALUES (2);
SELECT * FROM test.log ORDER BY x;
SELECT * FROM log ORDER BY x;
INSERT INTO log VALUES (0);
SELECT * FROM log ORDER BY x;
INSERT INTO log VALUES (1);
SELECT * FROM log ORDER BY x;
INSERT INTO log VALUES (2);
SELECT * FROM log ORDER BY x;
DROP TABLE test.log;
DROP TABLE log;
CREATE TABLE test.log (x UInt8) ENGINE = TinyLog;
CREATE TABLE log (x UInt8) ENGINE = TinyLog;
SELECT * FROM test.log ORDER BY x;
INSERT INTO test.log VALUES (0);
SELECT * FROM test.log ORDER BY x;
INSERT INTO test.log VALUES (1);
SELECT * FROM test.log ORDER BY x;
INSERT INTO test.log VALUES (2);
SELECT * FROM test.log ORDER BY x;
SELECT * FROM log ORDER BY x;
INSERT INTO log VALUES (0);
SELECT * FROM log ORDER BY x;
INSERT INTO log VALUES (1);
SELECT * FROM log ORDER BY x;
INSERT INTO log VALUES (2);
SELECT * FROM log ORDER BY x;
DROP TABLE test.log;
DROP TABLE log;
CREATE TABLE test.log (x UInt8) ENGINE = Log;
CREATE TABLE log (x UInt8) ENGINE = Log;
SELECT * FROM test.log ORDER BY x;
INSERT INTO test.log VALUES (0);
SELECT * FROM test.log ORDER BY x;
INSERT INTO test.log VALUES (1);
SELECT * FROM test.log ORDER BY x;
INSERT INTO test.log VALUES (2);
SELECT * FROM test.log ORDER BY x;
SELECT * FROM log ORDER BY x;
INSERT INTO log VALUES (0);
SELECT * FROM log ORDER BY x;
INSERT INTO log VALUES (1);
SELECT * FROM log ORDER BY x;
INSERT INTO log VALUES (2);
SELECT * FROM log ORDER BY x;
DROP TABLE test.log;
DROP TABLE log;

View File

@ -3,24 +3,24 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s String, n UInt64, d Date) ENGINE = Memory";
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64, d Date) ENGINE = Memory";
echo '"Hello, world", 123, "2016-01-01"
"Hello, ""world""", "456", 2016-01-02,
Hello "world", 789 ,2016-01-03
"Hello
world", 100, 2016-01-04,' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.csv FORMAT CSV";
world", 100, 2016-01-04,' | $CLICKHOUSE_CLIENT --query="INSERT INTO csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="DROP TABLE csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (t DateTime('Europe/Moscow'), s String) ENGINE = Memory";
$CLICKHOUSE_CLIENT --query="CREATE TABLE csv (t DateTime('Europe/Moscow'), s String) ENGINE = Memory";
echo '"2016-01-01 01:02:03","1"
2016-01-02 01:02:03, "2"
1502792101,"3"
99999,"4"' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.csv FORMAT CSV";
99999,"4"' | $CLICKHOUSE_CLIENT --query="INSERT INTO csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY s";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s";
$CLICKHOUSE_CLIENT --query="DROP TABLE csv";

View File

@ -13,11 +13,11 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=SELECT+name,value,changed+FROM+s
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=SELECT+name,value,changed+FROM+system.settings+WHERE+name+IN+('readonly','max_rows_to_read')&readonly=1&max_rows_to_read=10000&default_format=PrettySpaceNoEscapes" -d' '
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=SELECT+name,value,changed+FROM+system.settings+WHERE+name+IN+('readonly','max_rows_to_read')&readonly=2&max_rows_to_read=10000&default_format=PrettySpaceNoEscapes" -d' '
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?query=DROP+TABLE+IF+EXISTS+test.nonexistent" 2>&1 | grep -q '500 Internal Server Error' && echo 'Ok' || echo 'Fail'
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?readonly=0&query=DROP+TABLE+IF+EXISTS+test.nonexistent" 2>&1 | grep -q '500 Internal Server Error' && echo 'Ok' || echo 'Fail'
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?query=DROP+TABLE+IF+EXISTS+nonexistent" 2>&1 | grep -q '500 Internal Server Error' && echo 'Ok' || echo 'Fail'
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?readonly=0&query=DROP+TABLE+IF+EXISTS+nonexistent" 2>&1 | grep -q '500 Internal Server Error' && echo 'Ok' || echo 'Fail'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=DROP+TABLE+IF+EXISTS+test.nonexistent" -d ' ' | wc -l
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?readonly=0&query=DROP+TABLE+IF+EXISTS+test.nonexistent" -d ' ' | wc -l
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=DROP+TABLE+IF+EXISTS+nonexistent" -d ' ' | wc -l
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?readonly=0&query=DROP+TABLE+IF+EXISTS+nonexistent" -d ' ' | wc -l
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?readonly=1&query=DROP+TABLE+IF+EXISTS+test.nonexistent" -d ' ' 2>&1 | grep -q '500 Internal Server Error' && echo 'Ok' || echo 'Fail'
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?readonly=2&query=DROP+TABLE+IF+EXISTS+test.nonexistent" -d ' ' 2>&1 | grep -q '500 Internal Server Error' && echo 'Ok' || echo 'Fail'
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?readonly=1&query=DROP+TABLE+IF+EXISTS+nonexistent" -d ' ' 2>&1 | grep -q '500 Internal Server Error' && echo 'Ok' || echo 'Fail'
${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}?readonly=2&query=DROP+TABLE+IF+EXISTS+nonexistent" -d ' ' 2>&1 | grep -q '500 Internal Server Error' && echo 'Ok' || echo 'Fail'

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS test.insert;
CREATE TABLE test.insert (i UInt64, s String, u UUID, d Date, t DateTime, a Array(UInt32)) ENGINE = Memory;
DROP TABLE IF EXISTS insert;
CREATE TABLE insert (i UInt64, s String, u UUID, d Date, t DateTime, a Array(UInt32)) ENGINE = Memory;
INSERT INTO test.insert VALUES (1, 'Hello', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', '2016-01-01', '2016-01-02 03:04:05', [1, 2, 3]), (1 + 1, concat('Hello', ', world'), toUUID(0), toDate('2016-01-01') + 1, toStartOfMinute(toDateTime('2016-01-02 03:04:05')), [[0,1],[2]][1]), (round(pi()), concat('hello', ', world!'), toUUID(toString('ab41bdd6-5cd4-11e7-907b-a6006ad3dba0')), toDate(toDateTime('2016-01-03 03:04:05')), toStartOfHour(toDateTime('2016-01-02 03:04:05')), []), (4, 'World', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', '2016-01-04', '2016-12-11 10:09:08', [3,2,1]);
INSERT INTO insert VALUES (1, 'Hello', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', '2016-01-01', '2016-01-02 03:04:05', [1, 2, 3]), (1 + 1, concat('Hello', ', world'), toUUID(0), toDate('2016-01-01') + 1, toStartOfMinute(toDateTime('2016-01-02 03:04:05')), [[0,1],[2]][1]), (round(pi()), concat('hello', ', world!'), toUUID(toString('ab41bdd6-5cd4-11e7-907b-a6006ad3dba0')), toDate(toDateTime('2016-01-03 03:04:05')), toStartOfHour(toDateTime('2016-01-02 03:04:05')), []), (4, 'World', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', '2016-01-04', '2016-12-11 10:09:08', [3,2,1]);
SELECT * FROM test.insert ORDER BY i;
DROP TABLE test.insert;
SELECT * FROM insert ORDER BY i;
DROP TABLE insert;

View File

@ -3,15 +3,15 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.tskv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.tskv (tskv_format String, timestamp DateTime, timezone String, text String, binary_data String) ENGINE = Memory";
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS tskv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE tskv (tskv_format String, timestamp DateTime, timezone String, text String, binary_data String) ENGINE = Memory";
echo -n 'tskv tskv_format=custom-service-log timestamp=2013-01-01 00:00:00 timezone=+0400 text=multiline\ntext binary_data=can contain \0 symbol
binary_data=abc text=Hello, world
binary_data=def text=
tskv
' | $CLICKHOUSE_CLIENT --query="INSERT INTO test.tskv FORMAT TSKV";
' | $CLICKHOUSE_CLIENT --query="INSERT INTO tskv FORMAT TSKV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.tskv ORDER BY binary_data";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.tskv";
$CLICKHOUSE_CLIENT --query="SELECT * FROM tskv ORDER BY binary_data";
$CLICKHOUSE_CLIENT --query="DROP TABLE tskv";

View File

@ -1,18 +1,18 @@
DROP TABLE IF EXISTS test.array_pk;
CREATE TABLE test.array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1);
DROP TABLE IF EXISTS array_pk;
CREATE TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1);
INSERT INTO test.array_pk VALUES ([1, 2, 3], 'Hello, world!', 1);
INSERT INTO test.array_pk VALUES ([1, 2], 'Hello', 2);
INSERT INTO test.array_pk VALUES ([2], 'Goodbye', 3);
INSERT INTO test.array_pk VALUES ([], 'abc', 4);
INSERT INTO test.array_pk VALUES ([2, 3, 4], 'def', 5);
INSERT INTO test.array_pk VALUES ([5, 6], 'ghi', 6);
INSERT INTO array_pk VALUES ([1, 2, 3], 'Hello, world!', 1);
INSERT INTO array_pk VALUES ([1, 2], 'Hello', 2);
INSERT INTO array_pk VALUES ([2], 'Goodbye', 3);
INSERT INTO array_pk VALUES ([], 'abc', 4);
INSERT INTO array_pk VALUES ([2, 3, 4], 'def', 5);
INSERT INTO array_pk VALUES ([5, 6], 'ghi', 6);
SELECT * FROM test.array_pk ORDER BY n;
SELECT * FROM array_pk ORDER BY n;
DETACH TABLE test.array_pk;
ATTACH TABLE test.array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1);
DETACH TABLE array_pk;
ATTACH TABLE array_pk (key Array(UInt8), s String, n UInt64, d Date MATERIALIZED '2000-01-01') ENGINE = MergeTree(d, (key, s, n), 1);
SELECT * FROM test.array_pk ORDER BY n;
SELECT * FROM array_pk ORDER BY n;
DROP TABLE test.array_pk;
DROP TABLE array_pk;

View File

@ -1,20 +1,20 @@
DROP TABLE IF EXISTS test.sample1;
DROP TABLE IF EXISTS test.sample2;
DROP TABLE IF EXISTS test.sample_merge;
DROP TABLE IF EXISTS sample1;
DROP TABLE IF EXISTS sample2;
DROP TABLE IF EXISTS sample_merge;
CREATE TABLE test.sample1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
CREATE TABLE test.sample2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
CREATE TABLE sample1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
CREATE TABLE sample2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10);
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
INSERT INTO test.sample1 (x) SELECT number AS x FROM system.numbers LIMIT 1000000;
INSERT INTO test.sample2 (x) SELECT number AS x FROM system.numbers LIMIT 2000000;
INSERT INTO sample1 (x) SELECT number AS x FROM system.numbers LIMIT 1000000;
INSERT INTO sample2 (x) SELECT number AS x FROM system.numbers LIMIT 2000000;
CREATE TABLE test.sample_merge AS test.sample1 ENGINE = Merge(test, '^sample\\d$');
CREATE TABLE sample_merge AS sample1 ENGINE = Merge(test, '^sample\\d$');
SELECT abs(sum(_sample_factor) - 3000000) / 3000000 < 0.001 FROM test.sample_merge SAMPLE 100000;
SELECT abs(sum(_sample_factor) - 3000000) / 3000000 < 0.001 FROM sample_merge SAMPLE 100000;
SELECT abs(sum(_sample_factor) - 3000000) / 3000000 < 0.001 FROM merge(test, '^sample\\d$') SAMPLE 100000;
DROP TABLE test.sample1;
DROP TABLE test.sample2;
DROP TABLE test.sample_merge;
DROP TABLE sample1;
DROP TABLE sample2;
DROP TABLE sample_merge;

View File

@ -1,68 +1,68 @@
DROP TABLE IF EXISTS test.pk;
DROP TABLE IF EXISTS pk;
CREATE TABLE test.pk (d Date DEFAULT '2000-01-01', x UInt64, y UInt64, z UInt64) ENGINE = MergeTree(d, (x, y, z), 1);
CREATE TABLE pk (d Date DEFAULT '2000-01-01', x UInt64, y UInt64, z UInt64) ENGINE = MergeTree(d, (x, y, z), 1);
INSERT INTO test.pk (x, y, z) VALUES (1, 11, 1235), (1, 11, 4395), (1, 22, 3545), (1, 22, 6984), (1, 33, 4596), (2, 11, 4563), (2, 11, 4578), (2, 11, 3572), (2, 22, 5786), (2, 22, 5786), (2, 22, 2791), (2, 22, 2791), (3, 33, 2791), (3, 33, 2791), (3, 33, 1235), (3, 44, 4935), (3, 44, 4578), (3, 55, 5786), (3, 55, 2791), (3, 55, 1235);
INSERT INTO pk (x, y, z) VALUES (1, 11, 1235), (1, 11, 4395), (1, 22, 3545), (1, 22, 6984), (1, 33, 4596), (2, 11, 4563), (2, 11, 4578), (2, 11, 3572), (2, 22, 5786), (2, 22, 5786), (2, 22, 2791), (2, 22, 2791), (3, 33, 2791), (3, 33, 2791), (3, 33, 1235), (3, 44, 4935), (3, 44, 4578), (3, 55, 5786), (3, 55, 2791), (3, 55, 1235);
SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0;
SET max_block_size = 1;
SET max_rows_to_read = 4;
SELECT * FROM test.pk WHERE x = 2 AND y = 11;
SELECT * FROM pk WHERE x = 2 AND y = 11;
SET max_rows_to_read = 5;
SELECT * FROM test.pk WHERE x = 1;
SELECT * FROM pk WHERE x = 1;
SET max_rows_to_read = 9;
SELECT * FROM test.pk WHERE x = 3;
SELECT * FROM pk WHERE x = 3;
SET max_rows_to_read = 3;
SELECT * FROM test.pk WHERE x = 3 AND y = 44;
SELECT * FROM pk WHERE x = 3 AND y = 44;
SET max_rows_to_read = 2;
SELECT * FROM test.pk WHERE x = 3 AND y = 44 AND z = 4935;
SELECT * FROM test.pk WHERE x = 3 AND y = 44 AND z = 4578;
SELECT * FROM pk WHERE x = 3 AND y = 44 AND z = 4935;
SELECT * FROM pk WHERE x = 3 AND y = 44 AND z = 4578;
SET max_rows_to_read = 1;
SELECT * FROM test.pk WHERE x = 3 AND y = 44 AND z = 4934;
SELECT * FROM test.pk WHERE x = 3 AND y = 44 AND z = 4936;
SELECT * FROM test.pk WHERE x = 3 AND y = 44 AND z = 4577;
SELECT * FROM test.pk WHERE x = 3 AND y = 44 AND z = 4579;
SELECT * FROM pk WHERE x = 3 AND y = 44 AND z = 4934;
SELECT * FROM pk WHERE x = 3 AND y = 44 AND z = 4936;
SELECT * FROM pk WHERE x = 3 AND y = 44 AND z = 4577;
SELECT * FROM pk WHERE x = 3 AND y = 44 AND z = 4579;
SET max_rows_to_read = 1;
SELECT * FROM test.pk WHERE x = 3 AND y = 55 AND z > 5786;
SELECT * FROM pk WHERE x = 3 AND y = 55 AND z > 5786;
SET max_rows_to_read = 2;
SELECT * FROM test.pk WHERE x = 3 AND y = 55 AND z >= 5786;
SELECT * FROM pk WHERE x = 3 AND y = 55 AND z >= 5786;
SET max_rows_to_read = 3;
SELECT * FROM test.pk WHERE x = 3 AND y = 55 AND z > 1235;
SELECT * FROM pk WHERE x = 3 AND y = 55 AND z > 1235;
SET max_rows_to_read = 4;
SELECT * FROM test.pk WHERE x = 3 AND y = 55 AND z >= 1235;
SELECT * FROM test.pk WHERE x = 3 AND y = 55 AND z >= 1000;
SELECT * FROM test.pk WHERE x = 3 AND y = 55 AND z >= 1000 AND x < 10000;
SELECT * FROM test.pk WHERE x = 3 AND y = 55;
SELECT * FROM test.pk WHERE x = 3 AND y >= 50;
SELECT * FROM test.pk WHERE x = 3 AND y > 44;
SELECT * FROM test.pk WHERE x >= 3 AND y > 44;
SELECT * FROM test.pk WHERE x > 2 AND y > 44;
SELECT * FROM pk WHERE x = 3 AND y = 55 AND z >= 1235;
SELECT * FROM pk WHERE x = 3 AND y = 55 AND z >= 1000;
SELECT * FROM pk WHERE x = 3 AND y = 55 AND z >= 1000 AND x < 10000;
SELECT * FROM pk WHERE x = 3 AND y = 55;
SELECT * FROM pk WHERE x = 3 AND y >= 50;
SELECT * FROM pk WHERE x = 3 AND y > 44;
SELECT * FROM pk WHERE x >= 3 AND y > 44;
SELECT * FROM pk WHERE x > 2 AND y > 44;
SET max_rows_to_read = 2;
SELECT * FROM test.pk WHERE x = 3 AND y = 55 AND z = 5786;
SELECT * FROM pk WHERE x = 3 AND y = 55 AND z = 5786;
SET max_rows_to_read = 15;
SET merge_tree_min_rows_for_seek = 0;
SELECT * FROM test.pk WHERE z = 2791;
SELECT * FROM test.pk WHERE z = 5786;
SELECT * FROM test.pk WHERE z = 1235;
SELECT * FROM test.pk WHERE z = 4578;
SELECT * FROM pk WHERE z = 2791;
SELECT * FROM pk WHERE z = 5786;
SELECT * FROM pk WHERE z = 1235;
SELECT * FROM pk WHERE z = 4578;
SET max_rows_to_read = 10;
SELECT * FROM test.pk WHERE y = 11;
SELECT * FROM test.pk WHERE y = 22;
SELECT * FROM test.pk WHERE y = 33;
SELECT * FROM test.pk WHERE y = 44;
SELECT * FROM test.pk WHERE y = 55;
SELECT * FROM pk WHERE y = 11;
SELECT * FROM pk WHERE y = 22;
SELECT * FROM pk WHERE y = 33;
SELECT * FROM pk WHERE y = 44;
SELECT * FROM pk WHERE y = 55;
DROP TABLE test.pk;
DROP TABLE pk;

View File

@ -1,54 +1,54 @@
DROP TABLE IF EXISTS test.index_for_like;
CREATE TABLE test.index_for_like (s String, d Date DEFAULT today()) ENGINE = MergeTree(d, (s, d), 1);
DROP TABLE IF EXISTS index_for_like;
CREATE TABLE index_for_like (s String, d Date DEFAULT today()) ENGINE = MergeTree(d, (s, d), 1);
INSERT INTO test.index_for_like (s) VALUES ('Hello'), ('Hello, World'), ('Hello, World 1'), ('Hello 1'), ('Goodbye'), ('Goodbye, World'), ('Goodbye 1'), ('Goodbye, World 1');
INSERT INTO index_for_like (s) VALUES ('Hello'), ('Hello, World'), ('Hello, World 1'), ('Hello 1'), ('Goodbye'), ('Goodbye, World'), ('Goodbye 1'), ('Goodbye, World 1');
SET max_rows_to_read = 3;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello, World%';
SELECT s FROM index_for_like WHERE s LIKE 'Hello, World%';
SET max_rows_to_read = 2;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello, World %';
SELECT s FROM index_for_like WHERE s LIKE 'Hello, World %';
SET max_rows_to_read = 2;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello, World 1%';
SELECT s FROM index_for_like WHERE s LIKE 'Hello, World 1%';
SET max_rows_to_read = 1;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello, World 2%';
SELECT s FROM index_for_like WHERE s LIKE 'Hello, World 2%';
SET max_rows_to_read = 1;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello, Worle%';
SELECT s FROM index_for_like WHERE s LIKE 'Hello, Worle%';
SET max_rows_to_read = 3;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello, Wor%';
SELECT s FROM index_for_like WHERE s LIKE 'Hello, Wor%';
SET max_rows_to_read = 5;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello%';
SELECT s FROM index_for_like WHERE s LIKE 'Hello%';
SET max_rows_to_read = 2;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello %';
SELECT s FROM index_for_like WHERE s LIKE 'Hello %';
SET max_rows_to_read = 3;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello,%';
SELECT s FROM index_for_like WHERE s LIKE 'Hello,%';
SET max_rows_to_read = 1;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello;%';
SELECT s FROM index_for_like WHERE s LIKE 'Hello;%';
SET max_rows_to_read = 5;
SELECT s FROM test.index_for_like WHERE s LIKE 'H%';
SELECT s FROM index_for_like WHERE s LIKE 'H%';
SET max_rows_to_read = 4;
SELECT s FROM test.index_for_like WHERE s LIKE 'Good%';
SELECT s FROM index_for_like WHERE s LIKE 'Good%';
SET max_rows_to_read = 8;
SELECT s FROM test.index_for_like WHERE s LIKE '%';
SELECT s FROM test.index_for_like WHERE s LIKE '%Hello%';
SELECT s FROM test.index_for_like WHERE s LIKE '%Hello';
SELECT s FROM index_for_like WHERE s LIKE '%';
SELECT s FROM index_for_like WHERE s LIKE '%Hello%';
SELECT s FROM index_for_like WHERE s LIKE '%Hello';
SET max_rows_to_read = 3;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello, World% %';
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello, Worl_%';
SELECT s FROM index_for_like WHERE s LIKE 'Hello, World% %';
SELECT s FROM index_for_like WHERE s LIKE 'Hello, Worl_%';
SET max_rows_to_read = 1;
SELECT s FROM test.index_for_like WHERE s LIKE 'Hello, Worl\\_%';
SELECT s FROM index_for_like WHERE s LIKE 'Hello, Worl\\_%';
DROP TABLE test.index_for_like;
DROP TABLE index_for_like;

View File

@ -1,19 +1,19 @@
DROP TABLE IF EXISTS test.pk_set;
DROP TABLE IF EXISTS pk_set;
CREATE TABLE test.pk_set (d Date, n UInt64, host String, code UInt64) ENGINE = MergeTree(d, (n, host, code), 1);
INSERT INTO test.pk_set (n, host, code) VALUES (1, 'market', 100), (11, 'news', 100);
CREATE TABLE pk_set (d Date, n UInt64, host String, code UInt64) ENGINE = MergeTree(d, (n, host, code), 1);
INSERT INTO pk_set (n, host, code) VALUES (1, 'market', 100), (11, 'news', 100);
SELECT count() FROM test.pk_set WHERE host IN ('admin.market1', 'admin.market2') AND code = 100;
SELECT count() FROM test.pk_set WHERE host IN ('admin.market1', 'admin.market2') AND code = 100 AND n = 11;
SELECT count() FROM test.pk_set WHERE host IN ('admin.market1', 'admin.market2') AND code = 100 AND n >= 11;
SELECT count() FROM test.pk_set WHERE host IN ('market', 'admin.market2', 'admin.market3', 'admin.market4', 'abc') AND code = 100 AND n = 11;
SELECT count() FROM test.pk_set WHERE host IN ('market', 'admin.market2', 'admin.market3', 'admin.market4', 'abc') AND code = 100 AND n >= 11;
SELECT count() FROM test.pk_set WHERE host IN ('admin.market2', 'admin.market3', 'admin.market4', 'abc') AND code = 100 AND n = 11;
SELECT count() FROM test.pk_set WHERE host IN ('admin.market2', 'admin.market3', 'admin.market4', 'abc', 'news') AND code = 100 AND n = 11;
SELECT count() FROM pk_set WHERE host IN ('admin.market1', 'admin.market2') AND code = 100;
SELECT count() FROM pk_set WHERE host IN ('admin.market1', 'admin.market2') AND code = 100 AND n = 11;
SELECT count() FROM pk_set WHERE host IN ('admin.market1', 'admin.market2') AND code = 100 AND n >= 11;
SELECT count() FROM pk_set WHERE host IN ('market', 'admin.market2', 'admin.market3', 'admin.market4', 'abc') AND code = 100 AND n = 11;
SELECT count() FROM pk_set WHERE host IN ('market', 'admin.market2', 'admin.market3', 'admin.market4', 'abc') AND code = 100 AND n >= 11;
SELECT count() FROM pk_set WHERE host IN ('admin.market2', 'admin.market3', 'admin.market4', 'abc') AND code = 100 AND n = 11;
SELECT count() FROM pk_set WHERE host IN ('admin.market2', 'admin.market3', 'admin.market4', 'abc', 'news') AND code = 100 AND n = 11;
-- that barely reproduces the problem
-- better way:
-- for i in {1..1000}; do echo "SELECT count() FROM test.pk_set WHERE host IN ('a'"$(seq 1 $i | sed -r "s/.+/,'\\0'/")") AND code = 100 AND n = 11;"; done > queries.tsv
-- for i in {1..1000}; do echo "SELECT count() FROM pk_set WHERE host IN ('a'"$(seq 1 $i | sed -r "s/.+/,'\\0'/")") AND code = 100 AND n = 11;"; done > queries.tsv
-- clickhouse-benchmark < queries.tsv
DROP TABLE test.pk_set;
DROP TABLE pk_set;

View File

@ -1,96 +1,96 @@
DROP TABLE IF EXISTS test.replacing;
CREATE TABLE test.replacing (d Date, k UInt64, s String, v UInt16) ENGINE = ReplacingMergeTree(d, k, 8192, v);
DROP TABLE IF EXISTS replacing;
CREATE TABLE replacing (d Date, k UInt64, s String, v UInt16) ENGINE = ReplacingMergeTree(d, k, 8192, v);
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'Hello', 0);
SELECT _part_index, * FROM test.replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'Hello', 0);
SELECT _part_index, * FROM replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'World', 0);
SELECT _part_index, * FROM test.replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'World', 0);
SELECT _part_index, * FROM replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, v, _part_index;
OPTIMIZE TABLE test.replacing;
SELECT _part_index, * FROM test.replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, v, _part_index;
OPTIMIZE TABLE replacing;
SELECT _part_index, * FROM replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'Hello', 10);
SELECT _part_index, * FROM test.replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'Hello', 10);
SELECT _part_index, * FROM replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'Hello!', 9);
INSERT INTO test.replacing VALUES ('2000-01-01', 2, 'abc', 1);
INSERT INTO test.replacing VALUES ('2000-01-01', 2, 'def', 1);
INSERT INTO test.replacing VALUES ('2000-01-01', 2, 'ghi', 0);
SELECT * FROM test.replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'Hello!', 9);
INSERT INTO replacing VALUES ('2000-01-01', 2, 'abc', 1);
INSERT INTO replacing VALUES ('2000-01-01', 2, 'def', 1);
INSERT INTO replacing VALUES ('2000-01-01', 2, 'ghi', 0);
SELECT * FROM replacing FINAL ORDER BY k, v, _part_index;
OPTIMIZE TABLE test.replacing;
OPTIMIZE TABLE test.replacing;
OPTIMIZE TABLE test.replacing;
SELECT _part_index, * FROM test.replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, v, _part_index;
OPTIMIZE TABLE replacing;
OPTIMIZE TABLE replacing;
OPTIMIZE TABLE replacing;
SELECT _part_index, * FROM replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, v, _part_index;
DROP TABLE test.replacing;
DROP TABLE replacing;
DROP TABLE IF EXISTS test.replacing;
CREATE TABLE test.replacing (d Date, k UInt64, s String, v UInt16) ENGINE = ReplacingMergeTree(d, k, 1, v);
DROP TABLE IF EXISTS replacing;
CREATE TABLE replacing (d Date, k UInt64, s String, v UInt16) ENGINE = ReplacingMergeTree(d, k, 1, v);
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'Hello', 0);
SELECT _part_index, * FROM test.replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'Hello', 0);
SELECT _part_index, * FROM replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'World', 0);
SELECT _part_index, * FROM test.replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'World', 0);
SELECT _part_index, * FROM replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, v, _part_index;
OPTIMIZE TABLE test.replacing;
SELECT _part_index, * FROM test.replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, v, _part_index;
OPTIMIZE TABLE replacing;
SELECT _part_index, * FROM replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'Hello', 10);
SELECT _part_index, * FROM test.replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'Hello', 10);
SELECT _part_index, * FROM replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'Hello!', 9);
INSERT INTO test.replacing VALUES ('2000-01-01', 2, 'abc', 1);
INSERT INTO test.replacing VALUES ('2000-01-01', 2, 'def', 1);
INSERT INTO test.replacing VALUES ('2000-01-01', 2, 'ghi', 0);
SELECT * FROM test.replacing FINAL ORDER BY k, v, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'Hello!', 9);
INSERT INTO replacing VALUES ('2000-01-01', 2, 'abc', 1);
INSERT INTO replacing VALUES ('2000-01-01', 2, 'def', 1);
INSERT INTO replacing VALUES ('2000-01-01', 2, 'ghi', 0);
SELECT * FROM replacing FINAL ORDER BY k, v, _part_index;
OPTIMIZE TABLE test.replacing PARTITION 200001 FINAL;
SELECT _part_index, * FROM test.replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, v, _part_index;
OPTIMIZE TABLE replacing PARTITION 200001 FINAL;
SELECT _part_index, * FROM replacing ORDER BY k, v, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, v, _part_index;
DROP TABLE test.replacing;
DROP TABLE replacing;
DROP TABLE IF EXISTS test.replacing;
CREATE TABLE test.replacing (d Date, k UInt64, s String) ENGINE = ReplacingMergeTree(d, k, 2);
DROP TABLE IF EXISTS replacing;
CREATE TABLE replacing (d Date, k UInt64, s String) ENGINE = ReplacingMergeTree(d, k, 2);
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'Hello');
SELECT _part_index, * FROM test.replacing ORDER BY k, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'Hello');
SELECT _part_index, * FROM replacing ORDER BY k, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, _part_index;
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'World');
SELECT _part_index, * FROM test.replacing ORDER BY k, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'World');
SELECT _part_index, * FROM replacing ORDER BY k, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, _part_index;
OPTIMIZE TABLE test.replacing;
SELECT _part_index, * FROM test.replacing ORDER BY k, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, _part_index;
OPTIMIZE TABLE replacing;
SELECT _part_index, * FROM replacing ORDER BY k, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, _part_index;
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'Hello');
SELECT _part_index, * FROM test.replacing ORDER BY k, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'Hello');
SELECT _part_index, * FROM replacing ORDER BY k, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, _part_index;
INSERT INTO test.replacing VALUES ('2000-01-01', 1, 'Hello!');
INSERT INTO test.replacing VALUES ('2000-01-01', 2, 'abc');
INSERT INTO test.replacing VALUES ('2000-01-01', 2, 'def');
INSERT INTO test.replacing VALUES ('2000-01-01', 2, 'ghi');
SELECT * FROM test.replacing FINAL ORDER BY k, _part_index;
INSERT INTO replacing VALUES ('2000-01-01', 1, 'Hello!');
INSERT INTO replacing VALUES ('2000-01-01', 2, 'abc');
INSERT INTO replacing VALUES ('2000-01-01', 2, 'def');
INSERT INTO replacing VALUES ('2000-01-01', 2, 'ghi');
SELECT * FROM replacing FINAL ORDER BY k, _part_index;
OPTIMIZE TABLE test.replacing PARTITION 200001 FINAL;
SELECT _part_index, * FROM test.replacing ORDER BY k, _part_index;
SELECT _part_index, * FROM test.replacing FINAL ORDER BY k, _part_index;
OPTIMIZE TABLE replacing PARTITION 200001 FINAL;
SELECT _part_index, * FROM replacing ORDER BY k, _part_index;
SELECT _part_index, * FROM replacing FINAL ORDER BY k, _part_index;
DROP TABLE test.replacing;
DROP TABLE replacing;

File diff suppressed because it is too large Load Diff

View File

@ -1,28 +1,28 @@
DROP TABLE IF EXISTS test.summing_composite_key;
CREATE TABLE test.summing_composite_key (d Date, k UInt64, FirstMap Nested(k1 UInt32, k2ID Int8, s Float64), SecondMap Nested(k1ID UInt64, k2Key UInt32, k3Type Int32, s Int64)) ENGINE = SummingMergeTree(d, k, 1);
DROP TABLE IF EXISTS summing_composite_key;
CREATE TABLE summing_composite_key (d Date, k UInt64, FirstMap Nested(k1 UInt32, k2ID Int8, s Float64), SecondMap Nested(k1ID UInt64, k2Key UInt32, k3Type Int32, s Int64)) ENGINE = SummingMergeTree(d, k, 1);
INSERT INTO test.summing_composite_key VALUES ('2000-01-01', 1, [1,2], [3,4], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]), ('2000-01-01', 1, [2,1], [4,3], [20,22], [2,2,1], [5,5,0], [-3,-3,-33], [10,100,1000]), ('2000-01-01', 2, [1,2], [3,4], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]), ('2000-01-01', 2, [2,1,1], [4,3,3], [20,22,33], [2,2], [5,5], [-3,-3], [10,100]), ('2000-01-01', 2, [1,2], [3,4], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]);
INSERT INTO summing_composite_key VALUES ('2000-01-01', 1, [1,2], [3,4], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]), ('2000-01-01', 1, [2,1], [4,3], [20,22], [2,2,1], [5,5,0], [-3,-3,-33], [10,100,1000]), ('2000-01-01', 2, [1,2], [3,4], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]), ('2000-01-01', 2, [2,1,1], [4,3,3], [20,22,33], [2,2], [5,5], [-3,-3], [10,100]), ('2000-01-01', 2, [1,2], [3,4], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]);
SELECT * FROM test.summing_composite_key ORDER BY d, k, _part_index;
SELECT * FROM summing_composite_key ORDER BY d, k, _part_index;
SELECT d, k, m.k1, m.k2ID, m.s FROM test.summing_composite_key ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1, m.k2ID, sum(m.s) FROM test.summing_composite_key ARRAY JOIN FirstMap AS m GROUP BY d, k, m.k1, m.k2ID ORDER BY d, k, m.k1, m.k2ID;
SELECT d, k, m.k1, m.k2ID,m. s FROM test.summing_composite_key FINAL ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1, m.k2ID, m.s FROM summing_composite_key ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1, m.k2ID, sum(m.s) FROM summing_composite_key ARRAY JOIN FirstMap AS m GROUP BY d, k, m.k1, m.k2ID ORDER BY d, k, m.k1, m.k2ID;
SELECT d, k, m.k1, m.k2ID,m. s FROM summing_composite_key FINAL ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, m.s FROM test.summing_composite_key ARRAY JOIN SecondMap AS m ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type, m.s;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, sum(m.s) FROM test.summing_composite_key ARRAY JOIN SecondMap AS m GROUP BY d, k, m.k1ID, m.k2Key, m.k3Type ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, m.s FROM test.summing_composite_key FINAL ARRAY JOIN SecondMap AS m ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type, m.s;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, m.s FROM summing_composite_key ARRAY JOIN SecondMap AS m ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type, m.s;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, sum(m.s) FROM summing_composite_key ARRAY JOIN SecondMap AS m GROUP BY d, k, m.k1ID, m.k2Key, m.k3Type ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, m.s FROM summing_composite_key FINAL ARRAY JOIN SecondMap AS m ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type, m.s;
OPTIMIZE TABLE test.summing_composite_key PARTITION 200001 FINAL;
OPTIMIZE TABLE summing_composite_key PARTITION 200001 FINAL;
SELECT * FROM test.summing_composite_key ORDER BY d, k, _part_index;
SELECT * FROM summing_composite_key ORDER BY d, k, _part_index;
SELECT d, k, m.k1, m.k2ID, m.s FROM test.summing_composite_key ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1, m.k2ID, sum(m.s) FROM test.summing_composite_key ARRAY JOIN FirstMap AS m GROUP BY d, k, m.k1, m.k2ID ORDER BY d, k, m.k1, m.k2ID;
SELECT d, k, m.k1, m.k2ID, m.s FROM test.summing_composite_key FINAL ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1, m.k2ID, m.s FROM summing_composite_key ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1, m.k2ID, sum(m.s) FROM summing_composite_key ARRAY JOIN FirstMap AS m GROUP BY d, k, m.k1, m.k2ID ORDER BY d, k, m.k1, m.k2ID;
SELECT d, k, m.k1, m.k2ID, m.s FROM summing_composite_key FINAL ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, m.s FROM test.summing_composite_key ARRAY JOIN SecondMap AS m ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type, m.s;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, sum(m.s) FROM test.summing_composite_key ARRAY JOIN SecondMap AS m GROUP BY d, k, m.k1ID, m.k2Key, m.k3Type ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, m.s FROM test.summing_composite_key FINAL ARRAY JOIN SecondMap AS m ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type, m.s;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, m.s FROM summing_composite_key ARRAY JOIN SecondMap AS m ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type, m.s;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, sum(m.s) FROM summing_composite_key ARRAY JOIN SecondMap AS m GROUP BY d, k, m.k1ID, m.k2Key, m.k3Type ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, m.s FROM summing_composite_key FINAL ARRAY JOIN SecondMap AS m ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type, m.s;
DROP TABLE test.summing_composite_key;
DROP TABLE summing_composite_key;

File diff suppressed because it is too large Load Diff

View File

@ -1,11 +1,11 @@
DROP TABLE IF EXISTS test.v1;
DROP TABLE IF EXISTS test.v2;
DROP TABLE IF EXISTS v1;
DROP TABLE IF EXISTS v2;
CREATE VIEW test.v1 AS SELECT 1 FROM (SELECT 1);
SELECT * FROM test.v1;
CREATE VIEW v1 AS SELECT 1 FROM (SELECT 1);
SELECT * FROM v1;
CREATE VIEW test.v2 AS SELECT number * number FROM (SELECT number FROM system.numbers LIMIT 10);
SELECT * FROM test.v2;
CREATE VIEW v2 AS SELECT number * number FROM (SELECT number FROM system.numbers LIMIT 10);
SELECT * FROM v2;
DROP TABLE test.v1;
DROP TABLE test.v2;
DROP TABLE v1;
DROP TABLE v2;

View File

@ -1,12 +1,12 @@
DROP TABLE IF EXISTS test.replace;
DROP TABLE IF EXISTS replace;
CREATE TABLE test.replace ( EventDate Date, Id UInt64, Data String, Version UInt32) ENGINE = ReplacingMergeTree(EventDate, Id, 8192, Version);
INSERT INTO test.replace VALUES ('2016-06-02', 1, 'version 1', 1);
INSERT INTO test.replace VALUES ('2016-06-02', 2, 'version 1', 1);
INSERT INTO test.replace VALUES ('2016-06-02', 1, 'version 0', 0);
CREATE TABLE replace ( EventDate Date, Id UInt64, Data String, Version UInt32) ENGINE = ReplacingMergeTree(EventDate, Id, 8192, Version);
INSERT INTO replace VALUES ('2016-06-02', 1, 'version 1', 1);
INSERT INTO replace VALUES ('2016-06-02', 2, 'version 1', 1);
INSERT INTO replace VALUES ('2016-06-02', 1, 'version 0', 0);
SELECT * FROM test.replace ORDER BY Id, Version;
SELECT * FROM test.replace FINAL ORDER BY Id, Version;
SELECT * FROM test.replace FINAL WHERE Version = 0 ORDER BY Id, Version;
SELECT * FROM replace ORDER BY Id, Version;
SELECT * FROM replace FINAL ORDER BY Id, Version;
SELECT * FROM replace FINAL WHERE Version = 0 ORDER BY Id, Version;
DROP TABLE test.replace;
DROP TABLE replace;

File diff suppressed because one or more lines are too long

View File

@ -3,9 +3,9 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
echo 'DROP TABLE IF EXISTS test.bom' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
echo 'CREATE TABLE test.bom (a UInt8, b UInt8, c UInt8) ENGINE = Memory' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
echo -ne '1,2,3\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+test.bom+FORMAT+CSV" --data-binary @-
echo -ne '\xEF\xBB\xBF4,5,6\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+test.bom+FORMAT+CSV" --data-binary @-
echo 'SELECT * FROM test.bom ORDER BY a' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
echo 'DROP TABLE test.bom' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
echo 'DROP TABLE IF EXISTS bom' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
echo 'CREATE TABLE bom (a UInt8, b UInt8, c UInt8) ENGINE = Memory' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
echo -ne '1,2,3\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+bom+FORMAT+CSV" --data-binary @-
echo -ne '\xEF\xBB\xBF4,5,6\n' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+bom+FORMAT+CSV" --data-binary @-
echo 'SELECT * FROM bom ORDER BY a' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-
echo 'DROP TABLE bom' | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @-

View File

@ -1,28 +1,28 @@
DROP TABLE IF EXISTS test.bad_arrays;
CREATE TABLE test.bad_arrays (a Array(String), b Array(UInt8)) ENGINE = Memory;
DROP TABLE IF EXISTS bad_arrays;
CREATE TABLE bad_arrays (a Array(String), b Array(UInt8)) ENGINE = Memory;
INSERT INTO test.bad_arrays VALUES ([''],[]),([''],[1]);
INSERT INTO bad_arrays VALUES ([''],[]),([''],[1]);
SELECT a FROM test.bad_arrays ARRAY JOIN b;
SELECT a FROM bad_arrays ARRAY JOIN b;
DROP TABLE test.bad_arrays;
DROP TABLE bad_arrays;
DROP TABLE IF EXISTS test.bad_arrays;
CREATE TABLE test.bad_arrays (a Array(String), b Array(String)) ENGINE = Memory;
DROP TABLE IF EXISTS bad_arrays;
CREATE TABLE bad_arrays (a Array(String), b Array(String)) ENGINE = Memory;
INSERT INTO test.bad_arrays VALUES ([''],[]),([''],[]),([''],[]),([''],[]),([''],[]),([''],[]),([''],[]),([''],[]),([''],[]),(['abc'],['223750']),(['ноутбук acer aspire e5-532-p3p2'],[]),([''],[]),([''],[]),([''],[]),([''],[]),(['лучшие моноблоки 2016'],[]),(['лучшие моноблоки 2016'],[]),([''],[]),([''],[]);
INSERT INTO bad_arrays VALUES ([''],[]),([''],[]),([''],[]),([''],[]),([''],[]),([''],[]),([''],[]),([''],[]),([''],[]),(['abc'],['223750']),(['ноутбук acer aspire e5-532-p3p2'],[]),([''],[]),([''],[]),([''],[]),([''],[]),(['лучшие моноблоки 2016'],[]),(['лучшие моноблоки 2016'],[]),([''],[]),([''],[]);
SELECT a FROM test.bad_arrays ARRAY JOIN b;
SELECT a FROM bad_arrays ARRAY JOIN b;
DROP TABLE test.bad_arrays;
DROP TABLE bad_arrays;
DROP TABLE IF EXISTS test.bad_arrays;
CREATE TABLE test.bad_arrays (a Array(String), b Array(UInt8)) ENGINE = Memory;
DROP TABLE IF EXISTS bad_arrays;
CREATE TABLE bad_arrays (a Array(String), b Array(UInt8)) ENGINE = Memory;
INSERT INTO test.bad_arrays VALUES (['abc','def'],[1,2,3]),([],[1,2]),(['a','b'],[]),(['Hello'],[1,2]),([],[]),(['x','y','z'],[4,5,6]);
INSERT INTO bad_arrays VALUES (['abc','def'],[1,2,3]),([],[1,2]),(['a','b'],[]),(['Hello'],[1,2]),([],[]),(['x','y','z'],[4,5,6]);
SELECT a, b FROM test.bad_arrays ARRAY JOIN b;
SELECT a, b FROM bad_arrays ARRAY JOIN b;
DROP TABLE test.bad_arrays;
DROP TABLE bad_arrays;

View File

@ -3,9 +3,9 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'DROP TABLE IF EXISTS test.bad_arrays'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'CREATE TABLE test.bad_arrays (a Array(String)) ENGINE = Memory'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'INSERT INTO test.bad_arrays VALUES ([123])' 2>&1 | grep -c 'Exception'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "INSERT INTO test.bad_arrays VALUES (['123', concat('Hello', ' world!'), toString(123)])"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT * FROM test.bad_arrays'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'DROP TABLE test.bad_arrays'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'DROP TABLE IF EXISTS bad_arrays'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'CREATE TABLE bad_arrays (a Array(String)) ENGINE = Memory'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'INSERT INTO bad_arrays VALUES ([123])' 2>&1 | grep -c 'Exception'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "INSERT INTO bad_arrays VALUES (['123', concat('Hello', ' world!'), toString(123)])"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT * FROM bad_arrays'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'DROP TABLE bad_arrays'

View File

@ -1,18 +1,18 @@
DROP TABLE IF EXISTS test.numbers_squashed;
CREATE TABLE test.numbers_squashed AS system.numbers ENGINE = StripeLog;
DROP TABLE IF EXISTS numbers_squashed;
CREATE TABLE numbers_squashed AS system.numbers ENGINE = StripeLog;
SET max_block_size = 10000;
SET min_insert_block_size_rows = 1000000;
SET min_insert_block_size_bytes = 0;
INSERT INTO test.numbers_squashed SELECT * FROM system.numbers LIMIT 10000000;
SELECT blockSize() AS b, count() / b AS c FROM test.numbers_squashed GROUP BY blockSize() ORDER BY c DESC;
INSERT INTO numbers_squashed SELECT * FROM system.numbers LIMIT 10000000;
SELECT blockSize() AS b, count() / b AS c FROM numbers_squashed GROUP BY blockSize() ORDER BY c DESC;
SET min_insert_block_size_bytes = 1000000;
INSERT INTO test.numbers_squashed SELECT * FROM system.numbers LIMIT 10000000;
SELECT blockSize() AS b, count() / b AS c FROM test.numbers_squashed GROUP BY blockSize() ORDER BY c DESC;
INSERT INTO numbers_squashed SELECT * FROM system.numbers LIMIT 10000000;
SELECT blockSize() AS b, count() / b AS c FROM numbers_squashed GROUP BY blockSize() ORDER BY c DESC;
SELECT count() FROM test.numbers_squashed;
SELECT count() FROM numbers_squashed;
DROP TABLE test.numbers_squashed;
DROP TABLE numbers_squashed;

View File

@ -1,41 +1,41 @@
DROP TABLE IF EXISTS test.numbers_squashed;
CREATE TABLE test.numbers_squashed (number UInt8) ENGINE = Memory;
DROP TABLE IF EXISTS numbers_squashed;
CREATE TABLE numbers_squashed (number UInt8) ENGINE = Memory;
SET min_insert_block_size_rows = 100;
SET min_insert_block_size_bytes = 0;
SET max_threads = 1;
INSERT INTO test.numbers_squashed
INSERT INTO numbers_squashed
SELECT arrayJoin(range(10)) AS number
UNION ALL
SELECT arrayJoin(range(100))
UNION ALL
SELECT arrayJoin(range(10));
SELECT blockSize() AS b, count() / b AS c FROM test.numbers_squashed GROUP BY blockSize() ORDER BY c DESC, b ASC;
SELECT count() FROM test.numbers_squashed;
SELECT blockSize() AS b, count() / b AS c FROM numbers_squashed GROUP BY blockSize() ORDER BY c DESC, b ASC;
SELECT count() FROM numbers_squashed;
INSERT INTO test.numbers_squashed
INSERT INTO numbers_squashed
SELECT arrayJoin(range(100)) AS number
UNION ALL
SELECT arrayJoin(range(10))
UNION ALL
SELECT arrayJoin(range(100));
SELECT blockSize() AS b, count() / b AS c FROM test.numbers_squashed GROUP BY blockSize() ORDER BY c DESC, b ASC;
SELECT count() FROM test.numbers_squashed;
SELECT blockSize() AS b, count() / b AS c FROM numbers_squashed GROUP BY blockSize() ORDER BY c DESC, b ASC;
SELECT count() FROM numbers_squashed;
INSERT INTO test.numbers_squashed
INSERT INTO numbers_squashed
SELECT arrayJoin(range(10)) AS number
UNION ALL
SELECT arrayJoin(range(100))
UNION ALL
SELECT arrayJoin(range(100));
SELECT blockSize() AS b, count() / b AS c FROM test.numbers_squashed GROUP BY blockSize() ORDER BY c DESC, b ASC;
SELECT count() FROM test.numbers_squashed;
SELECT blockSize() AS b, count() / b AS c FROM numbers_squashed GROUP BY blockSize() ORDER BY c DESC, b ASC;
SELECT count() FROM numbers_squashed;
INSERT INTO test.numbers_squashed
INSERT INTO numbers_squashed
SELECT arrayJoin(range(10)) AS number
UNION ALL
SELECT arrayJoin(range(10))
@ -46,12 +46,12 @@ SELECT arrayJoin(range(100))
UNION ALL
SELECT arrayJoin(range(10));
SELECT blockSize() AS b, count() / b AS c FROM test.numbers_squashed GROUP BY blockSize() ORDER BY c DESC, b ASC;
SELECT count() FROM test.numbers_squashed;
SELECT blockSize() AS b, count() / b AS c FROM numbers_squashed GROUP BY blockSize() ORDER BY c DESC, b ASC;
SELECT count() FROM numbers_squashed;
SET min_insert_block_size_rows = 10;
INSERT INTO test.numbers_squashed
INSERT INTO numbers_squashed
SELECT arrayJoin(range(10)) AS number
UNION ALL
SELECT arrayJoin(range(10))
@ -62,7 +62,7 @@ SELECT arrayJoin(range(100))
UNION ALL
SELECT arrayJoin(range(10));
SELECT blockSize() AS b, count() / b AS c FROM test.numbers_squashed GROUP BY blockSize() ORDER BY c DESC, b ASC;
SELECT count() FROM test.numbers_squashed;
SELECT blockSize() AS b, count() / b AS c FROM numbers_squashed GROUP BY blockSize() ORDER BY c DESC, b ASC;
SELECT count() FROM numbers_squashed;
DROP TABLE test.numbers_squashed;
DROP TABLE numbers_squashed;

View File

@ -1,20 +1,20 @@
DROP TABLE IF EXISTS test.index;
DROP TABLE IF EXISTS index;
CREATE TABLE test.index
CREATE TABLE index
(
key Int32,
name String,
merge_date Date
) ENGINE = MergeTree(merge_date, key, 8192);
insert into test.index values (1,'1','2016-07-07');
insert into test.index values (-1,'-1','2016-07-07');
insert into index values (1,'1','2016-07-07');
insert into index values (-1,'-1','2016-07-07');
select * from test.index where key = 1;
select * from test.index where key = -1;
OPTIMIZE TABLE test.index;
select * from test.index where key = 1;
select * from test.index where key = -1;
select * from test.index where key < -0.5;
select * from index where key = 1;
select * from index where key = -1;
OPTIMIZE TABLE index;
select * from index where key = 1;
select * from index where key = -1;
select * from index where key < -0.5;
DROP TABLE test.index;
DROP TABLE index;

View File

@ -1,17 +1,17 @@
DROP TABLE IF EXISTS test.nested1;
DROP TABLE IF EXISTS test.nested2;
DROP TABLE IF EXISTS nested1;
DROP TABLE IF EXISTS nested2;
CREATE TABLE test.nested1 (d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, b String)) ENGINE = MergeTree(d, x, 1);
INSERT INTO test.nested1 (x, n.a, n.b) VALUES (1, ['Hello', 'World'], ['abc', 'def']), (2, [], []);
CREATE TABLE nested1 (d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, b String)) ENGINE = MergeTree(d, x, 1);
INSERT INTO nested1 (x, n.a, n.b) VALUES (1, ['Hello', 'World'], ['abc', 'def']), (2, [], []);
SET max_block_size = 1;
SELECT * FROM test.nested1 ORDER BY x;
SELECT * FROM nested1 ORDER BY x;
CREATE TABLE test.nested2 (d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, b String)) ENGINE = MergeTree(d, x, 1);
CREATE TABLE nested2 (d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, b String)) ENGINE = MergeTree(d, x, 1);
INSERT INTO test.nested2 SELECT * FROM test.nested1;
INSERT INTO nested2 SELECT * FROM nested1;
SELECT * FROM test.nested2 ORDER BY x;
SELECT * FROM nested2 ORDER BY x;
DROP TABLE test.nested1;
DROP TABLE test.nested2;
DROP TABLE nested1;
DROP TABLE nested2;

View File

@ -1,19 +1,19 @@
DROP TABLE IF EXISTS test.prewhere_defaults;
DROP TABLE IF EXISTS prewhere_defaults;
CREATE TABLE test.prewhere_defaults (d Date DEFAULT '2000-01-01', k UInt64 DEFAULT 0, x UInt16) ENGINE = MergeTree(d, k, 1);
CREATE TABLE prewhere_defaults (d Date DEFAULT '2000-01-01', k UInt64 DEFAULT 0, x UInt16) ENGINE = MergeTree(d, k, 1);
INSERT INTO test.prewhere_defaults (x) VALUES (1);
INSERT INTO prewhere_defaults (x) VALUES (1);
SET max_block_size = 1;
SELECT * FROM test.prewhere_defaults PREWHERE x != 0 ORDER BY x;
SELECT * FROM prewhere_defaults PREWHERE x != 0 ORDER BY x;
ALTER TABLE test.prewhere_defaults ADD COLUMN y UInt16 DEFAULT x;
ALTER TABLE prewhere_defaults ADD COLUMN y UInt16 DEFAULT x;
SELECT * FROM test.prewhere_defaults PREWHERE x != 0 ORDER BY x;
SELECT * FROM prewhere_defaults PREWHERE x != 0 ORDER BY x;
INSERT INTO test.prewhere_defaults (x) VALUES (2);
INSERT INTO prewhere_defaults (x) VALUES (2);
SELECT * FROM test.prewhere_defaults PREWHERE x != 0 ORDER BY x;
SELECT * FROM prewhere_defaults PREWHERE x != 0 ORDER BY x;
DROP TABLE test.prewhere_defaults;
DROP TABLE prewhere_defaults;

View File

@ -18,33 +18,33 @@ $CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2"
$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2;"
$CLICKHOUSE_CLIENT -n --query="SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error'
$CLICKHOUSE_CLIENT -n --query="DROP TABLE IF EXISTS test.t; CREATE TABLE test.t (x UInt64) ENGINE = TinyLog;"
$CLICKHOUSE_CLIENT -n --query="DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = TinyLog;"
$CLICKHOUSE_CLIENT --query="INSERT INTO test.t VALUES (1),(2),(3);"
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.t"
$CLICKHOUSE_CLIENT --query="INSERT INTO test.t VALUES" <<< "(4),(5),(6)"
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.t"
$CLICKHOUSE_CLIENT --query="INSERT INTO t VALUES (1),(2),(3);"
$CLICKHOUSE_CLIENT --query="SELECT * FROM t"
$CLICKHOUSE_CLIENT --query="INSERT INTO t VALUES" <<< "(4),(5),(6)"
$CLICKHOUSE_CLIENT --query="SELECT * FROM t"
$CLICKHOUSE_CLIENT -n --query="INSERT INTO test.t VALUES (1),(2),(3);"
$CLICKHOUSE_CLIENT -n --query="SELECT * FROM test.t"
$CLICKHOUSE_CLIENT -n --query="INSERT INTO test.t VALUES" <<< "(4),(5),(6)"
$CLICKHOUSE_CLIENT -n --query="SELECT * FROM test.t"
$CLICKHOUSE_CLIENT -n --query="INSERT INTO t VALUES (1),(2),(3);"
$CLICKHOUSE_CLIENT -n --query="SELECT * FROM t"
$CLICKHOUSE_CLIENT -n --query="INSERT INTO t VALUES" <<< "(4),(5),(6)"
$CLICKHOUSE_CLIENT -n --query="SELECT * FROM t"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1;"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1; "
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1 ; "
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}" -d "SELECT 1"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}" -d "SELECT 1;"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}" -d "SELECT 1; "
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}" -d "SELECT 1 ; "
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1; S" 2>&1 | grep -o 'Syntax error'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1; SELECT 2" 2>&1 | grep -o 'Syntax error'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1; SELECT 2;" 2>&1 | grep -o 'Syntax error'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}" -d "SELECT 1; S" 2>&1 | grep -o 'Syntax error'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}" -d "SELECT 1; SELECT 2" 2>&1 | grep -o 'Syntax error'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}" -d "SELECT 1; SELECT 2;" 2>&1 | grep -o 'Syntax error'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}" -d "SELECT 1; SELECT 2; SELECT" 2>&1 | grep -o 'Syntax error'
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "INSERT INTO test.t VALUES (1),(2),(3);"
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.t"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT" -d "INTO test.t VALUES (4),(5),(6);"
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.t"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?query=INSERT+INTO+test.t+VALUES" -d "(7),(8),(9)"
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.t"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}" -d "INSERT INTO t VALUES (1),(2),(3);"
$CLICKHOUSE_CLIENT --query="SELECT * FROM t"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}&query=INSERT" -d "INTO t VALUES (4),(5),(6);"
$CLICKHOUSE_CLIENT --query="SELECT * FROM t"
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL_PARAMS}&query=INSERT+INTO+t+VALUES" -d "(7),(8),(9)"
$CLICKHOUSE_CLIENT --query="SELECT * FROM t"
$CLICKHOUSE_CLIENT -n --query="DROP TABLE test.t;"
$CLICKHOUSE_CLIENT -n --query="DROP TABLE t;"

View File

@ -5,8 +5,8 @@ set -e
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.json_noisy"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test.json_noisy (d1 UInt8, d2 String) ENGINE = Memory"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS json_noisy"
$CLICKHOUSE_CLIENT -q "CREATE TABLE json_noisy (d1 UInt8, d2 String) ENGINE = Memory"
echo '{"d1" : 1, "d2" : "ok"}
{ }
@ -15,20 +15,20 @@ echo '{"d1" : 1, "d2" : "ok"}
{"d2":"ok","t1":[[[]],true, null, false, "1","2", 0.03, 1], "d1":"1", "t2":["1","2"]}
{"d2":"ok","t1":{"a":{"b": {} ,"c":false},"b":[true,null, false]}, "t2": { "a": [ ] } , "d1":1}
{"t0" : -0.1, "t1" : +1, "t2" : 0, "t3" : [0.0, -0.1], "d2" : "ok", "d1" : 1}' \
| $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 -q "INSERT INTO test.json_noisy FORMAT JSONEachRow"
| $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 -q "INSERT INTO json_noisy FORMAT JSONEachRow"
$CLICKHOUSE_CLIENT --max_threads=1 -q "SELECT * FROM test.json_noisy"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.json_noisy"
$CLICKHOUSE_CLIENT --max_threads=1 -q "SELECT * FROM json_noisy"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS json_noisy"
# Regular test for DateTime
echo
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.json_each_row"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test.json_each_row (d DateTime('Europe/Moscow')) ENGINE = Memory"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS json_each_row"
$CLICKHOUSE_CLIENT -q "CREATE TABLE json_each_row (d DateTime('Europe/Moscow')) ENGINE = Memory"
echo '{"d" : "2017-08-31 18:36:48", "t" : ""}
{"d" : "1504193808", "t" : -1}
{"d" : 1504193808, "t" : []}
{"d" : 01504193808, "t" : []}' \
| $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 -q "INSERT INTO test.json_each_row FORMAT JSONEachRow"
$CLICKHOUSE_CLIENT -q "SELECT DISTINCT * FROM test.json_each_row"
$CLICKHOUSE_CLIENT -q "DROP TABLE test.json_each_row"
| $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 -q "INSERT INTO json_each_row FORMAT JSONEachRow"
$CLICKHOUSE_CLIENT -q "SELECT DISTINCT * FROM json_each_row"
$CLICKHOUSE_CLIENT -q "DROP TABLE json_each_row"

View File

@ -1,22 +1,22 @@
DROP TABLE IF EXISTS test.aggregates;
CREATE TABLE test.aggregates (d Date, s AggregateFunction(uniq, UInt64)) ENGINE = MergeTree(d, d, 8192);
DROP TABLE IF EXISTS aggregates;
CREATE TABLE aggregates (d Date, s AggregateFunction(uniq, UInt64)) ENGINE = MergeTree(d, d, 8192);
INSERT INTO test.aggregates
INSERT INTO aggregates
SELECT toDate('2016-10-31') AS d, uniqState(toUInt64(arrayJoin(range(100)))) AS s
UNION ALL
SELECT toDate('2016-11-01') AS d, uniqState(toUInt64(arrayJoin(range(100)))) AS s;
INSERT INTO test.aggregates SELECT toDate('2016-10-31') + number AS d, uniqState(toUInt64(arrayJoin(range(100)))) AS s FROM (SELECT * FROM system.numbers LIMIT 2) GROUP BY d;
INSERT INTO aggregates SELECT toDate('2016-10-31') + number AS d, uniqState(toUInt64(arrayJoin(range(100)))) AS s FROM (SELECT * FROM system.numbers LIMIT 2) GROUP BY d;
SELECT d, uniqMerge(s) FROM test.aggregates GROUP BY d ORDER BY d;
SELECT d, uniqMerge(s) FROM aggregates GROUP BY d ORDER BY d;
INSERT INTO test.aggregates
INSERT INTO aggregates
SELECT toDate('2016-12-01') AS d, uniqState(toUInt64(arrayJoin(range(100)))) AS s
UNION ALL
SELECT toDate('2016-12-02') AS d, uniqState(toUInt64(arrayJoin(range(100)))) AS s
UNION ALL
SELECT toDate('2016-12-03') AS d, uniqState(toUInt64(arrayJoin(range(100)))) AS s;
SELECT d, uniqMerge(s) FROM test.aggregates GROUP BY d ORDER BY d;
SELECT d, uniqMerge(s) FROM aggregates GROUP BY d ORDER BY d;
DROP TABLE test.aggregates;
DROP TABLE aggregates;

View File

@ -8,23 +8,23 @@ TABLE_HASH="cityHash64(groupArray(cityHash64(*)))"
function pack_unpack_compare()
{
local buf_file="${CLICKHOUSE_TMP}/test.buf.'.$3"
local buf_file="${CLICKHOUSE_TMP}/buf.'.$3"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.buf"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.buf_file"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS buf"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS buf_file"
${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.buf ENGINE = Memory AS $1"
local res_orig=$(${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT $TABLE_HASH FROM test.buf")
${CLICKHOUSE_CLIENT} --query "CREATE TABLE buf ENGINE = Memory AS $1"
local res_orig=$(${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT $TABLE_HASH FROM buf")
${CLICKHOUSE_CLIENT} --max_threads=1 --query "CREATE TABLE test.buf_file ENGINE = File($3) AS SELECT * FROM test.buf"
local res_db_file=$(${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT $TABLE_HASH FROM test.buf_file")
${CLICKHOUSE_CLIENT} --max_threads=1 --query "CREATE TABLE buf_file ENGINE = File($3) AS SELECT * FROM buf"
local res_db_file=$(${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT $TABLE_HASH FROM buf_file")
${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT * FROM test.buf FORMAT $3" > "$buf_file"
${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT * FROM buf FORMAT $3" > "$buf_file"
local res_ch_local1=$(${CLICKHOUSE_LOCAL} --structure "$2" --file "$buf_file" --table "my super table" --input-format "$3" --output-format TabSeparated --query "SELECT $TABLE_HASH FROM \`my super table\`")
local res_ch_local2=$(${CLICKHOUSE_LOCAL} --structure "$2" --table "my super table" --input-format "$3" --output-format TabSeparated --query "SELECT $TABLE_HASH FROM \`my super table\`" < "$buf_file")
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.buf"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.buf_file"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS buf"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS buf_file"
rm -f "$buf_file" stderr
echo $((res_orig - res_db_file)) $((res_orig - res_ch_local1)) $((res_orig - res_ch_local2))

View File

@ -1,42 +1,42 @@
DROP TABLE IF EXISTS test.enum_pk;
CREATE TABLE test.enum_pk (date Date DEFAULT '0000-00-00', x Enum8('0' = 0, '1' = 1, '2' = 2), d Enum8('0' = 0, '1' = 1, '2' = 2)) ENGINE = MergeTree(date, x, 1);
INSERT INTO test.enum_pk (x, d) VALUES ('0', '0')('1', '1')('0', '0')('1', '1')('1', '1')('0', '0')('0', '0')('2', '2')('0', '0')('1', '1')('1', '1')('1', '1')('1', '1')('0', '0');
DROP TABLE IF EXISTS enum_pk;
CREATE TABLE enum_pk (date Date DEFAULT '0000-00-00', x Enum8('0' = 0, '1' = 1, '2' = 2), d Enum8('0' = 0, '1' = 1, '2' = 2)) ENGINE = MergeTree(date, x, 1);
INSERT INTO enum_pk (x, d) VALUES ('0', '0')('1', '1')('0', '0')('1', '1')('1', '1')('0', '0')('0', '0')('2', '2')('0', '0')('1', '1')('1', '1')('1', '1')('1', '1')('0', '0');
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE x = '0';
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE d = '0';
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE x = '0';
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE d = '0';
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE x != '0';
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE d != '0';
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE x != '0';
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE d != '0';
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE x = '1';
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE d = '1';
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE x = '1';
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE d = '1';
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE exp2(toInt64(x != '1')) > 1;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE exp2(toInt64(d != '1')) > 1;
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE exp2(toInt64(x != '1')) > 1;
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE exp2(toInt64(d != '1')) > 1;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE x = toString(0);
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE d = toString(0);
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE x = toString(0);
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE d = toString(0);
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE (x = toString(0)) > 0;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE (d = toString(0)) > 0;
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE (x = toString(0)) > 0;
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE (d = toString(0)) > 0;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE ((x != toString(1)) > 0) > 0;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE ((d != toString(1)) > 0) > 0;
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE ((x != toString(1)) > 0) > 0;
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE ((d != toString(1)) > 0) > 0;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE exp2((x != toString(0)) != 0) > 1;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE exp2((d != toString(0)) != 0) > 1;
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE exp2((x != toString(0)) != 0) > 1;
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE exp2((d != toString(0)) != 0) > 1;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE (-(x != toString(0)) = -1) > 0;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE (-(d != toString(0)) = -1) > 0;
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE (-(x != toString(0)) = -1) > 0;
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE (-(d != toString(0)) = -1) > 0;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE 1 = 1;
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE 1 = 1;
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE 1 = 1;
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE 1 = 1;
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE (x = '0' OR x = '1');
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE (d = '0' OR d = '1');
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE (x = '0' OR x = '1');
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE (d = '0' OR d = '1');
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE x IN ('0', '1');
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE d IN ('0', '1');
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE x IN ('0', '1');
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE d IN ('0', '1');
SELECT cityHash64(groupArray(x)) FROM test.enum_pk WHERE (x != '0' AND x != '1');
SELECT cityHash64(groupArray(d)) FROM test.enum_pk WHERE (d != '0' AND d != '1');
SELECT cityHash64(groupArray(x)) FROM enum_pk WHERE (x != '0' AND x != '1');
SELECT cityHash64(groupArray(d)) FROM enum_pk WHERE (d != '0' AND d != '1');

View File

@ -1,10 +1,10 @@
#!/usr/bin/env python
def gen_queries():
create_template = 'create table test.tab (a Int8, b String, c Tuple(Int8), d Tuple(Tuple(Int8)), e Tuple(Int8, String), f Tuple(Tuple(Int8, String))) engine = MergeTree order by ({}) partition by {}'
drop_query = 'drop table if exists test.tab'
create_template = 'create table tab (a Int8, b String, c Tuple(Int8), d Tuple(Tuple(Int8)), e Tuple(Int8, String), f Tuple(Tuple(Int8, String))) engine = MergeTree order by ({}) partition by {}'
drop_query = 'drop table if exists tab'
values = ('1', "'a'", 'tuple(1)', 'tuple(tuple(1))', "(1, 'a')", "tuple((1, 'a'))")
insert_query = "insert into test.tab values (1, 'a', tuple(1), tuple(tuple(1)), (1, 'a'), tuple((1, 'a')))"
insert_query = "insert into tab values (1, 'a', tuple(1), tuple(tuple(1)), (1, 'a'), tuple((1, 'a')))"
columns = tuple('a b c d'.split())
order_by_columns = tuple('a b c'.split())
partition_by_columns = tuple(' tuple() a'.split())
@ -17,24 +17,24 @@ def gen_queries():
yield q
for column, value in zip(columns, values):
yield 'select {} in {} from test.tab'.format(column, value)
yield 'select {} in tuple({}) from test.tab'.format(column, value)
yield 'select {} in (select {} from test.tab) from test.tab'.format(column, column)
yield 'select {} in {} from tab'.format(column, value)
yield 'select {} in tuple({}) from tab'.format(column, value)
yield 'select {} in (select {} from tab) from tab'.format(column, column)
for i in range(len(columns)):
for j in range(i, len(columns)):
yield 'select ({}, {}) in tuple({}, {}) from test.tab'.format(columns[i], columns[j], values[i], values[j])
yield 'select ({}, {}) in (select {}, {} from test.tab) from test.tab'.format(columns[i], columns[j], columns[i], columns[j])
yield 'select ({}, {}) in (select ({}, {}) from test.tab) from test.tab'.format(columns[i], columns[j], columns[i], columns[j])
yield 'select ({}, {}) in tuple({}, {}) from tab'.format(columns[i], columns[j], values[i], values[j])
yield 'select ({}, {}) in (select {}, {} from tab) from tab'.format(columns[i], columns[j], columns[i], columns[j])
yield 'select ({}, {}) in (select ({}, {}) from tab) from tab'.format(columns[i], columns[j], columns[i], columns[j])
yield "select e in (1, 'a') from test.tab"
yield "select f in tuple((1, 'a')) from test.tab"
yield "select f in tuple(tuple((1, 'a'))) from test.tab"
yield "select e in (1, 'a') from tab"
yield "select f in tuple((1, 'a')) from tab"
yield "select f in tuple(tuple((1, 'a'))) from tab"
yield 'select e in (select a, b from test.tab) from test.tab'
yield 'select e in (select (a, b) from test.tab) from test.tab'
yield 'select f in (select tuple((a, b)) from test.tab) from test.tab'
yield 'select tuple(f) in (select tuple(tuple((a, b))) from test.tab) from test.tab'
yield 'select e in (select a, b from tab) from tab'
yield 'select e in (select (a, b) from tab) from tab'
yield 'select f in (select tuple((a, b)) from tab) from tab'
yield 'select tuple(f) in (select tuple(tuple((a, b))) from tab) from tab'
import requests
import os

View File

@ -1,6 +1,6 @@
DROP TABLE IF EXISTS test.enum_totals;
CREATE TABLE test.enum_totals (e Enum8('hello' = 1, 'world' = 2)) ENGINE = Memory;
INSERT INTO test.enum_totals VALUES ('hello'), ('world'), ('world');
DROP TABLE IF EXISTS enum_totals;
CREATE TABLE enum_totals (e Enum8('hello' = 1, 'world' = 2)) ENGINE = Memory;
INSERT INTO enum_totals VALUES ('hello'), ('world'), ('world');
SELECT e, count() FROM test.enum_totals GROUP BY e WITH TOTALS ORDER BY e;
DROP TABLE test.enum_totals;
SELECT e, count() FROM enum_totals GROUP BY e WITH TOTALS ORDER BY e;
DROP TABLE enum_totals;

View File

@ -1,34 +1,34 @@
DROP TABLE IF EXISTS test.enum_nested_alter;
CREATE TABLE test.enum_nested_alter
DROP TABLE IF EXISTS enum_nested_alter;
CREATE TABLE enum_nested_alter
(d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, e Enum8('Hello' = 1), b UInt8))
ENGINE = MergeTree(d, x, 1);
INSERT INTO test.enum_nested_alter (x, n.e) VALUES (1, ['Hello']);
SELECT * FROM test.enum_nested_alter;
INSERT INTO enum_nested_alter (x, n.e) VALUES (1, ['Hello']);
SELECT * FROM enum_nested_alter;
ALTER TABLE test.enum_nested_alter MODIFY COLUMN n.e Array(Enum8('Hello' = 1, 'World' = 2));
INSERT INTO test.enum_nested_alter (x, n.e) VALUES (2, ['World']);
SELECT * FROM test.enum_nested_alter ORDER BY x;
ALTER TABLE enum_nested_alter MODIFY COLUMN n.e Array(Enum8('Hello' = 1, 'World' = 2));
INSERT INTO enum_nested_alter (x, n.e) VALUES (2, ['World']);
SELECT * FROM enum_nested_alter ORDER BY x;
ALTER TABLE test.enum_nested_alter MODIFY COLUMN n.e Array(Enum16('Hello' = 1, 'World' = 2, 'a' = 300));
SELECT * FROM test.enum_nested_alter ORDER BY x;
ALTER TABLE enum_nested_alter MODIFY COLUMN n.e Array(Enum16('Hello' = 1, 'World' = 2, 'a' = 300));
SELECT * FROM enum_nested_alter ORDER BY x;
ALTER TABLE test.enum_nested_alter MODIFY COLUMN n.e Array(UInt16);
SELECT * FROM test.enum_nested_alter ORDER BY x;
ALTER TABLE enum_nested_alter MODIFY COLUMN n.e Array(UInt16);
SELECT * FROM enum_nested_alter ORDER BY x;
ALTER TABLE test.enum_nested_alter MODIFY COLUMN n.e Array(Enum16('Hello' = 1, 'World' = 2, 'a' = 300));
SELECT * FROM test.enum_nested_alter ORDER BY x;
ALTER TABLE enum_nested_alter MODIFY COLUMN n.e Array(Enum16('Hello' = 1, 'World' = 2, 'a' = 300));
SELECT * FROM enum_nested_alter ORDER BY x;
ALTER TABLE test.enum_nested_alter MODIFY COLUMN n.e Array(String);
SELECT * FROM test.enum_nested_alter ORDER BY x;
ALTER TABLE enum_nested_alter MODIFY COLUMN n.e Array(String);
SELECT * FROM enum_nested_alter ORDER BY x;
ALTER TABLE test.enum_nested_alter MODIFY COLUMN n.e Array(Enum16('Hello' = 1, 'World' = 2, 'a' = 300));
SELECT * FROM test.enum_nested_alter ORDER BY x;
ALTER TABLE enum_nested_alter MODIFY COLUMN n.e Array(Enum16('Hello' = 1, 'World' = 2, 'a' = 300));
SELECT * FROM enum_nested_alter ORDER BY x;
DROP TABLE test.enum_nested_alter;
DROP TABLE enum_nested_alter;
CREATE TABLE test.enum_nested_alter
CREATE TABLE enum_nested_alter
(
d Date DEFAULT '2000-01-01',
x UInt64,
@ -39,30 +39,30 @@ CREATE TABLE test.enum_nested_alter
status Enum16('hello' = 1, 'world' = 2)))
ENGINE = MergeTree(d, x, 1);
INSERT INTO test.enum_nested_alter (x, tasks.errcategory, tasks.status) VALUES (1, ['system', 'rtb.client'], ['hello', 'world']);
SELECT * FROM test.enum_nested_alter ORDER BY x;
INSERT INTO enum_nested_alter (x, tasks.errcategory, tasks.status) VALUES (1, ['system', 'rtb.client'], ['hello', 'world']);
SELECT * FROM enum_nested_alter ORDER BY x;
ALTER TABLE test.enum_nested_alter
ALTER TABLE enum_nested_alter
MODIFY COLUMN tasks.errcategory Array(Enum8(
'undefined' = 0, 'system' = 1, 'generic' = 2, 'asio.netdb' = 3, 'asio.misc' = 4,
'asio.addrinfo' = 5, 'rtb.client' = 6, 'rtb.logic' = 7, 'http.status' = 8, 'http.code' = 9)),
MODIFY COLUMN tasks.status Array(Enum8('hello' = 1, 'world' = 2, 'goodbye' = 3));
INSERT INTO test.enum_nested_alter (x, tasks.errcategory, tasks.status) VALUES (2, ['http.status', 'http.code'], ['hello', 'goodbye']);
SELECT * FROM test.enum_nested_alter ORDER BY x;
INSERT INTO enum_nested_alter (x, tasks.errcategory, tasks.status) VALUES (2, ['http.status', 'http.code'], ['hello', 'goodbye']);
SELECT * FROM enum_nested_alter ORDER BY x;
DROP TABLE test.enum_nested_alter;
DROP TABLE enum_nested_alter;
DROP TABLE IF EXISTS test.enum_nested_alter;
CREATE TABLE test.enum_nested_alter
DROP TABLE IF EXISTS enum_nested_alter;
CREATE TABLE enum_nested_alter
(d Date DEFAULT '2000-01-01', x UInt64, n Nested(a String, e Enum8('Hello.world' = 1), b UInt8))
ENGINE = MergeTree(d, x, 1);
INSERT INTO test.enum_nested_alter (x, n.e) VALUES (1, ['Hello.world']);
SELECT * FROM test.enum_nested_alter;
INSERT INTO enum_nested_alter (x, n.e) VALUES (1, ['Hello.world']);
SELECT * FROM enum_nested_alter;
ALTER TABLE test.enum_nested_alter MODIFY COLUMN n.e Array(Enum8('Hello.world' = 1, 'a' = 2));
SELECT * FROM test.enum_nested_alter;
ALTER TABLE enum_nested_alter MODIFY COLUMN n.e Array(Enum8('Hello.world' = 1, 'a' = 2));
SELECT * FROM enum_nested_alter;
DROP TABLE test.enum_nested_alter;
DROP TABLE enum_nested_alter;

View File

@ -1,17 +1,17 @@
DROP TABLE IF EXISTS test.alter;
CREATE TABLE test.alter (d Date, k UInt64, i32 Int32, n Nested(ui8 UInt8, s String)) ENGINE=MergeTree(d, k, 8192);
DROP TABLE IF EXISTS alter;
CREATE TABLE alter (d Date, k UInt64, i32 Int32, n Nested(ui8 UInt8, s String)) ENGINE=MergeTree(d, k, 8192);
INSERT INTO test.alter VALUES ('2015-01-01', 3, 30, [1,2,3], ['1','12','123']);
INSERT INTO test.alter VALUES ('2015-01-01', 2, 20, [1,2], ['1','12']);
INSERT INTO test.alter VALUES ('2015-01-01', 1, 10, [1], ['1']);
INSERT INTO alter VALUES ('2015-01-01', 3, 30, [1,2,3], ['1','12','123']);
INSERT INTO alter VALUES ('2015-01-01', 2, 20, [1,2], ['1','12']);
INSERT INTO alter VALUES ('2015-01-01', 1, 10, [1], ['1']);
ALTER TABLE test.alter ADD COLUMN `n.i8` Array(Int8) AFTER i32;
ALTER TABLE alter ADD COLUMN `n.i8` Array(Int8) AFTER i32;
SELECT `n.i8`, `n.ui8`, `n.s` FROM test.alter ORDER BY k;
SELECT `n.i8` FROM test.alter ORDER BY k;
SELECT `n.i8`, `n.ui8`, `n.s` FROM alter ORDER BY k;
SELECT `n.i8` FROM alter ORDER BY k;
OPTIMIZE TABLE test.alter;
OPTIMIZE TABLE alter;
SELECT `n.i8` FROM test.alter ORDER BY k;
SELECT `n.i8` FROM alter ORDER BY k;
DROP TABLE IF EXISTS test.alter;
DROP TABLE IF EXISTS alter;

View File

@ -1,24 +1,24 @@
DROP TABLE IF EXISTS test.replaceall;
CREATE TABLE test.replaceall (str FixedString(3)) ENGINE = Memory;
DROP TABLE IF EXISTS replaceall;
CREATE TABLE replaceall (str FixedString(3)) ENGINE = Memory;
INSERT INTO test.replaceall VALUES ('foo');
INSERT INTO test.replaceall VALUES ('boa');
INSERT INTO test.replaceall VALUES ('bar');
INSERT INTO test.replaceall VALUES ('bao');
INSERT INTO replaceall VALUES ('foo');
INSERT INTO replaceall VALUES ('boa');
INSERT INTO replaceall VALUES ('bar');
INSERT INTO replaceall VALUES ('bao');
SELECT
str,
replaceAll(str, 'o', '*') AS replaced
FROM test.replaceall
FROM replaceall
ORDER BY str ASC;
DROP TABLE test.replaceall;
DROP TABLE replaceall;
CREATE TABLE test.replaceall (date Date DEFAULT today(), fs FixedString(16)) ENGINE = MergeTree(date, (date, fs), 8192);
INSERT INTO test.replaceall (fs) VALUES ('54db0d43009d\0\0\0\0'), ('fe2b58224766cf10'), ('54db0d43009d\0\0\0\0'), ('fe2b58224766cf10');
CREATE TABLE replaceall (date Date DEFAULT today(), fs FixedString(16)) ENGINE = MergeTree(date, (date, fs), 8192);
INSERT INTO replaceall (fs) VALUES ('54db0d43009d\0\0\0\0'), ('fe2b58224766cf10'), ('54db0d43009d\0\0\0\0'), ('fe2b58224766cf10');
SELECT fs, replaceAll(fs, '\0', '*')
FROM test.replaceall
FROM replaceall
ORDER BY fs ASC;
DROP TABLE test.replaceall;
DROP TABLE replaceall;

View File

@ -8,124 +8,124 @@ SELECT NULL + NULL;
SELECT '----- MergeTree engine -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(
col1 UInt64, col2 Nullable(UInt64),
col3 String, col4 Nullable(String),
col5 Array(UInt64), col6 Array(Nullable(UInt64)),
col7 Array(String), col8 Array(Nullable(String)),
d Date) Engine = MergeTree(d, (col1, d), 8192);
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, NULL, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', NULL, [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [NULL], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], [NULL], '2000-01-01');
SELECT * FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8 ASC;
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, NULL, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', NULL, [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [NULL], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], [NULL], '2000-01-01');
SELECT * FROM test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8 ASC;
SELECT '----- Memory engine -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(
col1 UInt64, col2 Nullable(UInt64),
col3 String, col4 Nullable(String),
col5 Array(UInt64), col6 Array(Nullable(UInt64)),
col7 Array(String), col8 Array(Nullable(String)),
d Date) Engine = Memory;
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, NULL, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', NULL, [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [NULL], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], [NULL], '2000-01-01');
SELECT * FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8 ASC;
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, NULL, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', NULL, [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [NULL], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], [NULL], '2000-01-01');
SELECT * FROM test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8 ASC;
SELECT '----- TinyLog engine -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(
col1 UInt64, col2 Nullable(UInt64),
col3 String, col4 Nullable(String),
col5 Array(UInt64), col6 Array(Nullable(UInt64)),
col7 Array(String), col8 Array(Nullable(String)),
d Date) Engine = TinyLog;
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, NULL, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', NULL, [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [NULL], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], [NULL], '2000-01-01');
SELECT * FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8 ASC;
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, NULL, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', NULL, [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [NULL], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], [NULL], '2000-01-01');
SELECT * FROM test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8 ASC;
SELECT '----- Log engine -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(
col1 UInt64, col2 Nullable(UInt64),
col3 String, col4 Nullable(String),
col5 Array(UInt64), col6 Array(Nullable(UInt64)),
col7 Array(String), col8 Array(Nullable(String)),
d Date) Engine = Log;
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, NULL, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', NULL, [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [NULL], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], [NULL], '2000-01-01');
SELECT * FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8 ASC;
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, NULL, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', NULL, [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [NULL], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], [NULL], '2000-01-01');
SELECT * FROM test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8 ASC;
SELECT '----- StripeLog engine -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(
col1 UInt64, col2 Nullable(UInt64),
col3 String, col4 Nullable(String),
col5 Array(UInt64), col6 Array(Nullable(UInt64)),
col7 Array(String), col8 Array(Nullable(String)),
d Date) Engine = StripeLog;
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, NULL, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', NULL, [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [NULL], ['a'], ['a'], '2000-01-01');
INSERT INTO test.test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], [NULL], '2000-01-01');
SELECT * FROM test.test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8 ASC;
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, NULL, 'a', 'a', [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', NULL, [1], [1], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [NULL], ['a'], ['a'], '2000-01-01');
INSERT INTO test1 VALUES (1, 1, 'a', 'a', [1], [1], ['a'], [NULL], '2000-01-01');
SELECT * FROM test1 ORDER BY col1,col2,col3,col4,col5,col6,col7,col8 ASC;
SELECT '----- Insert with expression -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64))) Engine=Memory;
INSERT INTO test.test1(col1) VALUES ([1+1]);
SELECT col1 FROM test.test1 ORDER BY col1 ASC;
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(UInt64))) Engine=Memory;
INSERT INTO test1(col1) VALUES ([1+1]);
SELECT col1 FROM test1 ORDER BY col1 ASC;
SELECT '----- Insert. Source and target columns have same types up to nullability. -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(UInt64), col2 UInt64) Engine=Memory;
DROP TABLE IF EXISTS test.test2;
CREATE TABLE test.test2(col1 UInt64, col2 Nullable(UInt64)) Engine=Memory;
INSERT INTO test.test1(col1,col2) VALUES (2,7)(6,9)(5,1)(4,3)(8,2);
INSERT INTO test.test2(col1,col2) SELECT col1,col2 FROM test.test1;
SELECT col1,col2 FROM test.test2 ORDER BY col1,col2 ASC;
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Nullable(UInt64), col2 UInt64) Engine=Memory;
DROP TABLE IF EXISTS test2;
CREATE TABLE test2(col1 UInt64, col2 Nullable(UInt64)) Engine=Memory;
INSERT INTO test1(col1,col2) VALUES (2,7)(6,9)(5,1)(4,3)(8,2);
INSERT INTO test2(col1,col2) SELECT col1,col2 FROM test1;
SELECT col1,col2 FROM test2 ORDER BY col1,col2 ASC;
SELECT '----- Apply functions and aggregate functions on columns that may contain null values -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(UInt64), col2 Nullable(UInt64)) Engine=Memory;
INSERT INTO test.test1(col1,col2) VALUES (2,7)(NULL,6)(9,NULL)(NULL,NULL)(5,1)(42,42);
SELECT col1, col2, col1 + col2, col1 * 7 FROM test.test1 ORDER BY col1,col2 ASC;
SELECT sum(col1) FROM test.test1;
SELECT sum(col1 * 7) FROM test.test1;
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Nullable(UInt64), col2 Nullable(UInt64)) Engine=Memory;
INSERT INTO test1(col1,col2) VALUES (2,7)(NULL,6)(9,NULL)(NULL,NULL)(5,1)(42,42);
SELECT col1, col2, col1 + col2, col1 * 7 FROM test1 ORDER BY col1,col2 ASC;
SELECT sum(col1) FROM test1;
SELECT sum(col1 * 7) FROM test1;
SELECT '----- isNull, isNotNull -----';
SELECT col1, col2, isNull(col1), isNotNull(col2) FROM test.test1 ORDER BY col1,col2 ASC;
SELECT col1, col2, isNull(col1), isNotNull(col2) FROM test1 ORDER BY col1,col2 ASC;
SELECT '----- ifNull, nullIf -----';
SELECT col1, col2, ifNull(col1,col2) FROM test.test1 ORDER BY col1,col2 ASC;
SELECT col1, col2, nullIf(col1,col2) FROM test.test1 ORDER BY col1,col2 ASC;
SELECT col1, col2, ifNull(col1,col2) FROM test1 ORDER BY col1,col2 ASC;
SELECT col1, col2, nullIf(col1,col2) FROM test1 ORDER BY col1,col2 ASC;
SELECT '----- coalesce -----';
@ -134,17 +134,17 @@ SELECT coalesce(NULL, 1);
SELECT coalesce(NULL, NULL, 1);
SELECT coalesce(NULL, 42, NULL, 1);
SELECT coalesce(NULL, NULL, NULL);
SELECT col1, col2, coalesce(col1, col2) FROM test.test1 ORDER BY col1, col2 ASC;
SELECT col1, col2, coalesce(col1, col2, 99) FROM test.test1 ORDER BY col1, col2 ASC;
SELECT col1, col2, coalesce(col1, col2) FROM test1 ORDER BY col1, col2 ASC;
SELECT col1, col2, coalesce(col1, col2, 99) FROM test1 ORDER BY col1, col2 ASC;
SELECT '----- assumeNotNull -----';
SELECT res FROM (SELECT col1, assumeNotNull(col1) AS res FROM test.test1) WHERE col1 IS NOT NULL ORDER BY res ASC;
SELECT res FROM (SELECT col1, assumeNotNull(col1) AS res FROM test1) WHERE col1 IS NOT NULL ORDER BY res ASC;
SELECT '----- IS NULL, IS NOT NULL -----';
SELECT col1 FROM test.test1 WHERE col1 IS NOT NULL ORDER BY col1 ASC;
SELECT col1 FROM test.test1 WHERE col1 IS NULL;
SELECT col1 FROM test1 WHERE col1 IS NOT NULL ORDER BY col1 ASC;
SELECT col1 FROM test1 WHERE col1 IS NULL;
SELECT '----- multiIf -----';
@ -158,15 +158,15 @@ SELECT multiIf(1, toFixedString('A', 16), 1, NULL, toFixedString('DEF', 16));
SELECT multiIf(NULL, 2, 1, 3, 4);
SELECT multiIf(1, 2, NULL, 3, 4);
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(Int8), col2 Nullable(UInt16), col3 Nullable(Float32)) Engine=TinyLog;
INSERT INTO test.test1(col1,col2,col3) VALUES (toInt8(1),toUInt16(2),toFloat32(3))(NULL,toUInt16(1),toFloat32(2))(toInt8(1),NULL,toFloat32(2))(toInt8(1),toUInt16(2),NULL);
SELECT multiIf(col1 == 1, col2, col2 == 2, col3, col3 == 3, col1, 42) FROM test.test1;
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Nullable(Int8), col2 Nullable(UInt16), col3 Nullable(Float32)) Engine=TinyLog;
INSERT INTO test1(col1,col2,col3) VALUES (toInt8(1),toUInt16(2),toFloat32(3))(NULL,toUInt16(1),toFloat32(2))(toInt8(1),NULL,toFloat32(2))(toInt8(1),toUInt16(2),NULL);
SELECT multiIf(col1 == 1, col2, col2 == 2, col3, col3 == 3, col1, 42) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(cond1 Nullable(UInt8), then1 Int8, cond2 UInt8, then2 Nullable(UInt16), then3 Nullable(Float32)) Engine=TinyLog;
INSERT INTO test.test1(cond1,then1,cond2,then2,then3) VALUES(1,1,1,42,99)(0,7,1,99,42)(NULL,6,2,99,NULL);
SELECT multiIf(cond1,then1,cond2,then2,then3) FROM test.test1;
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(cond1 Nullable(UInt8), then1 Int8, cond2 UInt8, then2 Nullable(UInt16), then3 Nullable(Float32)) Engine=TinyLog;
INSERT INTO test1(cond1,then1,cond2,then2,then3) VALUES(1,1,1,42,99)(0,7,1,99,42)(NULL,6,2,99,NULL);
SELECT multiIf(cond1,then1,cond2,then2,then3) FROM test1;
SELECT '----- Array functions -----';
@ -194,59 +194,59 @@ SELECT arrayElement(['a',NULL,'c','d'], 2);
SELECT arrayElement(['a',NULL,'c','d'], 3);
SELECT arrayElement(['a',NULL,'c','d'], 4);
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 UInt64) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES(1),(2),(3),(4);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 UInt64) Engine=TinyLog;
INSERT INTO test1(col1) VALUES(1),(2),(3),(4);
SELECT arrayElement([1,NULL,2,3], col1) FROM test.test1;
SELECT arrayElement([1,NULL,2,3], col1) FROM test1;
SELECT '----- variable arrays -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64))) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES([2,3,7,NULL]);
INSERT INTO test.test1(col1) VALUES([NULL,3,7,4]);
INSERT INTO test.test1(col1) VALUES([2,NULL,7,NULL]);
INSERT INTO test.test1(col1) VALUES([2,3,NULL,4]);
INSERT INTO test.test1(col1) VALUES([NULL,NULL,NULL,NULL]);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(UInt64))) Engine=TinyLog;
INSERT INTO test1(col1) VALUES([2,3,7,NULL]);
INSERT INTO test1(col1) VALUES([NULL,3,7,4]);
INSERT INTO test1(col1) VALUES([2,NULL,7,NULL]);
INSERT INTO test1(col1) VALUES([2,3,NULL,4]);
INSERT INTO test1(col1) VALUES([NULL,NULL,NULL,NULL]);
SELECT arrayElement(col1, 1) FROM test.test1;
SELECT arrayElement(col1, 2) FROM test.test1;
SELECT arrayElement(col1, 3) FROM test.test1;
SELECT arrayElement(col1, 4) FROM test.test1;
SELECT arrayElement(col1, 1) FROM test1;
SELECT arrayElement(col1, 2) FROM test1;
SELECT arrayElement(col1, 3) FROM test1;
SELECT arrayElement(col1, 4) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(String))) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES(['a','bc','def',NULL]);
INSERT INTO test.test1(col1) VALUES([NULL,'bc','def','ghij']);
INSERT INTO test.test1(col1) VALUES(['a',NULL,'def',NULL]);
INSERT INTO test.test1(col1) VALUES(['a','bc',NULL,'ghij']);
INSERT INTO test.test1(col1) VALUES([NULL,NULL,NULL,NULL]);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(String))) Engine=TinyLog;
INSERT INTO test1(col1) VALUES(['a','bc','def',NULL]);
INSERT INTO test1(col1) VALUES([NULL,'bc','def','ghij']);
INSERT INTO test1(col1) VALUES(['a',NULL,'def',NULL]);
INSERT INTO test1(col1) VALUES(['a','bc',NULL,'ghij']);
INSERT INTO test1(col1) VALUES([NULL,NULL,NULL,NULL]);
SELECT arrayElement(col1, 1) FROM test.test1;
SELECT arrayElement(col1, 2) FROM test.test1;
SELECT arrayElement(col1, 3) FROM test.test1;
SELECT arrayElement(col1, 4) FROM test.test1;
SELECT arrayElement(col1, 1) FROM test1;
SELECT arrayElement(col1, 2) FROM test1;
SELECT arrayElement(col1, 3) FROM test1;
SELECT arrayElement(col1, 4) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64)), col2 UInt64) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES([2,3,7,NULL], 1);
INSERT INTO test.test1(col1,col2) VALUES([NULL,3,7,4], 2);
INSERT INTO test.test1(col1,col2) VALUES([2,NULL,7,NULL], 3);
INSERT INTO test.test1(col1,col2) VALUES([2,3,NULL,4],4);
INSERT INTO test.test1(col1,col2) VALUES([NULL,NULL,NULL,NULL],3);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(UInt64)), col2 UInt64) Engine=TinyLog;
INSERT INTO test1(col1,col2) VALUES([2,3,7,NULL], 1);
INSERT INTO test1(col1,col2) VALUES([NULL,3,7,4], 2);
INSERT INTO test1(col1,col2) VALUES([2,NULL,7,NULL], 3);
INSERT INTO test1(col1,col2) VALUES([2,3,NULL,4],4);
INSERT INTO test1(col1,col2) VALUES([NULL,NULL,NULL,NULL],3);
SELECT arrayElement(col1,col2) FROM test.test1;
SELECT arrayElement(col1,col2) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(String)), col2 UInt64) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES(['a','bc','def',NULL], 1);
INSERT INTO test.test1(col1,col2) VALUES([NULL,'bc','def','ghij'], 2);
INSERT INTO test.test1(col1,col2) VALUES(['a',NULL,'def','ghij'], 3);
INSERT INTO test.test1(col1,col2) VALUES(['a','bc',NULL,'ghij'],4);
INSERT INTO test.test1(col1,col2) VALUES([NULL,NULL,NULL,NULL],3);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(String)), col2 UInt64) Engine=TinyLog;
INSERT INTO test1(col1,col2) VALUES(['a','bc','def',NULL], 1);
INSERT INTO test1(col1,col2) VALUES([NULL,'bc','def','ghij'], 2);
INSERT INTO test1(col1,col2) VALUES(['a',NULL,'def','ghij'], 3);
INSERT INTO test1(col1,col2) VALUES(['a','bc',NULL,'ghij'],4);
INSERT INTO test1(col1,col2) VALUES([NULL,NULL,NULL,NULL],3);
SELECT arrayElement(col1,col2) FROM test.test1;
SELECT arrayElement(col1,col2) FROM test1;
SELECT '----- has -----';
@ -263,220 +263,220 @@ SELECT has(['a',NULL,'def','ghij'], NULL);
SELECT has(['a',NULL,'def','ghij'], 'def');
SELECT has(['a',NULL,'def','ghij'], 'ghij');
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 UInt64) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES(1),(2),(3),(4);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 UInt64) Engine=TinyLog;
INSERT INTO test1(col1) VALUES(1),(2),(3),(4);
SELECT has([1,NULL,2,3], col1) FROM test.test1;
SELECT has([1,NULL,2,3], col1) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(UInt64)) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES(1),(2),(3),(4),(NULL);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Nullable(UInt64)) Engine=TinyLog;
INSERT INTO test1(col1) VALUES(1),(2),(3),(4),(NULL);
SELECT has([1,NULL,2,3], col1) FROM test.test1;
SELECT has([1,NULL,2,3], col1) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 String) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES('a'),('bc'),('def'),('ghij');
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 String) Engine=TinyLog;
INSERT INTO test1(col1) VALUES('a'),('bc'),('def'),('ghij');
SELECT has(['a',NULL,'def','ghij'], col1) FROM test.test1;
SELECT has(['a',NULL,'def','ghij'], col1) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(String)) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES('a'),('bc'),('def'),('ghij'),(NULL);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Nullable(String)) Engine=TinyLog;
INSERT INTO test1(col1) VALUES('a'),('bc'),('def'),('ghij'),(NULL);
SELECT has(['a',NULL,'def','ghij'], col1) FROM test.test1;
SELECT has(['a',NULL,'def','ghij'], col1) FROM test1;
SELECT '----- variable arrays -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64))) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES([2,3,7,NULL]);
INSERT INTO test.test1(col1) VALUES([NULL,3,7,4]);
INSERT INTO test.test1(col1) VALUES([2,NULL,7,NULL]);
INSERT INTO test.test1(col1) VALUES([2,3,NULL,4]);
INSERT INTO test.test1(col1) VALUES([NULL,NULL,NULL,NULL]);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(UInt64))) Engine=TinyLog;
INSERT INTO test1(col1) VALUES([2,3,7,NULL]);
INSERT INTO test1(col1) VALUES([NULL,3,7,4]);
INSERT INTO test1(col1) VALUES([2,NULL,7,NULL]);
INSERT INTO test1(col1) VALUES([2,3,NULL,4]);
INSERT INTO test1(col1) VALUES([NULL,NULL,NULL,NULL]);
SELECT has(col1, 2) FROM test.test1;
SELECT has(col1, 3) FROM test.test1;
SELECT has(col1, 4) FROM test.test1;
SELECT has(col1, 5) FROM test.test1;
SELECT has(col1, 7) FROM test.test1;
SELECT has(col1, NULL) FROM test.test1;
SELECT has(col1, 2) FROM test1;
SELECT has(col1, 3) FROM test1;
SELECT has(col1, 4) FROM test1;
SELECT has(col1, 5) FROM test1;
SELECT has(col1, 7) FROM test1;
SELECT has(col1, NULL) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(String))) Engine=TinyLog;
INSERT INTO test.test1(col1) VALUES(['a','bc','def',NULL]);
INSERT INTO test.test1(col1) VALUES([NULL,'bc','def','ghij']);
INSERT INTO test.test1(col1) VALUES(['a',NULL,'def',NULL]);
INSERT INTO test.test1(col1) VALUES(['a','bc',NULL,'ghij']);
INSERT INTO test.test1(col1) VALUES([NULL,NULL,NULL,NULL]);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(String))) Engine=TinyLog;
INSERT INTO test1(col1) VALUES(['a','bc','def',NULL]);
INSERT INTO test1(col1) VALUES([NULL,'bc','def','ghij']);
INSERT INTO test1(col1) VALUES(['a',NULL,'def',NULL]);
INSERT INTO test1(col1) VALUES(['a','bc',NULL,'ghij']);
INSERT INTO test1(col1) VALUES([NULL,NULL,NULL,NULL]);
SELECT has(col1, 'a') FROM test.test1;
SELECT has(col1, 'bc') FROM test.test1;
SELECT has(col1, 'def') FROM test.test1;
SELECT has(col1, 'ghij') FROM test.test1;
SELECT has(col1, NULL) FROM test.test1;
SELECT has(col1, 'a') FROM test1;
SELECT has(col1, 'bc') FROM test1;
SELECT has(col1, 'def') FROM test1;
SELECT has(col1, 'ghij') FROM test1;
SELECT has(col1, NULL) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64)), col2 UInt64) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES([2,3,7,NULL], 2);
INSERT INTO test.test1(col1,col2) VALUES([NULL,3,7,4], 3);
INSERT INTO test.test1(col1,col2) VALUES([2,NULL,7,NULL], 7);
INSERT INTO test.test1(col1,col2) VALUES([2,3,NULL,4],5);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(UInt64)), col2 UInt64) Engine=TinyLog;
INSERT INTO test1(col1,col2) VALUES([2,3,7,NULL], 2);
INSERT INTO test1(col1,col2) VALUES([NULL,3,7,4], 3);
INSERT INTO test1(col1,col2) VALUES([2,NULL,7,NULL], 7);
INSERT INTO test1(col1,col2) VALUES([2,3,NULL,4],5);
SELECT has(col1,col2) FROM test.test1;
SELECT has(col1,col2) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt64)), col2 Nullable(UInt64)) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES([2,3,7,NULL], 2);
INSERT INTO test.test1(col1,col2) VALUES([NULL,3,7,4], 3);
INSERT INTO test.test1(col1,col2) VALUES([2,NULL,7,NULL], 7);
INSERT INTO test.test1(col1,col2) VALUES([2,3,NULL,4],5);
INSERT INTO test.test1(col1,col2) VALUES([NULL,NULL,NULL,NULL],NULL);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(UInt64)), col2 Nullable(UInt64)) Engine=TinyLog;
INSERT INTO test1(col1,col2) VALUES([2,3,7,NULL], 2);
INSERT INTO test1(col1,col2) VALUES([NULL,3,7,4], 3);
INSERT INTO test1(col1,col2) VALUES([2,NULL,7,NULL], 7);
INSERT INTO test1(col1,col2) VALUES([2,3,NULL,4],5);
INSERT INTO test1(col1,col2) VALUES([NULL,NULL,NULL,NULL],NULL);
SELECT has(col1,col2) FROM test.test1;
SELECT has(col1,col2) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(String)), col2 String) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES(['a','bc','def',NULL], 'a');
INSERT INTO test.test1(col1,col2) VALUES([NULL,'bc','def','ghij'], 'bc');
INSERT INTO test.test1(col1,col2) VALUES(['a',NULL,'def','ghij'], 'def');
INSERT INTO test.test1(col1,col2) VALUES(['a','bc',NULL,'ghij'], 'ghij');
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(String)), col2 String) Engine=TinyLog;
INSERT INTO test1(col1,col2) VALUES(['a','bc','def',NULL], 'a');
INSERT INTO test1(col1,col2) VALUES([NULL,'bc','def','ghij'], 'bc');
INSERT INTO test1(col1,col2) VALUES(['a',NULL,'def','ghij'], 'def');
INSERT INTO test1(col1,col2) VALUES(['a','bc',NULL,'ghij'], 'ghij');
SELECT has(col1,col2) FROM test.test1;
SELECT has(col1,col2) FROM test1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(String)), col2 Nullable(String)) Engine=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES(['a','bc','def',NULL], 'a');
INSERT INTO test.test1(col1,col2) VALUES([NULL,'bc','def','ghij'], 'bc');
INSERT INTO test.test1(col1,col2) VALUES(['a',NULL,'def','ghij'], 'def');
INSERT INTO test.test1(col1,col2) VALUES(['a','bc',NULL,'ghij'], 'ghij');
INSERT INTO test.test1(col1,col2) VALUES([NULL,NULL,NULL,NULL], NULL);
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(String)), col2 Nullable(String)) Engine=TinyLog;
INSERT INTO test1(col1,col2) VALUES(['a','bc','def',NULL], 'a');
INSERT INTO test1(col1,col2) VALUES([NULL,'bc','def','ghij'], 'bc');
INSERT INTO test1(col1,col2) VALUES(['a',NULL,'def','ghij'], 'def');
INSERT INTO test1(col1,col2) VALUES(['a','bc',NULL,'ghij'], 'ghij');
INSERT INTO test1(col1,col2) VALUES([NULL,NULL,NULL,NULL], NULL);
SELECT has(col1,col2) FROM test.test1;
SELECT has(col1,col2) FROM test1;
SELECT '----- Aggregation -----';
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(String), col2 Nullable(UInt8), col3 String) ENGINE=TinyLog;
INSERT INTO test.test1(col1,col2,col3) VALUES('A', 0, 'ABCDEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('A', 0, 'BACDEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('A', 1, 'BCADEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('A', 1, 'BCDAEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('B', 1, 'BCDEAFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('B', 1, 'BCDEFAGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('B', 1, 'BCDEFGAH');
INSERT INTO test.test1(col1,col2,col3) VALUES('B', 1, 'BCDEFGHA');
INSERT INTO test.test1(col1,col2,col3) VALUES('C', 1, 'ACBDEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('C', NULL, 'ACDBEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('C', NULL, 'ACDEBFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('C', NULL, 'ACDEFBGH');
INSERT INTO test.test1(col1,col2,col3) VALUES(NULL, 1, 'ACDEFGBH');
INSERT INTO test.test1(col1,col2,col3) VALUES(NULL, NULL, 'ACDEFGHB');
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Nullable(String), col2 Nullable(UInt8), col3 String) ENGINE=TinyLog;
INSERT INTO test1(col1,col2,col3) VALUES('A', 0, 'ABCDEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('A', 0, 'BACDEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('A', 1, 'BCADEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('A', 1, 'BCDAEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('B', 1, 'BCDEAFGH');
INSERT INTO test1(col1,col2,col3) VALUES('B', 1, 'BCDEFAGH');
INSERT INTO test1(col1,col2,col3) VALUES('B', 1, 'BCDEFGAH');
INSERT INTO test1(col1,col2,col3) VALUES('B', 1, 'BCDEFGHA');
INSERT INTO test1(col1,col2,col3) VALUES('C', 1, 'ACBDEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('C', NULL, 'ACDBEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('C', NULL, 'ACDEBFGH');
INSERT INTO test1(col1,col2,col3) VALUES('C', NULL, 'ACDEFBGH');
INSERT INTO test1(col1,col2,col3) VALUES(NULL, 1, 'ACDEFGBH');
INSERT INTO test1(col1,col2,col3) VALUES(NULL, NULL, 'ACDEFGHB');
SELECT col1, col2, count() FROM test.test1 GROUP BY col1, col2 ORDER BY col1, col2;
SELECT col1, col2, count() FROM test1 GROUP BY col1, col2 ORDER BY col1, col2;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 String, col2 Nullable(UInt8), col3 String) ENGINE=TinyLog;
INSERT INTO test.test1(col1,col2,col3) VALUES('A', 0, 'ABCDEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('A', 0, 'BACDEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('A', 1, 'BCADEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('A', 1, 'BCDAEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('B', 1, 'BCDEAFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('B', 1, 'BCDEFAGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('B', 1, 'BCDEFGAH');
INSERT INTO test.test1(col1,col2,col3) VALUES('B', 1, 'BCDEFGHA');
INSERT INTO test.test1(col1,col2,col3) VALUES('C', 1, 'ACBDEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('C', NULL, 'ACDBEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('C', NULL, 'ACDEBFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES('C', NULL, 'ACDEFBGH');
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 String, col2 Nullable(UInt8), col3 String) ENGINE=TinyLog;
INSERT INTO test1(col1,col2,col3) VALUES('A', 0, 'ABCDEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('A', 0, 'BACDEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('A', 1, 'BCADEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('A', 1, 'BCDAEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('B', 1, 'BCDEAFGH');
INSERT INTO test1(col1,col2,col3) VALUES('B', 1, 'BCDEFAGH');
INSERT INTO test1(col1,col2,col3) VALUES('B', 1, 'BCDEFGAH');
INSERT INTO test1(col1,col2,col3) VALUES('B', 1, 'BCDEFGHA');
INSERT INTO test1(col1,col2,col3) VALUES('C', 1, 'ACBDEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('C', NULL, 'ACDBEFGH');
INSERT INTO test1(col1,col2,col3) VALUES('C', NULL, 'ACDEBFGH');
INSERT INTO test1(col1,col2,col3) VALUES('C', NULL, 'ACDEFBGH');
SELECT col1, col2, count() FROM test.test1 GROUP BY col1, col2 ORDER BY col1, col2;
SELECT col1, col2, count() FROM test1 GROUP BY col1, col2 ORDER BY col1, col2;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(String), col2 String) ENGINE=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES('A', 'ABCDEFGH');
INSERT INTO test.test1(col1,col2) VALUES('A', 'BACDEFGH');
INSERT INTO test.test1(col1,col2) VALUES('A', 'BCADEFGH');
INSERT INTO test.test1(col1,col2) VALUES('A', 'BCDAEFGH');
INSERT INTO test.test1(col1,col2) VALUES('B', 'BCDEAFGH');
INSERT INTO test.test1(col1,col2) VALUES('B', 'BCDEFAGH');
INSERT INTO test.test1(col1,col2) VALUES('B', 'BCDEFGAH');
INSERT INTO test.test1(col1,col2) VALUES('B', 'BCDEFGHA');
INSERT INTO test.test1(col1,col2) VALUES('C', 'ACBDEFGH');
INSERT INTO test.test1(col1,col2) VALUES('C', 'ACDBEFGH');
INSERT INTO test.test1(col1,col2) VALUES('C', 'ACDEBFGH');
INSERT INTO test.test1(col1,col2) VALUES('C', 'ACDEFBGH');
INSERT INTO test.test1(col1,col2) VALUES(NULL, 'ACDEFGBH');
INSERT INTO test.test1(col1,col2) VALUES(NULL, 'ACDEFGHB');
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Nullable(String), col2 String) ENGINE=TinyLog;
INSERT INTO test1(col1,col2) VALUES('A', 'ABCDEFGH');
INSERT INTO test1(col1,col2) VALUES('A', 'BACDEFGH');
INSERT INTO test1(col1,col2) VALUES('A', 'BCADEFGH');
INSERT INTO test1(col1,col2) VALUES('A', 'BCDAEFGH');
INSERT INTO test1(col1,col2) VALUES('B', 'BCDEAFGH');
INSERT INTO test1(col1,col2) VALUES('B', 'BCDEFAGH');
INSERT INTO test1(col1,col2) VALUES('B', 'BCDEFGAH');
INSERT INTO test1(col1,col2) VALUES('B', 'BCDEFGHA');
INSERT INTO test1(col1,col2) VALUES('C', 'ACBDEFGH');
INSERT INTO test1(col1,col2) VALUES('C', 'ACDBEFGH');
INSERT INTO test1(col1,col2) VALUES('C', 'ACDEBFGH');
INSERT INTO test1(col1,col2) VALUES('C', 'ACDEFBGH');
INSERT INTO test1(col1,col2) VALUES(NULL, 'ACDEFGBH');
INSERT INTO test1(col1,col2) VALUES(NULL, 'ACDEFGHB');
SELECT col1, count() FROM test.test1 GROUP BY col1 ORDER BY col1;
SELECT col1, count() FROM test1 GROUP BY col1 ORDER BY col1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(UInt8), col2 String) ENGINE=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES(0, 'ABCDEFGH');
INSERT INTO test.test1(col1,col2) VALUES(0, 'BACDEFGH');
INSERT INTO test.test1(col1,col2) VALUES(1, 'BCADEFGH');
INSERT INTO test.test1(col1,col2) VALUES(1, 'BCDAEFGH');
INSERT INTO test.test1(col1,col2) VALUES(1, 'BCDEAFGH');
INSERT INTO test.test1(col1,col2) VALUES(1, 'BCDEFAGH');
INSERT INTO test.test1(col1,col2) VALUES(1, 'BCDEFGAH');
INSERT INTO test.test1(col1,col2) VALUES(1, 'BCDEFGHA');
INSERT INTO test.test1(col1,col2) VALUES(1, 'ACBDEFGH');
INSERT INTO test.test1(col1,col2) VALUES(NULL, 'ACDBEFGH');
INSERT INTO test.test1(col1,col2) VALUES(NULL, 'ACDEBFGH');
INSERT INTO test.test1(col1,col2) VALUES(NULL, 'ACDEFBGH');
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Nullable(UInt8), col2 String) ENGINE=TinyLog;
INSERT INTO test1(col1,col2) VALUES(0, 'ABCDEFGH');
INSERT INTO test1(col1,col2) VALUES(0, 'BACDEFGH');
INSERT INTO test1(col1,col2) VALUES(1, 'BCADEFGH');
INSERT INTO test1(col1,col2) VALUES(1, 'BCDAEFGH');
INSERT INTO test1(col1,col2) VALUES(1, 'BCDEAFGH');
INSERT INTO test1(col1,col2) VALUES(1, 'BCDEFAGH');
INSERT INTO test1(col1,col2) VALUES(1, 'BCDEFGAH');
INSERT INTO test1(col1,col2) VALUES(1, 'BCDEFGHA');
INSERT INTO test1(col1,col2) VALUES(1, 'ACBDEFGH');
INSERT INTO test1(col1,col2) VALUES(NULL, 'ACDBEFGH');
INSERT INTO test1(col1,col2) VALUES(NULL, 'ACDEBFGH');
INSERT INTO test1(col1,col2) VALUES(NULL, 'ACDEFBGH');
SELECT col1, count() FROM test.test1 GROUP BY col1 ORDER BY col1;
SELECT col1, count() FROM test1 GROUP BY col1 ORDER BY col1;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(UInt64), col2 UInt64, col3 String) ENGINE=TinyLog;
INSERT INTO test.test1(col1,col2,col3) VALUES(0, 2, 'ABCDEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES(0, 3, 'BACDEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES(1, 5, 'BCADEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES(1, 2, 'BCDAEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES(1, 3, 'BCDEAFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES(1, 5, 'BCDEFAGH');
INSERT INTO test.test1(col1,col2,col3) VALUES(1, 2, 'BCDEFGAH');
INSERT INTO test.test1(col1,col2,col3) VALUES(1, 3, 'BCDEFGHA');
INSERT INTO test.test1(col1,col2,col3) VALUES(1, 5, 'ACBDEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES(NULL, 2, 'ACDBEFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES(NULL, 3, 'ACDEBFGH');
INSERT INTO test.test1(col1,col2,col3) VALUES(NULL, 3, 'ACDEFBGH');
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Nullable(UInt64), col2 UInt64, col3 String) ENGINE=TinyLog;
INSERT INTO test1(col1,col2,col3) VALUES(0, 2, 'ABCDEFGH');
INSERT INTO test1(col1,col2,col3) VALUES(0, 3, 'BACDEFGH');
INSERT INTO test1(col1,col2,col3) VALUES(1, 5, 'BCADEFGH');
INSERT INTO test1(col1,col2,col3) VALUES(1, 2, 'BCDAEFGH');
INSERT INTO test1(col1,col2,col3) VALUES(1, 3, 'BCDEAFGH');
INSERT INTO test1(col1,col2,col3) VALUES(1, 5, 'BCDEFAGH');
INSERT INTO test1(col1,col2,col3) VALUES(1, 2, 'BCDEFGAH');
INSERT INTO test1(col1,col2,col3) VALUES(1, 3, 'BCDEFGHA');
INSERT INTO test1(col1,col2,col3) VALUES(1, 5, 'ACBDEFGH');
INSERT INTO test1(col1,col2,col3) VALUES(NULL, 2, 'ACDBEFGH');
INSERT INTO test1(col1,col2,col3) VALUES(NULL, 3, 'ACDEBFGH');
INSERT INTO test1(col1,col2,col3) VALUES(NULL, 3, 'ACDEFBGH');
SELECT col1, col2, count() FROM test.test1 GROUP BY col1, col2 ORDER BY col1, col2;
SELECT col1, col2, count() FROM test1 GROUP BY col1, col2 ORDER BY col1, col2;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Nullable(UInt64), col2 UInt64, col3 Nullable(UInt64), col4 String) ENGINE=TinyLog;
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(0, 2, 1, 'ABCDEFGH');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(0, 3, NULL, 'BACDEFGH');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(1, 5, 1, 'BCADEFGH');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(1, 2, NULL, 'BCDAEFGH');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(1, 3, 1, 'BCDEAFGH');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(1, 5, NULL, 'BCDEFAGH');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(1, 2, 1, 'BCDEFGAH');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(1, 3, NULL, 'BCDEFGHA');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(1, 5, 1, 'ACBDEFGH');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(NULL, 2, NULL, 'ACDBEFGH');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(NULL, 3, 1, 'ACDEBFGH');
INSERT INTO test.test1(col1,col2,col3,col4) VALUES(NULL, 3, NULL, 'ACDEFBGH');
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Nullable(UInt64), col2 UInt64, col3 Nullable(UInt64), col4 String) ENGINE=TinyLog;
INSERT INTO test1(col1,col2,col3,col4) VALUES(0, 2, 1, 'ABCDEFGH');
INSERT INTO test1(col1,col2,col3,col4) VALUES(0, 3, NULL, 'BACDEFGH');
INSERT INTO test1(col1,col2,col3,col4) VALUES(1, 5, 1, 'BCADEFGH');
INSERT INTO test1(col1,col2,col3,col4) VALUES(1, 2, NULL, 'BCDAEFGH');
INSERT INTO test1(col1,col2,col3,col4) VALUES(1, 3, 1, 'BCDEAFGH');
INSERT INTO test1(col1,col2,col3,col4) VALUES(1, 5, NULL, 'BCDEFAGH');
INSERT INTO test1(col1,col2,col3,col4) VALUES(1, 2, 1, 'BCDEFGAH');
INSERT INTO test1(col1,col2,col3,col4) VALUES(1, 3, NULL, 'BCDEFGHA');
INSERT INTO test1(col1,col2,col3,col4) VALUES(1, 5, 1, 'ACBDEFGH');
INSERT INTO test1(col1,col2,col3,col4) VALUES(NULL, 2, NULL, 'ACDBEFGH');
INSERT INTO test1(col1,col2,col3,col4) VALUES(NULL, 3, 1, 'ACDEBFGH');
INSERT INTO test1(col1,col2,col3,col4) VALUES(NULL, 3, NULL, 'ACDEFBGH');
SELECT col1, col2, col3, count() FROM test.test1 GROUP BY col1, col2, col3 ORDER BY col1, col2, col3;
SELECT col1, col2, col3, count() FROM test1 GROUP BY col1, col2, col3 ORDER BY col1, col2, col3;
DROP TABLE IF EXISTS test.test1;
CREATE TABLE test.test1(col1 Array(Nullable(UInt8)), col2 String) ENGINE=TinyLog;
INSERT INTO test.test1(col1,col2) VALUES([0], 'ABCDEFGH');
INSERT INTO test.test1(col1,col2) VALUES([0], 'BACDEFGH');
INSERT INTO test.test1(col1,col2) VALUES([1], 'BCADEFGH');
INSERT INTO test.test1(col1,col2) VALUES([1], 'BCDAEFGH');
INSERT INTO test.test1(col1,col2) VALUES([1], 'BCDEAFGH');
INSERT INTO test.test1(col1,col2) VALUES([1], 'BCDEFAGH');
INSERT INTO test.test1(col1,col2) VALUES([1], 'BCDEFGAH');
INSERT INTO test.test1(col1,col2) VALUES([1], 'BCDEFGHA');
INSERT INTO test.test1(col1,col2) VALUES([1], 'ACBDEFGH');
INSERT INTO test.test1(col1,col2) VALUES([NULL], 'ACDBEFGH');
INSERT INTO test.test1(col1,col2) VALUES([NULL], 'ACDEBFGH');
INSERT INTO test.test1(col1,col2) VALUES([NULL], 'ACDEFBGH');
DROP TABLE IF EXISTS test1;
CREATE TABLE test1(col1 Array(Nullable(UInt8)), col2 String) ENGINE=TinyLog;
INSERT INTO test1(col1,col2) VALUES([0], 'ABCDEFGH');
INSERT INTO test1(col1,col2) VALUES([0], 'BACDEFGH');
INSERT INTO test1(col1,col2) VALUES([1], 'BCADEFGH');
INSERT INTO test1(col1,col2) VALUES([1], 'BCDAEFGH');
INSERT INTO test1(col1,col2) VALUES([1], 'BCDEAFGH');
INSERT INTO test1(col1,col2) VALUES([1], 'BCDEFAGH');
INSERT INTO test1(col1,col2) VALUES([1], 'BCDEFGAH');
INSERT INTO test1(col1,col2) VALUES([1], 'BCDEFGHA');
INSERT INTO test1(col1,col2) VALUES([1], 'ACBDEFGH');
INSERT INTO test1(col1,col2) VALUES([NULL], 'ACDBEFGH');
INSERT INTO test1(col1,col2) VALUES([NULL], 'ACDEBFGH');
INSERT INTO test1(col1,col2) VALUES([NULL], 'ACDEFBGH');
SELECT col1, count() FROM test.test1 GROUP BY col1 ORDER BY col1;
SELECT col1, count() FROM test1 GROUP BY col1 ORDER BY col1;

View File

@ -1,5 +1,5 @@
DROP TABLE IF EXISTS test.grop_uniq_array_date;
CREATE TABLE test.grop_uniq_array_date (d Date, dt DateTime) ENGINE = Memory;
INSERT INTO test.grop_uniq_array_date VALUES (toDate('2016-12-16'), toDateTime('2016-12-16 12:00:00')) (toDate('2016-12-16'), toDateTime('2016-12-16 12:00:00'));
SELECT groupUniqArray(d), groupUniqArray(dt) FROM test.grop_uniq_array_date;
DROP TABLE IF EXISTS test.grop_uniq_array_date;
DROP TABLE IF EXISTS grop_uniq_array_date;
CREATE TABLE grop_uniq_array_date (d Date, dt DateTime) ENGINE = Memory;
INSERT INTO grop_uniq_array_date VALUES (toDate('2016-12-16'), toDateTime('2016-12-16 12:00:00')) (toDate('2016-12-16'), toDateTime('2016-12-16 12:00:00'));
SELECT groupUniqArray(d), groupUniqArray(dt) FROM grop_uniq_array_date;
DROP TABLE IF EXISTS grop_uniq_array_date;

Some files were not shown because too many files have changed in this diff Show More