2021-08-18 22:19:14 +00:00
import pytest
2022-07-15 07:42:56 +00:00
import asyncio
2023-01-29 20:03:38 +00:00
import glob
2021-08-18 22:19:14 +00:00
import re
2022-12-02 14:05:46 +00:00
import random
2022-02-22 12:39:11 +00:00
import os . path
2023-11-14 20:31:49 +00:00
import sys
2023-01-29 20:03:38 +00:00
from collections import namedtuple
2021-08-18 22:19:14 +00:00
from helpers . cluster import ClickHouseCluster
2022-06-06 19:15:17 +00:00
from helpers . test_tools import assert_eq_with_retry , TSV
2021-08-18 22:19:14 +00:00
2023-01-29 20:03:38 +00:00
2023-11-14 20:31:49 +00:00
script_dir = os . path . dirname ( os . path . realpath ( __file__ ) )
2021-08-18 22:19:14 +00:00
cluster = ClickHouseCluster ( __file__ )
2022-03-22 16:39:58 +00:00
instance = cluster . add_instance (
2022-06-15 18:25:13 +00:00
" instance " ,
main_configs = [ " configs/backups_disk.xml " ] ,
2023-04-04 16:12:08 +00:00
user_configs = [ " configs/zookeeper_retries.xml " ] ,
2022-06-15 18:25:13 +00:00
external_dirs = [ " /backups/ " ] ,
2022-03-22 16:39:58 +00:00
)
2021-08-18 22:19:14 +00:00
2022-06-09 09:07:46 +00:00
def create_and_fill_table ( engine = " MergeTree " , n = 100 ) :
2021-10-26 09:48:31 +00:00
if engine == " MergeTree " :
engine = " MergeTree ORDER BY y PARTITION BY x % 10 "
2021-08-18 22:19:14 +00:00
instance . query ( " CREATE DATABASE test " )
2021-10-26 09:48:31 +00:00
instance . query ( f " CREATE TABLE test.table(x UInt32, y String) ENGINE= { engine } " )
2022-03-22 16:39:58 +00:00
instance . query (
2022-06-09 09:07:46 +00:00
f " INSERT INTO test.table SELECT number, toString(number) FROM numbers( { n } ) "
2022-03-22 16:39:58 +00:00
)
2021-08-18 22:19:14 +00:00
@pytest.fixture ( scope = " module " , autouse = True )
def start_cluster ( ) :
try :
cluster . start ( )
yield cluster
finally :
cluster . shutdown ( )
@pytest.fixture ( autouse = True )
def cleanup_after_test ( ) :
try :
yield
finally :
instance . query ( " DROP DATABASE IF EXISTS test " )
2022-06-09 09:07:46 +00:00
instance . query ( " DROP DATABASE IF EXISTS test2 " )
instance . query ( " DROP DATABASE IF EXISTS test3 " )
2022-06-18 22:01:08 +00:00
instance . query ( " DROP USER IF EXISTS u1 " )
instance . query ( " DROP ROLE IF EXISTS r1, r2 " )
2022-06-19 12:14:41 +00:00
instance . query ( " DROP SETTINGS PROFILE IF EXISTS prof1 " )
instance . query ( " DROP ROW POLICY IF EXISTS rowpol1 ON test.table " )
instance . query ( " DROP QUOTA IF EXISTS q1 " )
2021-08-18 22:19:14 +00:00
backup_id_counter = 0
2022-03-22 16:39:58 +00:00
2021-08-18 22:19:14 +00:00
def new_backup_name ( ) :
global backup_id_counter
backup_id_counter + = 1
2021-11-05 14:18:23 +00:00
return f " Disk( ' backups ' , ' { backup_id_counter } / ' ) "
2021-08-18 22:19:14 +00:00
2021-10-26 09:48:31 +00:00
2022-04-25 10:54:23 +00:00
def get_path_to_backup ( backup_name ) :
name = backup_name . split ( " , " ) [ 1 ] . strip ( " ' )/ " )
return os . path . join ( instance . cluster . instances_dir , " backups " , name )
2022-02-22 12:39:11 +00:00
2023-01-29 20:03:38 +00:00
def find_files_in_backup_folder ( backup_name ) :
path = get_path_to_backup ( backup_name )
files = [ f for f in glob . glob ( path + " /** " , recursive = True ) if os . path . isfile ( f ) ]
files + = [ f for f in glob . glob ( path + " /.** " , recursive = True ) if os . path . isfile ( f ) ]
return files
2022-06-09 16:19:54 +00:00
session_id_counter = 0
def new_session_id ( ) :
global session_id_counter
session_id_counter + = 1
return " Session # " + str ( session_id_counter )
2022-07-05 07:39:52 +00:00
def has_mutation_in_backup ( mutation_id , backup_name , database , table ) :
return os . path . exists (
os . path . join (
get_path_to_backup ( backup_name ) ,
f " data/ { database } / { table } /mutations/ { mutation_id } .txt " ,
)
)
2023-01-29 20:03:38 +00:00
BackupInfo = namedtuple (
" BackupInfo " ,
" name id status error num_files total_size num_entries uncompressed_size compressed_size files_read bytes_read " ,
)
def get_backup_info_from_system_backups ( by_id = None , by_name = None ) :
where_condition = " 1 "
if by_id :
where_condition = f " id = ' { by_id } ' "
elif by_name :
where_condition = f " name = ' { by_name } ' "
[
name ,
id ,
status ,
error ,
num_files ,
total_size ,
num_entries ,
uncompressed_size ,
compressed_size ,
files_read ,
bytes_read ,
] = (
instance . query (
f " SELECT name, id, status, error, num_files, total_size, num_entries, uncompressed_size, compressed_size, files_read, bytes_read "
f " FROM system.backups WHERE { where_condition } LIMIT 1 "
)
. strip ( " \n " )
. split ( " \t " )
)
num_files = int ( num_files )
total_size = int ( total_size )
num_entries = int ( num_entries )
uncompressed_size = int ( uncompressed_size )
compressed_size = int ( compressed_size )
files_read = int ( files_read )
bytes_read = int ( bytes_read )
return BackupInfo (
name = name ,
id = id ,
status = status ,
error = error ,
num_files = num_files ,
total_size = total_size ,
num_entries = num_entries ,
uncompressed_size = uncompressed_size ,
compressed_size = compressed_size ,
files_read = files_read ,
bytes_read = bytes_read ,
)
2022-03-22 16:39:58 +00:00
@pytest.mark.parametrize (
" engine " , [ " MergeTree " , " Log " , " TinyLog " , " StripeLog " , " Memory " ]
)
2021-10-26 09:48:31 +00:00
def test_restore_table ( engine ) :
2021-08-18 22:19:14 +00:00
backup_name = new_backup_name ( )
2021-10-26 09:48:31 +00:00
create_and_fill_table ( engine = engine )
2021-08-18 22:19:14 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
2021-08-18 22:19:14 +00:00
instance . query ( " DROP TABLE test.table " )
assert instance . query ( " EXISTS test.table " ) == " 0 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
2021-08-18 22:19:14 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-03-22 16:39:58 +00:00
@pytest.mark.parametrize (
" engine " , [ " MergeTree " , " Log " , " TinyLog " , " StripeLog " , " Memory " ]
)
2021-10-26 09:48:31 +00:00
def test_restore_table_into_existing_table ( engine ) :
2021-08-18 22:19:14 +00:00
backup_name = new_backup_name ( )
2021-10-26 09:48:31 +00:00
create_and_fill_table ( engine = engine )
2021-08-18 22:19:14 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
2021-08-18 22:19:14 +00:00
2022-05-24 12:07:33 +00:00
expected_error = " already contains some data "
2022-05-03 16:34:29 +00:00
assert expected_error in instance . query_and_get_error (
2022-05-07 15:14:43 +00:00
f " RESTORE TABLE test.table FROM { backup_name } "
2022-05-03 16:34:29 +00:00
)
2021-08-18 22:19:14 +00:00
2022-05-03 16:34:29 +00:00
instance . query (
2022-05-07 15:14:43 +00:00
f " RESTORE TABLE test.table FROM { backup_name } SETTINGS structure_only=true "
2022-05-03 16:34:29 +00:00
)
2022-05-02 22:01:11 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2021-08-18 22:19:14 +00:00
2022-05-03 16:34:29 +00:00
instance . query (
2022-05-07 15:14:43 +00:00
f " RESTORE TABLE test.table FROM { backup_name } SETTINGS allow_non_empty_tables=true "
2022-05-03 16:34:29 +00:00
)
2022-05-02 22:01:11 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 200 \t 9900 \n "
2022-05-03 16:34:29 +00:00
2021-08-18 22:19:14 +00:00
def test_restore_table_under_another_name ( ) :
backup_name = new_backup_name ( )
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
2021-08-18 22:19:14 +00:00
assert instance . query ( " EXISTS test.table2 " ) == " 0 \n "
2022-05-07 15:14:43 +00:00
instance . query ( f " RESTORE TABLE test.table AS test.table2 FROM { backup_name } " )
2021-08-18 22:19:14 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table2 " ) == " 100 \t 4950 \n "
def test_backup_table_under_another_name ( ) :
backup_name = new_backup_name ( )
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP TABLE test.table AS test.table2 TO { backup_name } " )
2021-08-18 22:19:14 +00:00
assert instance . query ( " EXISTS test.table2 " ) == " 0 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " RESTORE TABLE test.table2 FROM { backup_name } " )
2021-08-18 22:19:14 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table2 " ) == " 100 \t 4950 \n "
2022-06-22 22:56:41 +00:00
def test_materialized_view_select_1 ( ) :
2022-02-22 17:05:52 +00:00
backup_name = new_backup_name ( )
2022-03-22 16:39:58 +00:00
instance . query (
" CREATE MATERIALIZED VIEW mv_1(x UInt8) ENGINE=MergeTree ORDER BY tuple() POPULATE AS SELECT 1 AS x "
)
2022-02-22 17:05:52 +00:00
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP TABLE mv_1 TO { backup_name } " )
2022-02-22 17:05:52 +00:00
instance . query ( " DROP TABLE mv_1 " )
2022-04-26 16:33:19 +00:00
instance . query ( f " RESTORE TABLE mv_1 FROM { backup_name } " )
2022-02-22 17:05:52 +00:00
assert instance . query ( " SELECT * FROM mv_1 " ) == " 1 \n "
instance . query ( " DROP TABLE mv_1 " )
2021-08-18 22:19:14 +00:00
def test_incremental_backup ( ) :
backup_name = new_backup_name ( )
incremental_backup_name = new_backup_name ( )
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
2021-08-18 22:19:14 +00:00
instance . query ( " INSERT INTO test.table VALUES (65, ' a ' ), (66, ' b ' ) " )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 102 \t 5081 \n "
2022-03-22 16:39:58 +00:00
instance . query (
2022-04-26 16:33:19 +00:00
f " BACKUP TABLE test.table TO { incremental_backup_name } SETTINGS base_backup = { backup_name } "
2022-03-22 16:39:58 +00:00
)
2021-08-18 22:19:14 +00:00
2022-03-22 16:39:58 +00:00
instance . query (
2022-04-26 16:33:19 +00:00
f " RESTORE TABLE test.table AS test.table2 FROM { incremental_backup_name } "
2022-03-22 16:39:58 +00:00
)
2021-08-18 22:19:14 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table2 " ) == " 102 \t 5081 \n "
2022-10-23 19:50:30 +00:00
def test_increment_backup_without_changes ( ) :
backup_name = new_backup_name ( )
incremental_backup_name = new_backup_name ( )
2023-01-29 20:03:38 +00:00
create_and_fill_table ( n = 1 )
2022-10-23 19:50:30 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == TSV ( [ [ " 1 " , " 0 " ] ] )
# prepare first backup without base_backup
2023-01-29 20:03:38 +00:00
id_backup = instance . query ( f " BACKUP TABLE test.table TO { backup_name } " ) . split ( " \t " ) [
0
]
backup_info = get_backup_info_from_system_backups ( by_id = id_backup )
2022-10-23 19:50:30 +00:00
2023-01-29 20:03:38 +00:00
assert backup_info . status == " BACKUP_CREATED "
assert backup_info . error == " "
assert backup_info . num_files > 0
assert backup_info . total_size > 0
assert (
0 < backup_info . num_entries and backup_info . num_entries < = backup_info . num_files
2022-10-23 19:50:30 +00:00
)
2023-01-29 20:03:38 +00:00
assert backup_info . uncompressed_size > 0
assert backup_info . compressed_size == backup_info . uncompressed_size
2022-10-23 19:50:30 +00:00
# create second backup without changes based on the first one
2023-01-29 20:03:38 +00:00
id_backup2 = instance . query (
2022-10-23 19:50:30 +00:00
f " BACKUP TABLE test.table TO { incremental_backup_name } SETTINGS base_backup = { backup_name } "
2023-01-29 20:03:38 +00:00
) . split ( " \t " ) [ 0 ]
2022-10-23 19:50:30 +00:00
2023-01-29 20:03:38 +00:00
backup2_info = get_backup_info_from_system_backups ( by_id = id_backup2 )
2022-10-23 19:50:30 +00:00
2023-01-29 20:03:38 +00:00
assert backup2_info . status == " BACKUP_CREATED "
assert backup2_info . error == " "
assert backup2_info . num_files == backup_info . num_files
assert backup2_info . total_size == backup_info . total_size
assert backup2_info . num_entries == 0
assert backup2_info . uncompressed_size > 0
assert backup2_info . compressed_size == backup2_info . uncompressed_size
2022-10-23 19:50:30 +00:00
# restore the second backup
# we expect to see all files in the meta info of the restore and a sum of uncompressed and compressed sizes
2023-01-29 20:03:38 +00:00
id_restore = instance . query (
2022-10-23 19:50:30 +00:00
f " RESTORE TABLE test.table AS test.table2 FROM { incremental_backup_name } "
2023-01-29 20:03:38 +00:00
) . split ( " \t " ) [ 0 ]
2022-10-23 19:50:30 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table2 " ) == TSV (
[ [ " 1 " , " 0 " ] ]
)
2023-01-29 20:03:38 +00:00
restore_info = get_backup_info_from_system_backups ( by_id = id_restore )
2022-10-23 19:50:30 +00:00
2023-01-29 20:03:38 +00:00
assert restore_info . status == " RESTORED "
assert restore_info . error == " "
assert restore_info . num_files == backup2_info . num_files
assert restore_info . total_size == backup2_info . total_size
assert restore_info . num_entries == backup2_info . num_entries
assert restore_info . uncompressed_size == backup2_info . uncompressed_size
assert restore_info . compressed_size == backup2_info . compressed_size
assert restore_info . files_read == backup2_info . num_files
assert restore_info . bytes_read == backup2_info . total_size
2022-10-23 19:50:30 +00:00
2022-10-07 10:59:11 +00:00
def test_incremental_backup_overflow ( ) :
backup_name = new_backup_name ( )
incremental_backup_name = new_backup_name ( )
instance . query ( " CREATE DATABASE test " )
instance . query (
" CREATE TABLE test.table(y String CODEC(NONE)) ENGINE=MergeTree ORDER BY tuple() "
)
# Create a column of 4GB+10K
instance . query (
" INSERT INTO test.table SELECT toString(repeat( ' A ' , 1024)) FROM numbers((4*1024*1024)+10) "
)
# Force one part
instance . query ( " OPTIMIZE TABLE test.table FINAL " )
# ensure that the column's size on disk is indeed greater then 4GB
assert (
int (
instance . query (
" SELECT bytes_on_disk FROM system.parts_columns WHERE active AND database = ' test ' AND table = ' table ' AND column = ' y ' "
)
)
> 4 * 1024 * 1024 * 1024
)
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
instance . query (
f " BACKUP TABLE test.table TO { incremental_backup_name } SETTINGS base_backup = { backup_name } "
)
# And now check that incremental backup does not have any files
assert os . listdir ( os . path . join ( get_path_to_backup ( incremental_backup_name ) ) ) == [
" .backup "
]
2022-02-22 12:39:11 +00:00
def test_incremental_backup_after_renaming_table ( ) :
backup_name = new_backup_name ( )
incremental_backup_name = new_backup_name ( )
create_and_fill_table ( )
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
2022-02-22 12:39:11 +00:00
instance . query ( " RENAME TABLE test.table TO test.table2 " )
2022-03-22 16:39:58 +00:00
instance . query (
2022-04-26 16:33:19 +00:00
f " BACKUP TABLE test.table2 TO { incremental_backup_name } SETTINGS base_backup = { backup_name } "
2022-03-22 16:39:58 +00:00
)
2022-02-22 12:39:11 +00:00
# Files in a base backup can be searched by checksum, so an incremental backup with a renamed table actually
# contains only its changed metadata.
2022-04-19 09:02:34 +00:00
assert (
2022-04-25 10:54:23 +00:00
os . path . isdir ( os . path . join ( get_path_to_backup ( backup_name ) , " metadata " ) ) == True
)
assert os . path . isdir ( os . path . join ( get_path_to_backup ( backup_name ) , " data " ) ) == True
assert (
os . path . isdir (
os . path . join ( get_path_to_backup ( incremental_backup_name ) , " metadata " )
)
2022-04-19 09:02:34 +00:00
== True
)
assert (
2022-04-25 10:54:23 +00:00
os . path . isdir ( os . path . join ( get_path_to_backup ( incremental_backup_name ) , " data " ) )
2022-04-19 09:02:34 +00:00
== False
)
2022-02-22 12:39:11 +00:00
instance . query ( " DROP TABLE test.table2 " )
2022-04-26 16:33:19 +00:00
instance . query ( f " RESTORE TABLE test.table2 FROM { incremental_backup_name } " )
2022-02-22 12:39:11 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table2 " ) == " 100 \t 4950 \n "
2022-08-30 21:36:02 +00:00
def test_incremental_backup_for_log_family ( ) :
backup_name = new_backup_name ( )
create_and_fill_table ( engine = " Log " )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
instance . query ( " INSERT INTO test.table VALUES (65, ' a ' ), (66, ' b ' ) " )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 102 \t 5081 \n "
backup_name2 = new_backup_name ( )
instance . query ( f " BACKUP TABLE test.table TO { backup_name2 } " )
backup_name_inc = new_backup_name ( )
instance . query (
f " BACKUP TABLE test.table TO { backup_name_inc } SETTINGS base_backup = { backup_name } "
)
metadata_path = os . path . join (
get_path_to_backup ( backup_name ) , " metadata/test/table.sql "
)
metadata_path2 = os . path . join (
get_path_to_backup ( backup_name2 ) , " metadata/test/table.sql "
)
metadata_path_inc = os . path . join (
get_path_to_backup ( backup_name_inc ) , " metadata/test/table.sql "
)
assert os . path . isfile ( metadata_path )
assert os . path . isfile ( metadata_path2 )
assert not os . path . isfile ( metadata_path_inc )
assert os . path . getsize ( metadata_path ) > 0
assert os . path . getsize ( metadata_path ) == os . path . getsize ( metadata_path2 )
x_bin_path = os . path . join ( get_path_to_backup ( backup_name ) , " data/test/table/x.bin " )
y_bin_path = os . path . join ( get_path_to_backup ( backup_name ) , " data/test/table/y.bin " )
x_bin_path2 = os . path . join (
get_path_to_backup ( backup_name2 ) , " data/test/table/x.bin "
)
y_bin_path2 = os . path . join (
get_path_to_backup ( backup_name2 ) , " data/test/table/y.bin "
)
x_bin_path_inc = os . path . join (
get_path_to_backup ( backup_name_inc ) , " data/test/table/x.bin "
)
y_bin_path_inc = os . path . join (
get_path_to_backup ( backup_name_inc ) , " data/test/table/y.bin "
)
assert os . path . isfile ( x_bin_path )
assert os . path . isfile ( y_bin_path )
assert os . path . isfile ( x_bin_path2 )
assert os . path . isfile ( y_bin_path2 )
assert os . path . isfile ( x_bin_path_inc )
assert os . path . isfile ( y_bin_path_inc )
x_bin_size = os . path . getsize ( x_bin_path )
y_bin_size = os . path . getsize ( y_bin_path )
x_bin_size2 = os . path . getsize ( x_bin_path2 )
y_bin_size2 = os . path . getsize ( y_bin_path2 )
x_bin_size_inc = os . path . getsize ( x_bin_path_inc )
y_bin_size_inc = os . path . getsize ( y_bin_path_inc )
assert x_bin_size > 0
assert y_bin_size > 0
assert x_bin_size2 > 0
assert y_bin_size2 > 0
assert x_bin_size_inc > 0
assert y_bin_size_inc > 0
assert x_bin_size2 == x_bin_size + x_bin_size_inc
assert y_bin_size2 == y_bin_size + y_bin_size_inc
instance . query ( f " RESTORE TABLE test.table AS test.table2 FROM { backup_name_inc } " )
assert instance . query ( " SELECT count(), sum(x) FROM test.table2 " ) == " 102 \t 5081 \n "
2022-02-22 12:39:11 +00:00
2023-05-10 09:52:09 +00:00
def test_incremental_backup_append_table_def ( ) :
backup_name = new_backup_name ( )
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
instance . query ( " ALTER TABLE test.table MODIFY SETTING parts_to_throw_insert=100 " )
incremental_backup_name = new_backup_name ( )
instance . query (
f " BACKUP TABLE test.table TO { incremental_backup_name } SETTINGS base_backup = { backup_name } "
)
instance . query ( " DROP TABLE test.table " )
instance . query ( f " RESTORE TABLE test.table FROM { incremental_backup_name } " )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
assert " parts_to_throw_insert = 100 " in instance . query (
" SHOW CREATE TABLE test.table "
)
2021-08-18 22:19:14 +00:00
def test_backup_not_found_or_already_exists ( ) :
backup_name = new_backup_name ( )
expected_error = " Backup .* not found "
2022-03-22 16:39:58 +00:00
assert re . search (
expected_error ,
instance . query_and_get_error (
2022-04-26 16:33:19 +00:00
f " RESTORE TABLE test.table AS test.table2 FROM { backup_name } "
2022-03-22 16:39:58 +00:00
) ,
)
2021-08-18 22:19:14 +00:00
create_and_fill_table ( )
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
2021-08-18 22:19:14 +00:00
expected_error = " Backup .* already exists "
2022-03-22 16:39:58 +00:00
assert re . search (
expected_error ,
2022-04-26 16:33:19 +00:00
instance . query_and_get_error ( f " BACKUP TABLE test.table TO { backup_name } " ) ,
2022-03-22 16:39:58 +00:00
)
2021-11-05 14:18:23 +00:00
def test_file_engine ( ) :
2021-11-09 08:34:37 +00:00
backup_name = f " File( ' /backups/file/ ' ) "
2021-11-05 14:18:23 +00:00
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
2021-11-05 14:18:23 +00:00
instance . query ( " DROP TABLE test.table " )
assert instance . query ( " EXISTS test.table " ) == " 0 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
2021-11-05 14:18:23 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2021-11-09 08:33:28 +00:00
def test_database ( ) :
backup_name = new_backup_name ( )
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP DATABASE test TO { backup_name } " )
2022-08-21 16:18:35 +00:00
2021-11-09 08:33:28 +00:00
instance . query ( " DROP DATABASE test " )
2022-04-26 16:33:19 +00:00
instance . query ( f " RESTORE DATABASE test FROM { backup_name } " )
2021-11-09 08:33:28 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-01-28 17:33:35 +00:00
def test_zip_archive ( ) :
2022-04-25 10:54:23 +00:00
backup_name = f " Disk( ' backups ' , ' archive.zip ' ) "
2022-01-28 17:33:35 +00:00
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
2022-08-21 16:18:35 +00:00
2022-04-25 10:54:23 +00:00
assert os . path . isfile ( get_path_to_backup ( backup_name ) )
2022-01-28 17:33:35 +00:00
instance . query ( " DROP TABLE test.table " )
assert instance . query ( " EXISTS test.table " ) == " 0 \n "
2022-04-26 16:33:19 +00:00
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
2022-01-28 17:33:35 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
def test_zip_archive_with_settings ( ) :
2022-04-25 10:54:23 +00:00
backup_name = f " Disk( ' backups ' , ' archive_with_settings.zip ' ) "
2022-01-28 17:33:35 +00:00
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-03-22 16:39:58 +00:00
instance . query (
2022-04-26 16:33:19 +00:00
f " BACKUP TABLE test.table TO { backup_name } SETTINGS compression_method= ' lzma ' , compression_level=3, password= ' qwerty ' "
2022-03-22 16:39:58 +00:00
)
2022-01-28 17:33:35 +00:00
instance . query ( " DROP TABLE test.table " )
assert instance . query ( " EXISTS test.table " ) == " 0 \n "
2022-03-22 16:39:58 +00:00
instance . query (
2022-04-26 16:33:19 +00:00
f " RESTORE TABLE test.table FROM { backup_name } SETTINGS password= ' qwerty ' "
2022-03-22 16:39:58 +00:00
)
2022-01-28 17:33:35 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-05-12 17:42:05 +00:00
2023-01-29 03:35:00 +00:00
def test_zip_archive_with_bad_compression_method ( ) :
backup_name = f " Disk( ' backups ' , ' archive_with_bad_compression_method.zip ' ) "
2023-01-29 00:40:24 +00:00
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2023-01-29 03:35:00 +00:00
expected_error = " Unknown compression method specified for a zip archive "
2023-01-29 02:29:10 +00:00
assert expected_error in instance . query_and_get_error (
2023-01-29 03:35:00 +00:00
f " BACKUP TABLE test.table TO { backup_name } SETTINGS id= ' archive_with_bad_compression_method ' , compression_method= ' foobar ' "
2023-01-29 00:40:24 +00:00
)
2023-01-29 01:03:51 +00:00
assert (
instance . query (
2023-01-29 03:35:00 +00:00
" SELECT status FROM system.backups WHERE id= ' archive_with_bad_compression_method ' "
2023-01-29 01:03:51 +00:00
)
== " BACKUP_FAILED \n "
)
2023-01-29 00:40:24 +00:00
2024-01-19 15:05:36 +00:00
def test_tar_archive ( ) :
backup_name = f " Disk( ' backups ' , ' archive.tar ' ) "
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
assert os . path . isfile ( get_path_to_backup ( backup_name ) )
instance . query ( " DROP TABLE test.table " )
assert instance . query ( " EXISTS test.table " ) == " 0 \n "
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2024-02-20 23:34:02 +00:00
def test_tar_bz2_archive ( ) :
backup_name = f " Disk( ' backups ' , ' archive.tar.bz2 ' ) "
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
assert os . path . isfile ( get_path_to_backup ( backup_name ) )
instance . query ( " DROP TABLE test.table " )
assert instance . query ( " EXISTS test.table " ) == " 0 \n "
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
def test_tar_gz_archive ( ) :
backup_name = f " Disk( ' backups ' , ' archive.tar.gz ' ) "
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
assert os . path . isfile ( get_path_to_backup ( backup_name ) )
instance . query ( " DROP TABLE test.table " )
assert instance . query ( " EXISTS test.table " ) == " 0 \n "
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
def test_tar_lzma_archive ( ) :
backup_name = f " Disk( ' backups ' , ' archive.tar.lzma ' ) "
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
assert os . path . isfile ( get_path_to_backup ( backup_name ) )
instance . query ( " DROP TABLE test.table " )
assert instance . query ( " EXISTS test.table " ) == " 0 \n "
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2024-01-19 15:05:36 +00:00
def test_tar_archive_with_password ( ) :
backup_name = f " Disk( ' backups ' , ' archive_with_password.tar ' ) "
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2024-02-20 23:34:02 +00:00
expected_error = " Setting a password is not currently supported for libarchive "
2024-01-19 15:05:36 +00:00
assert expected_error in instance . query_and_get_error (
f " BACKUP TABLE test.table TO { backup_name } SETTINGS id= ' tar_archive_with_password ' , password= ' password123 ' "
)
assert (
instance . query (
" SELECT status FROM system.backups WHERE id= ' tar_archive_with_password ' "
)
== " BACKUP_FAILED \n "
)
def test_tar_archive_with_bad_compression_method ( ) :
backup_name = f " Disk( ' backups ' , ' archive_with_bad_compression_method.tar ' ) "
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2024-02-15 13:36:15 +00:00
expected_error = " Compressing tar archives is currently not supported "
2024-01-19 15:05:36 +00:00
assert expected_error in instance . query_and_get_error (
f " BACKUP TABLE test.table TO { backup_name } SETTINGS id= ' tar_archive_with_bad_compression_method ' , compression_method= ' foobar ' "
)
assert (
instance . query (
" SELECT status FROM system.backups WHERE id= ' tar_archive_with_bad_compression_method ' "
)
== " BACKUP_FAILED \n "
)
2024-02-14 13:49:00 +00:00
2022-05-12 17:42:05 +00:00
def test_async ( ) :
create_and_fill_table ( )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-05-15 08:22:37 +00:00
2022-05-12 17:42:05 +00:00
backup_name = new_backup_name ( )
2022-07-22 11:45:50 +00:00
[ id , status ] = instance . query (
2022-05-15 08:22:37 +00:00
f " BACKUP TABLE test.table TO { backup_name } ASYNC "
) . split ( " \t " )
2022-07-22 08:08:37 +00:00
2022-07-26 19:46:22 +00:00
assert status == " CREATING_BACKUP \n " or status == " BACKUP_CREATED \n "
2022-07-22 08:08:37 +00:00
2022-05-15 08:22:37 +00:00
assert_eq_with_retry (
instance ,
2022-07-22 16:01:24 +00:00
f " SELECT status, error FROM system.backups WHERE id= ' { id } ' " ,
2022-07-26 19:46:22 +00:00
TSV ( [ [ " BACKUP_CREATED " , " " ] ] ) ,
2022-05-15 08:22:37 +00:00
)
2022-05-12 17:42:05 +00:00
instance . query ( " DROP TABLE test.table " )
2022-07-22 11:45:50 +00:00
[ id , status ] = instance . query (
2022-05-15 08:22:37 +00:00
f " RESTORE TABLE test.table FROM { backup_name } ASYNC "
) . split ( " \t " )
2022-07-22 08:08:37 +00:00
2022-05-22 20:34:48 +00:00
assert status == " RESTORING \n " or status == " RESTORED \n "
2022-07-22 08:08:37 +00:00
2022-05-15 08:22:37 +00:00
assert_eq_with_retry (
2022-07-22 08:08:37 +00:00
instance ,
2022-07-22 16:01:24 +00:00
f " SELECT status, error FROM system.backups WHERE id= ' { id } ' " ,
2022-07-22 08:08:37 +00:00
TSV ( [ [ " RESTORED " , " " ] ] ) ,
2022-05-15 08:22:37 +00:00
)
2022-05-12 17:42:05 +00:00
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-06-06 19:15:17 +00:00
2022-07-15 07:42:56 +00:00
@pytest.mark.parametrize ( " interface " , [ " native " , " http " ] )
def test_async_backups_to_same_destination ( interface ) :
create_and_fill_table ( )
backup_name = new_backup_name ( )
2022-08-03 12:04:18 +00:00
# The first backup.
if interface == " http " :
res = instance . http_query ( f " BACKUP TABLE test.table TO { backup_name } ASYNC " )
else :
res = instance . query ( f " BACKUP TABLE test.table TO { backup_name } ASYNC " )
id1 = res . split ( " \t " ) [ 0 ]
# The second backup to the same destination.
if interface == " http " :
res , err = instance . http_query_and_get_answer_with_error (
f " BACKUP TABLE test.table TO { backup_name } ASYNC "
)
else :
res , err = instance . query_and_get_answer_with_error (
f " BACKUP TABLE test.table TO { backup_name } ASYNC "
)
2023-02-15 15:04:52 +00:00
# One of those two backups to the same destination is expected to fail.
# If the second backup is going to fail it can fail either immediately or after a while.
2022-08-03 12:04:18 +00:00
# If it fails immediately we won't even get its ID.
id2 = None if err else res . split ( " \t " ) [ 0 ]
2022-07-15 07:42:56 +00:00
2022-08-03 12:04:18 +00:00
ids = [ id1 ]
if id2 :
ids . append ( id2 )
ids_for_query = " [ " + " , " . join ( f " ' { id } ' " for id in ids ) + " ] "
2022-07-15 07:42:56 +00:00
assert_eq_with_retry (
instance ,
2022-08-03 12:04:18 +00:00
f " SELECT status FROM system.backups WHERE id IN { ids_for_query } AND status == ' CREATING_BACKUP ' " ,
2022-07-22 08:08:37 +00:00
" " ,
2022-07-15 07:42:56 +00:00
)
2023-02-25 22:29:51 +00:00
ids_succeeded = instance . query (
f " SELECT id FROM system.backups WHERE id IN { ids_for_query } AND status == ' BACKUP_CREATED ' "
) . splitlines ( )
2022-07-15 07:42:56 +00:00
2023-02-25 22:29:51 +00:00
ids_failed = instance . query (
f " SELECT id FROM system.backups WHERE id IN { ids_for_query } AND status == ' BACKUP_FAILED ' "
) . splitlines ( )
2023-02-15 15:04:52 +00:00
assert len ( ids_succeeded ) == 1
assert set ( ids_succeeded + ids_failed ) == set ( ids )
2022-07-15 07:42:56 +00:00
2022-08-03 12:04:18 +00:00
# Check that the first backup is all right.
2022-07-15 07:42:56 +00:00
instance . query ( " DROP TABLE test.table " )
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
2022-06-18 10:28:32 +00:00
def test_empty_files_in_backup ( ) :
instance . query ( " CREATE DATABASE test " )
2022-06-18 22:01:08 +00:00
instance . query (
" CREATE TABLE test.tbl1(x Array(UInt8)) ENGINE=MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0 "
)
2022-06-18 10:28:32 +00:00
instance . query ( " INSERT INTO test.tbl1 VALUES ([]) " )
backup_name = new_backup_name ( )
instance . query ( f " BACKUP TABLE test.tbl1 TO { backup_name } " )
instance . query ( " DROP TABLE test.tbl1 " )
instance . query ( f " RESTORE ALL FROM { backup_name } " )
assert instance . query ( " SELECT * FROM test.tbl1 " ) == " [] \n "
2022-06-06 19:15:17 +00:00
def test_dependencies ( ) :
create_and_fill_table ( )
instance . query ( " CREATE VIEW test.view AS SELECT x, y AS w FROM test.table " )
2022-06-09 09:07:46 +00:00
instance . query (
" CREATE DICTIONARY test.dict1(x UInt32, w String) PRIMARY KEY x SOURCE(CLICKHOUSE(HOST ' localhost ' PORT tcpPort() DB ' test ' TABLE ' view ' )) LAYOUT(FLAT()) LIFETIME(0) "
)
instance . query (
2023-04-12 22:32:19 +00:00
" CREATE DICTIONARY test.dict2(x UInt32, w String) PRIMARY KEY w SOURCE(CLICKHOUSE(HOST ' localhost ' PORT tcpPort() DB ' test ' TABLE ' dict1 ' )) LAYOUT(FLAT()) LIFETIME(0) "
2022-06-09 09:07:46 +00:00
)
instance . query (
" CREATE TABLE test.table2(k String, v Int32 DEFAULT dictGet( ' test.dict2 ' , ' x ' , k) - 1) ENGINE=MergeTree ORDER BY tuple() "
)
2022-06-06 19:15:17 +00:00
instance . query ( " INSERT INTO test.table2 (k) VALUES ( ' 7 ' ), ( ' 96 ' ), ( ' 124 ' ) " )
2022-06-09 09:07:46 +00:00
assert instance . query ( " SELECT * FROM test.table2 ORDER BY k " ) == TSV (
[ [ " 124 " , - 1 ] , [ " 7 " , 6 ] , [ " 96 " , 95 ] ]
)
2022-06-06 19:15:17 +00:00
backup_name = new_backup_name ( )
2022-06-08 02:11:41 +00:00
instance . query ( f " BACKUP DATABASE test AS test2 TO { backup_name } " )
2022-06-09 09:07:46 +00:00
2022-06-06 19:15:17 +00:00
instance . query ( " DROP DATABASE test " )
2022-06-08 02:11:41 +00:00
instance . query ( f " RESTORE DATABASE test2 AS test3 FROM { backup_name } " )
2022-06-06 19:15:17 +00:00
2022-06-09 09:07:46 +00:00
assert instance . query ( " SELECT * FROM test3.table2 ORDER BY k " ) == TSV (
[ [ " 124 " , - 1 ] , [ " 7 " , 6 ] , [ " 96 " , 95 ] ]
)
2022-06-08 02:11:41 +00:00
instance . query ( " INSERT INTO test3.table2 (k) VALUES ( ' 63 ' ), ( ' 152 ' ), ( ' 71 ' ) " )
2022-06-09 09:07:46 +00:00
assert instance . query ( " SELECT * FROM test3.table2 ORDER BY k " ) == TSV (
[ [ " 124 " , - 1 ] , [ " 152 " , - 1 ] , [ " 63 " , 62 ] , [ " 7 " , 6 ] , [ " 71 " , 70 ] , [ " 96 " , 95 ] ]
)
def test_materialized_view ( ) :
create_and_fill_table ( n = 5 )
instance . query (
" CREATE MATERIALIZED VIEW test.view ENGINE=MergeTree ORDER BY tuple() POPULATE AS SELECT y, x FROM test.table "
)
instance . query ( " INSERT INTO test.table VALUES (990, ' a ' ) " )
backup_name = new_backup_name ( )
instance . query ( f " BACKUP DATABASE test TO { backup_name } " )
2022-06-09 16:19:54 +00:00
assert sorted (
os . listdir ( os . path . join ( get_path_to_backup ( backup_name ) , " metadata/test " ) )
) == [ " table.sql " , " view.sql " ]
assert sorted (
os . listdir ( os . path . join ( get_path_to_backup ( backup_name ) , " data/test " ) )
) == [ " table " , " view " ]
view_create_query = open (
os . path . join ( get_path_to_backup ( backup_name ) , " metadata/test/view.sql " )
) . read ( )
assert view_create_query . startswith ( " CREATE MATERIALIZED VIEW test.view " )
assert " POPULATE " not in view_create_query
2022-06-09 09:07:46 +00:00
instance . query ( " DROP DATABASE test " )
instance . query ( f " RESTORE DATABASE test FROM { backup_name } " )
instance . query ( " INSERT INTO test.table VALUES (991, ' b ' ) " )
2022-06-09 16:19:54 +00:00
assert instance . query ( " SELECT * FROM test.view ORDER BY x " ) == TSV (
[ [ " 0 " , 0 ] , [ " 1 " , 1 ] , [ " 2 " , 2 ] , [ " 3 " , 3 ] , [ " 4 " , 4 ] , [ " a " , 990 ] , [ " b " , 991 ] ]
)
2022-06-09 09:07:46 +00:00
def test_materialized_view_with_target_table ( ) :
create_and_fill_table ( n = 5 )
instance . query (
" CREATE TABLE test.target(x Int64, y String) ENGINE=MergeTree ORDER BY tuple() "
)
instance . query (
" CREATE MATERIALIZED VIEW test.view TO test.target AS SELECT y, x FROM test.table "
)
instance . query ( " INSERT INTO test.table VALUES (990, ' a ' ) " )
backup_name = new_backup_name ( )
instance . query ( f " BACKUP DATABASE test TO { backup_name } " )
2022-06-09 16:19:54 +00:00
assert sorted (
os . listdir ( os . path . join ( get_path_to_backup ( backup_name ) , " metadata/test " ) )
) == [ " table.sql " , " target.sql " , " view.sql " ]
assert sorted (
os . listdir ( os . path . join ( get_path_to_backup ( backup_name ) , " data/test " ) )
) == [ " table " , " target " ]
2022-06-09 09:07:46 +00:00
instance . query ( " DROP DATABASE test " )
instance . query ( f " RESTORE DATABASE test FROM { backup_name } " )
instance . query ( " INSERT INTO test.table VALUES (991, ' b ' ) " )
2022-06-09 16:19:54 +00:00
assert instance . query ( " SELECT * FROM test.view ORDER BY x " ) == TSV (
[ [ " a " , 990 ] , [ " b " , 991 ] ]
)
def test_temporary_table ( ) :
session_id = new_session_id ( )
instance . http_query (
" CREATE TEMPORARY TABLE temp_tbl(s String) " , params = { " session_id " : session_id }
)
instance . http_query (
" INSERT INTO temp_tbl VALUES ( ' q ' ) " , params = { " session_id " : session_id }
)
instance . http_query (
" INSERT INTO temp_tbl VALUES ( ' w ' ), ( ' e ' ) " , params = { " session_id " : session_id }
)
backup_name = new_backup_name ( )
instance . http_query (
f " BACKUP TEMPORARY TABLE temp_tbl TO { backup_name } " ,
params = { " session_id " : session_id } ,
)
session_id = new_session_id ( )
instance . http_query (
f " RESTORE TEMPORARY TABLE temp_tbl FROM { backup_name } " ,
params = { " session_id " : session_id } ,
)
assert instance . http_query (
" SELECT * FROM temp_tbl ORDER BY s " , params = { " session_id " : session_id }
) == TSV ( [ [ " e " ] , [ " q " ] , [ " w " ] ] )
2022-06-22 22:56:41 +00:00
# The backup created by "BACKUP DATABASE _temporary_and_external_tables" must not contain tables from other sessions.
def test_temporary_database ( ) :
2022-06-09 16:19:54 +00:00
session_id = new_session_id ( )
instance . http_query (
" CREATE TEMPORARY TABLE temp_tbl(s String) " , params = { " session_id " : session_id }
)
2022-06-22 22:56:41 +00:00
other_session_id = new_session_id ( )
instance . http_query (
" CREATE TEMPORARY TABLE other_temp_tbl(s String) " ,
params = { " session_id " : other_session_id } ,
)
2022-06-09 16:19:54 +00:00
backup_name = new_backup_name ( )
2022-06-22 22:56:41 +00:00
instance . http_query (
f " BACKUP DATABASE _temporary_and_external_tables TO { backup_name } " ,
params = { " session_id " : session_id } ,
)
assert os . listdir (
os . path . join ( get_path_to_backup ( backup_name ) , " temporary_tables/metadata " )
) == [ " temp_tbl.sql " ]
2022-06-09 16:19:54 +00:00
2022-06-22 22:56:41 +00:00
assert sorted ( os . listdir ( get_path_to_backup ( backup_name ) ) ) == [
" .backup " ,
" temporary_tables " ,
2022-06-09 16:19:54 +00:00
]
2022-06-18 11:56:04 +00:00
def test_restore_all_restores_temporary_tables ( ) :
session_id = new_session_id ( )
instance . http_query (
" CREATE TEMPORARY TABLE temp_tbl(s String) " , params = { " session_id " : session_id }
)
instance . http_query (
2022-06-18 22:01:08 +00:00
" INSERT INTO temp_tbl VALUES ( ' q ' ), ( ' w ' ), ( ' e ' ) " ,
params = { " session_id " : session_id } ,
2022-06-18 11:56:04 +00:00
)
backup_name = new_backup_name ( )
instance . http_query (
f " BACKUP TEMPORARY TABLE temp_tbl TO { backup_name } " ,
params = { " session_id " : session_id } ,
)
session_id = new_session_id ( )
instance . http_query (
f " RESTORE ALL FROM { backup_name } " ,
params = { " session_id " : session_id } ,
2022-06-18 22:01:08 +00:00
method = " POST " ,
2022-06-18 11:56:04 +00:00
)
assert instance . http_query (
" SELECT * FROM temp_tbl ORDER BY s " , params = { " session_id " : session_id }
2022-06-18 22:01:08 +00:00
) == TSV ( [ [ " e " ] , [ " q " ] , [ " w " ] ] )
def test_required_privileges ( ) :
create_and_fill_table ( n = 5 )
instance . query ( " CREATE USER u1 " )
backup_name = new_backup_name ( )
2024-02-21 18:32:06 +00:00
expected_error = " necessary to have the grant BACKUP ON test.`table` "
2022-06-18 22:01:08 +00:00
assert expected_error in instance . query_and_get_error (
f " BACKUP TABLE test.table TO { backup_name } " , user = " u1 "
)
instance . query ( " GRANT BACKUP ON test.table TO u1 " )
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " , user = " u1 " )
2024-02-22 09:06:38 +00:00
expected_error = " necessary to have the grant INSERT, CREATE TABLE ON test.`table` "
2022-06-18 22:01:08 +00:00
assert expected_error in instance . query_and_get_error (
f " RESTORE TABLE test.table FROM { backup_name } " , user = " u1 "
)
2023-08-06 12:48:20 +00:00
expected_error = " necessary to have the grant INSERT, CREATE TABLE ON test.table2 "
2022-06-18 22:01:08 +00:00
assert expected_error in instance . query_and_get_error (
f " RESTORE TABLE test.table AS test.table2 FROM { backup_name } " , user = " u1 "
)
instance . query ( " GRANT INSERT, CREATE ON test.table2 TO u1 " )
instance . query (
f " RESTORE TABLE test.table AS test.table2 FROM { backup_name } " , user = " u1 "
)
instance . query ( " DROP TABLE test.table " )
2024-02-22 09:06:38 +00:00
expected_error = " necessary to have the grant INSERT, CREATE TABLE ON test.`table` "
2022-06-18 22:01:08 +00:00
assert expected_error in instance . query_and_get_error (
f " RESTORE ALL FROM { backup_name } " , user = " u1 "
)
instance . query ( " GRANT INSERT, CREATE ON test.table TO u1 " )
instance . query ( f " RESTORE ALL FROM { backup_name } " , user = " u1 " )
2022-06-18 11:56:04 +00:00
2022-06-09 16:19:54 +00:00
def test_system_table ( ) :
backup_name = new_backup_name ( )
instance . query ( f " BACKUP TABLE system.numbers TO { backup_name } " )
assert os . listdir (
os . path . join ( get_path_to_backup ( backup_name ) , " metadata/system " )
) == [ " numbers.sql " ]
assert not os . path . isdir ( os . path . join ( get_path_to_backup ( backup_name ) , " data " ) )
create_query = open (
os . path . join ( get_path_to_backup ( backup_name ) , " metadata/system/numbers.sql " )
) . read ( )
2022-06-14 08:53:22 +00:00
assert create_query == " CREATE TABLE system.numbers ENGINE = SystemNumbers "
2022-06-09 16:19:54 +00:00
instance . query ( f " RESTORE TABLE system.numbers FROM { backup_name } " )
def test_system_database ( ) :
backup_name = new_backup_name ( )
instance . query ( f " BACKUP DATABASE system TO { backup_name } " )
assert " numbers.sql " in os . listdir (
os . path . join ( get_path_to_backup ( backup_name ) , " metadata/system " )
)
create_query = open (
os . path . join ( get_path_to_backup ( backup_name ) , " metadata/system/numbers.sql " )
) . read ( )
2022-06-14 08:53:22 +00:00
assert create_query == " CREATE TABLE system.numbers ENGINE = SystemNumbers "
2022-06-15 18:25:13 +00:00
def test_system_users ( ) :
2022-06-18 22:01:08 +00:00
instance . query (
" CREATE USER u1 IDENTIFIED BY ' qwe123 ' SETTINGS PROFILE ' default ' , custom_a = 1 "
)
2022-06-15 18:25:13 +00:00
instance . query ( " GRANT SELECT ON test.* TO u1 " )
2022-06-19 12:14:41 +00:00
2022-06-15 18:25:13 +00:00
instance . query ( " CREATE ROLE r1, r2 " )
instance . query ( " GRANT r1 TO r2 WITH ADMIN OPTION " )
instance . query ( " GRANT r2 TO u1 " )
2024-02-26 23:54:20 +00:00
instance . query ( " CREATE SETTINGS PROFILE `prof1` SETTINGS custom_b=2 TO u1 " )
2022-06-19 12:14:41 +00:00
instance . query ( " CREATE ROW POLICY rowpol1 ON test.table USING x<50 TO u1 " )
instance . query ( " CREATE QUOTA q1 TO r1 " )
2022-06-15 18:25:13 +00:00
backup_name = new_backup_name ( )
2022-06-19 12:14:41 +00:00
instance . query (
f " BACKUP TABLE system.users, TABLE system.roles, TABLE system.settings_profiles, TABLE system.row_policies, TABLE system.quotas TO { backup_name } "
)
2022-06-15 18:25:13 +00:00
instance . query ( " DROP USER u1 " )
instance . query ( " DROP ROLE r1, r2 " )
2022-06-19 12:14:41 +00:00
instance . query ( " DROP SETTINGS PROFILE prof1 " )
instance . query ( " DROP ROW POLICY rowpol1 ON test.table " )
instance . query ( " DROP QUOTA q1 " )
2022-06-15 18:25:13 +00:00
2022-06-19 12:14:41 +00:00
instance . query (
f " RESTORE TABLE system.users, TABLE system.roles, TABLE system.settings_profiles, TABLE system.row_policies, TABLE system.quotas FROM { backup_name } "
)
2022-06-15 18:25:13 +00:00
2022-06-18 22:01:08 +00:00
assert (
instance . query ( " SHOW CREATE USER u1 " )
2024-02-22 09:06:38 +00:00
== " CREATE USER u1 IDENTIFIED WITH sha256_password SETTINGS PROFILE `default`, custom_a = 1 \n "
2022-06-18 22:01:08 +00:00
)
assert instance . query ( " SHOW GRANTS FOR u1 " ) == TSV (
[ " GRANT SELECT ON test.* TO u1 " , " GRANT r2 TO u1 " ]
)
2022-06-15 18:25:13 +00:00
assert instance . query ( " SHOW CREATE ROLE r1 " ) == " CREATE ROLE r1 \n "
assert instance . query ( " SHOW GRANTS FOR r1 " ) == " "
assert instance . query ( " SHOW CREATE ROLE r2 " ) == " CREATE ROLE r2 \n "
2022-06-18 22:01:08 +00:00
assert instance . query ( " SHOW GRANTS FOR r2 " ) == TSV (
[ " GRANT r1 TO r2 WITH ADMIN OPTION " ]
)
2022-06-19 12:14:41 +00:00
assert (
instance . query ( " SHOW CREATE SETTINGS PROFILE prof1 " )
2024-02-26 23:54:20 +00:00
== " CREATE SETTINGS PROFILE `prof1` SETTINGS custom_b = 2 TO u1 \n "
2022-06-19 12:14:41 +00:00
)
assert (
instance . query ( " SHOW CREATE ROW POLICY rowpol1 " )
2024-02-21 18:36:07 +00:00
== " CREATE ROW POLICY rowpol1 ON test.`table` FOR SELECT USING x < 50 TO u1 \n "
2022-06-19 12:14:41 +00:00
)
assert instance . query ( " SHOW CREATE QUOTA q1 " ) == " CREATE QUOTA q1 TO r1 \n "
2022-06-18 22:01:08 +00:00
def test_system_users_required_privileges ( ) :
instance . query ( " CREATE ROLE r1 " )
instance . query ( " CREATE USER u1 DEFAULT ROLE r1 " )
instance . query ( " GRANT SELECT ON test.* TO u1 " )
2022-06-19 10:49:50 +00:00
# SETTINGS allow_backup=false means the following user won't be included in backups.
instance . query ( " CREATE USER u2 SETTINGS allow_backup=false " )
2022-06-18 22:01:08 +00:00
backup_name = new_backup_name ( )
2023-08-06 12:48:20 +00:00
expected_error = " necessary to have the grant BACKUP ON system.users "
2022-06-18 22:01:08 +00:00
assert expected_error in instance . query_and_get_error (
f " BACKUP TABLE system.users, TABLE system.roles TO { backup_name } " , user = " u2 "
)
instance . query ( " GRANT BACKUP ON system.users TO u2 " )
2023-08-06 12:48:20 +00:00
expected_error = " necessary to have the grant BACKUP ON system.roles "
2022-06-18 22:01:08 +00:00
assert expected_error in instance . query_and_get_error (
f " BACKUP TABLE system.users, TABLE system.roles TO { backup_name } " , user = " u2 "
)
instance . query ( " GRANT BACKUP ON system.roles TO u2 " )
instance . query (
f " BACKUP TABLE system.users, TABLE system.roles TO { backup_name } " , user = " u2 "
)
instance . query ( " DROP USER u1 " )
instance . query ( " DROP ROLE r1 " )
2022-06-19 10:49:50 +00:00
expected_error = (
2023-08-06 12:48:20 +00:00
" necessary to have the grant CREATE USER, CREATE ROLE, ROLE ADMIN ON *.* "
2022-06-19 10:49:50 +00:00
)
assert expected_error in instance . query_and_get_error (
f " RESTORE ALL FROM { backup_name } " , user = " u2 "
)
2022-06-18 22:01:08 +00:00
2022-06-19 10:49:50 +00:00
instance . query ( " GRANT CREATE USER, CREATE ROLE, ROLE ADMIN ON *.* TO u2 " )
2022-06-18 22:01:08 +00:00
2023-08-06 12:48:20 +00:00
expected_error = " necessary to have the grant SELECT ON test.* WITH GRANT OPTION "
2022-06-19 10:49:50 +00:00
assert expected_error in instance . query_and_get_error (
f " RESTORE ALL FROM { backup_name } " , user = " u2 "
2022-06-18 22:01:08 +00:00
)
2022-06-19 10:49:50 +00:00
instance . query ( " GRANT SELECT ON test.* TO u2 WITH GRANT OPTION " )
instance . query ( f " RESTORE ALL FROM { backup_name } " , user = " u2 " )
assert instance . query ( " SHOW CREATE USER u1 " ) == " CREATE USER u1 DEFAULT ROLE r1 \n "
2022-06-18 22:01:08 +00:00
assert instance . query ( " SHOW GRANTS FOR u1 " ) == TSV (
2022-06-19 10:49:50 +00:00
[ " GRANT SELECT ON test.* TO u1 " , " GRANT r1 TO u1 " ]
2022-06-18 22:01:08 +00:00
)
2022-06-19 10:49:50 +00:00
2022-06-18 22:01:08 +00:00
assert instance . query ( " SHOW CREATE ROLE r1 " ) == " CREATE ROLE r1 \n "
assert instance . query ( " SHOW GRANTS FOR r1 " ) == " "
2022-06-21 12:59:43 +00:00
def test_system_users_async ( ) :
instance . query ( " CREATE USER u1 IDENTIFIED BY ' qwe123 ' SETTINGS custom_c = 3 " )
backup_name = new_backup_name ( )
2022-07-22 08:08:37 +00:00
id = instance . query (
2022-06-21 12:59:43 +00:00
f " BACKUP DATABASE default, TABLE system.users, TABLE system.roles, TABLE system.settings_profiles, TABLE system.row_policies, TABLE system.quotas TO { backup_name } ASYNC "
2022-07-22 08:08:37 +00:00
) . split ( " \t " ) [ 0 ]
2022-06-21 12:59:43 +00:00
assert_eq_with_retry (
instance ,
2022-07-22 16:01:24 +00:00
f " SELECT status, error FROM system.backups WHERE id= ' { id } ' " ,
2022-07-26 19:46:22 +00:00
TSV ( [ [ " BACKUP_CREATED " , " " ] ] ) ,
2022-06-21 12:59:43 +00:00
)
instance . query ( " DROP USER u1 " )
2022-07-22 08:08:37 +00:00
id = instance . query (
2022-06-21 12:59:43 +00:00
f " RESTORE DATABASE default, TABLE system.users, TABLE system.roles, TABLE system.settings_profiles, TABLE system.row_policies, TABLE system.quotas FROM { backup_name } ASYNC "
2022-07-22 08:08:37 +00:00
) . split ( " \t " ) [ 0 ]
2022-06-21 12:59:43 +00:00
assert_eq_with_retry (
instance ,
2022-07-22 16:01:24 +00:00
f " SELECT status, error FROM system.backups WHERE id= ' { id } ' " ,
2022-07-22 08:08:37 +00:00
TSV ( [ [ " RESTORED " , " " ] ] ) ,
2022-06-21 12:59:43 +00:00
)
assert (
instance . query ( " SHOW CREATE USER u1 " )
== " CREATE USER u1 IDENTIFIED WITH sha256_password SETTINGS custom_c = 3 \n "
)
2022-06-22 20:30:50 +00:00
def test_projection ( ) :
create_and_fill_table ( n = 3 )
instance . query ( " ALTER TABLE test.table ADD PROJECTION prjmax (SELECT MAX(x)) " )
instance . query ( f " INSERT INTO test.table VALUES (100, ' a ' ), (101, ' b ' ) " )
assert (
instance . query (
" SELECT count() FROM system.projection_parts WHERE database= ' test ' AND table= ' table ' AND name= ' prjmax ' "
)
== " 2 \n "
)
backup_name = new_backup_name ( )
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
assert os . path . exists (
os . path . join (
get_path_to_backup ( backup_name ) , " data/test/table/1_5_5_0/data.bin "
)
)
assert os . path . exists (
os . path . join (
get_path_to_backup ( backup_name ) ,
" data/test/table/1_5_5_0/prjmax.proj/data.bin " ,
)
)
instance . query ( " DROP TABLE test.table " )
assert (
instance . query (
" SELECT count() FROM system.projection_parts WHERE database= ' test ' AND table= ' table ' AND name= ' prjmax ' "
)
== " 0 \n "
)
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
assert instance . query ( " SELECT * FROM test.table ORDER BY x " ) == TSV (
[ [ 0 , " 0 " ] , [ 1 , " 1 " ] , [ 2 , " 2 " ] , [ 100 , " a " ] , [ 101 , " b " ] ]
)
assert (
instance . query (
" SELECT count() FROM system.projection_parts WHERE database= ' test ' AND table= ' table ' AND name= ' prjmax ' "
)
== " 2 \n "
)
2022-06-27 13:39:12 +00:00
def test_system_functions ( ) :
instance . query ( " CREATE FUNCTION linear_equation AS (x, k, b) -> k*x + b; " )
instance . query ( " CREATE FUNCTION parity_str AS (n) -> if(n % 2, ' odd ' , ' even ' ); " )
backup_name = new_backup_name ( )
instance . query ( f " BACKUP TABLE system.functions TO { backup_name } " )
instance . query ( " DROP FUNCTION linear_equation " )
instance . query ( " DROP FUNCTION parity_str " )
instance . query ( f " RESTORE TABLE system.functions FROM { backup_name } " )
assert instance . query (
" SELECT number, linear_equation(number, 2, 1) FROM numbers(3) "
) == TSV ( [ [ 0 , 1 ] , [ 1 , 3 ] , [ 2 , 5 ] ] )
2022-06-30 08:10:12 +00:00
2022-06-27 13:39:12 +00:00
assert instance . query ( " SELECT number, parity_str(number) FROM numbers(3) " ) == TSV (
[ [ 0 , " even " ] , [ 1 , " odd " ] , [ 2 , " even " ] ]
)
2022-06-29 12:42:23 +00:00
def test_backup_partition ( ) :
create_and_fill_table ( n = 30 )
backup_name = new_backup_name ( )
instance . query ( f " BACKUP TABLE test.table PARTITIONS ' 1 ' , ' 4 ' TO { backup_name } " )
instance . query ( " DROP TABLE test.table " )
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
2022-06-30 08:10:12 +00:00
assert instance . query ( " SELECT * FROM test.table ORDER BY x " ) == TSV (
[ [ 1 , " 1 " ] , [ 4 , " 4 " ] , [ 11 , " 11 " ] , [ 14 , " 14 " ] , [ 21 , " 21 " ] , [ 24 , " 24 " ] ]
)
2022-06-29 12:42:23 +00:00
def test_restore_partition ( ) :
create_and_fill_table ( n = 30 )
backup_name = new_backup_name ( )
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
instance . query ( " DROP TABLE test.table " )
instance . query ( f " RESTORE TABLE test.table PARTITIONS ' 2 ' , ' 3 ' FROM { backup_name } " )
2022-06-30 08:10:12 +00:00
assert instance . query ( " SELECT * FROM test.table ORDER BY x " ) == TSV (
[ [ 2 , " 2 " ] , [ 3 , " 3 " ] , [ 12 , " 12 " ] , [ 13 , " 13 " ] , [ 22 , " 22 " ] , [ 23 , " 23 " ] ]
)
2022-07-05 07:39:52 +00:00
2023-04-01 17:46:53 +00:00
@pytest.mark.parametrize ( " exclude_system_log_tables " , [ False , True ] )
def test_backup_all ( exclude_system_log_tables ) :
2023-03-29 22:11:13 +00:00
create_and_fill_table ( )
session_id = new_session_id ( )
instance . http_query (
" CREATE TEMPORARY TABLE temp_tbl(s String) " , params = { " session_id " : session_id }
)
instance . http_query (
" INSERT INTO temp_tbl VALUES ( ' q ' ), ( ' w ' ), ( ' e ' ) " ,
params = { " session_id " : session_id } ,
)
instance . query ( " CREATE FUNCTION two_and_half AS (x) -> x * 2.5 " )
instance . query ( " CREATE USER u1 IDENTIFIED BY ' qwe123 ' SETTINGS custom_a = 1 " )
backup_name = new_backup_name ( )
2023-04-01 17:46:53 +00:00
exclude_from_backup = [ ]
if exclude_system_log_tables :
2023-04-14 15:51:20 +00:00
# See the list of log tables in src/Interpreters/SystemLog.cpp
log_tables = [
" query_log " ,
" query_thread_log " ,
" part_log " ,
" trace_log " ,
" crash_log " ,
" text_log " ,
" metric_log " ,
" filesystem_cache_log " ,
" filesystem_read_prefetches_log " ,
" asynchronous_metric_log " ,
" opentelemetry_span_log " ,
" query_views_log " ,
" zookeeper_log " ,
" session_log " ,
" transactions_info_log " ,
" processors_profile_log " ,
" asynchronous_insert_log " ,
2023-09-03 12:04:23 +00:00
" backup_log " ,
2023-04-14 15:51:20 +00:00
]
exclude_from_backup + = [ " system. " + table_name for table_name in log_tables ]
2023-04-01 17:46:53 +00:00
backup_command = f " BACKUP ALL { ' EXCEPT TABLES ' + ' , ' . join ( exclude_from_backup ) if exclude_from_backup else ' ' } TO { backup_name } "
instance . http_query ( backup_command , params = { " session_id " : session_id } )
2023-03-29 22:11:13 +00:00
instance . query ( " DROP TABLE test.table " )
instance . query ( " DROP FUNCTION two_and_half " )
instance . query ( " DROP USER u1 " )
2023-04-01 17:46:53 +00:00
restore_settings = [ ]
if not exclude_system_log_tables :
restore_settings . append ( " allow_non_empty_tables=true " )
restore_command = f " RESTORE ALL FROM { backup_name } { ' SETTINGS ' + ' , ' . join ( restore_settings ) if restore_settings else ' ' } "
2023-03-29 22:11:13 +00:00
session_id = new_session_id ( )
instance . http_query (
2023-04-01 17:46:53 +00:00
restore_command , params = { " session_id " : session_id } , method = " POST "
2023-03-29 22:11:13 +00:00
)
assert instance . query ( " SELECT count(), sum(x) FROM test.table " ) == " 100 \t 4950 \n "
assert instance . http_query (
" SELECT * FROM temp_tbl ORDER BY s " , params = { " session_id " : session_id }
) == TSV ( [ [ " e " ] , [ " q " ] , [ " w " ] ] )
assert instance . query ( " SELECT two_and_half(6) " ) == " 15 \n "
assert (
instance . query ( " SHOW CREATE USER u1 " )
== " CREATE USER u1 IDENTIFIED WITH sha256_password SETTINGS custom_a = 1 \n "
)
2023-04-01 17:46:53 +00:00
instance . query ( " DROP TABLE test.table " )
instance . query ( " DROP FUNCTION two_and_half " )
instance . query ( " DROP USER u1 " )
2023-03-29 22:11:13 +00:00
2022-07-22 16:01:24 +00:00
def test_operation_id ( ) :
create_and_fill_table ( n = 30 )
backup_name = new_backup_name ( )
2022-07-26 20:20:19 +00:00
2022-07-22 16:01:24 +00:00
[ id , status ] = instance . query (
f " BACKUP TABLE test.table TO { backup_name } SETTINGS id= ' first ' ASYNC "
) . split ( " \t " )
2022-07-26 20:20:19 +00:00
2022-07-22 16:01:24 +00:00
assert id == " first "
2022-07-26 19:46:22 +00:00
assert status == " CREATING_BACKUP \n " or status == " BACKUP_CREATED \n "
2022-07-22 16:01:24 +00:00
assert_eq_with_retry (
instance ,
f " SELECT status, error FROM system.backups WHERE id= ' first ' " ,
2022-07-26 19:46:22 +00:00
TSV ( [ [ " BACKUP_CREATED " , " " ] ] ) ,
2022-07-22 16:01:24 +00:00
)
instance . query ( " DROP TABLE test.table " )
[ id , status ] = instance . query (
f " RESTORE TABLE test.table FROM { backup_name } SETTINGS id= ' second ' ASYNC "
) . split ( " \t " )
assert id == " second "
assert status == " RESTORING \n " or status == " RESTORED \n "
assert_eq_with_retry (
instance ,
f " SELECT status, error FROM system.backups WHERE id= ' second ' " ,
TSV ( [ [ " RESTORED " , " " ] ] ) ,
)
# Reuse the same ID again
instance . query ( " DROP TABLE test.table " )
[ id , status ] = instance . query (
f " RESTORE TABLE test.table FROM { backup_name } SETTINGS id= ' first ' "
) . split ( " \t " )
2022-07-26 20:20:19 +00:00
2022-07-22 16:01:24 +00:00
assert id == " first "
assert status == " RESTORED \n "
2022-07-26 20:20:19 +00:00
def test_system_backups ( ) :
2023-01-29 20:03:38 +00:00
# Backup
2022-07-26 20:20:19 +00:00
create_and_fill_table ( n = 30 )
backup_name = new_backup_name ( )
id = instance . query ( f " BACKUP TABLE test.table TO { backup_name } " ) . split ( " \t " ) [ 0 ]
2023-01-29 20:03:38 +00:00
info = get_backup_info_from_system_backups ( by_id = id )
2022-07-26 20:20:19 +00:00
escaped_backup_name = backup_name . replace ( " ' " , " \\ ' " )
2023-01-29 20:03:38 +00:00
assert info . name == escaped_backup_name
assert info . status == " BACKUP_CREATED "
assert info . error == " "
assert info . num_files > 0
assert info . total_size > 0
assert 0 < info . num_entries and info . num_entries < = info . num_files
assert info . uncompressed_size > 0
assert info . compressed_size == info . uncompressed_size
assert info . files_read == 0
assert info . bytes_read == 0
files_in_backup_folder = find_files_in_backup_folder ( backup_name )
assert info . num_entries == len ( files_in_backup_folder ) - 1
assert info . uncompressed_size == sum (
os . path . getsize ( f ) for f in files_in_backup_folder
)
# The concrete values can change.
info . num_files == 91
info . total_size == 4973
info . num_entries == 55
info . uncompressed_size == 19701
instance . query ( " DROP TABLE test.table " )
2022-07-26 20:20:19 +00:00
2023-01-29 20:03:38 +00:00
# Restore
id = instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " ) . split ( " \t " ) [ 0 ]
restore_info = get_backup_info_from_system_backups ( by_id = id )
assert restore_info . name == escaped_backup_name
assert restore_info . status == " RESTORED "
assert restore_info . error == " "
assert restore_info . num_files == info . num_files
assert restore_info . total_size == info . total_size
assert restore_info . num_entries == info . num_entries
assert restore_info . uncompressed_size == info . uncompressed_size
assert restore_info . compressed_size == info . compressed_size
assert restore_info . files_read == restore_info . num_files
assert restore_info . bytes_read == restore_info . total_size
# Failed backup.
2022-07-27 08:36:56 +00:00
backup_name = new_backup_name ( )
2022-07-26 20:20:19 +00:00
expected_error = " Table test.non_existent_table was not found "
assert expected_error in instance . query_and_get_error (
2022-07-27 08:36:56 +00:00
f " BACKUP TABLE test.non_existent_table TO { backup_name } "
2022-07-26 20:20:19 +00:00
)
2022-07-27 08:36:56 +00:00
escaped_backup_name = backup_name . replace ( " ' " , " \\ ' " )
2023-01-29 20:03:38 +00:00
info = get_backup_info_from_system_backups ( by_name = escaped_backup_name )
assert info . status == " BACKUP_FAILED "
assert expected_error in info . error
assert info . num_files == 0
assert info . total_size == 0
assert info . num_entries == 0
assert info . uncompressed_size == 0
assert info . compressed_size == 0
assert info . files_read == 0
assert info . bytes_read == 0
2022-07-26 20:20:19 +00:00
2022-07-05 07:39:52 +00:00
def test_mutation ( ) :
create_and_fill_table ( engine = " MergeTree ORDER BY tuple() " , n = 5 )
instance . query (
" INSERT INTO test.table SELECT number, toString(number) FROM numbers(5, 5) "
)
instance . query (
" INSERT INTO test.table SELECT number, toString(number) FROM numbers(10, 5) "
)
instance . query ( " ALTER TABLE test.table UPDATE x=x+1 WHERE 1 " )
2022-07-22 08:08:37 +00:00
instance . query ( " ALTER TABLE test.table UPDATE x=x+1+sleep(3) WHERE 1 " )
instance . query ( " ALTER TABLE test.table UPDATE x=x+1+sleep(3) WHERE 1 " )
2022-07-05 07:39:52 +00:00
backup_name = new_backup_name ( )
instance . query ( f " BACKUP TABLE test.table TO { backup_name } " )
assert not has_mutation_in_backup ( " 0000000004 " , backup_name , " test " , " table " )
assert has_mutation_in_backup ( " 0000000005 " , backup_name , " test " , " table " )
assert has_mutation_in_backup ( " 0000000006 " , backup_name , " test " , " table " )
assert not has_mutation_in_backup ( " 0000000007 " , backup_name , " test " , " table " )
instance . query ( " DROP TABLE test.table " )
instance . query ( f " RESTORE TABLE test.table FROM { backup_name } " )
2022-12-02 14:05:46 +00:00
def test_tables_dependency ( ) :
instance . query ( " CREATE DATABASE test " )
instance . query ( " CREATE DATABASE test2 " )
# For this test we use random names of tables to check they're created according to their dependency (not just in alphabetic order).
2023-02-07 17:59:24 +00:00
random_table_names = [ f " { chr ( ord ( ' A ' ) + i ) } " for i in range ( 0 , 15 ) ]
2022-12-02 14:05:46 +00:00
random . shuffle ( random_table_names )
random_table_names = [
random . choice ( [ " test " , " test2 " ] ) + " . " + table_name
for table_name in random_table_names
]
print ( f " random_table_names= { random_table_names } " )
2023-02-07 17:59:24 +00:00
t1 , t2 , t3 , t4 , t5 , t6 , t7 , t8 , t9 , t10 , t11 , t12 , t13 , t14 , t15 = tuple (
random_table_names
)
2022-12-02 14:05:46 +00:00
# Create a materialized view and a dictionary with a local table as source.
instance . query (
f " CREATE TABLE { t1 } (x Int64, y String) ENGINE=MergeTree ORDER BY tuple() "
)
instance . query (
f " CREATE TABLE { t2 } (x Int64, y String) ENGINE=MergeTree ORDER BY tuple() "
)
instance . query ( f " CREATE MATERIALIZED VIEW { t3 } TO { t2 } AS SELECT x, y FROM { t1 } " )
instance . query (
2023-04-12 22:32:19 +00:00
f " CREATE DICTIONARY { t4 } (x Int64, y String) PRIMARY KEY x SOURCE(CLICKHOUSE(HOST ' localhost ' PORT tcpPort() TABLE ' { t1 . split ( ' . ' ) [ 1 ] } ' DB ' { t1 . split ( ' . ' ) [ 0 ] } ' )) LAYOUT(FLAT()) LIFETIME(4) "
2022-12-02 14:05:46 +00:00
)
instance . query ( f " CREATE TABLE { t5 } AS dictionary( { t4 } ) " )
instance . query (
f " CREATE TABLE { t6 } (x Int64, y String DEFAULT dictGet( { t4 } , ' y ' , x)) ENGINE=MergeTree ORDER BY tuple() "
)
2022-12-04 01:04:50 +00:00
instance . query ( f " CREATE VIEW { t7 } AS SELECT sum(x) FROM (SELECT x FROM { t6 } ) " )
instance . query (
2023-04-12 22:32:19 +00:00
f " CREATE DICTIONARY { t8 } (x Int64, y String) PRIMARY KEY x SOURCE(CLICKHOUSE(TABLE ' { t1 . split ( ' . ' ) [ 1 ] } ' DB ' { t1 . split ( ' . ' ) [ 0 ] } ' )) LAYOUT(FLAT()) LIFETIME(9) "
2023-02-07 17:59:24 +00:00
)
instance . query ( f " CREATE TABLE { t9 } (a Int64) ENGINE=Log " )
instance . query (
f " CREATE VIEW { t10 } (x Int64, y String) AS SELECT * FROM { t1 } WHERE x IN { t9 } "
)
instance . query (
f " CREATE VIEW { t11 } (x Int64, y String) AS SELECT * FROM { t2 } WHERE x NOT IN (SELECT a FROM { t9 } ) "
)
instance . query (
f " CREATE TABLE { t12 } AS { t1 } ENGINE = Buffer( { t2 . split ( ' . ' ) [ 0 ] } , { t2 . split ( ' . ' ) [ 1 ] } , 16, 10, 100, 10000, 1000000, 10000000, 100000000) "
)
instance . query (
f " CREATE TABLE { t13 } AS { t1 } ENGINE = Buffer((SELECT ' { t2 . split ( ' . ' ) [ 0 ] } ' ), (SELECT ' { t2 . split ( ' . ' ) [ 1 ] } ' ), 16, 10, 100, 10000, 1000000, 10000000, 100000000) "
2022-12-04 01:04:50 +00:00
)
2022-12-05 18:37:41 +00:00
instance . query (
2023-02-07 17:59:24 +00:00
f " CREATE TABLE { t14 } AS { t1 } ENGINE = Buffer( ' ' , { t2 . split ( ' . ' ) [ 1 ] } , 16, 10, 100, 10000, 1000000, 10000000, 100000000) " ,
database = t2 . split ( " . " ) [ 0 ] ,
)
instance . query (
f " CREATE TABLE { t15 } AS { t1 } ENGINE = Buffer( ' ' , ' ' , 16, 10, 100, 10000, 1000000, 10000000, 100000000) "
2022-12-05 18:37:41 +00:00
)
2022-12-02 14:05:46 +00:00
# Make backup.
backup_name = new_backup_name ( )
instance . query ( f " BACKUP DATABASE test, DATABASE test2 TO { backup_name } " )
# Drop everything in reversive order.
def drop ( ) :
2023-05-03 18:06:46 +00:00
instance . query ( f " DROP TABLE { t15 } SYNC " )
instance . query ( f " DROP TABLE { t14 } SYNC " )
instance . query ( f " DROP TABLE { t13 } SYNC " )
instance . query ( f " DROP TABLE { t12 } SYNC " )
instance . query ( f " DROP TABLE { t11 } SYNC " )
instance . query ( f " DROP TABLE { t10 } SYNC " )
instance . query ( f " DROP TABLE { t9 } SYNC " )
2023-02-07 17:59:24 +00:00
instance . query ( f " DROP DICTIONARY { t8 } " )
2023-05-03 18:06:46 +00:00
instance . query ( f " DROP TABLE { t7 } SYNC " )
instance . query ( f " DROP TABLE { t6 } SYNC " )
instance . query ( f " DROP TABLE { t5 } SYNC " )
2022-12-02 14:05:46 +00:00
instance . query ( f " DROP DICTIONARY { t4 } " )
2023-05-03 18:06:46 +00:00
instance . query ( f " DROP TABLE { t3 } SYNC " )
instance . query ( f " DROP TABLE { t2 } SYNC " )
instance . query ( f " DROP TABLE { t1 } SYNC " )
instance . query ( " DROP DATABASE test SYNC " )
instance . query ( " DROP DATABASE test2 SYNC " )
2022-12-02 14:05:46 +00:00
drop ( )
2022-12-04 01:04:50 +00:00
# Restore everything.
2022-12-02 14:05:46 +00:00
instance . query ( f " RESTORE ALL FROM { backup_name } " )
2022-12-04 01:04:50 +00:00
# Check everything is restored.
2022-12-02 14:05:46 +00:00
assert instance . query (
" SELECT concat(database, ' . ' , name) AS c FROM system.tables WHERE database IN [ ' test ' , ' test2 ' ] ORDER BY c "
2023-02-07 17:59:24 +00:00
) == TSV ( sorted ( random_table_names ) )
2022-12-04 01:04:50 +00:00
# Check logs.
instance . query ( " SYSTEM FLUSH LOGS " )
expect_in_logs = [
f " Table { t1 } has no dependencies (level 0) " ,
f " Table { t2 } has no dependencies (level 0) " ,
(
f " Table { t3 } has 2 dependencies: { t1 } , { t2 } (level 1) " ,
f " Table { t3 } has 2 dependencies: { t2 } , { t1 } (level 1) " ,
) ,
f " Table { t4 } has 1 dependencies: { t1 } (level 1) " ,
f " Table { t5 } has 1 dependencies: { t4 } (level 2) " ,
f " Table { t6 } has 1 dependencies: { t4 } (level 2) " ,
f " Table { t7 } has 1 dependencies: { t6 } (level 3) " ,
2023-02-07 17:59:24 +00:00
f " Table { t8 } has 1 dependencies: { t1 } (level 1) " ,
f " Table { t9 } has no dependencies (level 0) " ,
(
f " Table { t10 } has 2 dependencies: { t1 } , { t9 } (level 1) " ,
f " Table { t10 } has 2 dependencies: { t9 } , { t1 } (level 1) " ,
) ,
(
f " Table { t11 } has 2 dependencies: { t2 } , { t9 } (level 1) " ,
f " Table { t11 } has 2 dependencies: { t9 } , { t2 } (level 1) " ,
) ,
f " Table { t12 } has 1 dependencies: { t2 } (level 1) " ,
f " Table { t13 } has 1 dependencies: { t2 } (level 1) " ,
f " Table { t14 } has 1 dependencies: { t2 } (level 1) " ,
f " Table { t15 } has no dependencies (level 0) " ,
2022-12-04 01:04:50 +00:00
]
for expect in expect_in_logs :
assert any (
[
instance . contains_in_log ( f " RestorerFromBackup: { x } " )
for x in tuple ( expect )
]
)
2022-12-02 14:05:46 +00:00
drop ( )
2023-11-14 20:31:49 +00:00
# Test for the "clickhouse_backupview" utility.
test_backupview_dir = os . path . abspath (
os . path . join ( script_dir , " ../../../utils/backupview/test " )
)
if test_backupview_dir not in sys . path :
sys . path . append ( test_backupview_dir )
import test_backupview as test_backupview_module
def test_backupview ( ) :
if instance . is_built_with_sanitizer ( ) :
return # This test is actually for clickhouse_backupview, not for ClickHouse itself.
test_backupview_module . test_backupview_1 ( )