mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #11285 from ClickHouse/fix_fixed_database_tests
Add retries in clickhouse-test before server start
This commit is contained in:
commit
94887f0c74
@ -333,6 +333,19 @@ def run_tests_array(all_tests_with_params):
|
||||
server_logs_level = "warning"
|
||||
|
||||
|
||||
def check_server_started(client, retry_count):
|
||||
while retry_count > 0:
|
||||
clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
|
||||
(stdout, stderr) = clickhouse_proc.communicate("SELECT 1")
|
||||
if clickhouse_proc.returncode != 0 or not stdout.startswith("1"):
|
||||
retry_count -= 1
|
||||
sleep(0.5)
|
||||
else:
|
||||
return True
|
||||
|
||||
return False
|
||||
|
||||
|
||||
def main(args):
|
||||
global SERVER_DIED
|
||||
global exit_code
|
||||
@ -346,6 +359,9 @@ def main(args):
|
||||
|
||||
return stdout.startswith('1')
|
||||
|
||||
if not check_server_started(args.client, args.server_check_retries):
|
||||
raise Exception("clickhouse-server is not responding. Cannot execute 'SELECT 1' query.")
|
||||
|
||||
base_dir = os.path.abspath(args.queries)
|
||||
tmp_dir = os.path.abspath(args.tmp)
|
||||
|
||||
@ -362,7 +378,7 @@ def main(args):
|
||||
os.environ.setdefault("CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL", server_logs_level)
|
||||
|
||||
if args.zookeeper is None:
|
||||
code, out = commands.getstatusoutput(args.extract_from_config +" --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l')
|
||||
code, out = commands.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l')
|
||||
try:
|
||||
if int(out) > 0:
|
||||
args.zookeeper = True
|
||||
@ -572,6 +588,7 @@ if __name__ == '__main__':
|
||||
parser.add_argument('--parallel', default='1/1', help='One parallel test run number/total')
|
||||
parser.add_argument('-j', '--jobs', default=1, nargs='?', type=int, help='Run all tests in parallel')
|
||||
parser.add_argument('-U', '--unified', default=3, type=int, help='output NUM lines of unified context')
|
||||
parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started')
|
||||
|
||||
parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests')
|
||||
parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests')
|
||||
|
@ -3,16 +3,16 @@
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.numbers";
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.numbers (number UInt64) engine = MergeTree order by number";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.numbers select * from system.numbers limit 10";
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS numbers";
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE numbers (number UInt64) engine = MergeTree order by number";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO numbers select * from system.numbers limit 10";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT number FROM test.numbers LIMIT 10 FORMAT JSON" | grep 'rows_read';
|
||||
$CLICKHOUSE_CLIENT --query="SELECT number FROM test.numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read';
|
||||
$CLICKHOUSE_CLIENT --query="SELECT number FROM test.numbers LIMIT 10 FORMAT XML" | grep 'rows_read';
|
||||
$CLICKHOUSE_CLIENT --query="SELECT number FROM numbers LIMIT 10 FORMAT JSON" | grep 'rows_read';
|
||||
$CLICKHOUSE_CLIENT --query="SELECT number FROM numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read';
|
||||
$CLICKHOUSE_CLIENT --query="SELECT number FROM numbers LIMIT 10 FORMAT XML" | grep 'rows_read';
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM test.numbers LIMIT 10 FORMAT JSON" | grep 'rows_read';
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM test.numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read';
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM test.numbers LIMIT 10 FORMAT XML" | grep 'rows_read';
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM numbers LIMIT 10 FORMAT JSON" | grep 'rows_read';
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read';
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM numbers LIMIT 10 FORMAT XML" | grep 'rows_read';
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.numbers";
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS numbers";
|
||||
|
@ -1,14 +1,14 @@
|
||||
-- Check that settings are correctly passed through Distributed table
|
||||
DROP TABLE IF EXISTS test.simple;
|
||||
CREATE TABLE test.simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/test/simple', '1') ORDER BY d;
|
||||
DROP TABLE IF EXISTS simple;
|
||||
CREATE TABLE simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/test/simple', '1') ORDER BY d;
|
||||
|
||||
-- TODO: replace '127.0.0.2' -> '127.0.0.1' after a fix
|
||||
INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (1);
|
||||
INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (1);
|
||||
INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (1);
|
||||
INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (1);
|
||||
|
||||
SET insert_deduplicate=0;
|
||||
INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (2);
|
||||
INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (2);
|
||||
INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (2);
|
||||
INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (2);
|
||||
|
||||
SELECT * FROM remote('127.0.0.2', 'test', 'simple') ORDER BY d;
|
||||
DROP TABLE test.simple;
|
||||
SELECT * FROM remote('127.0.0.2', currentDatabase(), 'simple') ORDER BY d;
|
||||
DROP TABLE simple;
|
||||
|
@ -26,144 +26,143 @@ function query_with_retry
|
||||
echo "Query '$1' failed with '$result'"
|
||||
}
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.src;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r2;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.dst_r1 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '1') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.dst_r2 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '2') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE dst_r1 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '1') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE dst_r2 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '2') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (0, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (2, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (0, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (2, '0', 1);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT 'Initial';"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (0, '1', 2);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2), (1, '2', 2);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (2, '1', 2);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (0, '1', 2);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2), (1, '2', 2);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (2, '1', 2);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.src;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM src;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT 'REPLACE simple';"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;"
|
||||
query_with_retry "ALTER TABLE test.src DROP PARTITION 1;"
|
||||
query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;"
|
||||
query_with_retry "ALTER TABLE src DROP PARTITION 1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.src;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM src;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT 'REPLACE empty';"
|
||||
query_with_retry "ALTER TABLE test.src DROP PARTITION 1;"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;"
|
||||
query_with_retry "ALTER TABLE src DROP PARTITION 1;"
|
||||
query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT 'REPLACE recursive';"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 DROP PARTITION 1;"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2), (1, '2', 2);"
|
||||
query_with_retry "ALTER TABLE dst_r1 DROP PARTITION 1;"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2), (1, '2', 2);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="CREATE table test_block_numbers (m UInt64) ENGINE MergeTree() ORDER BY tuple();"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database='test' AND table='dst_r1' AND active AND name LIKE '1_%';"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database=currentDatabase() AND table='dst_r1' AND active AND name LIKE '1_%';"
|
||||
|
||||
query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;"
|
||||
query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database='test' AND table='dst_r1' AND active AND name LIKE '1_%';"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database=currentDatabase() AND table='dst_r1' AND active AND name LIKE '1_%';"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT (max(m) - min(m) > 1) AS new_block_is_generated FROM test_block_numbers;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test_block_numbers;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT 'ATTACH FROM';"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 DROP PARTITION 1;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;"
|
||||
query_with_retry "ALTER TABLE dst_r1 DROP PARTITION 1;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE src;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r2 VALUES (1, '1', 2);"
|
||||
query_with_retry "ALTER TABLE test.dst_r2 ATTACH PARTITION 1 FROM test.src;"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r2 VALUES (1, '1', 2);"
|
||||
query_with_retry "ALTER TABLE dst_r2 ATTACH PARTITION 1 FROM src;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT 'REPLACE with fetch';"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2); -- trash part to be deleted"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE src;"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2); -- trash part to be deleted"
|
||||
|
||||
# Stop replication at the second replica and remove source table to use fetch instead of copying
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES test.dst_r2;"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES dst_r2;"
|
||||
query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE src;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES dst_r2;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT 'REPLACE with fetch of merged';"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.src;"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 DROP PARTITION 1;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;"
|
||||
query_with_retry "ALTER TABLE dst_r1 DROP PARTITION 1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2); -- trash part to be deleted"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2); -- trash part to be deleted"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES test.dst_r2;"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES dst_r2;"
|
||||
query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE src;"
|
||||
|
||||
# do not wait other replicas to execute OPTIMIZE
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r1;"
|
||||
query_with_retry "OPTIMIZE TABLE test.dst_r1 PARTITION 1;" "--replication_alter_partitions_sync=0 --optimize_throw_if_noop=1"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;"
|
||||
query_with_retry "OPTIMIZE TABLE dst_r1 PARTITION 1;" "--replication_alter_partitions_sync=0 --optimize_throw_if_noop=1"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM dst_r1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM dst_r2;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT 'After restart';"
|
||||
$CLICKHOUSE_CLIENT --query="USE test;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM RESTART REPLICA test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM RESTART REPLICA dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM RESTART REPLICAS;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT 'DETACH+ATTACH PARTITION';"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 DETACH PARTITION 0;"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 DETACH PARTITION 1;"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 DETACH PARTITION 2;"
|
||||
query_with_retry "ALTER TABLE test.dst_r1 ATTACH PARTITION 1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;"
|
||||
query_with_retry "ALTER TABLE dst_r1 DETACH PARTITION 0;"
|
||||
query_with_retry "ALTER TABLE dst_r1 DETACH PARTITION 1;"
|
||||
query_with_retry "ALTER TABLE dst_r1 DETACH PARTITION 2;"
|
||||
query_with_retry "ALTER TABLE dst_r1 ATTACH PARTITION 1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.src;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r2;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r1;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r2;"
|
||||
|
@ -4,20 +4,20 @@ set -e
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.mt_00763_2"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.buffer_00763_2"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mt_00763_2"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS buffer_00763_2"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.buffer_00763_2 (s String) ENGINE = Buffer(test, mt_00763_2, 1, 1, 1, 1, 1, 1, 1)"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_2 (s String) ENGINE = Buffer('$CLICKHOUSE_DATABASE', mt_00763_2, 1, 1, 1, 1, 1, 1, 1)"
|
||||
|
||||
|
||||
function thread1()
|
||||
{
|
||||
seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS test.mt_00763_2; CREATE TABLE test.mt_00763_2 (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO test.mt_00763_2 SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||:
|
||||
seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS mt_00763_2; CREATE TABLE mt_00763_2 (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO mt_00763_2 SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||:
|
||||
}
|
||||
|
||||
function thread2()
|
||||
{
|
||||
seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM test.buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473'
|
||||
seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473'
|
||||
}
|
||||
|
||||
thread1 &
|
||||
@ -25,5 +25,5 @@ thread2 &
|
||||
|
||||
wait
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE test.mt_00763_2"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE test.buffer_00763_2"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE mt_00763_2"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE buffer_00763_2"
|
||||
|
@ -6,21 +6,21 @@ CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.mt_00763_1"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.buffer_00763_1"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mt_00763_1"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS buffer_00763_1"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.buffer_00763_1 (s String) ENGINE = Buffer(test, mt_00763_1, 1, 1, 1, 1, 1, 1, 1)"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.mt_00763_1 (x UInt32, s String) ENGINE = MergeTree ORDER BY x"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO test.mt_00763_1 VALUES (1, '1'), (2, '2'), (3, '3')"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_1 (s String) ENGINE = Buffer($CLICKHOUSE_DATABASE, mt_00763_1, 1, 1, 1, 1, 1, 1, 1)"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE mt_00763_1 (x UInt32, s String) ENGINE = MergeTree ORDER BY x"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO mt_00763_1 VALUES (1, '1'), (2, '2'), (3, '3')"
|
||||
|
||||
function thread1()
|
||||
{
|
||||
seq 1 300 | sed -r -e 's/.+/ALTER TABLE test.mt_00763_1 MODIFY column s UInt32; ALTER TABLE test.mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||:
|
||||
seq 1 300 | sed -r -e 's/.+/ALTER TABLE mt_00763_1 MODIFY column s UInt32; ALTER TABLE mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||:
|
||||
}
|
||||
|
||||
function thread2()
|
||||
{
|
||||
seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM test.buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)'
|
||||
seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)'
|
||||
}
|
||||
|
||||
thread1 &
|
||||
@ -28,5 +28,5 @@ thread2 &
|
||||
|
||||
wait
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE test.mt_00763_1"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE test.buffer_00763_1"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE mt_00763_1"
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE buffer_00763_1"
|
||||
|
@ -1,15 +1,15 @@
|
||||
DROP TABLE IF EXISTS test.part_header_r1;
|
||||
DROP TABLE IF EXISTS test.part_header_r2;
|
||||
DROP TABLE IF EXISTS part_header_r1;
|
||||
DROP TABLE IF EXISTS part_header_r2;
|
||||
|
||||
SET replication_alter_partitions_sync = 2;
|
||||
|
||||
CREATE TABLE test.part_header_r1(x UInt32, y UInt32)
|
||||
CREATE TABLE part_header_r1(x UInt32, y UInt32)
|
||||
ENGINE ReplicatedMergeTree('/clickhouse/tables/test/part_header', '1') ORDER BY x
|
||||
SETTINGS use_minimalistic_part_header_in_zookeeper = 0,
|
||||
old_parts_lifetime = 1,
|
||||
cleanup_delay_period = 0,
|
||||
cleanup_delay_period_random_add = 0;
|
||||
CREATE TABLE test.part_header_r2(x UInt32, y UInt32)
|
||||
CREATE TABLE part_header_r2(x UInt32, y UInt32)
|
||||
ENGINE ReplicatedMergeTree('/clickhouse/tables/test/part_header', '2') ORDER BY x
|
||||
SETTINGS use_minimalistic_part_header_in_zookeeper = 1,
|
||||
old_parts_lifetime = 1,
|
||||
@ -17,47 +17,47 @@ CREATE TABLE test.part_header_r2(x UInt32, y UInt32)
|
||||
cleanup_delay_period_random_add = 0;
|
||||
|
||||
SELECT '*** Test fetches ***';
|
||||
INSERT INTO test.part_header_r1 VALUES (1, 1);
|
||||
INSERT INTO test.part_header_r2 VALUES (2, 2);
|
||||
SYSTEM SYNC REPLICA test.part_header_r1;
|
||||
SYSTEM SYNC REPLICA test.part_header_r2;
|
||||
INSERT INTO part_header_r1 VALUES (1, 1);
|
||||
INSERT INTO part_header_r2 VALUES (2, 2);
|
||||
SYSTEM SYNC REPLICA part_header_r1;
|
||||
SYSTEM SYNC REPLICA part_header_r2;
|
||||
SELECT '*** replica 1 ***';
|
||||
SELECT x, y FROM test.part_header_r1 ORDER BY x;
|
||||
SELECT x, y FROM part_header_r1 ORDER BY x;
|
||||
SELECT '*** replica 2 ***';
|
||||
SELECT x, y FROM test.part_header_r2 ORDER BY x;
|
||||
SELECT x, y FROM part_header_r2 ORDER BY x;
|
||||
|
||||
SELECT '*** Test merges ***';
|
||||
OPTIMIZE TABLE test.part_header_r1;
|
||||
SYSTEM SYNC REPLICA test.part_header_r2;
|
||||
OPTIMIZE TABLE part_header_r1;
|
||||
SYSTEM SYNC REPLICA part_header_r2;
|
||||
SELECT '*** replica 1 ***';
|
||||
SELECT _part, x FROM test.part_header_r1 ORDER BY x;
|
||||
SELECT _part, x FROM part_header_r1 ORDER BY x;
|
||||
SELECT '*** replica 2 ***';
|
||||
SELECT _part, x FROM test.part_header_r2 ORDER BY x;
|
||||
SELECT _part, x FROM part_header_r2 ORDER BY x;
|
||||
|
||||
SELECT sleep(3) FORMAT Null;
|
||||
|
||||
SELECT '*** Test part removal ***';
|
||||
SELECT '*** replica 1 ***';
|
||||
SELECT name FROM system.parts WHERE active AND database = 'test' AND table = 'part_header_r1';
|
||||
SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1';
|
||||
SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test/part_header/replicas/1/parts';
|
||||
SELECT '*** replica 2 ***';
|
||||
SELECT name FROM system.parts WHERE active AND database = 'test' AND table = 'part_header_r2';
|
||||
SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2';
|
||||
SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test/part_header/replicas/1/parts';
|
||||
|
||||
SELECT '*** Test ALTER ***';
|
||||
ALTER TABLE test.part_header_r1 MODIFY COLUMN y String;
|
||||
ALTER TABLE part_header_r1 MODIFY COLUMN y String;
|
||||
SELECT '*** replica 1 ***';
|
||||
SELECT x, length(y) FROM test.part_header_r1 ORDER BY x;
|
||||
SELECT x, length(y) FROM part_header_r1 ORDER BY x;
|
||||
SELECT '*** replica 2 ***';
|
||||
SELECT x, length(y) FROM test.part_header_r2 ORDER BY x;
|
||||
SELECT x, length(y) FROM part_header_r2 ORDER BY x;
|
||||
|
||||
SELECT '*** Test CLEAR COLUMN ***';
|
||||
SET replication_alter_partitions_sync = 2;
|
||||
ALTER TABLE test.part_header_r1 CLEAR COLUMN y IN PARTITION tuple();
|
||||
ALTER TABLE part_header_r1 CLEAR COLUMN y IN PARTITION tuple();
|
||||
SELECT '*** replica 1 ***';
|
||||
SELECT x, length(y) FROM test.part_header_r1 ORDER BY x;
|
||||
SELECT x, length(y) FROM part_header_r1 ORDER BY x;
|
||||
SELECT '*** replica 2 ***';
|
||||
SELECT x, length(y) FROM test.part_header_r2 ORDER BY x;
|
||||
SELECT x, length(y) FROM part_header_r2 ORDER BY x;
|
||||
|
||||
DROP TABLE test.part_header_r1;
|
||||
DROP TABLE test.part_header_r2;
|
||||
DROP TABLE part_header_r1;
|
||||
DROP TABLE part_header_r2;
|
||||
|
@ -1,214 +1,214 @@
|
||||
----- Group of very similar simple tests ------
|
||||
select '----HORIZONTAL MERGE TESTS----';
|
||||
DROP TABLE IF EXISTS test.zero_rows_per_granule1;
|
||||
DROP TABLE IF EXISTS test.zero_rows_per_granule2;
|
||||
DROP TABLE IF EXISTS zero_rows_per_granule1;
|
||||
DROP TABLE IF EXISTS zero_rows_per_granule2;
|
||||
|
||||
CREATE TABLE test.zero_rows_per_granule1 (
|
||||
CREATE TABLE zero_rows_per_granule1 (
|
||||
p Date,
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
v2 Int64
|
||||
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/zero_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0;
|
||||
|
||||
CREATE TABLE test.zero_rows_per_granule2 (
|
||||
CREATE TABLE zero_rows_per_granule2 (
|
||||
p Date,
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
v2 Int64
|
||||
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/zero_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0;
|
||||
|
||||
INSERT INTO test.zero_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
INSERT INTO zero_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
SYSTEM SYNC REPLICA test.zero_rows_per_granule2;
|
||||
SYSTEM SYNC REPLICA zero_rows_per_granule2;
|
||||
|
||||
SELECT 'Replica synced';
|
||||
|
||||
SELECT COUNT(*) FROM test.zero_rows_per_granule1;
|
||||
SELECT COUNT(*) FROM zero_rows_per_granule1;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database=currentDatabase() and active=1;
|
||||
|
||||
SELECT COUNT(*) FROM test.zero_rows_per_granule2;
|
||||
SELECT COUNT(*) FROM zero_rows_per_granule2;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database=currentDatabase() and active=1;
|
||||
|
||||
INSERT INTO test.zero_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000);
|
||||
INSERT INTO zero_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000);
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database=currentDatabase() and active=1;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database=currentDatabase() and active=1;
|
||||
|
||||
SELECT sleep(0.7) Format Null;
|
||||
|
||||
OPTIMIZE TABLE test.zero_rows_per_granule2 FINAL;
|
||||
OPTIMIZE TABLE zero_rows_per_granule2 FINAL;
|
||||
|
||||
SELECT 'Parts optimized';
|
||||
|
||||
SYSTEM SYNC REPLICA test.zero_rows_per_granule1;
|
||||
SYSTEM SYNC REPLICA zero_rows_per_granule1;
|
||||
|
||||
SELECT 'Replica synced';
|
||||
|
||||
SELECT COUNT(*) FROM test.zero_rows_per_granule2;
|
||||
SELECT COUNT(*) FROM zero_rows_per_granule2;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database=currentDatabase() and active=1;
|
||||
|
||||
SELECT COUNT(*) FROM test.zero_rows_per_granule1;
|
||||
SELECT COUNT(*) FROM zero_rows_per_granule1;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database=currentDatabase() and active=1;
|
||||
|
||||
DROP TABLE IF EXISTS test.zero_rows_per_granule1;
|
||||
DROP TABLE IF EXISTS test.zero_rows_per_granule2;
|
||||
DROP TABLE IF EXISTS zero_rows_per_granule1;
|
||||
DROP TABLE IF EXISTS zero_rows_per_granule2;
|
||||
|
||||
SELECT '-----';
|
||||
|
||||
DROP TABLE IF EXISTS test.four_rows_per_granule1;
|
||||
DROP TABLE IF EXISTS test.four_rows_per_granule2;
|
||||
DROP TABLE IF EXISTS four_rows_per_granule1;
|
||||
DROP TABLE IF EXISTS four_rows_per_granule2;
|
||||
|
||||
CREATE TABLE test.four_rows_per_granule1 (
|
||||
CREATE TABLE four_rows_per_granule1 (
|
||||
p Date,
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
v2 Int64
|
||||
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/four_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0;
|
||||
|
||||
CREATE TABLE test.four_rows_per_granule2 (
|
||||
CREATE TABLE four_rows_per_granule2 (
|
||||
p Date,
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
v2 Int64
|
||||
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/four_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0;
|
||||
|
||||
INSERT INTO test.four_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
INSERT INTO four_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
SELECT COUNT(*) FROM test.four_rows_per_granule1;
|
||||
SELECT COUNT(*) FROM four_rows_per_granule1;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database=currentDatabase() and active=1;
|
||||
|
||||
SYSTEM SYNC REPLICA test.four_rows_per_granule2;
|
||||
SYSTEM SYNC REPLICA four_rows_per_granule2;
|
||||
|
||||
SELECT 'Replica synced';
|
||||
|
||||
SELECT COUNT(*) FROM test.four_rows_per_granule2;
|
||||
SELECT COUNT(*) FROM four_rows_per_granule2;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database=currentDatabase() and active=1;
|
||||
|
||||
DETACH TABLE test.four_rows_per_granule2;
|
||||
ATTACH TABLE test.four_rows_per_granule2;
|
||||
DETACH TABLE four_rows_per_granule2;
|
||||
ATTACH TABLE four_rows_per_granule2;
|
||||
|
||||
SELECT 'Table attached';
|
||||
|
||||
INSERT INTO test.four_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000);
|
||||
INSERT INTO four_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000);
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database=currentDatabase() and active=1;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database=currentDatabase() and active=1;
|
||||
|
||||
SELECT sleep(0.7) Format Null;
|
||||
|
||||
OPTIMIZE TABLE test.four_rows_per_granule2 FINAL;
|
||||
OPTIMIZE TABLE four_rows_per_granule2 FINAL;
|
||||
SELECT 'Parts optimized';
|
||||
|
||||
DETACH TABLE test.four_rows_per_granule2;
|
||||
DETACH TABLE four_rows_per_granule2;
|
||||
|
||||
ATTACH TABLE test.four_rows_per_granule2;
|
||||
ATTACH TABLE four_rows_per_granule2;
|
||||
|
||||
SELECT COUNT(*) FROM test.four_rows_per_granule2;
|
||||
SELECT COUNT(*) FROM four_rows_per_granule2;
|
||||
|
||||
--SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database='test' and active=1;
|
||||
--SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database=currentDatabase() and active=1;
|
||||
|
||||
SYSTEM SYNC REPLICA test.four_rows_per_granule1;
|
||||
SYSTEM SYNC REPLICA four_rows_per_granule1;
|
||||
SELECT 'Replica synced';
|
||||
|
||||
SELECT COUNT(*) FROM test.four_rows_per_granule1;
|
||||
SELECT COUNT(*) FROM four_rows_per_granule1;
|
||||
|
||||
--SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database='test' and active=1;
|
||||
--SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database=currentDatabase() and active=1;
|
||||
|
||||
DROP TABLE IF EXISTS test.four_rows_per_granule1;
|
||||
DROP TABLE IF EXISTS test.four_rows_per_granule2;
|
||||
DROP TABLE IF EXISTS four_rows_per_granule1;
|
||||
DROP TABLE IF EXISTS four_rows_per_granule2;
|
||||
|
||||
SELECT '-----';
|
||||
|
||||
DROP TABLE IF EXISTS test.adaptive_granularity_alter1;
|
||||
DROP TABLE IF EXISTS test.adaptive_granularity_alter2;
|
||||
DROP TABLE IF EXISTS adaptive_granularity_alter1;
|
||||
DROP TABLE IF EXISTS adaptive_granularity_alter2;
|
||||
|
||||
CREATE TABLE test.adaptive_granularity_alter1 (
|
||||
CREATE TABLE adaptive_granularity_alter1 (
|
||||
p Date,
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
v2 Int64
|
||||
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/adaptive_granularity_alter', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0;
|
||||
|
||||
CREATE TABLE test.adaptive_granularity_alter2 (
|
||||
CREATE TABLE adaptive_granularity_alter2 (
|
||||
p Date,
|
||||
k UInt64,
|
||||
v1 UInt64,
|
||||
v2 Int64
|
||||
) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/adaptive_granularity_alter', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0;
|
||||
|
||||
INSERT INTO test.adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000);
|
||||
|
||||
SELECT COUNT(*) FROM test.adaptive_granularity_alter1;
|
||||
SELECT COUNT(*) FROM adaptive_granularity_alter1;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1;
|
||||
|
||||
SYSTEM SYNC REPLICA test.adaptive_granularity_alter2;
|
||||
SYSTEM SYNC REPLICA adaptive_granularity_alter2;
|
||||
SELECT 'Replica synced';
|
||||
|
||||
SELECT COUNT(*) FROM test.adaptive_granularity_alter2;
|
||||
SELECT COUNT(*) FROM adaptive_granularity_alter2;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1;
|
||||
|
||||
ALTER TABLE test.adaptive_granularity_alter2 MODIFY COLUMN v1 Int16;
|
||||
ALTER TABLE adaptive_granularity_alter2 MODIFY COLUMN v1 Int16;
|
||||
|
||||
DETACH TABLE test.adaptive_granularity_alter2;
|
||||
DETACH TABLE adaptive_granularity_alter2;
|
||||
|
||||
ATTACH TABLE test.adaptive_granularity_alter2;
|
||||
ATTACH TABLE adaptive_granularity_alter2;
|
||||
|
||||
SELECT COUNT(*) FROM test.adaptive_granularity_alter2;
|
||||
SELECT COUNT(*) FROM adaptive_granularity_alter2;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1;
|
||||
|
||||
SYSTEM SYNC REPLICA test.adaptive_granularity_alter1;
|
||||
SYSTEM SYNC REPLICA adaptive_granularity_alter1;
|
||||
SELECT 'Replica synced';
|
||||
|
||||
SELECT COUNT(*) FROM test.adaptive_granularity_alter1;
|
||||
SELECT COUNT(*) FROM adaptive_granularity_alter1;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1;
|
||||
|
||||
INSERT INTO test.adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 5, 3000, 4000), ('2018-05-17', 6, 5000, 6000), ('2018-05-19', 42, 42, 42);
|
||||
INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 5, 3000, 4000), ('2018-05-17', 6, 5000, 6000), ('2018-05-19', 42, 42, 42);
|
||||
|
||||
SELECT COUNT(*) FROM test.adaptive_granularity_alter1;
|
||||
SELECT COUNT(*) FROM adaptive_granularity_alter1;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1;
|
||||
|
||||
SYSTEM SYNC REPLICA test.adaptive_granularity_alter2;
|
||||
SYSTEM SYNC REPLICA adaptive_granularity_alter2;
|
||||
|
||||
SELECT COUNT(*) FROM test.adaptive_granularity_alter2;
|
||||
SELECT COUNT(*) FROM adaptive_granularity_alter2;
|
||||
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1;
|
||||
SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1;
|
||||
|
||||
ALTER TABLE test.adaptive_granularity_alter1 MODIFY COLUMN v2 String;
|
||||
ALTER TABLE adaptive_granularity_alter1 MODIFY COLUMN v2 String;
|
||||
|
||||
DETACH TABLE test.adaptive_granularity_alter1;
|
||||
DETACH TABLE adaptive_granularity_alter1;
|
||||
|
||||
ATTACH TABLE test.adaptive_granularity_alter1;
|
||||
ATTACH TABLE adaptive_granularity_alter1;
|
||||
|
||||
INSERT INTO test.adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 100, 1000, 'aaaa'), ('2018-05-16', 101, 3000, 'bbbb'), ('2018-05-17', 102, 5000, 'cccc'), ('2018-05-19', 103, 7000, 'dddd');
|
||||
INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 100, 1000, 'aaaa'), ('2018-05-16', 101, 3000, 'bbbb'), ('2018-05-17', 102, 5000, 'cccc'), ('2018-05-19', 103, 7000, 'dddd');
|
||||
|
||||
SELECT sleep(0.7) Format Null;
|
||||
|
||||
OPTIMIZE TABLE test.adaptive_granularity_alter1 FINAL;
|
||||
OPTIMIZE TABLE adaptive_granularity_alter1 FINAL;
|
||||
SELECT 'Parts optimized';
|
||||
|
||||
SELECT k, v2 FROM test.adaptive_granularity_alter1 WHERE k >= 100 OR k = 42 ORDER BY k;
|
||||
SELECT k, v2 FROM adaptive_granularity_alter1 WHERE k >= 100 OR k = 42 ORDER BY k;
|
||||
|
||||
SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1;
|
||||
SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1;
|
||||
|
||||
SYSTEM SYNC REPLICA test.adaptive_granularity_alter2;
|
||||
SYSTEM SYNC REPLICA adaptive_granularity_alter2;
|
||||
SELECT 'Replica synced';
|
||||
|
||||
SELECT k, v2 FROM test.adaptive_granularity_alter2 WHERE k >= 100 OR k = 42 ORDER BY k;
|
||||
SELECT k, v2 FROM adaptive_granularity_alter2 WHERE k >= 100 OR k = 42 ORDER BY k;
|
||||
|
||||
SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1;
|
||||
SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1;
|
||||
|
||||
DROP TABLE IF EXISTS test.adaptive_granularity_alter1;
|
||||
DROP TABLE IF EXISTS test.adaptive_granularity_alter2;
|
||||
DROP TABLE IF EXISTS adaptive_granularity_alter1;
|
||||
DROP TABLE IF EXISTS adaptive_granularity_alter2;
|
||||
|
@ -3,10 +3,10 @@
|
||||
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 (d Date, u UInt8, str String) ENGINE = TinyLog"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS csv"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE csv (d Date, u UInt8, str String) ENGINE = TinyLog"
|
||||
|
||||
INSERT_QUERY='$CLICKHOUSE_CLIENT --query="INSERT INTO test.csv FORMAT CSVWithNames"'
|
||||
INSERT_QUERY='$CLICKHOUSE_CLIENT --query="INSERT INTO csv FORMAT CSVWithNames"'
|
||||
USE_HEADER='--input_format_with_names_use_header=1'
|
||||
SKIP_UNKNOWN='--input_format_skip_unknown_fields=1'
|
||||
|
||||
@ -32,5 +32,5 @@ echo -ne 'str,u\nLine16,1\nLine17,2\n' | eval $INSERT_QUERY $US
|
||||
echo -ne 'd,str\n2019-04-18,Line18\n2019-04-18,Line19\n'| eval $INSERT_QUERY $USE_HEADER
|
||||
echo -ne 'unknown\n\n\n' | eval $INSERT_QUERY $USE_HEADER $SKIP_UNKNOWN
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.csv"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM csv"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS csv"
|
||||
|
@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
. $CURDIR/mergetree_mutations.lib
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
CREATE TABLE test.minmax_idx
|
||||
CREATE TABLE minmax_idx
|
||||
(
|
||||
u64 UInt64,
|
||||
i64 Int64,
|
||||
@ -19,7 +19,7 @@ ORDER BY u64
|
||||
SETTINGS index_granularity = 2;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
|
||||
(0, 1, 1),
|
||||
(1, 1, 2),
|
||||
(2, 1, 3),
|
||||
@ -31,13 +31,13 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES
|
||||
(8, 1, 9),
|
||||
(9, 1, 10)"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 5;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx UPDATE i64 = 5 WHERE i64 = 1;"
|
||||
wait_for_mutation "minmax_idx" "mutation_2.txt" "test"
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx UPDATE i64 = 5 WHERE i64 = 1;"
|
||||
wait_for_mutation "minmax_idx" "mutation_2.txt" "$CLICKHOUSE_DATABASE"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 1;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 5;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx"
|
||||
|
@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
. $CURDIR/mergetree_mutations.lib
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
CREATE TABLE test.minmax_idx
|
||||
CREATE TABLE minmax_idx
|
||||
(
|
||||
u64 UInt64,
|
||||
i64 Int64,
|
||||
@ -19,7 +19,7 @@ ORDER BY u64
|
||||
SETTINGS index_granularity = 2;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
|
||||
(0, 2, 1),
|
||||
(1, 1, 1),
|
||||
(2, 1, 1),
|
||||
@ -31,35 +31,35 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES
|
||||
(8, 1, 2),
|
||||
(9, 1, 2)"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
ALTER TABLE test.minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;"
|
||||
ALTER TABLE minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;"
|
||||
wait_for_mutation "minmax_idx" "mutation_3.txt" "test"
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;"
|
||||
wait_for_mutation "minmax_idx" "mutation_3.txt" "$CLICKHOUSE_DATABASE"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;"
|
||||
wait_for_mutation "minmax_idx" "mutation_4.txt" "test"
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;"
|
||||
wait_for_mutation "minmax_idx" "mutation_4.txt" "$CLICKHOUSE_DATABASE"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;"
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 2;"
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 1;"
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 2;"
|
||||
sleep 0.5
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;"
|
||||
wait_for_mutation "minmax_idx" "mutation_5.txt" "test"
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx;"
|
||||
wait_for_mutation "minmax_idx" "mutation_5.txt" "$CLICKHOUSE_DATABASE"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx"
|
||||
|
@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
. $CURDIR/mergetree_mutations.lib
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
CREATE TABLE test.minmax_idx
|
||||
CREATE TABLE minmax_idx
|
||||
(
|
||||
u64 UInt64,
|
||||
i64 Int64,
|
||||
@ -20,7 +20,7 @@ ORDER BY u64
|
||||
SETTINGS index_granularity = 2;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
|
||||
(0, 2, 1),
|
||||
(1, 1, 1),
|
||||
(2, 1, 1),
|
||||
@ -32,19 +32,19 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES
|
||||
(8, 1, 2),
|
||||
(9, 1, 2)"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;"
|
||||
wait_for_mutation "minmax_idx" "mutation_3.txt" "test"
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;"
|
||||
wait_for_mutation "minmax_idx" "mutation_3.txt" "$CLICKHOUSE_DATABASE"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx"
|
||||
|
@ -4,12 +4,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
. $CURDIR/mergetree_mutations.lib
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions1;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions2;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS indices_mutaions1;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS indices_mutaions2;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
CREATE TABLE test.indices_mutaions1
|
||||
CREATE TABLE indices_mutaions1
|
||||
(
|
||||
u64 UInt64,
|
||||
i64 Int64,
|
||||
@ -19,7 +19,7 @@ CREATE TABLE test.indices_mutaions1
|
||||
PARTITION BY i32
|
||||
ORDER BY u64
|
||||
SETTINGS index_granularity = 2;
|
||||
CREATE TABLE test.indices_mutaions2
|
||||
CREATE TABLE indices_mutaions2
|
||||
(
|
||||
u64 UInt64,
|
||||
i64 Int64,
|
||||
@ -31,7 +31,7 @@ ORDER BY u64
|
||||
SETTINGS index_granularity = 2;"
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test.indices_mutaions1 VALUES
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO indices_mutaions1 VALUES
|
||||
(0, 2, 1),
|
||||
(1, 1, 1),
|
||||
(2, 1, 1),
|
||||
@ -43,18 +43,18 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.indices_mutaions1 VALUES
|
||||
(8, 1, 2),
|
||||
(9, 1, 2)"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions1"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions2"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE indices_mutaions1"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE indices_mutaions2"
|
||||
|
@ -3,17 +3,17 @@
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.a;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.b;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS a;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS b;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.a (x UInt64) ENGINE = Memory;"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.b (x UInt64) ENGINE = Memory;"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE a (x UInt64) ENGINE = Memory;"
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE b (x UInt64) ENGINE = Memory;"
|
||||
|
||||
function thread1()
|
||||
{
|
||||
for attempt_thread1 in {1..10}
|
||||
do
|
||||
seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=11 --query="INSERT INTO test.a FORMAT TSV" &
|
||||
seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=11 --query="INSERT INTO a FORMAT TSV" &
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT --query="KILL QUERY WHERE query_id='11' SYNC" | grep -q "cant_cancel" && sleep .1 || break ||:
|
||||
done
|
||||
@ -27,7 +27,7 @@ function thread2()
|
||||
{
|
||||
for attempt_thread2 in {1..10}
|
||||
do
|
||||
seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=22 --query="INSERT INTO test.b FORMAT TSV" &
|
||||
seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=22 --query="INSERT INTO b FORMAT TSV" &
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT --query="KILL QUERY WHERE query_id='22' SYNC" | grep -q "cant_cancel" && sleep .1 || break ||:
|
||||
done
|
||||
@ -46,5 +46,5 @@ bash -c thread2 > /dev/null 2>&1 &
|
||||
wait
|
||||
echo OK
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE test.a"
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE test.b"
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE a"
|
||||
$CLICKHOUSE_CLIENT --query "DROP TABLE b"
|
||||
|
@ -1,14 +1,14 @@
|
||||
DROP TABLE IF EXISTS test.rollup_having;
|
||||
CREATE TABLE test.rollup_having (
|
||||
DROP TABLE IF EXISTS rollup_having;
|
||||
CREATE TABLE rollup_having (
|
||||
a Nullable(String),
|
||||
b Nullable(String)
|
||||
) ENGINE = Memory;
|
||||
|
||||
INSERT INTO test.rollup_having VALUES (NULL, NULL);
|
||||
INSERT INTO test.rollup_having VALUES ('a', NULL);
|
||||
INSERT INTO test.rollup_having VALUES ('a', 'b');
|
||||
INSERT INTO rollup_having VALUES (NULL, NULL);
|
||||
INSERT INTO rollup_having VALUES ('a', NULL);
|
||||
INSERT INTO rollup_having VALUES ('a', 'b');
|
||||
|
||||
SELECT a, b, count(*) FROM test.rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL; -- { serverError 48 }
|
||||
SELECT a, b, count(*) FROM test.rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL and b IS NOT NULL; -- { serverError 48 }
|
||||
SELECT a, b, count(*) FROM rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL; -- { serverError 48 }
|
||||
SELECT a, b, count(*) FROM rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL and b IS NOT NULL; -- { serverError 48 }
|
||||
|
||||
DROP TABLE test.rollup_having;
|
||||
DROP TABLE rollup_having;
|
||||
|
@ -1,11 +1,11 @@
|
||||
DROP TABLE IF EXISTS test.table;
|
||||
CREATE TABLE test.table (x UInt16) ENGINE = TinyLog;
|
||||
INSERT INTO test.table SELECT * FROM system.numbers LIMIT 10;
|
||||
DROP TABLE IF EXISTS source_table;
|
||||
CREATE TABLE source_table (x UInt16) ENGINE = TinyLog;
|
||||
INSERT INTO source_table SELECT * FROM system.numbers LIMIT 10;
|
||||
|
||||
DROP TABLE IF EXISTS test.view;
|
||||
CREATE VIEW test.view (x UInt64) AS SELECT * FROM test.table;
|
||||
DROP TABLE IF EXISTS dest_view;
|
||||
CREATE VIEW dest_view (x UInt64) AS SELECT * FROM source_table;
|
||||
|
||||
SELECT x, any(x) FROM test.view GROUP BY x ORDER BY x;
|
||||
SELECT x, any(x) FROM dest_view GROUP BY x ORDER BY x;
|
||||
|
||||
DROP TABLE test.view;
|
||||
DROP TABLE test.table;
|
||||
DROP TABLE dest_view;
|
||||
DROP TABLE source_table;
|
||||
|
@ -0,0 +1,22 @@
|
||||
DROP TABLE IF EXISTS date_datetime_key_condition;
|
||||
|
||||
CREATE TABLE date_datetime_key_condition (dt DateTime) ENGINE = MergeTree() ORDER BY dt;
|
||||
INSERT INTO date_datetime_key_condition VALUES ('2020-01-01 00:00:00'), ('2020-01-01 10:00:00'), ('2020-01-02 00:00:00');
|
||||
|
||||
-- partial
|
||||
SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2020-01-01') AND dt < toDate('2020-01-02');
|
||||
SELECT groupArray(dt) from date_datetime_key_condition WHERE dt >= toDate('2020-01-02');
|
||||
SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2020-01-02');
|
||||
|
||||
-- inside
|
||||
SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2019-01-02');
|
||||
SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2021-01-02');
|
||||
SELECT groupArray(dt) from date_datetime_key_condition WHERE dt >= toDate('2019-01-02') AND dt < toDate('2021-01-02');
|
||||
SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2019-01-02') OR dt <= toDate('2021-01-02');
|
||||
|
||||
-- outside
|
||||
SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2019-01-02');
|
||||
SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2021-01-02');
|
||||
SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2019-01-02') OR dt > toDate('2021-01-02');
|
||||
|
||||
DROP TABLE date_datetime_key_condition;
|
@ -1,22 +0,0 @@
|
||||
DROP TABLE IF EXISTS test.date_datetime_key_condition;
|
||||
|
||||
CREATE TABLE test.date_datetime_key_condition (dt DateTime) ENGINE = MergeTree() ORDER BY dt;
|
||||
INSERT INTO test.date_datetime_key_condition VALUES ('2020-01-01 00:00:00'), ('2020-01-01 10:00:00'), ('2020-01-02 00:00:00');
|
||||
|
||||
-- partial
|
||||
SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2020-01-01') AND dt < toDate('2020-01-02');
|
||||
SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt >= toDate('2020-01-02');
|
||||
SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2020-01-02');
|
||||
|
||||
-- inside
|
||||
SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2019-01-02');
|
||||
SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2021-01-02');
|
||||
SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt >= toDate('2019-01-02') AND dt < toDate('2021-01-02');
|
||||
SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2019-01-02') OR dt <= toDate('2021-01-02');
|
||||
|
||||
-- outside
|
||||
SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2019-01-02');
|
||||
SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2021-01-02');
|
||||
SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2019-01-02') OR dt > toDate('2021-01-02');
|
||||
|
||||
DROP TABLE test.date_datetime_key_condition;
|
Loading…
Reference in New Issue
Block a user