2019-08-08 15:51:17 +00:00
import pytest
2021-06-11 08:06:37 +00:00
import logging
from helpers . cluster import ClickHouseCluster
2019-08-08 15:51:17 +00:00
from helpers . test_tools import TSV
2022-09-30 18:09:18 +00:00
from helpers . test_tools import assert_eq_with_retry
2022-09-18 23:26:12 +00:00
from helpers . wait_for_helpers import wait_for_delete_inactive_parts
from helpers . wait_for_helpers import wait_for_delete_empty_parts
2019-08-08 15:51:17 +00:00
cluster = ClickHouseCluster ( __file__ )
2022-09-30 18:09:18 +00:00
instance = cluster . add_instance (
" instance " ,
main_configs = [
" configs/testkeeper.xml " ,
] ,
)
2019-08-08 15:51:17 +00:00
q = instance . query
2022-03-22 16:39:58 +00:00
path_to_data = " /var/lib/clickhouse/ "
2019-08-08 15:51:17 +00:00
@pytest.fixture ( scope = " module " )
def started_cluster ( ) :
try :
cluster . start ( )
2022-03-22 16:39:58 +00:00
q (
2022-06-23 19:40:05 +00:00
" CREATE DATABASE test ENGINE = Ordinary " ,
settings = { " allow_deprecated_database_ordinary " : 1 } ,
2022-03-22 16:39:58 +00:00
) # Different path in shadow/ with Atomic
2019-08-08 15:51:17 +00:00
yield cluster
finally :
cluster . shutdown ( )
@pytest.fixture
def partition_table_simple ( started_cluster ) :
2021-06-11 08:06:37 +00:00
q ( " DROP TABLE IF EXISTS test.partition_simple " )
2022-03-22 16:39:58 +00:00
q (
" CREATE TABLE test.partition_simple (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) "
2022-11-17 17:06:59 +00:00
" SETTINGS index_granularity=8192, index_granularity_bytes=0, compress_marks=false, compress_primary_key=false "
2022-03-22 16:39:58 +00:00
)
2021-06-11 08:06:37 +00:00
q ( " INSERT INTO test.partition_simple ( x ) VALUES ( now() ) " )
q ( " INSERT INTO test.partition_simple ( x ) VALUES ( now()+1 ) " )
2019-08-08 15:51:17 +00:00
yield
2022-03-22 16:39:58 +00:00
q ( " DROP TABLE test.partition_simple " )
2019-08-08 15:51:17 +00:00
def test_partition_simple ( partition_table_simple ) :
2021-06-11 08:06:37 +00:00
q ( " ALTER TABLE test.partition_simple DETACH PARTITION 197001 " )
q ( " ALTER TABLE test.partition_simple ATTACH PARTITION 197001 " )
q ( " OPTIMIZE TABLE test.partition_simple " )
2019-08-08 15:51:17 +00:00
def partition_complex_assert_columns_txt ( ) :
2022-03-22 16:39:58 +00:00
path_to_parts = path_to_data + " data/test/partition_complex/ "
parts = TSV (
q (
" SELECT name FROM system.parts WHERE database= ' test ' AND table= ' partition_complex ' "
)
)
2021-06-11 08:06:37 +00:00
assert len ( parts ) > 0
2019-08-08 15:51:17 +00:00
for part_name in parts . lines :
2022-03-22 16:39:58 +00:00
path_to_columns = path_to_parts + part_name + " /columns.txt "
2019-08-08 15:51:17 +00:00
# 2 header lines + 3 columns
2022-03-22 16:39:58 +00:00
assert (
instance . exec_in_container ( [ " wc " , " -l " , path_to_columns ] ) . split ( ) [ 0 ] == " 5 "
)
2019-08-08 15:51:17 +00:00
def partition_complex_assert_checksums ( ) :
# Do not check increment.txt - it can be changed by other tests with FREEZE
2022-03-22 16:39:58 +00:00
cmd = [
" bash " ,
" -c " ,
f " cd { path_to_data } && find shadow -type f -exec "
+ " md5sum {} \\ ; | grep partition_complex "
" | sed ' s shadow/[0-9]*/data/[a-z0-9_-]*/ shadow/1/data/test/ g ' | sort | uniq " ,
]
checksums = (
" 082814b5aa5109160d5c0c5aff10d4df \t shadow/1/data/test/partition_complex/19700102_2_2_0/k.bin \n "
" 082814b5aa5109160d5c0c5aff10d4df \t shadow/1/data/test/partition_complex/19700201_1_1_0/v1.bin \n "
" 13cae8e658e0ca4f75c56b1fc424e150 \t shadow/1/data/test/partition_complex/19700102_2_2_0/minmax_p.idx \n "
" 25daad3d9e60b45043a70c4ab7d3b1c6 \t shadow/1/data/test/partition_complex/19700102_2_2_0/partition.dat \n "
" 3726312af62aec86b64a7708d5751787 \t shadow/1/data/test/partition_complex/19700201_1_1_0/partition.dat \n "
" 37855b06a39b79a67ea4e86e4a3299aa \t shadow/1/data/test/partition_complex/19700102_2_2_0/checksums.txt \n "
" 38e62ff37e1e5064e9a3f605dfe09d13 \t shadow/1/data/test/partition_complex/19700102_2_2_0/v1.bin \n "
" 4ae71336e44bf9bf79d2752e234818a5 \t shadow/1/data/test/partition_complex/19700102_2_2_0/k.mrk \n "
" 4ae71336e44bf9bf79d2752e234818a5 \t shadow/1/data/test/partition_complex/19700102_2_2_0/p.mrk \n "
" 4ae71336e44bf9bf79d2752e234818a5 \t shadow/1/data/test/partition_complex/19700102_2_2_0/v1.mrk \n "
" 4ae71336e44bf9bf79d2752e234818a5 \t shadow/1/data/test/partition_complex/19700201_1_1_0/k.mrk \n "
" 4ae71336e44bf9bf79d2752e234818a5 \t shadow/1/data/test/partition_complex/19700201_1_1_0/p.mrk \n "
" 4ae71336e44bf9bf79d2752e234818a5 \t shadow/1/data/test/partition_complex/19700201_1_1_0/v1.mrk \n "
" 55a54008ad1ba589aa210d2629c1df41 \t shadow/1/data/test/partition_complex/19700201_1_1_0/primary.idx \n "
" 5f087cb3e7071bf9407e095821e2af8f \t shadow/1/data/test/partition_complex/19700201_1_1_0/checksums.txt \n "
" 77d5af402ada101574f4da114f242e02 \t shadow/1/data/test/partition_complex/19700102_2_2_0/columns.txt \n "
" 77d5af402ada101574f4da114f242e02 \t shadow/1/data/test/partition_complex/19700201_1_1_0/columns.txt \n "
" 88cdc31ded355e7572d68d8cde525d3a \t shadow/1/data/test/partition_complex/19700201_1_1_0/p.bin \n "
" 9e688c58a5487b8eaf69c9e1005ad0bf \t shadow/1/data/test/partition_complex/19700102_2_2_0/primary.idx \n "
" c0904274faa8f3f06f35666cc9c5bd2f \t shadow/1/data/test/partition_complex/19700102_2_2_0/default_compression_codec.txt \n "
" c0904274faa8f3f06f35666cc9c5bd2f \t shadow/1/data/test/partition_complex/19700201_1_1_0/default_compression_codec.txt \n "
" c4ca4238a0b923820dcc509a6f75849b \t shadow/1/data/test/partition_complex/19700102_2_2_0/count.txt \n "
" c4ca4238a0b923820dcc509a6f75849b \t shadow/1/data/test/partition_complex/19700201_1_1_0/count.txt \n "
" cfcb770c3ecd0990dcceb1bde129e6c6 \t shadow/1/data/test/partition_complex/19700102_2_2_0/p.bin \n "
" e2af3bef1fd129aea73a890ede1e7a30 \t shadow/1/data/test/partition_complex/19700201_1_1_0/k.bin \n "
" f2312862cc01adf34a93151377be2ddf \t shadow/1/data/test/partition_complex/19700201_1_1_0/minmax_p.idx \n "
)
assert TSV ( instance . exec_in_container ( cmd ) . replace ( " " , " \t " ) ) == TSV ( checksums )
2019-08-08 15:51:17 +00:00
@pytest.fixture
def partition_table_complex ( started_cluster ) :
2021-06-11 08:06:37 +00:00
q ( " DROP TABLE IF EXISTS test.partition_complex " )
2022-03-22 16:39:58 +00:00
q (
" CREATE TABLE test.partition_complex (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) "
2022-11-17 17:06:59 +00:00
" ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0, compress_marks=false, compress_primary_key=false "
2022-03-22 16:39:58 +00:00
)
2021-06-11 08:06:37 +00:00
q ( " INSERT INTO test.partition_complex (p, k) VALUES(toDate(31), 1) " )
q ( " INSERT INTO test.partition_complex (p, k) VALUES(toDate(1), 2) " )
2019-08-08 15:51:17 +00:00
yield
2021-06-11 08:06:37 +00:00
q ( " DROP TABLE test.partition_complex " )
2019-08-08 15:51:17 +00:00
def test_partition_complex ( partition_table_complex ) :
partition_complex_assert_columns_txt ( )
2021-06-11 08:06:37 +00:00
q ( " ALTER TABLE test.partition_complex FREEZE " )
2019-08-08 15:51:17 +00:00
partition_complex_assert_checksums ( )
2021-06-11 08:06:37 +00:00
q ( " ALTER TABLE test.partition_complex DETACH PARTITION 197001 " )
q ( " ALTER TABLE test.partition_complex ATTACH PARTITION 197001 " )
2019-08-08 15:51:17 +00:00
partition_complex_assert_columns_txt ( )
2021-06-11 08:06:37 +00:00
q ( " ALTER TABLE test.partition_complex MODIFY COLUMN v1 Int8 " )
2019-08-08 15:51:17 +00:00
# Check the backup hasn't changed
partition_complex_assert_checksums ( )
2021-06-11 08:06:37 +00:00
q ( " OPTIMIZE TABLE test.partition_complex " )
2019-08-08 15:51:17 +00:00
2022-03-22 16:39:58 +00:00
expected = TSV ( " 31 \t 1 \t 2 \n " " 1 \t 2 \t 3 " )
2021-06-11 08:06:37 +00:00
res = q ( " SELECT toUInt16(p), k, v1 FROM test.partition_complex ORDER BY k " )
2022-03-22 16:39:58 +00:00
assert TSV ( res ) == expected
2019-08-08 15:51:17 +00:00
@pytest.fixture
def cannot_attach_active_part_table ( started_cluster ) :
q ( " DROP TABLE IF EXISTS test.attach_active " )
2022-03-22 16:39:58 +00:00
q (
2022-11-17 17:06:59 +00:00
" CREATE TABLE test.attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n SETTINGS compress_marks=false, compress_primary_key=false "
2022-03-22 16:39:58 +00:00
)
2019-08-08 15:51:17 +00:00
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 ) :
2022-03-22 16:39:58 +00:00
error = instance . client . query_and_get_error (
" ALTER TABLE test.attach_active ATTACH PART ' ../1_2_2_0 ' "
)
2020-10-02 16:54:07 +00:00
print ( error )
2022-03-22 16:39:58 +00:00
assert 0 < = error . find ( " Invalid part name " )
2019-08-08 15:51:17 +00:00
2022-03-22 16:39:58 +00:00
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 \n 1_2_2_0 \n 2_3_3_0 \n 3_4_4_0 " )
assert TSV ( q ( " SElECT count(), sum(n) FROM test.attach_active " ) ) == TSV ( " 16 \t 120 " )
2019-08-08 15:51:17 +00:00
@pytest.fixture
def attach_check_all_parts_table ( started_cluster ) :
q ( " SYSTEM STOP MERGES " )
q ( " DROP TABLE IF EXISTS test.attach_partition " )
2022-03-22 16:39:58 +00:00
q (
2023-01-17 11:02:18 +00:00
" CREATE TABLE test.attach_partition (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n "
" SETTINGS compress_marks=false, compress_primary_key=false, old_parts_lifetime=0 "
2022-03-22 16:39:58 +00:00
)
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 "
)
2019-08-08 15:51:17 +00:00
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 " )
2022-09-18 23:26:12 +00:00
wait_for_delete_empty_parts ( instance , " test.attach_partition " )
2022-12-12 11:37:52 +00:00
wait_for_delete_inactive_parts ( instance , " test.attach_partition " )
2022-09-18 23:26:12 +00:00
2022-03-22 16:39:58 +00:00
path_to_detached = path_to_data + " data/test/attach_partition/detached/ "
instance . exec_in_container ( [ " mkdir " , " {} " . format ( path_to_detached + " 0_5_5_0 " ) ] )
instance . exec_in_container (
[
" cp " ,
" -pr " ,
path_to_detached + " 0_1_1_0 " ,
path_to_detached + " attaching_0_6_6_0 " ,
]
)
instance . exec_in_container (
[
" cp " ,
" -pr " ,
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 " ) or 0 < = error . find (
" No columns.txt in part 0_5_5_0 "
)
parts = q (
2022-09-22 22:51:13 +00:00
" SElECT name FROM system.parts "
" WHERE table= ' attach_partition ' AND database= ' test ' AND active ORDER BY name "
2022-03-22 16:39:58 +00:00
)
assert TSV ( parts ) == TSV ( " 1_2_2_0 \n 1_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 \n 0_3_3_0 \n 0_5_5_0 \n attaching_0_6_6_0 \n deleting_0_7_7_0 "
)
instance . exec_in_container ( [ " rm " , " -r " , path_to_detached + " 0_5_5_0 " ] )
2019-08-08 15:51:17 +00:00
q ( " ALTER TABLE test.attach_partition ATTACH PARTITION 0 " )
2022-03-22 16:39:58 +00:00
parts = q (
" SElECT name FROM system.parts WHERE table= ' attach_partition ' AND database= ' test ' ORDER BY name "
)
expected = " 0_5_5_0 \n 0_6_6_0 \n 1_2_2_0 \n 1_4_4_0 "
2019-08-08 15:51:17 +00:00
assert TSV ( parts ) == TSV ( expected )
2022-03-22 16:39:58 +00:00
assert TSV ( q ( " SElECT count(), sum(n) FROM test.attach_partition " ) ) == TSV ( " 16 \t 120 " )
2019-08-08 15:51:17 +00:00
2022-03-22 16:39:58 +00:00
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 \n deleting_0_7_7_0 " )
2019-08-08 15:51:17 +00:00
@pytest.fixture
def drop_detached_parts_table ( started_cluster ) :
q ( " SYSTEM STOP MERGES " )
q ( " DROP TABLE IF EXISTS test.drop_detached " )
2022-03-22 16:39:58 +00:00
q (
2022-11-17 17:06:59 +00:00
" CREATE TABLE test.drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n SETTINGS compress_marks=false, compress_primary_key=false "
2022-03-22 16:39:58 +00:00
)
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 "
)
2019-08-08 15:51:17 +00:00
yield
q ( " DROP TABLE test.drop_detached " )
q ( " SYSTEM START MERGES " )
def test_drop_detached_parts ( drop_detached_parts_table ) :
2019-08-08 16:08:43 +00:00
s = { " allow_drop_detached " : 1 }
2019-08-08 15:51:17 +00:00
q ( " ALTER TABLE test.drop_detached DETACH PARTITION 0 " )
q ( " ALTER TABLE test.drop_detached DETACH PARTITION 1 " )
2022-03-22 16:39:58 +00:00
path_to_detached = path_to_data + " data/test/drop_detached/detached/ "
instance . exec_in_container (
[ " mkdir " , " {} " . format ( path_to_detached + " attaching_0_6_6_0 " ) ]
)
instance . exec_in_container (
[ " mkdir " , " {} " . format ( path_to_detached + " deleting_0_7_7_0 " ) ]
)
instance . exec_in_container (
[ " mkdir " , " {} " . format ( path_to_detached + " any_other_name " ) ]
)
instance . exec_in_container (
[ " 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 " )
2019-08-08 15:51:17 +00:00
q ( " ALTER TABLE test.drop_detached DROP DETACHED PART ' 0_1_1_0 ' " , settings = s )
2022-03-22 16:39:58 +00:00
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 " )
2019-08-08 15:51:17 +00:00
2022-03-22 16:39:58 +00:00
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 " )
2019-08-08 15:51:17 +00:00
q ( " ALTER TABLE test.drop_detached DROP DETACHED PART ' any_other_name ' " , settings = s )
2022-03-22 16:39:58 +00:00
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 \n 1_2_2_0 \n 1_4_4_0 \n attaching_0_6_6_0 \n deleting_0_7_7_0 \n prefix_1_2_2_0_0 "
)
2019-08-08 15:51:17 +00:00
q ( " ALTER TABLE test.drop_detached DROP DETACHED PARTITION 1 " , settings = s )
2022-03-22 16:39:58 +00:00
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 \n attaching_0_6_6_0 \n deleting_0_7_7_0 " )
2021-08-04 14:42:48 +00:00
def test_system_detached_parts ( drop_detached_parts_table ) :
2022-11-17 17:13:23 +00:00
q (
" create table sdp_0 (n int, x int) engine=MergeTree order by n SETTINGS compress_marks=false, compress_primary_key=false "
)
q (
" create table sdp_1 (n int, x int) engine=MergeTree order by n partition by x SETTINGS compress_marks=false, compress_primary_key=false "
)
q (
" create table sdp_2 (n int, x String) engine=MergeTree order by n partition by x SETTINGS compress_marks=false, compress_primary_key=false "
)
2022-03-22 16:39:58 +00:00
q (
" create table sdp_3 (n int, x Enum( ' broken ' = 0, ' all ' = 1)) engine=MergeTree order by n partition by x "
)
2021-08-04 14:42:48 +00:00
for i in range ( 0 , 4 ) :
q ( " system stop merges sdp_ {} " . format ( i ) )
q ( " insert into sdp_ {} values (0, 0) " . format ( i ) )
q ( " insert into sdp_ {} values (1, 1) " . format ( i ) )
2022-03-22 16:39:58 +00:00
for p in q (
" select distinct partition_id from system.parts where table= ' sdp_ {} ' " . format (
i
)
) [ : - 1 ] . split ( " \n " ) :
2021-08-04 14:42:48 +00:00
q ( " alter table sdp_ {} detach partition id ' {} ' " . format ( i , p ) )
2022-03-22 16:39:58 +00:00
path_to_detached = path_to_data + " data/default/sdp_ {} /detached/ {} "
2021-08-04 14:42:48 +00:00
for i in range ( 0 , 4 ) :
2022-03-22 16:39:58 +00:00
instance . exec_in_container (
[ " mkdir " , path_to_detached . format ( i , " attaching_0_6_6_0 " ) ]
)
instance . exec_in_container (
[ " mkdir " , path_to_detached . format ( i , " deleting_0_7_7_0 " ) ]
)
instance . exec_in_container (
[ " mkdir " , path_to_detached . format ( i , " any_other_name " ) ]
)
instance . exec_in_container (
[ " mkdir " , path_to_detached . format ( i , " prefix_1_2_2_0_0 " ) ]
)
instance . exec_in_container (
[ " mkdir " , path_to_detached . format ( i , " ignored_202107_714380_714380_0 " ) ]
)
instance . exec_in_container (
[ " mkdir " , path_to_detached . format ( i , " broken_202107_714380_714380_123 " ) ]
)
instance . exec_in_container (
[ " mkdir " , path_to_detached . format ( i , " clone_all_714380_714380_42 " ) ]
)
instance . exec_in_container (
[ " mkdir " , path_to_detached . format ( i , " clone_all_714380_714380_42_123 " ) ]
)
instance . exec_in_container (
[
" mkdir " ,
path_to_detached . format (
i ,
" broken-on-start_6711e2b2592d86d18fc0f260cf33ef2b_714380_714380_42_123 " ,
) ,
]
)
res = q (
2022-10-18 06:53:21 +00:00
" select system.detached_parts.* except (bytes_on_disk, `path`) from system.detached_parts where table like ' sdp_ % ' order by table, name "
2022-03-22 16:39:58 +00:00
)
assert (
res == " default \t sdp_0 \t all \t all_1_1_0 \t default \t \t 1 \t 1 \t 0 \n "
" default \t sdp_0 \t all \t all_2_2_0 \t default \t \t 2 \t 2 \t 0 \n "
" default \t sdp_0 \t \\ N \t any_other_name \t default \t \\ N \t \\ N \t \\ N \t \\ N \n "
" default \t sdp_0 \t 0 \t attaching_0_6_6_0 \t default \t attaching \t 6 \t 6 \t 0 \n "
" default \t sdp_0 \t 6711e2b2592d86d18fc0f260cf33ef2b \t broken-on-start_6711e2b2592d86d18fc0f260cf33ef2b_714380_714380_42_123 \t default \t broken-on-start \t 714380 \t 714380 \t 42 \n "
" default \t sdp_0 \t 202107 \t broken_202107_714380_714380_123 \t default \t broken \t 714380 \t 714380 \t 123 \n "
" default \t sdp_0 \t all \t clone_all_714380_714380_42 \t default \t clone \t 714380 \t 714380 \t 42 \n "
" default \t sdp_0 \t all \t clone_all_714380_714380_42_123 \t default \t clone \t 714380 \t 714380 \t 42 \n "
" default \t sdp_0 \t 0 \t deleting_0_7_7_0 \t default \t deleting \t 7 \t 7 \t 0 \n "
" default \t sdp_0 \t 202107 \t ignored_202107_714380_714380_0 \t default \t ignored \t 714380 \t 714380 \t 0 \n "
" default \t sdp_0 \t 1 \t prefix_1_2_2_0_0 \t default \t prefix \t 2 \t 2 \t 0 \n "
" default \t sdp_1 \t 0 \t 0_1_1_0 \t default \t \t 1 \t 1 \t 0 \n "
" default \t sdp_1 \t 1 \t 1_2_2_0 \t default \t \t 2 \t 2 \t 0 \n "
" default \t sdp_1 \t \\ N \t any_other_name \t default \t \\ N \t \\ N \t \\ N \t \\ N \n "
" default \t sdp_1 \t 0 \t attaching_0_6_6_0 \t default \t attaching \t 6 \t 6 \t 0 \n "
" default \t sdp_1 \t 6711e2b2592d86d18fc0f260cf33ef2b \t broken-on-start_6711e2b2592d86d18fc0f260cf33ef2b_714380_714380_42_123 \t default \t broken-on-start \t 714380 \t 714380 \t 42 \n "
" default \t sdp_1 \t 202107 \t broken_202107_714380_714380_123 \t default \t broken \t 714380 \t 714380 \t 123 \n "
" default \t sdp_1 \t all \t clone_all_714380_714380_42 \t default \t clone \t 714380 \t 714380 \t 42 \n "
" default \t sdp_1 \t all \t clone_all_714380_714380_42_123 \t default \t clone \t 714380 \t 714380 \t 42 \n "
" default \t sdp_1 \t 0 \t deleting_0_7_7_0 \t default \t deleting \t 7 \t 7 \t 0 \n "
" default \t sdp_1 \t 202107 \t ignored_202107_714380_714380_0 \t default \t ignored \t 714380 \t 714380 \t 0 \n "
" default \t sdp_1 \t 1 \t prefix_1_2_2_0_0 \t default \t prefix \t 2 \t 2 \t 0 \n "
" default \t sdp_2 \t 58ed7160db50ea45e1c6aa694c8cbfd1 \t 58ed7160db50ea45e1c6aa694c8cbfd1_1_1_0 \t default \t \t 1 \t 1 \t 0 \n "
" default \t sdp_2 \t 6711e2b2592d86d18fc0f260cf33ef2b \t 6711e2b2592d86d18fc0f260cf33ef2b_2_2_0 \t default \t \t 2 \t 2 \t 0 \n "
" default \t sdp_2 \t \\ N \t any_other_name \t default \t \\ N \t \\ N \t \\ N \t \\ N \n "
" default \t sdp_2 \t 0 \t attaching_0_6_6_0 \t default \t attaching \t 6 \t 6 \t 0 \n "
" default \t sdp_2 \t 6711e2b2592d86d18fc0f260cf33ef2b \t broken-on-start_6711e2b2592d86d18fc0f260cf33ef2b_714380_714380_42_123 \t default \t broken-on-start \t 714380 \t 714380 \t 42 \n "
" default \t sdp_2 \t 202107 \t broken_202107_714380_714380_123 \t default \t broken \t 714380 \t 714380 \t 123 \n "
" default \t sdp_2 \t all \t clone_all_714380_714380_42 \t default \t clone \t 714380 \t 714380 \t 42 \n "
" default \t sdp_2 \t all \t clone_all_714380_714380_42_123 \t default \t clone \t 714380 \t 714380 \t 42 \n "
" default \t sdp_2 \t 0 \t deleting_0_7_7_0 \t default \t deleting \t 7 \t 7 \t 0 \n "
" default \t sdp_2 \t 202107 \t ignored_202107_714380_714380_0 \t default \t ignored \t 714380 \t 714380 \t 0 \n "
" default \t sdp_2 \t 1 \t prefix_1_2_2_0_0 \t default \t prefix \t 2 \t 2 \t 0 \n "
" default \t sdp_3 \t 0 \t 0_1_1_0 \t default \t \t 1 \t 1 \t 0 \n "
" default \t sdp_3 \t 1 \t 1_2_2_0 \t default \t \t 2 \t 2 \t 0 \n "
" default \t sdp_3 \t \\ N \t any_other_name \t default \t \\ N \t \\ N \t \\ N \t \\ N \n "
" default \t sdp_3 \t 0 \t attaching_0_6_6_0 \t default \t attaching \t 6 \t 6 \t 0 \n "
" default \t sdp_3 \t 6711e2b2592d86d18fc0f260cf33ef2b \t broken-on-start_6711e2b2592d86d18fc0f260cf33ef2b_714380_714380_42_123 \t default \t broken-on-start \t 714380 \t 714380 \t 42 \n "
" default \t sdp_3 \t 202107 \t broken_202107_714380_714380_123 \t default \t broken \t 714380 \t 714380 \t 123 \n "
" default \t sdp_3 \t all \t clone_all_714380_714380_42 \t default \t clone \t 714380 \t 714380 \t 42 \n "
" default \t sdp_3 \t all \t clone_all_714380_714380_42_123 \t default \t clone \t 714380 \t 714380 \t 42 \n "
" default \t sdp_3 \t 0 \t deleting_0_7_7_0 \t default \t deleting \t 7 \t 7 \t 0 \n "
" default \t sdp_3 \t 202107 \t ignored_202107_714380_714380_0 \t default \t ignored \t 714380 \t 714380 \t 0 \n "
2021-08-04 14:42:48 +00:00
" default \t sdp_3 \t 1 \t prefix_1_2_2_0_0 \t default \t prefix \t 2 \t 2 \t 0 \n "
2022-03-22 16:39:58 +00:00
)
2021-08-04 14:42:48 +00:00
for i in range ( 0 , 4 ) :
2022-03-22 16:39:58 +00:00
for p in q (
" select distinct partition_id from system.detached_parts where table= ' sdp_ {} ' and partition_id is not null " . format (
i
)
) [ : - 1 ] . split ( " \n " ) :
2021-08-04 14:42:48 +00:00
q ( " alter table sdp_ {} attach partition id ' {} ' " . format ( i , p ) )
2022-03-22 16:39:58 +00:00
assert (
q ( " select n, x, count() from merge( ' default ' , ' sdp_ ' ) group by n, x " )
== " 0 \t 0 \t 4 \n 1 \t 1 \t 4 \n "
)
2021-12-01 14:24:26 +00:00
def test_detached_part_dir_exists ( started_cluster ) :
2022-11-17 17:13:23 +00:00
q (
2023-01-17 11:02:18 +00:00
" create table detached_part_dir_exists (n int) engine=MergeTree order by n "
" SETTINGS compress_marks=false, compress_primary_key=false, old_parts_lifetime=0 "
2022-11-17 17:13:23 +00:00
)
2021-12-01 14:24:26 +00:00
q ( " insert into detached_part_dir_exists select 1 " ) # will create all_1_1_0
2022-03-22 16:39:58 +00:00
q (
" alter table detached_part_dir_exists detach partition id ' all ' "
2022-09-18 23:26:12 +00:00
) # will move all_1_1_0 to detached/all_1_1_0 and create all_1_1_1
wait_for_delete_empty_parts ( instance , " detached_part_dir_exists " )
2022-12-05 16:39:27 +00:00
wait_for_delete_inactive_parts ( instance , " detached_part_dir_exists " )
2022-09-18 23:26:12 +00:00
2021-12-01 14:24:26 +00:00
q ( " detach table detached_part_dir_exists " )
q ( " attach table detached_part_dir_exists " )
2022-12-05 16:39:27 +00:00
q ( " insert into detached_part_dir_exists select 1 " ) # will create all_1_1_0
2021-12-01 14:24:26 +00:00
q ( " insert into detached_part_dir_exists select 1 " ) # will create all_2_2_0
2022-12-05 16:39:27 +00:00
assert (
q (
" select name from system.parts where table= ' detached_part_dir_exists ' and active order by name "
)
== " all_1_1_0 \n all_2_2_0 \n "
)
2022-03-22 16:39:58 +00:00
instance . exec_in_container (
[
" bash " ,
" -c " ,
" mkdir /var/lib/clickhouse/data/default/detached_part_dir_exists/detached/all_2_2_0 " ,
] ,
privileged = True ,
)
instance . exec_in_container (
[
" bash " ,
" -c " ,
" touch /var/lib/clickhouse/data/default/detached_part_dir_exists/detached/all_2_2_0/file " ,
] ,
privileged = True ,
)
q (
" alter table detached_part_dir_exists detach partition id ' all ' "
) # directories already exist, but it's ok
assert (
q (
" select name from system.detached_parts where table= ' detached_part_dir_exists ' order by name "
)
== " all_1_1_0 \n all_1_1_0_try1 \n all_2_2_0 \n all_2_2_0_try1 \n "
)
2021-12-01 14:24:26 +00:00
q ( " drop table detached_part_dir_exists " )
2022-09-30 18:09:18 +00:00
def test_make_clone_in_detached ( started_cluster ) :
q (
2022-11-17 17:06:59 +00:00
" create table clone_in_detached (n int, m String) engine=ReplicatedMergeTree( ' /clone_in_detached ' , ' 1 ' ) order by n SETTINGS compress_marks=false, compress_primary_key=false "
2022-09-30 18:09:18 +00:00
)
path = path_to_data + " data/default/clone_in_detached/ "
# broken part already detached
2022-12-05 16:39:27 +00:00
q ( " insert into clone_in_detached values (42, ' ¯-_(ツ)_-¯ ' ) " )
2022-09-30 18:09:18 +00:00
instance . exec_in_container ( [ " rm " , path + " all_0_0_0/data.bin " ] )
instance . exec_in_container (
[ " cp " , " -r " , path + " all_0_0_0 " , path + " detached/broken_all_0_0_0 " ]
)
assert_eq_with_retry ( instance , " select * from clone_in_detached " , " \n " )
assert [ " broken_all_0_0_0 " , ] == sorted (
instance . exec_in_container ( [ " ls " , path + " detached/ " ] ) . strip ( ) . split ( " \n " )
)
# there's a directory with the same name, but different content
2022-12-05 16:39:27 +00:00
q ( " insert into clone_in_detached values (43, ' ¯-_(ツ)_-¯ ' ) " )
2022-09-30 18:09:18 +00:00
instance . exec_in_container ( [ " rm " , path + " all_1_1_0/data.bin " ] )
instance . exec_in_container (
[ " cp " , " -r " , path + " all_1_1_0 " , path + " detached/broken_all_1_1_0 " ]
)
instance . exec_in_container ( [ " rm " , path + " detached/broken_all_1_1_0/primary.idx " ] )
instance . exec_in_container (
[ " cp " , " -r " , path + " all_1_1_0 " , path + " detached/broken_all_1_1_0_try0 " ]
)
instance . exec_in_container (
[
" bash " ,
" -c " ,
" echo ' broken ' > {} " . format (
path + " detached/broken_all_1_1_0_try0/checksums.txt "
) ,
]
)
assert_eq_with_retry ( instance , " select * from clone_in_detached " , " \n " )
assert [
" broken_all_0_0_0 " ,
" broken_all_1_1_0 " ,
" broken_all_1_1_0_try0 " ,
" broken_all_1_1_0_try1 " ,
] == sorted (
instance . exec_in_container ( [ " ls " , path + " detached/ " ] ) . strip ( ) . split ( " \n " )
)
# there are directories with the same name, but different content, and part already detached
2022-12-05 16:39:27 +00:00
q ( " insert into clone_in_detached values (44, ' ¯-_(ツ)_-¯ ' ) " )
2022-09-30 18:09:18 +00:00
instance . exec_in_container ( [ " rm " , path + " all_2_2_0/data.bin " ] )
instance . exec_in_container (
[ " cp " , " -r " , path + " all_2_2_0 " , path + " detached/broken_all_2_2_0 " ]
)
instance . exec_in_container ( [ " rm " , path + " detached/broken_all_2_2_0/primary.idx " ] )
instance . exec_in_container (
[ " cp " , " -r " , path + " all_2_2_0 " , path + " detached/broken_all_2_2_0_try0 " ]
)
instance . exec_in_container (
[
" bash " ,
" -c " ,
" echo ' broken ' > {} " . format (
path + " detached/broken_all_2_2_0_try0/checksums.txt "
) ,
]
)
instance . exec_in_container (
[ " cp " , " -r " , path + " all_2_2_0 " , path + " detached/broken_all_2_2_0_try1 " ]
)
assert_eq_with_retry ( instance , " select * from clone_in_detached " , " \n " )
assert [
" broken_all_0_0_0 " ,
" broken_all_1_1_0 " ,
" broken_all_1_1_0_try0 " ,
" broken_all_1_1_0_try1 " ,
" broken_all_2_2_0 " ,
" broken_all_2_2_0_try0 " ,
" broken_all_2_2_0_try1 " ,
] == sorted (
instance . exec_in_container ( [ " ls " , path + " detached/ " ] ) . strip ( ) . split ( " \n " )
)