2022-04-05 19:12:48 +00:00
# include <Storages/MergeTree/DataPartStorageOnDisk.h>
# include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
# include <Disks/IVolume.h>
2022-04-19 19:34:41 +00:00
# include <Disks/TemporaryFileOnDisk.h>
2022-04-05 19:12:48 +00:00
# include <IO/WriteBufferFromFileBase.h>
2022-04-22 16:58:09 +00:00
# include <IO/ReadBufferFromFileBase.h>
2022-05-03 15:48:05 +00:00
# include <IO/ReadHelpers.h>
2022-05-03 17:58:47 +00:00
# include <Common/logger_useful.h>
2022-04-07 11:58:38 +00:00
# include <Disks/IStoragePolicy.h>
2022-04-19 19:34:41 +00:00
# include <Backups/BackupEntryFromSmallFile.h>
# include <Backups/BackupEntryFromImmutableFile.h>
# include <Storages/MergeTree/localBackup.h>
# include <Disks/SingleDiskVolume.h>
2022-05-03 15:48:05 +00:00
# include <Interpreters/TransactionVersionMetadata.h>
2022-04-05 19:12:48 +00:00
namespace DB
{
2022-04-07 11:58:38 +00:00
namespace ErrorCodes
{
extern const int DIRECTORY_ALREADY_EXISTS ;
2022-04-12 18:59:49 +00:00
extern const int NOT_ENOUGH_SPACE ;
2022-05-05 09:23:23 +00:00
extern const int LOGICAL_ERROR ;
2022-04-07 11:58:38 +00:00
}
2022-04-12 18:59:49 +00:00
DataPartStorageOnDisk : : DataPartStorageOnDisk ( VolumePtr volume_ , std : : string root_path_ , std : : string part_dir_ )
: volume ( std : : move ( volume_ ) ) , root_path ( std : : move ( root_path_ ) ) , part_dir ( std : : move ( part_dir_ ) )
2022-05-05 09:19:12 +00:00
{
2022-04-05 19:12:48 +00:00
}
2022-06-20 18:18:17 +00:00
std : : string DataPartStorageOnDisk : : getFullPath ( ) const
2022-04-19 19:34:41 +00:00
{
2022-06-20 18:18:17 +00:00
return fs : : path ( volume - > getDisk ( ) - > getPath ( ) ) / root_path / part_dir / " " ;
2022-04-19 19:34:41 +00:00
}
2022-06-20 18:18:17 +00:00
std : : string DataPartStorageOnDisk : : getRelativePath ( ) const
2022-04-12 18:59:49 +00:00
{
2022-04-22 20:38:18 +00:00
return fs : : path ( root_path ) / part_dir / " " ;
2022-04-12 18:59:49 +00:00
}
2022-06-20 18:18:17 +00:00
void DataPartStorageOnDisk : : setRelativePath ( const std : : string & path )
2022-04-07 11:58:38 +00:00
{
2022-06-20 18:18:17 +00:00
part_dir = path ;
2022-04-07 11:58:38 +00:00
}
2022-04-19 19:34:41 +00:00
std : : string DataPartStorageOnDisk : : getFullRootPath ( ) const
{
2022-04-22 20:38:18 +00:00
return fs : : path ( volume - > getDisk ( ) - > getPath ( ) ) / root_path / " " ;
2022-04-19 19:34:41 +00:00
}
2022-06-20 18:18:17 +00:00
DataPartStoragePtr DataPartStorageOnDisk : : getProjection ( const std : : string & name ) const
2022-04-22 16:58:09 +00:00
{
2022-06-20 18:18:17 +00:00
return std : : make_shared < DataPartStorageOnDisk > ( volume , std : : string ( fs : : path ( root_path ) / part_dir ) , name ) ;
2022-04-22 16:58:09 +00:00
}
2022-04-05 19:12:48 +00:00
bool DataPartStorageOnDisk : : exists ( ) const
{
2022-04-12 18:59:49 +00:00
return volume - > getDisk ( ) - > exists ( fs : : path ( root_path ) / part_dir ) ;
2022-04-05 19:12:48 +00:00
}
2022-06-20 18:18:17 +00:00
bool DataPartStorageOnDisk : : exists ( const std : : string & name ) const
2022-04-08 18:56:08 +00:00
{
2022-06-20 18:18:17 +00:00
return volume - > getDisk ( ) - > exists ( fs : : path ( root_path ) / part_dir / name ) ;
2022-04-08 18:56:08 +00:00
}
2022-06-20 18:18:17 +00:00
bool DataPartStorageOnDisk : : isDirectory ( const std : : string & name ) const
2022-04-05 19:12:48 +00:00
{
2022-06-20 18:18:17 +00:00
return volume - > getDisk ( ) - > isDirectory ( fs : : path ( root_path ) / part_dir / name ) ;
2022-04-05 19:12:48 +00:00
}
2022-06-20 18:18:17 +00:00
Poco : : Timestamp DataPartStorageOnDisk : : getLastModified ( ) const
2022-04-05 19:12:48 +00:00
{
2022-06-20 18:18:17 +00:00
return volume - > getDisk ( ) - > getLastModified ( fs : : path ( root_path ) / part_dir ) ;
2022-04-05 19:12:48 +00:00
}
2022-04-21 19:19:13 +00:00
class DataPartStorageIteratorOnDisk final : public IDataPartStorageIterator
2022-04-05 19:12:48 +00:00
{
2022-04-21 19:19:13 +00:00
public :
2022-06-15 16:32:22 +00:00
DataPartStorageIteratorOnDisk ( DiskPtr disk_ , DirectoryIteratorPtr it_ )
2022-04-21 19:19:13 +00:00
: disk ( std : : move ( disk_ ) ) , it ( std : : move ( it_ ) )
{
}
void next ( ) override { it - > next ( ) ; }
bool isValid ( ) const override { return it - > isValid ( ) ; }
bool isFile ( ) const override { return isValid ( ) & & disk - > isFile ( it - > path ( ) ) ; }
std : : string name ( ) const override { return it - > name ( ) ; }
2022-04-05 19:12:48 +00:00
2022-04-21 19:19:13 +00:00
private :
DiskPtr disk ;
2022-06-15 16:32:22 +00:00
DirectoryIteratorPtr it ;
2022-04-21 19:19:13 +00:00
} ;
2022-04-19 19:34:41 +00:00
2022-04-21 19:19:13 +00:00
DataPartStorageIteratorPtr DataPartStorageOnDisk : : iterate ( ) const
{
return std : : make_unique < DataPartStorageIteratorOnDisk > (
volume - > getDisk ( ) ,
volume - > getDisk ( ) - > iterateDirectory ( fs : : path ( root_path ) / part_dir ) ) ;
}
2022-06-20 18:18:17 +00:00
size_t DataPartStorageOnDisk : : getFileSize ( const String & file_name ) const
2022-04-21 19:19:13 +00:00
{
2022-06-20 18:18:17 +00:00
return volume - > getDisk ( ) - > getFileSize ( fs : : path ( root_path ) / part_dir / file_name ) ;
}
UInt32 DataPartStorageOnDisk : : getRefCount ( const String & file_name ) const
{
return volume - > getDisk ( ) - > getRefCount ( fs : : path ( root_path ) / part_dir / file_name ) ;
}
static UInt64 calculateTotalSizeOnDiskImpl ( const DiskPtr & disk , const String & from )
{
if ( disk - > isFile ( from ) )
return disk - > getFileSize ( from ) ;
std : : vector < std : : string > files ;
disk - > listFiles ( from , files ) ;
UInt64 res = 0 ;
for ( const auto & file : files )
res + = calculateTotalSizeOnDiskImpl ( disk , fs : : path ( from ) / file ) ;
return res ;
}
UInt64 DataPartStorageOnDisk : : calculateTotalSizeOnDisk ( ) const
{
return calculateTotalSizeOnDiskImpl ( volume - > getDisk ( ) , fs : : path ( root_path ) / part_dir ) ;
}
std : : unique_ptr < ReadBufferFromFileBase > DataPartStorageOnDisk : : readFile (
const std : : string & name ,
const ReadSettings & settings ,
std : : optional < size_t > read_hint ,
std : : optional < size_t > file_size ) const
{
return volume - > getDisk ( ) - > readFile ( fs : : path ( root_path ) / part_dir / name , settings , read_hint , file_size ) ;
}
static std : : unique_ptr < ReadBufferFromFileBase > openForReading ( const DiskPtr & disk , const String & path )
{
size_t file_size = disk - > getFileSize ( path ) ;
return disk - > readFile ( path , ReadSettings ( ) . adjustBufferSize ( file_size ) , file_size ) ;
}
void DataPartStorageOnDisk : : loadVersionMetadata ( VersionMetadata & version , Poco : : Logger * log ) const
{
std : : string version_file_name = fs : : path ( root_path ) / part_dir / " txn_version.txt " ;
String tmp_version_file_name = version_file_name + " .tmp " ;
DiskPtr disk = volume - > getDisk ( ) ;
auto remove_tmp_file = [ & ] ( )
{
auto last_modified = disk - > getLastModified ( tmp_version_file_name ) ;
auto buf = openForReading ( disk , tmp_version_file_name ) ;
String content ;
readStringUntilEOF ( content , * buf ) ;
LOG_WARNING ( log , " Found file {} that was last modified on {}, has size {} and the following content: {} " ,
tmp_version_file_name , last_modified . epochTime ( ) , content . size ( ) , content ) ;
disk - > removeFile ( tmp_version_file_name ) ;
} ;
if ( disk - > exists ( version_file_name ) )
{
auto buf = openForReading ( disk , version_file_name ) ;
version . read ( * buf ) ;
if ( disk - > exists ( tmp_version_file_name ) )
remove_tmp_file ( ) ;
return ;
}
/// Four (?) cases are possible:
/// 1. Part was created without transactions.
/// 2. Version metadata file was not renamed from *.tmp on part creation.
/// 3. Version metadata were written to *.tmp file, but hard restart happened before fsync.
/// 4. Fsyncs in storeVersionMetadata() work incorrectly.
if ( ! disk - > exists ( tmp_version_file_name ) )
{
/// Case 1.
/// We do not have version metadata and transactions history for old parts,
/// so let's consider that such parts were created by some ancient transaction
/// and were committed with some prehistoric CSN.
/// NOTE It might be Case 3, but version metadata file is written on part creation before other files,
/// so it's not Case 3 if part is not broken.
version . setCreationTID ( Tx : : PrehistoricTID , nullptr ) ;
version . creation_csn = Tx : : PrehistoricCSN ;
return ;
}
/// Case 2.
/// Content of *.tmp file may be broken, just use fake TID.
/// Transaction was not committed if *.tmp file was not renamed, so we should complete rollback by removing part.
version . setCreationTID ( Tx : : DummyTID , nullptr ) ;
version . creation_csn = Tx : : RolledBackCSN ;
remove_tmp_file ( ) ;
}
void DataPartStorageOnDisk : : checkConsistency ( const MergeTreeDataPartChecksums & checksums ) const
{
checksums . checkSizes ( volume - > getDisk ( ) , getRelativePath ( ) ) ;
2022-04-21 19:19:13 +00:00
}
2022-04-19 19:34:41 +00:00
2022-06-27 19:41:29 +00:00
DataPartStorageBuilderPtr DataPartStorageOnDisk : : getBuilder ( ) const
{
return std : : make_shared < DataPartStorageBuilderOnDisk > ( volume , root_path , part_dir ) ;
}
2022-04-19 19:34:41 +00:00
void DataPartStorageOnDisk : : remove (
2022-05-03 15:48:05 +00:00
bool can_remove_shared_data ,
const NameSet & names_not_to_remove ,
2022-05-05 09:19:12 +00:00
const MergeTreeDataPartChecksums & checksums ,
2022-04-19 19:34:41 +00:00
std : : list < ProjectionChecksums > projections ,
2022-08-12 10:47:25 +00:00
bool is_temp ,
MergeTreeDataPartState state ,
2022-04-19 19:34:41 +00:00
Poco : : Logger * log ) const
{
/// NOTE We rename part to delete_tmp_<relative_path> instead of delete_tmp_<name> to avoid race condition
/// when we try to remove two parts with the same name, but different relative paths,
/// for example all_1_2_1 (in Deleting state) and tmp_merge_all_1_2_1 (in Temporary state).
fs : : path from = fs : : path ( root_path ) / part_dir ;
2022-06-15 13:35:26 +00:00
// fs::path to = fs::path(root_path) / ("delete_tmp_" + part_dir);
2022-04-19 19:34:41 +00:00
// TODO directory delete_tmp_<name> is never removed if server crashes before returning from this function
2022-06-15 13:35:26 +00:00
/// Cut last "/" if it exists (it shouldn't). Otherwise fs::path behave differently.
fs : : path part_dir_without_slash = part_dir . ends_with ( " / " ) ? part_dir . substr ( 0 , part_dir . size ( ) - 1 ) : part_dir ;
/// NOTE relative_path can contain not only part name itself, but also some prefix like
/// "moving/all_1_1_1" or "detached/all_2_3_5". We should handle this case more properly.
if ( part_dir_without_slash . has_parent_path ( ) )
{
auto parent_path = part_dir_without_slash . parent_path ( ) ;
if ( parent_path = = " detached " )
2022-06-15 16:32:22 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Trying to remove detached part {} with path {} in remove function. It shouldn't happen " , part_dir , root_path ) ;
2022-06-15 13:35:26 +00:00
part_dir_without_slash = parent_path / ( " delete_tmp_ " + std : : string { part_dir_without_slash . filename ( ) } ) ;
}
else
{
part_dir_without_slash = ( " delete_tmp_ " + std : : string { part_dir_without_slash . filename ( ) } ) ;
}
fs : : path to = fs : : path ( root_path ) / part_dir_without_slash ;
2022-04-19 19:34:41 +00:00
auto disk = volume - > getDisk ( ) ;
if ( disk - > exists ( to ) )
{
LOG_WARNING ( log , " Directory {} (to which part must be renamed before removing) already exists. Most likely this is due to unclean restart or race condition. Removing it. " , fullPath ( disk , to ) ) ;
try
{
2022-05-03 15:48:05 +00:00
disk - > removeSharedRecursive ( fs : : path ( to ) / " " , ! can_remove_shared_data , names_not_to_remove ) ;
2022-04-19 19:34:41 +00:00
}
catch ( . . . )
{
LOG_ERROR ( log , " Cannot recursively remove directory {}. Exception: {} " , fullPath ( disk , to ) , getCurrentExceptionMessage ( false ) ) ;
throw ;
}
}
try
{
disk - > moveDirectory ( from , to ) ;
}
catch ( const fs : : filesystem_error & e )
{
if ( e . code ( ) = = std : : errc : : no_such_file_or_directory )
{
LOG_ERROR ( log , " Directory {} (part to remove) doesn't exist or one of nested files has gone. Most likely this is due to manual removing. This should be discouraged. Ignoring. " , fullPath ( disk , to ) ) ;
return ;
}
throw ;
}
// Record existing projection directories so we don't remove them twice
std : : unordered_set < String > projection_directories ;
2022-07-12 08:28:22 +00:00
std : : string proj_suffix = " .proj " ;
2022-04-19 19:34:41 +00:00
for ( const auto & projection : projections )
{
2022-07-12 08:28:22 +00:00
std : : string proj_dir_name = projection . name + proj_suffix ;
2022-04-19 19:34:41 +00:00
projection_directories . emplace ( proj_dir_name ) ;
2022-05-05 09:19:12 +00:00
clearDirectory (
fs : : path ( to ) / proj_dir_name ,
2022-08-12 10:47:25 +00:00
can_remove_shared_data , names_not_to_remove , projection . checksums , { } , is_temp , state , log , true ) ;
2022-04-19 19:34:41 +00:00
}
2022-07-12 08:28:22 +00:00
/// It is possible that we are removing the part which have a written but not loaded projection.
/// Such a part can appear server was restarted after DROP PROJECTION but before old part was removed.
/// In this case, the old part will load only projections from metadata.
/// See test 01701_clear_projection_and_part.
for ( const auto & [ name , _ ] : checksums . files )
{
if ( endsWith ( name , proj_suffix ) & & ! projection_directories . contains ( name ) & & disk - > isDirectory ( fs : : path ( to ) / name ) )
{
/// If we have a directory with suffix '.proj' it is likely a projection.
2022-07-12 09:37:00 +00:00
/// Try to load checksums for it (to avoid recursive removing fallback).
2022-07-12 08:28:22 +00:00
std : : string checksum_path = fs : : path ( to ) / name / " checksums.txt " ;
if ( disk - > exists ( checksum_path ) )
{
try
{
MergeTreeDataPartChecksums tmp_checksums ;
auto in = disk - > readFile ( checksum_path , { } ) ;
tmp_checksums . read ( * in ) ;
projection_directories . emplace ( name ) ;
clearDirectory (
fs : : path ( to ) / name ,
2022-08-12 10:47:25 +00:00
can_remove_shared_data , names_not_to_remove , tmp_checksums , { } , is_temp , state , log , true ) ;
2022-07-12 08:28:22 +00:00
}
catch ( . . . )
{
LOG_ERROR ( log , " Cannot load checksums from {} " , checksum_path ) ;
}
}
}
}
2022-08-12 10:47:25 +00:00
clearDirectory ( to , can_remove_shared_data , names_not_to_remove , checksums , projection_directories , is_temp , state , log , false ) ;
2022-04-19 19:34:41 +00:00
}
void DataPartStorageOnDisk : : clearDirectory (
const std : : string & dir ,
2022-05-03 15:48:05 +00:00
bool can_remove_shared_data ,
const NameSet & names_not_to_remove ,
2022-05-05 09:19:12 +00:00
const MergeTreeDataPartChecksums & checksums ,
const std : : unordered_set < String > & skip_directories ,
2022-08-12 10:47:25 +00:00
bool is_temp ,
MergeTreeDataPartState state ,
2022-04-19 19:34:41 +00:00
Poco : : Logger * log ,
bool is_projection ) const
{
auto disk = volume - > getDisk ( ) ;
2022-08-12 10:47:25 +00:00
/// It does not make sense to try fast path for incomplete temporary parts, because some files are probably absent.
/// Sometimes we add something to checksums.files before actually writing checksums and columns on disk.
/// Also sometimes we write checksums.txt and columns.txt in arbitrary order, so this check becomes complex...
2022-08-15 14:03:03 +00:00
bool is_temporary_part = is_temp | | state = = MergeTreeDataPartState : : Temporary ;
bool incomplete_temporary_part = is_temporary_part & & ( ! disk - > exists ( fs : : path ( dir ) / " checksums.txt " ) | | ! disk - > exists ( fs : : path ( dir ) / " columns.txt " ) ) ;
2022-08-12 10:47:25 +00:00
if ( checksums . empty ( ) | | incomplete_temporary_part )
2022-04-19 19:34:41 +00:00
{
/// If the part is not completely written, we cannot use fast path by listing files.
2022-05-03 15:48:05 +00:00
disk - > removeSharedRecursive ( fs : : path ( dir ) / " " , ! can_remove_shared_data , names_not_to_remove ) ;
2022-04-19 19:34:41 +00:00
return ;
}
try
{
/// Remove each expected file in directory, then remove directory itself.
2022-06-21 11:15:06 +00:00
RemoveBatchRequest request ;
2022-04-19 19:34:41 +00:00
# if !defined(__clang__)
# pragma GCC diagnostic push
# pragma GCC diagnostic ignored "-Wunused-variable"
# endif
for ( const auto & [ file , _ ] : checksums . files )
{
if ( skip_directories . find ( file ) = = skip_directories . end ( ) )
request . emplace_back ( fs : : path ( dir ) / file ) ;
}
# if !defined(__clang__)
# pragma GCC diagnostic pop
# endif
for ( const auto & file : { " checksums.txt " , " columns.txt " } )
request . emplace_back ( fs : : path ( dir ) / file ) ;
request . emplace_back ( fs : : path ( dir ) / " default_compression_codec.txt " , true ) ;
request . emplace_back ( fs : : path ( dir ) / " delete-on-destroy.txt " , true ) ;
if ( ! is_projection )
request . emplace_back ( fs : : path ( dir ) / " txn_version.txt " , true ) ;
2022-05-03 15:48:05 +00:00
disk - > removeSharedFiles ( request , ! can_remove_shared_data , names_not_to_remove ) ;
2022-04-19 19:34:41 +00:00
disk - > removeDirectory ( dir ) ;
}
catch ( . . . )
{
/// Recursive directory removal does many excessive "stat" syscalls under the hood.
LOG_ERROR ( log , " Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {} " , fullPath ( disk , dir ) , getCurrentExceptionMessage ( false ) ) ;
2022-07-12 08:31:09 +00:00
disk - > removeSharedRecursive ( fs : : path ( dir ) / " " , ! can_remove_shared_data , names_not_to_remove ) ;
2022-04-19 19:34:41 +00:00
}
}
2022-06-20 18:18:17 +00:00
std : : string DataPartStorageOnDisk : : getRelativePathForPrefix ( Poco : : Logger * log , const String & prefix , bool detached ) const
2022-04-22 16:58:09 +00:00
{
2022-06-20 18:18:17 +00:00
String res ;
2022-04-22 16:58:09 +00:00
2022-06-20 18:18:17 +00:00
auto full_relative_path = fs : : path ( root_path ) ;
if ( detached )
full_relative_path / = " detached " ;
for ( int try_no = 0 ; try_no < 10 ; + + try_no )
{
res = ( prefix . empty ( ) ? " " : prefix + " _ " ) + part_dir + ( try_no ? " _try " + DB : : toString ( try_no ) : " " ) ;
if ( ! volume - > getDisk ( ) - > exists ( full_relative_path / res ) )
return res ;
LOG_WARNING ( log , " Directory {} (to detach to) already exists. Will detach to directory with '_tryN' suffix. " , res ) ;
}
return res ;
2022-04-05 19:12:48 +00:00
}
2022-06-20 18:18:17 +00:00
void DataPartStorageBuilderOnDisk : : setRelativePath ( const std : : string & path )
2022-04-21 19:19:13 +00:00
{
2022-06-20 18:18:17 +00:00
part_dir = path ;
2022-04-21 19:19:13 +00:00
}
2022-06-20 18:18:17 +00:00
std : : string DataPartStorageOnDisk : : getDiskName ( ) const
2022-04-05 19:12:48 +00:00
{
2022-06-20 18:18:17 +00:00
return volume - > getDisk ( ) - > getName ( ) ;
2022-04-05 19:12:48 +00:00
}
2022-06-20 18:18:17 +00:00
std : : string DataPartStorageOnDisk : : getDiskType ( ) const
2022-04-05 19:12:48 +00:00
{
2022-08-19 14:58:30 +00:00
return toString ( volume - > getDisk ( ) - > getDataSourceDescription ( ) . type ) ;
2022-04-05 19:12:48 +00:00
}
2022-04-08 18:56:08 +00:00
bool DataPartStorageOnDisk : : isStoredOnRemoteDisk ( ) const
{
return volume - > getDisk ( ) - > isRemote ( ) ;
}
bool DataPartStorageOnDisk : : supportZeroCopyReplication ( ) const
{
return volume - > getDisk ( ) - > supportZeroCopyReplication ( ) ;
}
2022-04-22 16:58:09 +00:00
bool DataPartStorageOnDisk : : supportParallelWrite ( ) const
{
return volume - > getDisk ( ) - > supportParallelWrite ( ) ;
}
2022-04-19 19:34:41 +00:00
bool DataPartStorageOnDisk : : isBroken ( ) const
{
return volume - > getDisk ( ) - > isBroken ( ) ;
}
2022-06-20 18:18:17 +00:00
void DataPartStorageOnDisk : : syncRevision ( UInt64 revision )
{
volume - > getDisk ( ) - > syncRevision ( revision ) ;
}
UInt64 DataPartStorageOnDisk : : getRevision ( ) const
{
return volume - > getDisk ( ) - > getRevision ( ) ;
}
std : : unordered_map < String , String > DataPartStorageOnDisk : : getSerializedMetadata ( const std : : vector < String > & paths ) const
{
return volume - > getDisk ( ) - > getSerializedMetadata ( paths ) ;
}
std : : string DataPartStorageOnDisk : : getDiskPath ( ) const
2022-04-19 19:34:41 +00:00
{
return volume - > getDisk ( ) - > getPath ( ) ;
}
2022-06-20 18:18:17 +00:00
DataPartStorageOnDisk : : DisksSet : : const_iterator DataPartStorageOnDisk : : isStoredOnDisk ( const DisksSet & disks ) const
{
return disks . find ( volume - > getDisk ( ) ) ;
}
ReservationPtr DataPartStorageOnDisk : : reserve ( UInt64 bytes ) const
{
auto res = volume - > reserve ( bytes ) ;
if ( ! res )
throw Exception ( ErrorCodes : : NOT_ENOUGH_SPACE , " Cannot reserve {}, not enough space " , ReadableSize ( bytes ) ) ;
return res ;
}
ReservationPtr DataPartStorageOnDisk : : tryReserve ( UInt64 bytes ) const
{
return volume - > reserve ( bytes ) ;
}
size_t DataPartStorageOnDisk : : getVolumeIndex ( const IStoragePolicy & storage_policy ) const
{
return storage_policy . getVolumeIndexByDisk ( volume - > getDisk ( ) ) ;
}
2022-05-03 15:48:05 +00:00
void DataPartStorageOnDisk : : writeChecksums ( const MergeTreeDataPartChecksums & checksums , const WriteSettings & settings ) const
2022-04-05 19:12:48 +00:00
{
2022-04-12 18:59:49 +00:00
std : : string path = fs : : path ( root_path ) / part_dir / " checksums.txt " ;
2022-04-05 19:12:48 +00:00
2022-04-21 19:19:13 +00:00
try
2022-04-05 19:12:48 +00:00
{
2022-04-21 19:19:13 +00:00
{
2022-05-03 15:48:05 +00:00
auto out = volume - > getDisk ( ) - > writeFile ( path + " .tmp " , 4096 , WriteMode : : Rewrite , settings ) ;
2022-04-21 19:19:13 +00:00
checksums . write ( * out ) ;
}
volume - > getDisk ( ) - > moveFile ( path + " .tmp " , path ) ;
2022-04-05 19:12:48 +00:00
}
2022-04-21 19:19:13 +00:00
catch ( . . . )
{
try
{
if ( volume - > getDisk ( ) - > exists ( path + " .tmp " ) )
volume - > getDisk ( ) - > removeFile ( path + " .tmp " ) ;
}
catch ( . . . )
{
tryLogCurrentException ( " DataPartStorageOnDisk " ) ;
}
2022-04-05 19:12:48 +00:00
2022-04-21 19:19:13 +00:00
throw ;
}
2022-04-05 19:12:48 +00:00
}
2022-05-03 15:48:05 +00:00
void DataPartStorageOnDisk : : writeColumns ( const NamesAndTypesList & columns , const WriteSettings & settings ) const
2022-04-05 19:12:48 +00:00
{
2022-04-12 18:59:49 +00:00
std : : string path = fs : : path ( root_path ) / part_dir / " columns.txt " ;
2022-04-05 19:12:48 +00:00
2022-04-21 19:19:13 +00:00
try
2022-04-05 19:12:48 +00:00
{
2022-06-20 18:18:17 +00:00
auto buf = volume - > getDisk ( ) - > writeFile ( path + " .tmp " , 4096 , WriteMode : : Rewrite , settings ) ;
columns . writeText ( * buf ) ;
buf - > finalize ( ) ;
2022-04-21 19:19:13 +00:00
volume - > getDisk ( ) - > moveFile ( path + " .tmp " , path ) ;
2022-04-05 19:12:48 +00:00
}
2022-04-21 19:19:13 +00:00
catch ( . . . )
{
try
{
if ( volume - > getDisk ( ) - > exists ( path + " .tmp " ) )
volume - > getDisk ( ) - > removeFile ( path + " .tmp " ) ;
}
catch ( . . . )
{
tryLogCurrentException ( " DataPartStorageOnDisk " ) ;
}
2022-04-05 19:12:48 +00:00
2022-04-21 19:19:13 +00:00
throw ;
}
2022-04-05 19:12:48 +00:00
}
2022-05-03 15:48:05 +00:00
void DataPartStorageOnDisk : : writeVersionMetadata ( const VersionMetadata & version , bool fsync_part_dir ) const
{
std : : string path = fs : : path ( root_path ) / part_dir / " txn_version.txt " ;
try
{
{
/// TODO IDisk interface does not allow to open file with O_EXCL flag (for DiskLocal),
/// so we create empty file at first (expecting that createFile throws if file already exists)
/// and then overwrite it.
2022-06-03 18:49:12 +00:00
volume - > getDisk ( ) - > createFile ( path + " .tmp " ) ;
auto buf = volume - > getDisk ( ) - > writeFile ( path + " .tmp " , 256 ) ;
2022-05-03 15:48:05 +00:00
version . write ( * buf ) ;
buf - > finalize ( ) ;
buf - > sync ( ) ;
}
SyncGuardPtr sync_guard ;
if ( fsync_part_dir )
2022-06-20 18:18:17 +00:00
sync_guard = volume - > getDisk ( ) - > getDirectorySyncGuard ( getRelativePath ( ) ) ;
2022-05-03 15:48:05 +00:00
volume - > getDisk ( ) - > replaceFile ( path + " .tmp " , path ) ;
}
catch ( . . . )
{
try
{
if ( volume - > getDisk ( ) - > exists ( path + " .tmp " ) )
volume - > getDisk ( ) - > removeFile ( path + " .tmp " ) ;
}
catch ( . . . )
{
tryLogCurrentException ( " DataPartStorageOnDisk " ) ;
}
throw ;
}
}
void DataPartStorageOnDisk : : appendCSNToVersionMetadata ( const VersionMetadata & version , VersionMetadata : : WhichCSN which_csn ) const
{
/// Small enough appends to file are usually atomic,
/// so we append new metadata instead of rewriting file to reduce number of fsyncs.
/// We don't need to do fsync when writing CSN, because in case of hard restart
/// we will be able to restore CSN from transaction log in Keeper.
std : : string version_file_name = fs : : path ( root_path ) / part_dir / " txn_version.txt " ;
DiskPtr disk = volume - > getDisk ( ) ;
auto out = disk - > writeFile ( version_file_name , 256 , WriteMode : : Append ) ;
version . writeCSN ( * out , which_csn ) ;
out - > finalize ( ) ;
}
void DataPartStorageOnDisk : : appendRemovalTIDToVersionMetadata ( const VersionMetadata & version , bool clear ) const
{
String version_file_name = fs : : path ( root_path ) / part_dir / " txn_version.txt " ;
DiskPtr disk = volume - > getDisk ( ) ;
auto out = disk - > writeFile ( version_file_name , 256 , WriteMode : : Append ) ;
version . writeRemovalTID ( * out , clear ) ;
out - > finalize ( ) ;
/// fsync is not required when we clearing removal TID, because after hard restart we will fix metadata
if ( ! clear )
out - > sync ( ) ;
}
2022-04-08 18:56:08 +00:00
void DataPartStorageOnDisk : : writeDeleteOnDestroyMarker ( Poco : : Logger * log ) const
{
2022-04-12 18:59:49 +00:00
String marker_path = fs : : path ( root_path ) / part_dir / " delete-on-destroy.txt " ;
2022-04-08 18:56:08 +00:00
auto disk = volume - > getDisk ( ) ;
try
{
volume - > getDisk ( ) - > createFile ( marker_path ) ;
}
catch ( Poco : : Exception & e )
{
LOG_ERROR ( log , " {} (while creating DeleteOnDestroy marker: {}) " , e . what ( ) , backQuote ( fullPath ( disk , marker_path ) ) ) ;
}
}
2022-05-03 15:48:05 +00:00
void DataPartStorageOnDisk : : removeDeleteOnDestroyMarker ( ) const
{
std : : string delete_on_destroy_file_name = fs : : path ( root_path ) / part_dir / " delete-on-destroy.txt " ;
volume - > getDisk ( ) - > removeFileIfExists ( delete_on_destroy_file_name ) ;
}
void DataPartStorageOnDisk : : removeVersionMetadata ( ) const
{
std : : string version_file_name = fs : : path ( root_path ) / part_dir / " txn_version.txt " ;
volume - > getDisk ( ) - > removeFileIfExists ( version_file_name ) ;
}
2022-06-20 18:18:17 +00:00
String DataPartStorageOnDisk : : getUniqueId ( ) const
2022-05-05 09:19:12 +00:00
{
2022-06-21 07:26:43 +00:00
auto disk = volume - > getDisk ( ) ;
if ( ! disk - > supportZeroCopyReplication ( ) )
throw Exception ( fmt : : format ( " Disk {} doesn't support zero-copy replication " , disk - > getName ( ) ) , ErrorCodes : : LOGICAL_ERROR ) ;
return disk - > getUniqueId ( fs : : path ( getRelativePath ( ) ) / " checksums.txt " ) ;
2022-05-05 09:19:12 +00:00
}
2022-04-07 11:58:38 +00:00
bool DataPartStorageOnDisk : : shallParticipateInMerges ( const IStoragePolicy & storage_policy ) const
{
/// `IMergeTreeDataPart::volume` describes space where current part belongs, and holds
/// `SingleDiskVolume` object which does not contain up-to-date settings of corresponding volume.
/// Therefore we shall obtain volume from storage policy.
auto volume_ptr = storage_policy . getVolume ( storage_policy . getVolumeIndexByDisk ( volume - > getDisk ( ) ) ) ;
return ! volume_ptr - > areMergesAvoided ( ) ;
}
2022-04-19 19:34:41 +00:00
void DataPartStorageOnDisk : : backup (
TemporaryFilesOnDisks & temp_dirs ,
const MergeTreeDataPartChecksums & checksums ,
const NameSet & files_without_checksums ,
2022-06-22 20:30:50 +00:00
const String & path_in_backup ,
2022-04-19 19:34:41 +00:00
BackupEntries & backup_entries ) const
{
2022-06-22 20:30:50 +00:00
fs : : path part_path_on_disk = fs : : path { root_path } / part_dir ;
fs : : path part_path_in_backup = fs : : path { path_in_backup } / part_dir ;
2022-04-19 19:34:41 +00:00
2022-06-22 20:30:50 +00:00
auto disk = volume - > getDisk ( ) ;
2022-04-19 19:34:41 +00:00
auto temp_dir_it = temp_dirs . find ( disk ) ;
if ( temp_dir_it = = temp_dirs . end ( ) )
2022-07-03 14:32:11 +00:00
temp_dir_it = temp_dirs . emplace ( disk , std : : make_shared < TemporaryFileOnDisk > ( disk , " tmp/ " ) ) . first ;
2022-04-19 19:34:41 +00:00
auto temp_dir_owner = temp_dir_it - > second ;
fs : : path temp_dir = temp_dir_owner - > getPath ( ) ;
2022-06-22 20:30:50 +00:00
fs : : path temp_part_dir = temp_dir / part_path_in_backup . relative_path ( ) ;
2022-04-19 19:34:41 +00:00
disk - > createDirectories ( temp_part_dir ) ;
2022-06-22 20:30:50 +00:00
/// For example,
/// part_path_in_backup = /data/test/table/0_1_1_0
/// part_path_on_disk = store/f57/f5728353-44bb-4575-85e8-28deb893657a/0_1_1_0
2022-07-03 14:32:11 +00:00
/// tmp_part_dir = tmp/1aaaaaa/data/test/table/0_1_1_0
2022-06-22 20:30:50 +00:00
/// Or, for projections:
/// part_path_in_backup = /data/test/table/0_1_1_0/prjmax.proj
/// part_path_on_disk = store/f57/f5728353-44bb-4575-85e8-28deb893657a/0_1_1_0/prjmax.proj
2022-07-03 14:32:11 +00:00
/// tmp_part_dir = tmp/1aaaaaa/data/test/table/0_1_1_0/prjmax.proj
2022-06-22 20:30:50 +00:00
2022-04-19 19:34:41 +00:00
for ( const auto & [ filepath , checksum ] : checksums . files )
{
2022-06-22 20:30:50 +00:00
if ( filepath . ends_with ( " .proj " ) )
continue ; /// Skip *.proj files - they're actually directories and will be handled.
String filepath_on_disk = part_path_on_disk / filepath ;
String filepath_in_backup = part_path_in_backup / filepath ;
2022-04-19 19:34:41 +00:00
String hardlink_filepath = temp_part_dir / filepath ;
2022-06-30 08:10:12 +00:00
2022-06-22 20:30:50 +00:00
disk - > createHardLink ( filepath_on_disk , hardlink_filepath ) ;
2022-04-19 19:34:41 +00:00
UInt128 file_hash { checksum . file_hash . first , checksum . file_hash . second } ;
backup_entries . emplace_back (
2022-06-22 20:30:50 +00:00
filepath_in_backup ,
2022-04-19 19:34:41 +00:00
std : : make_unique < BackupEntryFromImmutableFile > ( disk , hardlink_filepath , checksum . file_size , file_hash , temp_dir_owner ) ) ;
}
for ( const auto & filepath : files_without_checksums )
{
2022-06-22 20:30:50 +00:00
String filepath_on_disk = part_path_on_disk / filepath ;
String filepath_in_backup = part_path_in_backup / filepath ;
backup_entries . emplace_back ( filepath_in_backup , std : : make_unique < BackupEntryFromSmallFile > ( disk , filepath_on_disk ) ) ;
2022-04-19 19:34:41 +00:00
}
}
DataPartStoragePtr DataPartStorageOnDisk : : freeze (
2022-05-05 09:19:12 +00:00
const std : : string & to ,
2022-04-19 19:34:41 +00:00
const std : : string & dir_path ,
2022-05-03 15:48:05 +00:00
bool make_source_readonly ,
std : : function < void ( const DiskPtr & ) > save_metadata_callback ,
bool copy_instead_of_hardlink ) const
2022-04-19 19:34:41 +00:00
{
auto disk = volume - > getDisk ( ) ;
disk - > createDirectories ( to ) ;
2022-06-20 18:18:17 +00:00
localBackup ( disk , getRelativePath ( ) , fs : : path ( to ) / dir_path , make_source_readonly , { } , copy_instead_of_hardlink ) ;
2022-04-19 19:34:41 +00:00
if ( save_metadata_callback )
save_metadata_callback ( disk ) ;
disk - > removeFileIfExists ( fs : : path ( to ) / dir_path / " delete-on-destroy.txt " ) ;
2022-05-03 15:48:05 +00:00
disk - > removeFileIfExists ( fs : : path ( to ) / dir_path / " txn_version.txt " ) ;
2022-04-19 19:34:41 +00:00
auto single_disk_volume = std : : make_shared < SingleDiskVolume > ( disk - > getName ( ) , disk , 0 ) ;
return std : : make_shared < DataPartStorageOnDisk > ( single_disk_volume , to , dir_path ) ;
}
2022-06-02 15:00:07 +00:00
DataPartStoragePtr DataPartStorageOnDisk : : clone (
2022-04-19 19:34:41 +00:00
const std : : string & to ,
const std : : string & dir_path ,
2022-06-15 11:41:08 +00:00
const DiskPtr & disk ,
2022-04-19 19:34:41 +00:00
Poco : : Logger * log ) const
{
String path_to_clone = fs : : path ( to ) / dir_path / " " ;
if ( disk - > exists ( path_to_clone ) )
{
LOG_WARNING ( log , " Path {} already exists. Will remove it and clone again. " , fullPath ( disk , path_to_clone ) ) ;
disk - > removeRecursive ( path_to_clone ) ;
}
disk - > createDirectories ( to ) ;
2022-06-20 18:18:17 +00:00
volume - > getDisk ( ) - > copy ( getRelativePath ( ) , disk , to ) ;
2022-04-19 19:34:41 +00:00
volume - > getDisk ( ) - > removeFileIfExists ( fs : : path ( path_to_clone ) / " delete-on-destroy.txt " ) ;
auto single_disk_volume = std : : make_shared < SingleDiskVolume > ( disk - > getName ( ) , disk , 0 ) ;
return std : : make_shared < DataPartStorageOnDisk > ( single_disk_volume , to , dir_path ) ;
}
2022-06-27 19:41:29 +00:00
void DataPartStorageOnDisk : : onRename ( const std : : string & new_root_path , const std : : string & new_part_dir )
{
part_dir = new_part_dir ;
root_path = new_root_path ;
}
2022-06-26 16:43:28 +00:00
void DataPartStorageBuilderOnDisk : : rename (
const std : : string & new_root_path ,
const std : : string & new_part_dir ,
Poco : : Logger * log ,
bool remove_new_dir_if_exists ,
bool fsync_part_dir )
2022-06-20 18:18:17 +00:00
{
String to = fs : : path ( new_root_path ) / new_part_dir / " " ;
if ( volume - > getDisk ( ) - > exists ( to ) )
{
if ( remove_new_dir_if_exists )
{
Names files ;
volume - > getDisk ( ) - > listFiles ( to , files ) ;
if ( log )
LOG_WARNING ( log ,
" Part directory {} already exists and contains {} files. Removing it. " ,
fullPath ( volume - > getDisk ( ) , to ) , files . size ( ) ) ;
2022-06-27 19:41:29 +00:00
transaction - > removeRecursive ( to ) ;
2022-06-20 18:18:17 +00:00
}
else
{
throw Exception (
ErrorCodes : : DIRECTORY_ALREADY_EXISTS ,
" Part directory {} already exists " ,
fullPath ( volume - > getDisk ( ) , to ) ) ;
}
}
String from = getRelativePath ( ) ;
/// Why?
2022-06-27 19:41:29 +00:00
transaction - > setLastModified ( from , Poco : : Timestamp : : fromEpochTime ( time ( nullptr ) ) ) ;
transaction - > moveDirectory ( from , to ) ;
2022-06-20 18:18:17 +00:00
part_dir = new_part_dir ;
root_path = new_root_path ;
SyncGuardPtr sync_guard ;
if ( fsync_part_dir )
sync_guard = volume - > getDisk ( ) - > getDirectorySyncGuard ( getRelativePath ( ) ) ;
}
void DataPartStorageOnDisk : : changeRootPath ( const std : : string & from_root , const std : : string & to_root )
{
/// This is a very dumb implementation, here for root path like
/// "some/current/path/to/part" and change like
/// "some/current" -> "other/different", we just replace prefix to make new root like
/// "other/different/path/to/part".
/// Here we expect that actual move was done by somebody else.
size_t prefix_size = from_root . size ( ) ;
if ( prefix_size > 0 & & from_root . back ( ) = = ' / ' )
- - prefix_size ;
if ( prefix_size > root_path . size ( )
| | std : : string_view ( from_root ) . substr ( 0 , prefix_size ) ! = std : : string_view ( root_path ) . substr ( 0 , prefix_size ) )
throw Exception (
ErrorCodes : : LOGICAL_ERROR ,
" Cannot change part root to {} because it is not a prefix of current root {} " ,
from_root , root_path ) ;
size_t dst_size = to_root . size ( ) ;
if ( dst_size > 0 & & to_root . back ( ) = = ' / ' )
- - dst_size ;
root_path = to_root . substr ( 0 , dst_size ) + root_path . substr ( prefix_size ) ;
}
2022-04-19 19:34:41 +00:00
2022-06-23 16:21:46 +00:00
DataPartStorageBuilderOnDisk : : DataPartStorageBuilderOnDisk (
VolumePtr volume_ ,
std : : string root_path_ ,
std : : string part_dir_ )
: volume ( std : : move ( volume_ ) )
, root_path ( std : : move ( root_path_ ) )
, part_dir ( std : : move ( part_dir_ ) )
, transaction ( volume - > getDisk ( ) - > createTransaction ( ) )
2022-05-05 09:19:12 +00:00
{
2022-04-12 18:59:49 +00:00
}
std : : unique_ptr < WriteBufferFromFileBase > DataPartStorageBuilderOnDisk : : writeFile (
2022-06-20 18:18:17 +00:00
const String & name ,
2022-05-03 15:48:05 +00:00
size_t buf_size ,
const WriteSettings & settings )
2022-04-12 18:59:49 +00:00
{
2022-06-23 16:21:46 +00:00
return transaction - > writeFile ( fs : : path ( root_path ) / part_dir / name , buf_size , WriteMode : : Rewrite , settings , /* autocommit = */ false ) ;
2022-04-12 18:59:49 +00:00
}
2022-06-20 18:18:17 +00:00
void DataPartStorageBuilderOnDisk : : removeFile ( const String & name )
2022-04-12 18:59:49 +00:00
{
2022-06-23 16:21:46 +00:00
transaction - > removeFile ( fs : : path ( root_path ) / part_dir / name ) ;
2022-04-12 18:59:49 +00:00
}
2022-06-23 12:01:26 +00:00
void DataPartStorageBuilderOnDisk : : removeFileIfExists ( const String & name )
{
2022-06-23 16:21:46 +00:00
transaction - > removeFileIfExists ( fs : : path ( root_path ) / part_dir / name ) ;
2022-06-23 12:01:26 +00:00
}
2022-04-12 18:59:49 +00:00
void DataPartStorageBuilderOnDisk : : removeRecursive ( )
{
2022-06-23 16:21:46 +00:00
transaction - > removeRecursive ( fs : : path ( root_path ) / part_dir ) ;
2022-04-12 18:59:49 +00:00
}
2022-04-21 19:19:13 +00:00
void DataPartStorageBuilderOnDisk : : removeSharedRecursive ( bool keep_in_remote_fs )
{
2022-06-23 16:21:46 +00:00
transaction - > removeSharedRecursive ( fs : : path ( root_path ) / part_dir , keep_in_remote_fs , { } ) ;
2022-04-21 19:19:13 +00:00
}
SyncGuardPtr DataPartStorageBuilderOnDisk : : getDirectorySyncGuard ( ) const
{
return volume - > getDisk ( ) - > getDirectorySyncGuard ( fs : : path ( root_path ) / part_dir ) ;
}
void DataPartStorageBuilderOnDisk : : createHardLinkFrom ( const IDataPartStorage & source , const std : : string & from , const std : : string & to ) const
{
const auto * source_on_disk = typeid_cast < const DataPartStorageOnDisk * > ( & source ) ;
if ( ! source_on_disk )
throw Exception (
2022-05-05 09:19:12 +00:00
ErrorCodes : : LOGICAL_ERROR ,
2022-04-21 19:19:13 +00:00
" Cannot create hardlink from different storage. Expected DataPartStorageOnDisk, got {} " ,
typeid ( source ) . name ( ) ) ;
2022-06-23 16:21:46 +00:00
transaction - > createHardLink (
2022-06-20 18:18:17 +00:00
fs : : path ( source_on_disk - > getRelativePath ( ) ) / from ,
2022-04-21 19:19:13 +00:00
fs : : path ( root_path ) / part_dir / to ) ;
}
2022-04-12 18:59:49 +00:00
bool DataPartStorageBuilderOnDisk : : exists ( ) const
{
return volume - > getDisk ( ) - > exists ( fs : : path ( root_path ) / part_dir ) ;
}
std : : string DataPartStorageBuilderOnDisk : : getFullPath ( ) const
{
return fs : : path ( volume - > getDisk ( ) - > getPath ( ) ) / root_path / part_dir ;
}
2022-06-20 18:18:17 +00:00
std : : string DataPartStorageBuilderOnDisk : : getRelativePath ( ) const
2022-04-21 19:19:13 +00:00
{
return fs : : path ( root_path ) / part_dir ;
}
2022-04-12 18:59:49 +00:00
void DataPartStorageBuilderOnDisk : : createDirectories ( )
{
2022-06-23 16:21:46 +00:00
transaction - > createDirectories ( fs : : path ( root_path ) / part_dir ) ;
2022-04-12 18:59:49 +00:00
}
2022-04-21 19:19:13 +00:00
void DataPartStorageBuilderOnDisk : : createProjection ( const std : : string & name )
{
2022-06-23 16:21:46 +00:00
transaction - > createDirectory ( fs : : path ( root_path ) / part_dir / name ) ;
2022-04-21 19:19:13 +00:00
}
2022-04-12 18:59:49 +00:00
ReservationPtr DataPartStorageBuilderOnDisk : : reserve ( UInt64 bytes )
{
auto res = volume - > reserve ( bytes ) ;
if ( ! res )
throw Exception ( ErrorCodes : : NOT_ENOUGH_SPACE , " Cannot reserve {}, not enough space " , ReadableSize ( bytes ) ) ;
return res ;
}
DataPartStorageBuilderPtr DataPartStorageBuilderOnDisk : : getProjection ( const std : : string & name ) const
{
return std : : make_shared < DataPartStorageBuilderOnDisk > ( volume , std : : string ( fs : : path ( root_path ) / part_dir ) , name ) ;
}
2022-04-19 19:34:41 +00:00
DataPartStoragePtr DataPartStorageBuilderOnDisk : : getStorage ( ) const
{
return std : : make_shared < DataPartStorageOnDisk > ( volume , root_path , part_dir ) ;
}
2022-06-23 16:21:46 +00:00
void DataPartStorageBuilderOnDisk : : commit ( )
{
transaction - > commit ( ) ;
}
2022-04-05 19:12:48 +00:00
}