mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
move tests with sudo to integration tests
This commit is contained in:
parent
f0836553d4
commit
2f36d80705
0
dbms/tests/integration/test_partition/__init__.py
Normal file
0
dbms/tests/integration/test_partition/__init__.py
Normal file
244
dbms/tests/integration/test_partition/test.py
Normal file
244
dbms/tests/integration/test_partition/test.py
Normal file
@ -0,0 +1,244 @@
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import TSV
|
||||
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance')
|
||||
q = instance.query
|
||||
path_to_data = '/var/lib/clickhouse/'
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
q('CREATE DATABASE test')
|
||||
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def partition_table_simple(started_cluster):
|
||||
q("DROP TABLE IF EXISTS test.partition")
|
||||
q("CREATE TABLE test.partition (date MATERIALIZED toDate(0), x UInt64, sample_key MATERIALIZED intHash64(x)) "
|
||||
"ENGINE=MergeTree PARTITION BY date SAMPLE BY sample_key ORDER BY (date,x,sample_key) "
|
||||
"SETTINGS index_granularity=8192, index_granularity_bytes=0")
|
||||
q("INSERT INTO test.partition ( x ) VALUES ( now() )")
|
||||
q("INSERT INTO test.partition ( x ) VALUES ( now()+1 )")
|
||||
|
||||
yield
|
||||
|
||||
q('DROP TABLE test.partition')
|
||||
|
||||
|
||||
def test_partition_simple(partition_table_simple):
|
||||
q("ALTER TABLE test.partition DETACH PARTITION 197001")
|
||||
q("ALTER TABLE test.partition ATTACH PARTITION 197001")
|
||||
q("OPTIMIZE TABLE test.partition")
|
||||
|
||||
|
||||
def exec_bash(cmd):
|
||||
cmd = '/bin/bash -c "{}"'.format(cmd.replace('"', '\\"'))
|
||||
return instance.exec_in_container(cmd)
|
||||
|
||||
|
||||
def partition_complex_assert_columns_txt():
|
||||
path_to_parts = path_to_data + 'data/test/partition/'
|
||||
parts = TSV(q("SELECT name FROM system.parts WHERE database='test' AND table='partition'"))
|
||||
for part_name in parts.lines:
|
||||
path_to_columns = path_to_parts + part_name + '/columns.txt'
|
||||
# 2 header lines + 3 columns
|
||||
assert exec_bash('cat {} | wc -l'.format(path_to_columns)) == u'5\n'
|
||||
|
||||
|
||||
def partition_complex_assert_checksums():
|
||||
# Do `cd` for consistent output for reference
|
||||
# Do not check increment.txt - it can be changed by other tests with FREEZE
|
||||
cmd = 'cd ' + path_to_data + " && find shadow -type f -exec md5sum {} \\;" \
|
||||
" | grep partition" \
|
||||
" | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g'" \
|
||||
" | sort" \
|
||||
" | uniq"
|
||||
|
||||
checksums = "082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition/19700102_2_2_0/k.bin\n" \
|
||||
"082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition/19700201_1_1_0/v1.bin\n" \
|
||||
"13cae8e658e0ca4f75c56b1fc424e150\tshadow/1/data/test/partition/19700102_2_2_0/minmax_p.idx\n" \
|
||||
"25daad3d9e60b45043a70c4ab7d3b1c6\tshadow/1/data/test/partition/19700102_2_2_0/partition.dat\n" \
|
||||
"3726312af62aec86b64a7708d5751787\tshadow/1/data/test/partition/19700201_1_1_0/partition.dat\n" \
|
||||
"37855b06a39b79a67ea4e86e4a3299aa\tshadow/1/data/test/partition/19700102_2_2_0/checksums.txt\n" \
|
||||
"38e62ff37e1e5064e9a3f605dfe09d13\tshadow/1/data/test/partition/19700102_2_2_0/v1.bin\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/k.mrk\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/p.mrk\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/v1.mrk\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/k.mrk\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/p.mrk\n" \
|
||||
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/v1.mrk\n" \
|
||||
"55a54008ad1ba589aa210d2629c1df41\tshadow/1/data/test/partition/19700201_1_1_0/primary.idx\n" \
|
||||
"5f087cb3e7071bf9407e095821e2af8f\tshadow/1/data/test/partition/19700201_1_1_0/checksums.txt\n" \
|
||||
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700102_2_2_0/columns.txt\n" \
|
||||
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700201_1_1_0/columns.txt\n" \
|
||||
"88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition/19700201_1_1_0/p.bin\n" \
|
||||
"9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition/19700102_2_2_0/primary.idx\n" \
|
||||
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700102_2_2_0/count.txt\n" \
|
||||
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700201_1_1_0/count.txt\n" \
|
||||
"cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition/19700102_2_2_0/p.bin\n" \
|
||||
"e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition/19700201_1_1_0/k.bin\n" \
|
||||
"f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition/19700201_1_1_0/minmax_p.idx\n"
|
||||
|
||||
assert TSV(exec_bash(cmd).replace(' ', '\t')) == TSV(checksums)
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def partition_table_complex(started_cluster):
|
||||
q("DROP TABLE IF EXISTS test.partition")
|
||||
q("CREATE TABLE test.partition (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) "
|
||||
"ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0")
|
||||
q("INSERT INTO test.partition (p, k) VALUES(toDate(31), 1)")
|
||||
q("INSERT INTO test.partition (p, k) VALUES(toDate(1), 2)")
|
||||
|
||||
yield
|
||||
|
||||
q("DROP TABLE test.partition")
|
||||
|
||||
|
||||
def test_partition_complex(partition_table_complex):
|
||||
|
||||
partition_complex_assert_columns_txt()
|
||||
|
||||
q("ALTER TABLE test.partition FREEZE")
|
||||
|
||||
partition_complex_assert_checksums()
|
||||
|
||||
q("ALTER TABLE test.partition DETACH PARTITION 197001")
|
||||
q("ALTER TABLE test.partition ATTACH PARTITION 197001")
|
||||
|
||||
partition_complex_assert_columns_txt()
|
||||
|
||||
q("ALTER TABLE test.partition MODIFY COLUMN v1 Int8")
|
||||
|
||||
# Check the backup hasn't changed
|
||||
partition_complex_assert_checksums()
|
||||
|
||||
q("OPTIMIZE TABLE test.partition")
|
||||
|
||||
expected = TSV('31\t1\t2\n'
|
||||
'1\t2\t3')
|
||||
res = q("SELECT toUInt16(p), k, v1 FROM test.partition ORDER BY k")
|
||||
assert(TSV(res) == expected)
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def cannot_attach_active_part_table(started_cluster):
|
||||
q("DROP TABLE IF EXISTS test.attach_active")
|
||||
q("CREATE TABLE test.attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n")
|
||||
q("INSERT INTO test.attach_active SELECT number FROM system.numbers LIMIT 16")
|
||||
|
||||
yield
|
||||
|
||||
q("DROP TABLE test.attach_active")
|
||||
|
||||
|
||||
def test_cannot_attach_active_part(cannot_attach_active_part_table):
|
||||
error = instance.client.query_and_get_error("ALTER TABLE test.attach_active ATTACH PART '../1_2_2_0'")
|
||||
print error
|
||||
assert 0 <= error.find('Invalid part name')
|
||||
|
||||
res = q("SElECT name FROM system.parts WHERE table='attach_active' AND database='test' ORDER BY name")
|
||||
assert TSV(res) == TSV('0_1_1_0\n1_2_2_0\n2_3_3_0\n3_4_4_0')
|
||||
assert TSV(q("SElECT count(), sum(n) FROM test.attach_active")) == TSV('16\t120')
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def attach_check_all_parts_table(started_cluster):
|
||||
q("SYSTEM STOP MERGES")
|
||||
q("DROP TABLE IF EXISTS test.attach_partition")
|
||||
q("CREATE TABLE test.attach_partition (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n")
|
||||
q("INSERT INTO test.attach_partition SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8")
|
||||
q("INSERT INTO test.attach_partition SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8")
|
||||
|
||||
yield
|
||||
|
||||
q("DROP TABLE test.attach_partition")
|
||||
q("SYSTEM START MERGES")
|
||||
|
||||
|
||||
def test_attach_check_all_parts(attach_check_all_parts_table):
|
||||
q("ALTER TABLE test.attach_partition DETACH PARTITION 0")
|
||||
|
||||
path_to_detached = path_to_data + 'data/test/attach_partition/detached/'
|
||||
exec_bash('mkdir {}'.format(path_to_detached + '0_5_5_0'))
|
||||
exec_bash('cp -pr {} {}'.format(path_to_detached + '0_1_1_0', path_to_detached + 'attaching_0_6_6_0'))
|
||||
exec_bash('cp -pr {} {}'.format(path_to_detached + '0_3_3_0', path_to_detached + 'deleting_0_7_7_0'))
|
||||
|
||||
error = instance.client.query_and_get_error("ALTER TABLE test.attach_partition ATTACH PARTITION 0")
|
||||
assert 0 <= error.find('No columns in part 0_5_5_0')
|
||||
|
||||
parts = q("SElECT name FROM system.parts WHERE table='attach_partition' AND database='test' ORDER BY name")
|
||||
assert TSV(parts) == TSV('1_2_2_0\n1_4_4_0')
|
||||
detached = q("SELECT name FROM system.detached_parts "
|
||||
"WHERE table='attach_partition' AND database='test' ORDER BY name")
|
||||
assert TSV(detached) == TSV('0_1_1_0\n0_3_3_0\n0_5_5_0\nattaching_0_6_6_0\ndeleting_0_7_7_0')
|
||||
|
||||
exec_bash('rm -r {}'.format(path_to_detached + '0_5_5_0'))
|
||||
|
||||
q("ALTER TABLE test.attach_partition ATTACH PARTITION 0")
|
||||
parts = q("SElECT name FROM system.parts WHERE table='attach_partition' AND database='test' ORDER BY name")
|
||||
expected = '0_5_5_0\n0_6_6_0\n1_2_2_0\n1_4_4_0'
|
||||
assert TSV(parts) == TSV(expected)
|
||||
assert TSV(q("SElECT count(), sum(n) FROM test.attach_partition")) == TSV('16\t120')
|
||||
|
||||
detached = q("SELECT name FROM system.detached_parts "
|
||||
"WHERE table='attach_partition' AND database='test' ORDER BY name")
|
||||
assert TSV(detached) == TSV('attaching_0_6_6_0\ndeleting_0_7_7_0')
|
||||
|
||||
|
||||
@pytest.fixture
|
||||
def drop_detached_parts_table(started_cluster):
|
||||
q("SYSTEM STOP MERGES")
|
||||
q("DROP TABLE IF EXISTS test.drop_detached")
|
||||
q("CREATE TABLE test.drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n")
|
||||
q("INSERT INTO test.drop_detached SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8")
|
||||
q("INSERT INTO test.drop_detached SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8")
|
||||
|
||||
yield
|
||||
|
||||
q("DROP TABLE test.drop_detached")
|
||||
q("SYSTEM START MERGES")
|
||||
|
||||
|
||||
def test_drop_detached_parts(drop_detached_parts_table):
|
||||
s = {"allow_drop_detached_part": 1}
|
||||
q("ALTER TABLE test.drop_detached DETACH PARTITION 0")
|
||||
q("ALTER TABLE test.drop_detached DETACH PARTITION 1")
|
||||
|
||||
path_to_detached = path_to_data + 'data/test/drop_detached/detached/'
|
||||
exec_bash('mkdir {}'.format(path_to_detached + 'attaching_0_6_6_0'))
|
||||
exec_bash('mkdir {}'.format(path_to_detached + 'deleting_0_7_7_0'))
|
||||
exec_bash('mkdir {}'.format(path_to_detached + 'any_other_name'))
|
||||
exec_bash('mkdir {}'.format(path_to_detached + 'prefix_1_2_2_0_0'))
|
||||
|
||||
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART '../1_2_2_0'", settings=s)
|
||||
assert 0 <= error.find('Invalid part name')
|
||||
|
||||
q("ALTER TABLE test.drop_detached DROP DETACHED PART '0_1_1_0'", settings=s)
|
||||
|
||||
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART 'attaching_0_6_6_0'", settings=s)
|
||||
assert 0 <= error.find('Cannot drop part')
|
||||
|
||||
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART 'deleting_0_7_7_0'", settings=s)
|
||||
assert 0 <= error.find('Cannot drop part')
|
||||
|
||||
q("ALTER TABLE test.drop_detached DROP DETACHED PART 'any_other_name'", settings=s)
|
||||
|
||||
detached = q("SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='test' ORDER BY name")
|
||||
assert TSV(detached) == TSV('0_3_3_0\n1_2_2_0\n1_4_4_0\nattaching_0_6_6_0\ndeleting_0_7_7_0\nprefix_1_2_2_0_0')
|
||||
|
||||
q("ALTER TABLE test.drop_detached DROP DETACHED PARTITION 1", settings=s)
|
||||
detached = q("SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='test' ORDER BY name")
|
||||
assert TSV(detached) == TSV('0_3_3_0\nattaching_0_6_6_0\ndeleting_0_7_7_0')
|
||||
|
@ -1,54 +0,0 @@
|
||||
5
|
||||
5
|
||||
082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700102_2_2_0/k.bin
|
||||
082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700201_1_1_0/v1.bin
|
||||
13cae8e658e0ca4f75c56b1fc424e150 shadow/1/data/test/partition_428/19700102_2_2_0/minmax_p.idx
|
||||
25daad3d9e60b45043a70c4ab7d3b1c6 shadow/1/data/test/partition_428/19700102_2_2_0/partition.dat
|
||||
3726312af62aec86b64a7708d5751787 shadow/1/data/test/partition_428/19700201_1_1_0/partition.dat
|
||||
37855b06a39b79a67ea4e86e4a3299aa shadow/1/data/test/partition_428/19700102_2_2_0/checksums.txt
|
||||
38e62ff37e1e5064e9a3f605dfe09d13 shadow/1/data/test/partition_428/19700102_2_2_0/v1.bin
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/k.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/p.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/v1.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/k.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/p.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/v1.mrk
|
||||
55a54008ad1ba589aa210d2629c1df41 shadow/1/data/test/partition_428/19700201_1_1_0/primary.idx
|
||||
5f087cb3e7071bf9407e095821e2af8f shadow/1/data/test/partition_428/19700201_1_1_0/checksums.txt
|
||||
77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700102_2_2_0/columns.txt
|
||||
77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700201_1_1_0/columns.txt
|
||||
88cdc31ded355e7572d68d8cde525d3a shadow/1/data/test/partition_428/19700201_1_1_0/p.bin
|
||||
9e688c58a5487b8eaf69c9e1005ad0bf shadow/1/data/test/partition_428/19700102_2_2_0/primary.idx
|
||||
c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700102_2_2_0/count.txt
|
||||
c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700201_1_1_0/count.txt
|
||||
cfcb770c3ecd0990dcceb1bde129e6c6 shadow/1/data/test/partition_428/19700102_2_2_0/p.bin
|
||||
e2af3bef1fd129aea73a890ede1e7a30 shadow/1/data/test/partition_428/19700201_1_1_0/k.bin
|
||||
f2312862cc01adf34a93151377be2ddf shadow/1/data/test/partition_428/19700201_1_1_0/minmax_p.idx
|
||||
5
|
||||
5
|
||||
082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700102_2_2_0/k.bin
|
||||
082814b5aa5109160d5c0c5aff10d4df shadow/1/data/test/partition_428/19700201_1_1_0/v1.bin
|
||||
13cae8e658e0ca4f75c56b1fc424e150 shadow/1/data/test/partition_428/19700102_2_2_0/minmax_p.idx
|
||||
25daad3d9e60b45043a70c4ab7d3b1c6 shadow/1/data/test/partition_428/19700102_2_2_0/partition.dat
|
||||
3726312af62aec86b64a7708d5751787 shadow/1/data/test/partition_428/19700201_1_1_0/partition.dat
|
||||
37855b06a39b79a67ea4e86e4a3299aa shadow/1/data/test/partition_428/19700102_2_2_0/checksums.txt
|
||||
38e62ff37e1e5064e9a3f605dfe09d13 shadow/1/data/test/partition_428/19700102_2_2_0/v1.bin
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/k.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/p.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700102_2_2_0/v1.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/k.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/p.mrk
|
||||
4ae71336e44bf9bf79d2752e234818a5 shadow/1/data/test/partition_428/19700201_1_1_0/v1.mrk
|
||||
55a54008ad1ba589aa210d2629c1df41 shadow/1/data/test/partition_428/19700201_1_1_0/primary.idx
|
||||
5f087cb3e7071bf9407e095821e2af8f shadow/1/data/test/partition_428/19700201_1_1_0/checksums.txt
|
||||
77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700102_2_2_0/columns.txt
|
||||
77d5af402ada101574f4da114f242e02 shadow/1/data/test/partition_428/19700201_1_1_0/columns.txt
|
||||
88cdc31ded355e7572d68d8cde525d3a shadow/1/data/test/partition_428/19700201_1_1_0/p.bin
|
||||
9e688c58a5487b8eaf69c9e1005ad0bf shadow/1/data/test/partition_428/19700102_2_2_0/primary.idx
|
||||
c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700102_2_2_0/count.txt
|
||||
c4ca4238a0b923820dcc509a6f75849b shadow/1/data/test/partition_428/19700201_1_1_0/count.txt
|
||||
cfcb770c3ecd0990dcceb1bde129e6c6 shadow/1/data/test/partition_428/19700102_2_2_0/p.bin
|
||||
e2af3bef1fd129aea73a890ede1e7a30 shadow/1/data/test/partition_428/19700201_1_1_0/k.bin
|
||||
f2312862cc01adf34a93151377be2ddf shadow/1/data/test/partition_428/19700201_1_1_0/minmax_p.idx
|
||||
31,1,2
|
||||
1,2,3
|
@ -1,60 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
set -e
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
# Not found column date in block. There are only columns: x.
|
||||
|
||||
# Test 1. Complex test checking columns.txt
|
||||
|
||||
chl="$CLICKHOUSE_CLIENT -q"
|
||||
ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path`
|
||||
|
||||
$chl "DROP TABLE IF EXISTS test.partition_428"
|
||||
$chl "CREATE TABLE test.partition_428 (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0"
|
||||
$chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(31), 1)"
|
||||
$chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(1), 2)"
|
||||
|
||||
for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do
|
||||
# 2 header lines + 3 columns
|
||||
(sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt 2>/dev/null || \
|
||||
cat $ch_dir/data/test/partition_428/$part/columns.txt) | wc -l
|
||||
done
|
||||
|
||||
$chl "ALTER TABLE test.partition_428 FREEZE"
|
||||
|
||||
# Do `cd` for consistent output for reference
|
||||
# Do not check increment.txt - it can be changed by other tests with FREEZE
|
||||
cd $ch_dir && find shadow -type f -exec md5sum {} \; | grep "partition_428" | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g' | sort | uniq
|
||||
|
||||
$chl "ALTER TABLE test.partition_428 DETACH PARTITION 197001"
|
||||
$chl "ALTER TABLE test.partition_428 ATTACH PARTITION 197001"
|
||||
|
||||
for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do
|
||||
# 2 header lines + 3 columns
|
||||
(sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt 2>/dev/null || \
|
||||
cat $ch_dir/data/test/partition_428/$part/columns.txt) | wc -l
|
||||
done
|
||||
|
||||
$chl "ALTER TABLE test.partition_428 MODIFY COLUMN v1 Int8"
|
||||
|
||||
# Check the backup hasn't changed
|
||||
cd $ch_dir && find shadow -type f -exec md5sum {} \; | grep "partition_428" | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g' | sort | uniq
|
||||
|
||||
$chl "OPTIMIZE TABLE test.partition_428"
|
||||
|
||||
$chl "SELECT toUInt16(p), k, v1 FROM test.partition_428 ORDER BY k FORMAT CSV"
|
||||
$chl "DROP TABLE test.partition_428"
|
||||
|
||||
# Test 2. Simple test
|
||||
|
||||
$chl "drop table if exists test.partition_428"
|
||||
$chl "create table test.partition_428 (date MATERIALIZED toDate(0), x UInt64, sample_key MATERIALIZED intHash64(x)) ENGINE=MergeTree PARTITION BY date SAMPLE BY sample_key ORDER BY (date,x,sample_key) SETTINGS index_granularity=8192, index_granularity_bytes=0"
|
||||
$chl "insert into test.partition_428 ( x ) VALUES ( now() )"
|
||||
$chl "insert into test.partition_428 ( x ) VALUES ( now()+1 )"
|
||||
$chl "alter table test.partition_428 detach partition 197001"
|
||||
$chl "alter table test.partition_428 attach partition 197001"
|
||||
$chl "optimize table test.partition_428"
|
||||
$chl "drop table test.partition_428"
|
@ -1,26 +0,0 @@
|
||||
=== cannot attach active ===
|
||||
OK1
|
||||
0_1_1_0
|
||||
1_2_2_0
|
||||
2_3_3_0
|
||||
3_4_4_0
|
||||
16 120
|
||||
=== check all parts before attaching ===
|
||||
OK2
|
||||
1_2_2_0
|
||||
1_4_4_0
|
||||
=== detached ===
|
||||
0_1_1_0
|
||||
0_3_3_0
|
||||
0_5_5_0
|
||||
attaching_0_6_6_0
|
||||
deleting_0_7_7_0
|
||||
=== attach ===
|
||||
0_5_5_0
|
||||
0_6_6_0
|
||||
1_2_2_0
|
||||
1_4_4_0
|
||||
16 120
|
||||
=== detached ===
|
||||
attaching_0_6_6_0
|
||||
deleting_0_7_7_0
|
@ -1,53 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
set -e
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path`
|
||||
cur_db=`${CLICKHOUSE_CLIENT} --query "SELECT currentDatabase()"`
|
||||
|
||||
echo '=== cannot attach active ===';
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_active";
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO attach_active SELECT number FROM system.numbers LIMIT 16";
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_active ATTACH PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK1'
|
||||
$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_active' AND database='${cur_db}' ORDER BY name FORMAT TSV";
|
||||
$CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_active FORMAT TSV";
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE attach_active";
|
||||
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES";
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_partitions";
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_partitions (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions DETACH PARTITION 0";
|
||||
sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ 2>/dev/null || \
|
||||
mkdir --mode=777 $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ # broken part
|
||||
sudo -n cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/ 2>/dev/null || \
|
||||
cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/
|
||||
sudo -n cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/ 2>/dev/null || \
|
||||
cp -pr $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/deleting_0_7_7_0/
|
||||
|
||||
echo '=== check all parts before attaching ===';
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0" 2>&1 | grep "No columns in part 0_5_5_0" > /dev/null && echo 'OK2';
|
||||
$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV";
|
||||
echo '=== detached ===';
|
||||
$CLICKHOUSE_CLIENT --query="SELECT name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV";
|
||||
|
||||
echo '=== attach ===';
|
||||
sudo -n rm -r $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ 2>/dev/null || \
|
||||
rm -r $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0";
|
||||
$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV";
|
||||
$CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_partitions FORMAT TSV";
|
||||
|
||||
echo '=== detached ===';
|
||||
$CLICKHOUSE_CLIENT --query="SELECT name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE attach_partitions";
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES";
|
@ -1,15 +0,0 @@
|
||||
=== validate part name ===
|
||||
OK1
|
||||
OK2
|
||||
OK3
|
||||
=== drop detached part ===
|
||||
0_3_3_0
|
||||
1_2_2_0
|
||||
1_4_4_0
|
||||
attaching_0_6_6_0
|
||||
deleting_0_7_7_0
|
||||
prefix_1_2_2_0_0
|
||||
=== drop detached partition ===
|
||||
0_3_3_0
|
||||
attaching_0_6_6_0
|
||||
deleting_0_7_7_0
|
@ -1,45 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
set -e
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path`
|
||||
cur_db=`${CLICKHOUSE_CLIENT} --query "SELECT currentDatabase()"`
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES";
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS drop_detached";
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8";
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE drop_detached DETACH PARTITION 0";
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE drop_detached DETACH PARTITION 1";
|
||||
sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/ 2>/dev/null || \
|
||||
mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/
|
||||
sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/ 2>/dev/null || \
|
||||
mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/deleting_0_7_7_0/
|
||||
sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/ 2>/dev/null || \
|
||||
mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/
|
||||
sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0_0/ 2>/dev/null || \
|
||||
mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0_0/
|
||||
#sudo -n mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0/ 2>/dev/null || \
|
||||
# mkdir --mode=777 $ch_dir/data/$cur_db/drop_detached/detached/prefix_1_2_2_0/
|
||||
|
||||
echo '=== validate part name ==='
|
||||
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK1'
|
||||
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '0_1_1_0'"
|
||||
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'attaching_0_6_6_0'" 2>&1 | grep "Cannot drop part" > /dev/null && echo 'OK2'
|
||||
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'deleting_0_7_7_0'" 2>&1 | grep "Cannot drop part" > /dev/null && echo 'OK3'
|
||||
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'any_other_name'"
|
||||
|
||||
echo '=== drop detached part ==='
|
||||
$CLICKHOUSE_CLIENT --query="SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' ORDER BY name FORMAT TSV";
|
||||
|
||||
echo '=== drop detached partition ==='
|
||||
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PARTITION 1"
|
||||
$CLICKHOUSE_CLIENT --query="SElECT name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' ORDER BY name FORMAT TSV";
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE drop_detached";
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES";
|
Loading…
Reference in New Issue
Block a user