Merge pull request #7812 from infinivision/fix_part_mutation

fix part mutation
This commit is contained in:
alexey-milovidov 2019-12-13 03:45:59 +03:00 committed by GitHub
commit 399995d60b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 278 additions and 123 deletions

View File

@ -2069,7 +2069,7 @@ void MergeTreeData::renameTempPartAndReplace(
*/
if (increment)
{
part_info.min_block = part_info.max_block = increment->get();
part_info.min_block = part_info.max_block = part_info.mutation = increment->get();
part_name = part->getNewName(part_info);
}
else

View File

@ -88,14 +88,18 @@ void FutureMergedMutatedPart::assign(MergeTreeData::DataPartsVector parts_)
parts = std::move(parts_);
UInt32 max_level = 0;
Int64 max_mutation = 0;
for (const auto & part : parts)
{
max_level = std::max(max_level, part->info.level);
max_mutation = std::max(max_mutation, part->info.mutation);
}
part_info.partition_id = parts.front()->info.partition_id;
part_info.min_block = parts.front()->info.min_block;
part_info.max_block = parts.back()->info.max_block;
part_info.level = max_level + 1;
part_info.mutation = parts.front()->info.mutation;
part_info.mutation = max_mutation;
if (parts.front()->storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{

View File

@ -0,0 +1,151 @@
import os.path
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()
def exec_bash(cmd):
cmd = '/bin/bash -c "{}"'.format(cmd.replace('"', '\\"'))
return instance.exec_in_container(cmd)
def copy_backup_to_detached(database, src_table, dst_table):
fp_increment = os.path.join(path_to_data, 'shadow/increment.txt')
increment = exec_bash('cat ' + fp_increment).strip()
fp_backup = os.path.join(path_to_data, 'shadow', increment, 'data', database, src_table)
fp_detached = os.path.join(path_to_data, 'data', database, dst_table, 'detached')
exec_bash('cp -r {}/* {}/'.format(fp_backup, fp_detached))
@pytest.fixture
def backup_restore(started_cluster):
q("DROP TABLE IF EXISTS test.tbl")
q("CREATE TABLE test.tbl (p Date, k Int8) ENGINE = MergeTree PARTITION BY toYYYYMM(p) ORDER BY p")
for i in range(1, 4):
q('INSERT INTO test.tbl (p, k) VALUES(toDate({}), {})'.format(i, i))
for i in range(31, 34):
q('INSERT INTO test.tbl (p, k) VALUES(toDate({}), {})'.format(i, i))
expected = TSV('1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33')
res = q("SELECT * FROM test.tbl ORDER BY p")
assert(TSV(res) == expected)
q("ALTER TABLE test.tbl FREEZE")
yield
q("DROP TABLE IF EXISTS test.tbl")
def test_restore(backup_restore):
q("CREATE TABLE test.tbl1 AS test.tbl")
copy_backup_to_detached('test', 'tbl', 'tbl1')
# The data_version of parts to be attached are larger than the newly created table's data_version.
q("ALTER TABLE test.tbl1 ATTACH PARTITION 197001")
q("ALTER TABLE test.tbl1 ATTACH PARTITION 197002")
q("SELECT sleep(2)")
# Validate the attached parts are identical to the backup.
expected = TSV('1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33')
res = q("SELECT * FROM test.tbl1 ORDER BY p")
assert(TSV(res) == expected)
q("ALTER TABLE test.tbl1 UPDATE k=10 WHERE 1")
q("SELECT sleep(2)")
# Validate mutation has been applied to all attached parts.
expected = TSV('1970-01-02\t10\n1970-01-03\t10\n1970-01-04\t10\n1970-02-01\t10\n1970-02-02\t10\n1970-02-03\t10')
res = q("SELECT * FROM test.tbl1 ORDER BY p")
assert(TSV(res) == expected)
q("DROP TABLE IF EXISTS test.tbl1")
def test_attach_partition(backup_restore):
q("CREATE TABLE test.tbl2 AS test.tbl")
for i in range(3, 5):
q('INSERT INTO test.tbl2(p, k) VALUES(toDate({}), {})'.format(i, i))
for i in range(33, 35):
q('INSERT INTO test.tbl2(p, k) VALUES(toDate({}), {})'.format(i, i))
expected = TSV('1970-01-04\t3\n1970-01-05\t4\n1970-02-03\t33\n1970-02-04\t34')
res = q("SELECT * FROM test.tbl2 ORDER BY p")
assert(TSV(res) == expected)
copy_backup_to_detached('test', 'tbl', 'tbl2')
# The data_version of parts to be attached
# - may be less than, equal to or larger than the current table's data_version.
# - may intersect with the existing parts of a partition.
q("ALTER TABLE test.tbl2 ATTACH PARTITION 197001")
q("ALTER TABLE test.tbl2 ATTACH PARTITION 197002")
q("SELECT sleep(2)")
expected = TSV('1970-01-02\t1\n1970-01-03\t2\n1970-01-04\t3\n1970-01-04\t3\n1970-01-05\t4\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33\n1970-02-03\t33\n1970-02-04\t34')
res = q("SELECT * FROM test.tbl2 ORDER BY p")
assert(TSV(res) == expected)
q("ALTER TABLE test.tbl2 UPDATE k=10 WHERE 1")
q("SELECT sleep(2)")
# Validate mutation has been applied to all attached parts.
expected = TSV('1970-01-02\t10\n1970-01-03\t10\n1970-01-04\t10\n1970-01-04\t10\n1970-01-05\t10\n1970-02-01\t10\n1970-02-02\t10\n1970-02-03\t10\n1970-02-03\t10\n1970-02-04\t10')
res = q("SELECT * FROM test.tbl2 ORDER BY p")
assert(TSV(res) == expected)
q("DROP TABLE IF EXISTS test.tbl2")
def test_replace_partition(backup_restore):
q("CREATE TABLE test.tbl3 AS test.tbl")
for i in range(3, 5):
q('INSERT INTO test.tbl3(p, k) VALUES(toDate({}), {})'.format(i, i))
for i in range(33, 35):
q('INSERT INTO test.tbl3(p, k) VALUES(toDate({}), {})'.format(i, i))
expected = TSV('1970-01-04\t3\n1970-01-05\t4\n1970-02-03\t33\n1970-02-04\t34')
res = q("SELECT * FROM test.tbl3 ORDER BY p")
assert(TSV(res) == expected)
copy_backup_to_detached('test', 'tbl', 'tbl3')
# The data_version of parts to be copied
# - may be less than, equal to or larger than the current table data_version.
# - may intersect with the existing parts of a partition.
q("ALTER TABLE test.tbl3 REPLACE PARTITION 197002 FROM test.tbl")
q("SELECT sleep(2)")
expected = TSV('1970-01-04\t3\n1970-01-05\t4\n1970-02-01\t31\n1970-02-02\t32\n1970-02-03\t33')
res = q("SELECT * FROM test.tbl3 ORDER BY p")
assert(TSV(res) == expected)
q("ALTER TABLE test.tbl3 UPDATE k=10 WHERE 1")
q("SELECT sleep(2)")
# Validate mutation has been applied to all copied parts.
expected = TSV('1970-01-04\t10\n1970-01-05\t10\n1970-02-01\t10\n1970-02-02\t10\n1970-02-03\t10')
res = q("SELECT * FROM test.tbl3 ORDER BY p")
assert(TSV(res) == expected)
q("DROP TABLE IF EXISTS test.tbl3")

View File

@ -50,35 +50,35 @@ def remove_part_from_disk(node, table, part_name):
def test_check_normal_table_corruption(started_cluster):
node1.query("INSERT INTO non_replicated_mt VALUES (toDate('2019-02-01'), 1, 10), (toDate('2019-02-01'), 2, 12)")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902", settings={"check_query_single_value_result": 0}) == "201902_1_1_0\t1\t\n"
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902", settings={"check_query_single_value_result": 0}) == "201902_1_1_0_1\t1\t\n"
remove_checksums_on_disk(node1, "non_replicated_mt", "201902_1_1_0")
remove_checksums_on_disk(node1, "non_replicated_mt", "201902_1_1_0_1")
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk."
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0_1\t1\tChecksums recounted and written to disk."
assert node1.query("SELECT COUNT() FROM non_replicated_mt") == "2\n"
remove_checksums_on_disk(node1, "non_replicated_mt", "201902_1_1_0")
remove_checksums_on_disk(node1, "non_replicated_mt", "201902_1_1_0_1")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t1\tChecksums recounted and written to disk."
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201902", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0_1\t1\tChecksums recounted and written to disk."
assert node1.query("SELECT COUNT() FROM non_replicated_mt") == "2\n"
corrupt_data_part_on_disk(node1, "non_replicated_mt", "201902_1_1_0")
corrupt_data_part_on_disk(node1, "non_replicated_mt", "201902_1_1_0_1")
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0_1\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
assert node1.query("CHECK TABLE non_replicated_mt", settings={"check_query_single_value_result": 0}).strip() == "201902_1_1_0_1\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16."
node1.query("INSERT INTO non_replicated_mt VALUES (toDate('2019-01-01'), 1, 10), (toDate('2019-01-01'), 2, 12)")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_2_2_0\t1\t\n"
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_2_2_0_2\t1\t\n"
corrupt_data_part_on_disk(node1, "non_replicated_mt", "201901_2_2_0")
corrupt_data_part_on_disk(node1, "non_replicated_mt", "201901_2_2_0_2")
remove_checksums_on_disk(node1, "non_replicated_mt", "201901_2_2_0")
remove_checksums_on_disk(node1, "non_replicated_mt", "201901_2_2_0_2")
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_2_2_0\t0\tCheck of part finished with error: \\'Cannot read all data. Bytes read: 2. Bytes expected: 16.\\'\n"
assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901", settings={"check_query_single_value_result": 0}) == "201901_2_2_0_2\t0\tCheck of part finished with error: \\'Cannot read all data. Bytes read: 2. Bytes expected: 16.\\'\n"
def test_check_replicated_table_simple(started_cluster):

View File

@ -37,8 +37,8 @@ def test_allow_databases(start_cluster):
assert node5.query("SELECT name FROM system.databases WHERE name = 'db1'") == "db1\n"
assert node5.query("SELECT name FROM system.tables WHERE database = 'db1' AND name = 'test_table' ") == "test_table\n"
assert node5.query("SELECT name FROM system.columns WHERE database = 'db1' AND table = 'test_table'") == "date\nk1\nv1\n"
assert node5.query("SELECT name FROM system.parts WHERE database = 'db1' AND table = 'test_table'") == "20000101_20000101_1_1_0\n"
assert node5.query("SELECT name FROM system.parts_columns WHERE database = 'db1' AND table = 'test_table'") == "20000101_20000101_1_1_0\n20000101_20000101_1_1_0\n20000101_20000101_1_1_0\n"
assert node5.query("SELECT name FROM system.parts WHERE database = 'db1' AND table = 'test_table'") == "20000101_20000101_1_1_0_1\n"
assert node5.query("SELECT name FROM system.parts_columns WHERE database = 'db1' AND table = 'test_table'") == "20000101_20000101_1_1_0_1\n20000101_20000101_1_1_0_1\n20000101_20000101_1_1_0_1\n"
assert node5.query("SELECT name FROM system.databases WHERE name = 'db1'", user="test_allow").strip() == ""
assert node5.query("SELECT name FROM system.tables WHERE database = 'db1' AND name = 'test_table'", user="test_allow").strip() == ""

View File

@ -24,6 +24,6 @@ def test_file_path_escaping(started_cluster):
node.query('''INSERT INTO test.`T.a_b,l-e!` VALUES (1);''')
node.query('''ALTER TABLE test.`T.a_b,l-e!` FREEZE;''')
node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/data/test/T%2Ea_b%2Cl%2De%21/1_1_1_0/%7EId.bin"])
node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/shadow/1/data/test/T%2Ea_b%2Cl%2De%21/1_1_1_0/%7EId.bin"])
node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/data/test/T%2Ea_b%2Cl%2De%21/1_1_1_0_1/%7EId.bin"])
node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/shadow/1/data/test/T%2Ea_b%2Cl%2De%21/1_1_1_0_1/%7EId.bin"])

View File

@ -146,11 +146,11 @@ def test_inserts_batching(started_cluster):
# 4. Full batch of inserts after ALTER (that have different block structure).
# 5. What was left to insert with the column structure before ALTER.
expected = '''\
20000101_20000101_1_1_0\t[1]
20000101_20000101_2_2_0\t[2,3,4]
20000101_20000101_3_3_0\t[5,6,7]
20000101_20000101_4_4_0\t[10,11,12]
20000101_20000101_5_5_0\t[8,9]
20000101_20000101_1_1_0_1\t[1]
20000101_20000101_2_2_0_2\t[2,3,4]
20000101_20000101_3_3_0_3\t[5,6,7]
20000101_20000101_4_4_0_4\t[10,11,12]
20000101_20000101_5_5_0_5\t[8,9]
'''
assert TSV(result) == TSV(expected)

View File

@ -65,30 +65,30 @@ def partition_complex_assert_checksums():
" | 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"
checksums = "082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition/19700102_2_2_0_2/k.bin\n" \
"082814b5aa5109160d5c0c5aff10d4df\tshadow/1/data/test/partition/19700201_1_1_0_1/v1.bin\n" \
"13cae8e658e0ca4f75c56b1fc424e150\tshadow/1/data/test/partition/19700102_2_2_0_2/minmax_p.idx\n" \
"25daad3d9e60b45043a70c4ab7d3b1c6\tshadow/1/data/test/partition/19700102_2_2_0_2/partition.dat\n" \
"3726312af62aec86b64a7708d5751787\tshadow/1/data/test/partition/19700201_1_1_0_1/partition.dat\n" \
"37855b06a39b79a67ea4e86e4a3299aa\tshadow/1/data/test/partition/19700102_2_2_0_2/checksums.txt\n" \
"38e62ff37e1e5064e9a3f605dfe09d13\tshadow/1/data/test/partition/19700102_2_2_0_2/v1.bin\n" \
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0_2/k.mrk\n" \
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0_2/p.mrk\n" \
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0_2/v1.mrk\n" \
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0_1/k.mrk\n" \
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0_1/p.mrk\n" \
"4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0_1/v1.mrk\n" \
"55a54008ad1ba589aa210d2629c1df41\tshadow/1/data/test/partition/19700201_1_1_0_1/primary.idx\n" \
"5f087cb3e7071bf9407e095821e2af8f\tshadow/1/data/test/partition/19700201_1_1_0_1/checksums.txt\n" \
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700102_2_2_0_2/columns.txt\n" \
"77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700201_1_1_0_1/columns.txt\n" \
"88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition/19700201_1_1_0_1/p.bin\n" \
"9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition/19700102_2_2_0_2/primary.idx\n" \
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700102_2_2_0_2/count.txt\n" \
"c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700201_1_1_0_1/count.txt\n" \
"cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition/19700102_2_2_0_2/p.bin\n" \
"e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition/19700201_1_1_0_1/k.bin\n" \
"f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition/19700201_1_1_0_1/minmax_p.idx\n"
assert TSV(exec_bash(cmd).replace(' ', '\t')) == TSV(checksums)
@ -149,7 +149,7 @@ def test_cannot_attach_active_part(cannot_attach_active_part_table):
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(res) == TSV('0_1_1_0_1\n1_2_2_0_2\n2_3_3_0_3\n3_4_4_0_4')
assert TSV(q("SElECT count(), sum(n) FROM test.attach_active")) == TSV('16\t120')
@ -171,30 +171,30 @@ 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'))
exec_bash('mkdir {}'.format(path_to_detached + '0_5_5_0_5'))
exec_bash('cp -pr {} {}'.format(path_to_detached + '0_1_1_0_1', path_to_detached + 'attaching_0_6_6_0_6'))
exec_bash('cp -pr {} {}'.format(path_to_detached + '0_3_3_0_3', path_to_detached + 'deleting_0_7_7_0_7'))
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')
assert TSV(parts) == TSV('1_2_2_0_2\n1_4_4_0_4')
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')
assert TSV(detached) == TSV('0_1_1_0_1\n0_3_3_0_3\n0_5_5_0_5\nattaching_0_6_6_0_6\ndeleting_0_7_7_0_7')
exec_bash('rm -r {}'.format(path_to_detached + '0_5_5_0'))
exec_bash('rm -r {}'.format(path_to_detached + '0_5_5_0_5'))
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'
expected = '0_5_5_0_5\n0_6_6_0_6\n1_2_2_0_2\n1_4_4_0_4'
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')
assert TSV(detached) == TSV('attaching_0_6_6_0_6\ndeleting_0_7_7_0_7')
@pytest.fixture
@ -217,28 +217,28 @@ def test_drop_detached_parts(drop_detached_parts_table):
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 + 'attaching_0_6_6_0_6'))
exec_bash('mkdir {}'.format(path_to_detached + 'deleting_0_7_7_0_7'))
exec_bash('mkdir {}'.format(path_to_detached + 'any_other_name'))
exec_bash('mkdir {}'.format(path_to_detached + 'prefix_1_2_2_0_0'))
exec_bash('mkdir {}'.format(path_to_detached + 'prefix_1_2_2_0_2'))
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART '../1_2_2_0'", settings=s)
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART '../1_2_2_0_2'", settings=s)
assert 0 <= error.find('Invalid part name')
q("ALTER TABLE test.drop_detached DROP DETACHED PART '0_1_1_0'", settings=s)
q("ALTER TABLE test.drop_detached DROP DETACHED PART '0_1_1_0_1'", settings=s)
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART 'attaching_0_6_6_0'", settings=s)
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART 'attaching_0_6_6_0_6'", 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)
error = instance.client.query_and_get_error("ALTER TABLE test.drop_detached DROP DETACHED PART 'deleting_0_7_7_0_7'", 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')
assert TSV(detached) == TSV('0_3_3_0_3\n1_2_2_0_2\n1_4_4_0_4\nattaching_0_6_6_0_6\ndeleting_0_7_7_0_7\nprefix_1_2_2_0_2')
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')
assert TSV(detached) == TSV('0_3_3_0_3\nattaching_0_6_6_0_6\ndeleting_0_7_7_0_7')

View File

@ -1,60 +1,60 @@
*** Not partitioned ***
Parts before OPTIMIZE:
tuple() all_1_1_0
tuple() all_2_2_0
tuple() all_1_1_0_1
tuple() all_2_2_0_2
Parts after OPTIMIZE:
tuple() all_1_2_1
tuple() all_1_2_1_2
Sum before DETACH PARTITION:
15
Sum after DETACH PARTITION:
0
system.detached_parts after DETACH PARTITION:
default not_partitioned all all_1_2_1 default 1 2 1
default not_partitioned all all_1_2_1_2 default 1 2 1
*** Partitioned by week ***
Parts before OPTIMIZE:
1999-12-27 19991227_1_1_0
2000-01-03 20000103_2_2_0
2000-01-03 20000103_3_3_0
1999-12-27 19991227_1_1_0_1
2000-01-03 20000103_2_2_0_2
2000-01-03 20000103_3_3_0_3
Parts after OPTIMIZE:
1999-12-27 19991227_1_1_0
2000-01-03 20000103_2_3_1
1999-12-27 19991227_1_1_0_1
2000-01-03 20000103_2_3_1_3
Sum before DROP PARTITION:
15
Sum after DROP PARTITION:
12
*** Partitioned by a (Date, UInt8) tuple ***
Parts before OPTIMIZE:
(\'2000-01-01\',1) 20000101-1_1_1_0
(\'2000-01-01\',1) 20000101-1_5_5_0
(\'2000-01-01\',2) 20000101-2_2_2_0
(\'2000-01-02\',1) 20000102-1_3_3_0
(\'2000-01-02\',1) 20000102-1_4_4_0
(\'2000-01-01\',1) 20000101-1_1_1_0_1
(\'2000-01-01\',1) 20000101-1_5_5_0_5
(\'2000-01-01\',2) 20000101-2_2_2_0_2
(\'2000-01-02\',1) 20000102-1_3_3_0_3
(\'2000-01-02\',1) 20000102-1_4_4_0_4
Parts after OPTIMIZE:
(\'2000-01-01\',1) 20000101-1_1_5_1
(\'2000-01-01\',2) 20000101-2_2_2_0
(\'2000-01-02\',1) 20000102-1_3_4_1
(\'2000-01-01\',1) 20000101-1_1_5_1_5
(\'2000-01-01\',2) 20000101-2_2_2_0_2
(\'2000-01-02\',1) 20000102-1_3_4_1_4
Sum before DETACH PARTITION:
15
Sum after DETACH PARTITION:
9
*** Partitioned by String ***
Parts before OPTIMIZE:
bbb 7d878f3d88441d2b3dc371e2a3050f6d_2_2_0
bbb 7d878f3d88441d2b3dc371e2a3050f6d_3_3_0
aaa 9b50856126a8a6064f11f027d455bf58_1_1_0
aaa 9b50856126a8a6064f11f027d455bf58_4_4_0
bbb 7d878f3d88441d2b3dc371e2a3050f6d_2_2_0_2
bbb 7d878f3d88441d2b3dc371e2a3050f6d_3_3_0_3
aaa 9b50856126a8a6064f11f027d455bf58_1_1_0_1
aaa 9b50856126a8a6064f11f027d455bf58_4_4_0_4
Parts after OPTIMIZE:
bbb 7d878f3d88441d2b3dc371e2a3050f6d_2_2_0
bbb 7d878f3d88441d2b3dc371e2a3050f6d_3_3_0
aaa 9b50856126a8a6064f11f027d455bf58_1_4_1
bbb 7d878f3d88441d2b3dc371e2a3050f6d_2_2_0_2
bbb 7d878f3d88441d2b3dc371e2a3050f6d_3_3_0_3
aaa 9b50856126a8a6064f11f027d455bf58_1_4_1_4
Sum before DROP PARTITION:
15
Sum after DROP PARTITION:
8
*** Table without columns with fixed size ***
Parts:
1 1_1_1_0 2
2 2_2_2_0 2
1 1_1_1_0_1 2
2 2_2_2_0_2 2
Before DROP PARTITION:
a
aa

View File

@ -17,7 +17,7 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill a single invalid mutat
${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE toUInt32(s) = 1"
sleep 0.1
${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_1_1_0', '20010101_2_2_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation'"
${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_1_1_0_1', '20010101_2_2_0_2'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation'"
${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation'"
@ -29,7 +29,7 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill invalid mutation that
${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE toUInt32(s) = 1"
${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation DELETE WHERE x = 1"
${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_1_1_0', '20010101_2_2_0'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation' AND mutation_id = 'mutation_4.txt'"
${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, latest_failed_part IN ('20000101_1_1_0_1', '20010101_2_2_0_2'), latest_fail_time != 0, substr(latest_fail_reason, 1, 8) FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation' AND mutation_id = 'mutation_4.txt'"
sleep 0.1
${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = 'kill_mutation' AND mutation_id = 'mutation_4.txt'"

View File

@ -1,19 +1,19 @@
19701001_1_1_0 0
19701002_2_2_0 0
19701003_3_3_0 0
19701001_1_1_0_1 0
19701002_2_2_0_2 0
19701003_3_3_0_3 0
freeze one
19701001_1_1_0 0
19701002_2_2_0 1
19701003_3_3_0 0
19701001_1_1_0_1 0
19701002_2_2_0_2 1
19701003_3_3_0_3 0
freeze all
19701001_1_1_0 1
19701002_2_2_0 1
19701003_3_3_0 1
19701001_1_1_0_1 1
19701002_2_2_0_2 1
19701003_3_3_0_3 1
1970-10-01 00:00:01
1970-10-02 00:00:01
1970-10-02 00:00:02
1970-10-03 00:00:01
19701001_1_1_0 1
19701002_2_2_0 1
19701002_4_4_0 0
19701003_3_3_0 1
19701001_1_1_0_1 1
19701002_2_2_0_2 1
19701002_4_4_0_4 0
19701003_3_3_0_3 1

View File

@ -1,11 +1,11 @@
201901_1_1_0 1
201901_1_1_0_1 1
========
201901_1_1_0 1
201901_2_2_0 1
201901_1_1_0_1 1
201901_2_2_0_2 1
========
201901_1_2_1 1
201901_1_2_1_2 1
========
201901_1_2_1 1
201902_3_3_0 1
201901_1_2_1_2 1
201902_3_3_0_3 1
========
201902_3_4_1 1
201902_3_4_1_4 1

View File

@ -1 +1 @@
20000101_20000101_1_1_0 test_00961 1c63ae7a38eb76e2a71c28aaf0b3ae4d 0053df9b467cc5483e752ec62e91cfd4 da96ff1e527a8a1f908ddf2b1d0af239
20000101_20000101_1_1_0_1 test_00961 1c63ae7a38eb76e2a71c28aaf0b3ae4d 0053df9b467cc5483e752ec62e91cfd4 da96ff1e527a8a1f908ddf2b1d0af239

View File

@ -15,7 +15,7 @@ ALTER TABLE table_01 DETACH PARTITION ID '20191001';
SELECT COUNT() FROM table_01;
ALTER TABLE table_01 ATTACH PART '20191001_1_1_0';
ALTER TABLE table_01 ATTACH PART '20191001_1_1_0_1';
SELECT COUNT() FROM table_01;

View File

@ -1,2 +1,2 @@
all_1_1_0 1
all_1_1_0_1 1
all_0_0_0 1

View File

@ -1,7 +1,7 @@
all_1_1_0 1
all_1_1_0 1
all_1_1_0 1
all_1_1_0 1
all_1_1_0 1
all_2_2_0 1
all_1_2_1 1
all_1_1_0_1 1
all_1_1_0_1 1
all_1_1_0_1 1
all_1_1_0_1 1
all_1_1_0_1 1
all_2_2_0_2 1
all_1_2_1_2 1