2019-10-10 16:30:30 +00:00
# include "IMergeTreeDataPart.h"
# include <optional>
2020-02-27 16:47:40 +00:00
# include <Core/Defines.h>
# include <IO/HashingWriteBuffer.h>
# include <IO/ReadBufferFromString.h>
2019-10-10 16:30:30 +00:00
# include <IO/ReadHelpers.h>
# include <IO/WriteHelpers.h>
# include <Storages/MergeTree/MergeTreeData.h>
2020-03-19 16:37:55 +00:00
# include <Storages/MergeTree/localBackup.h>
2020-02-27 16:47:40 +00:00
# include <Common/StringUtils/StringUtils.h>
# include <Common/escapeForFileName.h>
2019-10-10 16:30:30 +00:00
# include <common/JSON.h>
2020-02-27 16:47:40 +00:00
# include <common/logger_useful.h>
2019-10-10 16:30:30 +00:00
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int DIRECTORY_ALREADY_EXISTS ;
extern const int CANNOT_READ_ALL_DATA ;
extern const int LOGICAL_ERROR ;
2019-10-10 16:30:30 +00:00
extern const int FILE_DOESNT_EXIST ;
extern const int NO_FILE_IN_DATA_PART ;
extern const int EXPECTED_END_OF_FILE ;
extern const int CORRUPTED_DATA ;
extern const int NOT_FOUND_EXPECTED_DATA_PART ;
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART ;
extern const int BAD_TTL_FILE ;
2019-11-18 15:18:50 +00:00
extern const int NOT_IMPLEMENTED ;
2019-10-10 16:30:30 +00:00
}
2020-02-27 08:37:52 +00:00
extern const char * DELETE_ON_DESTROY_MARKER_PATH ;
2020-02-27 16:47:40 +00:00
static std : : unique_ptr < ReadBufferFromFileBase > openForReading ( const DiskPtr & disk , const String & path )
2019-10-10 16:30:30 +00:00
{
2020-02-27 16:47:40 +00:00
return disk - > readFile ( path , std : : min ( size_t ( DBMS_DEFAULT_BUFFER_SIZE ) , disk - > getFileSize ( path ) ) ) ;
2019-10-10 16:30:30 +00:00
}
2020-02-27 16:47:40 +00:00
void IMergeTreeDataPart : : MinMaxIndex : : load ( const MergeTreeData & data , const DiskPtr & disk_ , const String & part_path )
2019-10-10 16:30:30 +00:00
{
size_t minmax_idx_size = data . minmax_idx_column_types . size ( ) ;
2020-03-10 14:56:55 +00:00
hyperrectangle . reserve ( minmax_idx_size ) ;
2019-10-10 16:30:30 +00:00
for ( size_t i = 0 ; i < minmax_idx_size ; + + i )
{
String file_name = part_path + " minmax_ " + escapeForFileName ( data . minmax_idx_columns [ i ] ) + " .idx " ;
2020-02-27 16:47:40 +00:00
auto file = openForReading ( disk_ , file_name ) ;
2020-01-14 13:23:51 +00:00
const DataTypePtr & data_type = data . minmax_idx_column_types [ i ] ;
2019-10-10 16:30:30 +00:00
Field min_val ;
2020-02-27 16:47:40 +00:00
data_type - > deserializeBinary ( min_val , * file ) ;
2019-10-10 16:30:30 +00:00
Field max_val ;
2020-02-27 16:47:40 +00:00
data_type - > deserializeBinary ( max_val , * file ) ;
2019-10-10 16:30:30 +00:00
2020-03-10 14:56:55 +00:00
hyperrectangle . emplace_back ( min_val , true , max_val , true ) ;
2019-10-10 16:30:30 +00:00
}
initialized = true ;
}
2020-02-27 16:47:40 +00:00
void IMergeTreeDataPart : : MinMaxIndex : : store (
const MergeTreeData & data , const DiskPtr & disk_ , const String & part_path , Checksums & out_checksums ) const
2019-10-10 16:30:30 +00:00
{
2020-02-27 16:47:40 +00:00
store ( data . minmax_idx_columns , data . minmax_idx_column_types , disk_ , part_path , out_checksums ) ;
2019-10-10 16:30:30 +00:00
}
2020-02-27 16:47:40 +00:00
void IMergeTreeDataPart : : MinMaxIndex : : store (
const Names & column_names ,
const DataTypes & data_types ,
const DiskPtr & disk_ ,
const String & part_path ,
Checksums & out_checksums ) const
2019-10-10 16:30:30 +00:00
{
if ( ! initialized )
throw Exception ( " Attempt to store uninitialized MinMax index for part " + part_path + " . This is a bug. " ,
ErrorCodes : : LOGICAL_ERROR ) ;
for ( size_t i = 0 ; i < column_names . size ( ) ; + + i )
{
String file_name = " minmax_ " + escapeForFileName ( column_names [ i ] ) + " .idx " ;
2020-01-14 13:23:51 +00:00
const DataTypePtr & data_type = data_types . at ( i ) ;
2019-10-10 16:30:30 +00:00
2020-02-27 16:47:40 +00:00
auto out = disk_ - > writeFile ( part_path + file_name ) ;
HashingWriteBuffer out_hashing ( * out ) ;
2020-03-10 14:56:55 +00:00
data_type - > serializeBinary ( hyperrectangle [ i ] . left , out_hashing ) ;
data_type - > serializeBinary ( hyperrectangle [ i ] . right , out_hashing ) ;
2019-10-10 16:30:30 +00:00
out_hashing . next ( ) ;
out_checksums . files [ file_name ] . file_size = out_hashing . count ( ) ;
out_checksums . files [ file_name ] . file_hash = out_hashing . getHash ( ) ;
}
}
void IMergeTreeDataPart : : MinMaxIndex : : update ( const Block & block , const Names & column_names )
{
if ( ! initialized )
2020-03-10 14:56:55 +00:00
hyperrectangle . reserve ( column_names . size ( ) ) ;
2019-10-10 16:30:30 +00:00
for ( size_t i = 0 ; i < column_names . size ( ) ; + + i )
{
2020-04-02 17:27:07 +00:00
FieldRef min_value ;
FieldRef max_value ;
2019-10-10 16:30:30 +00:00
const ColumnWithTypeAndName & column = block . getByName ( column_names [ i ] ) ;
column . column - > getExtremes ( min_value , max_value ) ;
if ( ! initialized )
2020-03-10 14:56:55 +00:00
hyperrectangle . emplace_back ( min_value , true , max_value , true ) ;
2019-10-10 16:30:30 +00:00
else
{
2020-03-10 14:56:55 +00:00
hyperrectangle [ i ] . left = std : : min ( hyperrectangle [ i ] . left , min_value ) ;
hyperrectangle [ i ] . right = std : : max ( hyperrectangle [ i ] . right , max_value ) ;
2019-10-10 16:30:30 +00:00
}
}
initialized = true ;
}
void IMergeTreeDataPart : : MinMaxIndex : : merge ( const MinMaxIndex & other )
{
if ( ! other . initialized )
return ;
if ( ! initialized )
{
2020-03-10 14:56:55 +00:00
hyperrectangle = other . hyperrectangle ;
2019-10-10 16:30:30 +00:00
initialized = true ;
}
else
{
2020-03-10 14:56:55 +00:00
for ( size_t i = 0 ; i < hyperrectangle . size ( ) ; + + i )
2019-10-10 16:30:30 +00:00
{
2020-03-10 14:56:55 +00:00
hyperrectangle [ i ] . left = std : : min ( hyperrectangle [ i ] . left , other . hyperrectangle [ i ] . left ) ;
hyperrectangle [ i ] . right = std : : max ( hyperrectangle [ i ] . right , other . hyperrectangle [ i ] . right ) ;
2019-10-10 16:30:30 +00:00
}
}
}
IMergeTreeDataPart : : IMergeTreeDataPart (
2020-05-09 21:24:15 +00:00
MergeTreeData & storage_ , const String & name_ , const VolumePtr & volume_ , const std : : optional < String > & relative_path_ , Type part_type_ )
2019-10-10 16:30:30 +00:00
: storage ( storage_ )
, name ( name_ )
, info ( MergeTreePartInfo : : fromPartName ( name_ , storage . format_version ) )
2020-05-09 21:24:15 +00:00
, volume ( volume_ )
2019-10-10 16:30:30 +00:00
, relative_path ( relative_path_ . value_or ( name_ ) )
2020-01-14 13:23:51 +00:00
, index_granularity_info ( storage_ , part_type_ )
, part_type ( part_type_ )
2019-10-16 18:27:53 +00:00
{
}
2019-10-10 16:30:30 +00:00
IMergeTreeDataPart : : IMergeTreeDataPart (
2020-02-27 16:47:40 +00:00
const MergeTreeData & storage_ ,
const String & name_ ,
const MergeTreePartInfo & info_ ,
2020-05-09 21:24:15 +00:00
const VolumePtr & volume_ ,
2020-02-27 16:47:40 +00:00
const std : : optional < String > & relative_path_ ,
Type part_type_ )
2019-10-10 16:30:30 +00:00
: storage ( storage_ )
, name ( name_ )
, info ( info_ )
2020-05-09 21:24:15 +00:00
, volume ( volume_ )
2019-10-10 16:30:30 +00:00
, relative_path ( relative_path_ . value_or ( name_ ) )
2020-01-14 13:23:51 +00:00
, index_granularity_info ( storage_ , part_type_ )
, part_type ( part_type_ )
2019-10-16 18:27:53 +00:00
{
}
2019-10-10 16:30:30 +00:00
String IMergeTreeDataPart : : getNewName ( const MergeTreePartInfo & new_part_info ) const
{
if ( storage . format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING )
{
/// NOTE: getting min and max dates from the part name (instead of part data) because we want
/// the merged part name be determined only by source part names.
/// It is simpler this way when the real min and max dates for the block range can change
/// (e.g. after an ALTER DELETE command).
DayNum min_date ;
DayNum max_date ;
MergeTreePartInfo : : parseMinMaxDatesFromPartName ( name , min_date , max_date ) ;
return new_part_info . getPartNameV0 ( min_date , max_date ) ;
}
else
return new_part_info . getPartName ( ) ;
}
2019-12-03 00:23:11 +00:00
std : : optional < size_t > IMergeTreeDataPart : : getColumnPosition ( const String & column_name ) const
2019-10-31 14:44:17 +00:00
{
2019-12-25 20:06:16 +00:00
auto it = column_name_to_position . find ( column_name ) ;
if ( it = = column_name_to_position . end ( ) )
2019-12-03 00:23:11 +00:00
return { } ;
2019-12-25 20:06:16 +00:00
return it - > second ;
2019-12-18 16:41:11 +00:00
}
2019-10-31 14:44:17 +00:00
2019-10-10 16:30:30 +00:00
DayNum IMergeTreeDataPart : : getMinDate ( ) const
{
if ( storage . minmax_idx_date_column_pos ! = - 1 & & minmax_idx . initialized )
2020-03-10 14:56:55 +00:00
return DayNum ( minmax_idx . hyperrectangle [ storage . minmax_idx_date_column_pos ] . left . get < UInt64 > ( ) ) ;
2019-10-10 16:30:30 +00:00
else
return DayNum ( ) ;
}
DayNum IMergeTreeDataPart : : getMaxDate ( ) const
{
if ( storage . minmax_idx_date_column_pos ! = - 1 & & minmax_idx . initialized )
2020-03-10 14:56:55 +00:00
return DayNum ( minmax_idx . hyperrectangle [ storage . minmax_idx_date_column_pos ] . right . get < UInt64 > ( ) ) ;
2019-10-10 16:30:30 +00:00
else
return DayNum ( ) ;
}
time_t IMergeTreeDataPart : : getMinTime ( ) const
{
if ( storage . minmax_idx_time_column_pos ! = - 1 & & minmax_idx . initialized )
2020-03-10 14:56:55 +00:00
return minmax_idx . hyperrectangle [ storage . minmax_idx_time_column_pos ] . left . get < UInt64 > ( ) ;
2019-10-10 16:30:30 +00:00
else
return 0 ;
}
time_t IMergeTreeDataPart : : getMaxTime ( ) const
{
if ( storage . minmax_idx_time_column_pos ! = - 1 & & minmax_idx . initialized )
2020-03-10 14:56:55 +00:00
return minmax_idx . hyperrectangle [ storage . minmax_idx_time_column_pos ] . right . get < UInt64 > ( ) ;
2019-10-10 16:30:30 +00:00
else
return 0 ;
}
2019-12-25 20:06:16 +00:00
void IMergeTreeDataPart : : setColumns ( const NamesAndTypesList & new_columns )
2019-11-18 12:22:27 +00:00
{
2019-12-25 20:06:16 +00:00
columns = new_columns ;
column_name_to_position . clear ( ) ;
column_name_to_position . reserve ( new_columns . size ( ) ) ;
size_t pos = 0 ;
2019-11-18 12:22:27 +00:00
for ( const auto & column : columns )
2019-12-25 20:06:16 +00:00
column_name_to_position . emplace ( column . name , pos + + ) ;
2019-11-18 12:22:27 +00:00
}
IMergeTreeDataPart : : ~ IMergeTreeDataPart ( ) = default ;
void IMergeTreeDataPart : : removeIfNeeded ( )
2019-10-10 16:30:30 +00:00
{
2019-10-31 14:44:17 +00:00
if ( state = = State : : DeleteOnDestroy | | is_temp )
{
try
{
2020-02-28 17:14:55 +00:00
auto path = getFullRelativePath ( ) ;
2019-10-31 14:44:17 +00:00
2020-05-09 21:24:15 +00:00
if ( ! volume - > getDisk ( ) - > exists ( path ) )
2019-10-31 14:44:17 +00:00
return ;
if ( is_temp )
{
2020-03-19 16:37:55 +00:00
String file_name = fileName ( relative_path ) ;
2019-10-10 16:30:30 +00:00
2019-10-31 14:44:17 +00:00
if ( file_name . empty ( ) )
throw Exception ( " relative_path " + relative_path + " of part " + name + " is invalid or not set " , ErrorCodes : : LOGICAL_ERROR ) ;
2019-10-10 16:30:30 +00:00
2019-10-31 14:44:17 +00:00
if ( ! startsWith ( file_name , " tmp " ) )
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( storage . log , " ~DataPart() should remove part {} but its name doesn't start with tmp. Too suspicious, keeping the part. " , path ) ;
2019-10-31 14:44:17 +00:00
return ;
}
}
2019-10-10 16:30:30 +00:00
2020-01-22 06:50:29 +00:00
remove ( ) ;
2019-11-19 09:38:17 +00:00
2019-11-19 10:08:08 +00:00
if ( state = = State : : DeleteOnDestroy )
2019-11-19 09:38:17 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_TRACE ( storage . log , " Removed part from old location {} " , path ) ;
2019-11-19 10:08:08 +00:00
}
2019-10-31 14:44:17 +00:00
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
}
2019-10-10 16:30:30 +00:00
}
UInt64 IMergeTreeDataPart : : getIndexSizeInBytes ( ) const
{
UInt64 res = 0 ;
for ( const ColumnPtr & column : index )
res + = column - > byteSize ( ) ;
return res ;
}
UInt64 IMergeTreeDataPart : : getIndexSizeInAllocatedBytes ( ) const
{
UInt64 res = 0 ;
for ( const ColumnPtr & column : index )
res + = column - > allocatedBytes ( ) ;
return res ;
}
String IMergeTreeDataPart : : stateToString ( IMergeTreeDataPart : : State state )
{
switch ( state )
{
case State : : Temporary :
return " Temporary " ;
case State : : PreCommitted :
return " PreCommitted " ;
case State : : Committed :
return " Committed " ;
case State : : Outdated :
return " Outdated " ;
case State : : Deleting :
return " Deleting " ;
case State : : DeleteOnDestroy :
return " DeleteOnDestroy " ;
}
__builtin_unreachable ( ) ;
}
String IMergeTreeDataPart : : stateString ( ) const
{
return stateToString ( state ) ;
}
void IMergeTreeDataPart : : assertState ( const std : : initializer_list < IMergeTreeDataPart : : State > & affordable_states ) const
{
if ( ! checkState ( affordable_states ) )
{
String states_str ;
for ( auto affordable_state : affordable_states )
states_str + = stateToString ( affordable_state ) + " " ;
throw Exception ( " Unexpected state of part " + getNameWithState ( ) + " . Expected: " + states_str , ErrorCodes : : NOT_FOUND_EXPECTED_DATA_PART ) ;
}
}
void IMergeTreeDataPart : : assertOnDisk ( ) const
{
if ( ! isStoredOnDisk ( ) )
2019-11-05 11:53:22 +00:00
throw Exception ( " Data part ' " + name + " ' with type ' "
2020-02-11 13:41:26 +00:00
+ getType ( ) . toString ( ) + " ' is not stored on disk " , ErrorCodes : : LOGICAL_ERROR ) ;
2019-10-10 16:30:30 +00:00
}
UInt64 IMergeTreeDataPart : : getMarksCount ( ) const
{
return index_granularity . getMarksCount ( ) ;
}
size_t IMergeTreeDataPart : : getFileSizeOrZero ( const String & file_name ) const
{
auto checksum = checksums . files . find ( file_name ) ;
if ( checksum = = checksums . files . end ( ) )
return 0 ;
return checksum - > second . file_size ;
}
2020-01-15 18:24:10 +00:00
String IMergeTreeDataPart : : getColumnNameWithMinumumCompressedSize ( ) const
{
const auto & storage_columns = storage . getColumns ( ) . getAllPhysical ( ) ;
2020-05-15 10:26:44 +00:00
auto alter_conversions = storage . getAlterConversionsForPart ( shared_from_this ( ) ) ;
std : : optional < std : : string > minimum_size_column ;
2020-01-15 18:24:10 +00:00
UInt64 minimum_size = std : : numeric_limits < UInt64 > : : max ( ) ;
for ( const auto & column : storage_columns )
{
2020-05-15 10:26:44 +00:00
auto column_name = column . name ;
auto column_type = column . type ;
if ( alter_conversions . isColumnRenamed ( column . name ) )
column_name = alter_conversions . getColumnOldName ( column . name ) ;
if ( ! hasColumnFiles ( column_name , * column_type ) )
2020-01-15 18:24:10 +00:00
continue ;
2020-05-15 10:26:44 +00:00
const auto size = getColumnSize ( column_name , * column_type ) . data_compressed ;
2020-01-15 18:24:10 +00:00
if ( size < minimum_size )
{
minimum_size = size ;
2020-05-15 10:26:44 +00:00
minimum_size_column = column_name ;
2020-01-15 18:24:10 +00:00
}
}
if ( ! minimum_size_column )
throw Exception ( " Could not find a column of minimum size in MergeTree, part " + getFullPath ( ) , ErrorCodes : : LOGICAL_ERROR ) ;
return * minimum_size_column ;
}
2019-10-10 16:30:30 +00:00
String IMergeTreeDataPart : : getFullPath ( ) const
{
2019-11-18 12:22:27 +00:00
assertOnDisk ( ) ;
2019-10-10 16:30:30 +00:00
if ( relative_path . empty ( ) )
throw Exception ( " Part relative_path cannot be empty. It's bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-05-09 21:24:15 +00:00
return storage . getFullPathOnDisk ( volume - > getDisk ( ) ) + relative_path + " / " ;
2019-10-10 16:30:30 +00:00
}
2020-02-27 16:47:40 +00:00
String IMergeTreeDataPart : : getFullRelativePath ( ) const
{
assertOnDisk ( ) ;
if ( relative_path . empty ( ) )
throw Exception ( " Part relative_path cannot be empty. It's bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
return storage . relative_data_path + relative_path + " / " ;
}
2019-11-18 15:18:50 +00:00
void IMergeTreeDataPart : : loadColumnsChecksumsIndexes ( bool require_columns_checksums , bool check_consistency )
2019-10-31 14:44:17 +00:00
{
assertOnDisk ( ) ;
/// Memory should not be limited during ATTACH TABLE query.
/// This is already true at the server startup but must be also ensured for manual table ATTACH.
/// Motivation: memory for index is shared between queries - not belong to the query itself.
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock ( ) ;
loadColumns ( require_columns_checksums ) ;
loadChecksums ( require_columns_checksums ) ;
loadIndexGranularity ( ) ;
2020-03-29 00:20:38 +00:00
calculateColumnsSizesOnDisk ( ) ;
2020-02-27 16:47:40 +00:00
loadIndex ( ) ; /// Must be called after loadIndexGranularity as it uses the value of `index_granularity`
2019-10-31 14:44:17 +00:00
loadRowsCount ( ) ; /// Must be called after loadIndex() as it uses the value of `index_granularity`.
loadPartitionAndMinMaxIndex ( ) ;
loadTTLInfos ( ) ;
2019-11-18 15:18:50 +00:00
if ( check_consistency )
checkConsistency ( require_columns_checksums ) ;
2019-10-31 14:44:17 +00:00
}
void IMergeTreeDataPart : : loadIndexGranularity ( )
{
2020-02-11 13:41:26 +00:00
throw Exception ( " Method 'loadIndexGranularity' is not implemented for part with type " + getType ( ) . toString ( ) , ErrorCodes : : NOT_IMPLEMENTED ) ;
2019-10-31 14:44:17 +00:00
}
void IMergeTreeDataPart : : loadIndex ( )
{
/// It can be empty in case of mutations
if ( ! index_granularity . isInitialized ( ) )
throw Exception ( " Index granularity is not loaded before index loading " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-05-20 18:11:38 +00:00
const auto & primary_key = storage . getPrimaryKey ( ) ;
2020-05-21 19:46:03 +00:00
size_t key_size = primary_key . column_names . size ( ) ;
2019-10-31 14:44:17 +00:00
if ( key_size )
{
MutableColumns loaded_index ;
loaded_index . resize ( key_size ) ;
for ( size_t i = 0 ; i < key_size ; + + i )
{
2020-05-20 18:11:38 +00:00
loaded_index [ i ] = primary_key . data_types [ i ] - > createColumn ( ) ;
2019-10-31 14:44:17 +00:00
loaded_index [ i ] - > reserve ( index_granularity . getMarksCount ( ) ) ;
}
2020-02-27 16:47:40 +00:00
String index_path = getFullRelativePath ( ) + " primary.idx " ;
2020-05-09 21:24:15 +00:00
auto index_file = openForReading ( volume - > getDisk ( ) , index_path ) ;
2019-10-31 14:44:17 +00:00
2020-02-27 16:47:40 +00:00
for ( size_t i = 0 ; i < index_granularity . getMarksCount ( ) ; + + i ) //-V756
2019-10-31 14:44:17 +00:00
for ( size_t j = 0 ; j < key_size ; + + j )
2020-05-20 18:11:38 +00:00
primary_key . data_types [ j ] - > deserializeBinary ( * loaded_index [ j ] , * index_file ) ;
2019-10-31 14:44:17 +00:00
for ( size_t i = 0 ; i < key_size ; + + i )
{
loaded_index [ i ] - > protect ( ) ;
if ( loaded_index [ i ] - > size ( ) ! = index_granularity . getMarksCount ( ) )
throw Exception ( " Cannot read all data from index file " + index_path
+ " (expected size: " + toString ( index_granularity . getMarksCount ( ) ) + " , read: " + toString ( loaded_index [ i ] - > size ( ) ) + " ) " ,
ErrorCodes : : CANNOT_READ_ALL_DATA ) ;
}
2020-02-27 16:47:40 +00:00
if ( ! index_file - > eof ( ) )
2020-05-09 21:24:15 +00:00
throw Exception ( " Index file " + fullPath ( volume - > getDisk ( ) , index_path ) + " is unexpectedly long " , ErrorCodes : : EXPECTED_END_OF_FILE ) ;
2019-10-31 14:44:17 +00:00
index . assign ( std : : make_move_iterator ( loaded_index . begin ( ) ) , std : : make_move_iterator ( loaded_index . end ( ) ) ) ;
}
}
void IMergeTreeDataPart : : loadPartitionAndMinMaxIndex ( )
{
if ( storage . format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING )
{
DayNum min_date ;
DayNum max_date ;
MergeTreePartInfo : : parseMinMaxDatesFromPartName ( name , min_date , max_date ) ;
const auto & date_lut = DateLUT : : instance ( ) ;
partition = MergeTreePartition ( date_lut . toNumYYYYMM ( min_date ) ) ;
minmax_idx = MinMaxIndex ( min_date , max_date ) ;
}
else
{
2020-02-27 16:47:40 +00:00
String path = getFullRelativePath ( ) ;
2020-05-09 21:24:15 +00:00
partition . load ( storage , volume - > getDisk ( ) , path ) ;
2019-10-31 14:44:17 +00:00
if ( ! isEmpty ( ) )
2020-05-09 21:24:15 +00:00
minmax_idx . load ( storage , volume - > getDisk ( ) , path ) ;
2019-10-31 14:44:17 +00:00
}
2020-05-20 12:16:55 +00:00
String calculated_partition_id = partition . getID ( storage . getPartitionKey ( ) . sample_block ) ;
2019-10-31 14:44:17 +00:00
if ( calculated_partition_id ! = info . partition_id )
throw Exception (
" While loading part " + getFullPath ( ) + " : calculated partition ID: " + calculated_partition_id
+ " differs from partition ID in part name: " + info . partition_id ,
ErrorCodes : : CORRUPTED_DATA ) ;
}
void IMergeTreeDataPart : : loadChecksums ( bool require )
{
2020-02-27 16:47:40 +00:00
String path = getFullRelativePath ( ) + " checksums.txt " ;
2020-05-09 21:24:15 +00:00
if ( volume - > getDisk ( ) - > exists ( path ) )
2019-10-31 14:44:17 +00:00
{
2020-05-09 21:24:15 +00:00
auto buf = openForReading ( volume - > getDisk ( ) , path ) ;
2020-02-27 16:47:40 +00:00
if ( checksums . read ( * buf ) )
2019-10-31 14:44:17 +00:00
{
2020-02-27 16:47:40 +00:00
assertEOF ( * buf ) ;
2019-10-31 14:44:17 +00:00
bytes_on_disk = checksums . getTotalSizeOnDisk ( ) ;
}
else
2020-05-09 21:24:15 +00:00
bytes_on_disk = calculateTotalSizeOnDisk ( volume - > getDisk ( ) , getFullRelativePath ( ) ) ;
2019-10-31 14:44:17 +00:00
}
else
{
if ( require )
throw Exception ( " No checksums.txt in part " + name , ErrorCodes : : NO_FILE_IN_DATA_PART ) ;
2020-05-09 21:24:15 +00:00
bytes_on_disk = calculateTotalSizeOnDisk ( volume - > getDisk ( ) , getFullRelativePath ( ) ) ;
2019-10-31 14:44:17 +00:00
}
}
void IMergeTreeDataPart : : loadRowsCount ( )
{
2020-02-27 16:47:40 +00:00
String path = getFullRelativePath ( ) + " count.txt " ;
2019-10-31 14:44:17 +00:00
if ( index_granularity . empty ( ) )
{
rows_count = 0 ;
}
2020-01-22 14:10:35 +00:00
else if ( storage . format_version > = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING | | part_type = = Type : : COMPACT )
2019-10-31 14:44:17 +00:00
{
2020-05-09 21:24:15 +00:00
if ( ! volume - > getDisk ( ) - > exists ( path ) )
2019-10-31 14:44:17 +00:00
throw Exception ( " No count.txt in part " + name , ErrorCodes : : NO_FILE_IN_DATA_PART ) ;
2020-05-09 21:24:15 +00:00
auto buf = openForReading ( volume - > getDisk ( ) , path ) ;
2020-02-27 16:47:40 +00:00
readIntText ( rows_count , * buf ) ;
assertEOF ( * buf ) ;
2019-10-31 14:44:17 +00:00
}
else
{
for ( const NameAndTypePair & column : columns )
{
ColumnPtr column_col = column . type - > createColumn ( ) ;
if ( ! column_col - > isFixedAndContiguous ( ) | | column_col - > lowCardinality ( ) )
continue ;
2019-12-03 00:23:11 +00:00
size_t column_size = getColumnSize ( column . name , * column . type ) . data_uncompressed ;
2019-10-31 14:44:17 +00:00
if ( ! column_size )
continue ;
size_t sizeof_field = column_col - > sizeOfValueIfFixed ( ) ;
rows_count = column_size / sizeof_field ;
if ( column_size % sizeof_field ! = 0 )
{
throw Exception (
" Uncompressed size of column " + column . name + " ( " + toString ( column_size )
+ " ) is not divisible by the size of value ( " + toString ( sizeof_field ) + " ) " ,
ErrorCodes : : LOGICAL_ERROR ) ;
}
size_t last_mark_index_granularity = index_granularity . getLastNonFinalMarkRows ( ) ;
size_t rows_approx = index_granularity . getTotalRows ( ) ;
if ( ! ( rows_count < = rows_approx & & rows_approx < rows_count + last_mark_index_granularity ) )
throw Exception (
" Unexpected size of column " + column . name + " : " + toString ( rows_count ) + " rows, expected "
+ toString ( rows_approx ) + " +- " + toString ( last_mark_index_granularity ) + " rows according to the index " ,
ErrorCodes : : LOGICAL_ERROR ) ;
return ;
}
throw Exception ( " Data part doesn't contain fixed size column (even Date column) " , ErrorCodes::LOGICAL_ERROR) ;
}
}
void IMergeTreeDataPart : : loadTTLInfos ( )
{
2020-02-27 16:47:40 +00:00
String path = getFullRelativePath ( ) + " ttl.txt " ;
2020-05-09 21:24:15 +00:00
if ( volume - > getDisk ( ) - > exists ( path ) )
2019-10-31 14:44:17 +00:00
{
2020-05-09 21:24:15 +00:00
auto in = openForReading ( volume - > getDisk ( ) , path ) ;
2020-02-27 16:47:40 +00:00
assertString ( " ttl format version: " , * in ) ;
2019-10-31 14:44:17 +00:00
size_t format_version ;
2020-02-27 16:47:40 +00:00
readText ( format_version , * in ) ;
assertChar ( ' \n ' , * in ) ;
2019-10-31 14:44:17 +00:00
if ( format_version = = 1 )
{
try
{
2020-02-27 16:47:40 +00:00
ttl_infos . read ( * in ) ;
2019-10-31 14:44:17 +00:00
}
catch ( const JSONException & )
{
throw Exception ( " Error while parsing file ttl.txt in part: " + name , ErrorCodes : : BAD_TTL_FILE ) ;
}
}
else
throw Exception ( " Unknown ttl format version: " + toString ( format_version ) , ErrorCodes : : BAD_TTL_FILE ) ;
}
}
void IMergeTreeDataPart : : loadColumns ( bool require )
{
2020-02-27 16:47:40 +00:00
String path = getFullRelativePath ( ) + " columns.txt " ;
2020-05-09 21:24:15 +00:00
if ( ! volume - > getDisk ( ) - > exists ( path ) )
2019-10-31 14:44:17 +00:00
{
2020-01-15 18:24:10 +00:00
/// We can get list of columns only from columns.txt in compact parts.
if ( require | | part_type = = Type : : COMPACT )
2019-10-31 14:44:17 +00:00
throw Exception ( " No columns.txt in part " + name , ErrorCodes : : NO_FILE_IN_DATA_PART ) ;
/// If there is no file with a list of columns, write it down.
for ( const NameAndTypePair & column : storage . getColumns ( ) . getAllPhysical ( ) )
2020-05-09 21:24:15 +00:00
if ( volume - > getDisk ( ) - > exists ( getFullRelativePath ( ) + getFileNameForColumn ( column ) + " .bin " ) )
2019-10-31 14:44:17 +00:00
columns . push_back ( column ) ;
if ( columns . empty ( ) )
throw Exception ( " No columns in part " + name , ErrorCodes : : NO_FILE_IN_DATA_PART ) ;
{
2020-05-09 21:24:15 +00:00
auto buf = volume - > getDisk ( ) - > writeFile ( path + " .tmp " , 4096 ) ;
2020-02-27 16:47:40 +00:00
columns . writeText ( * buf ) ;
2019-10-31 14:44:17 +00:00
}
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > moveFile ( path + " .tmp " , path ) ;
2019-12-09 21:21:17 +00:00
}
else
{
2020-05-09 21:24:15 +00:00
columns . readText ( * volume - > getDisk ( ) - > readFile ( path ) ) ;
2019-10-31 14:44:17 +00:00
}
2019-12-25 20:06:16 +00:00
size_t pos = 0 ;
for ( const auto & column : columns )
column_name_to_position . emplace ( column . name , pos + + ) ;
2019-10-31 14:44:17 +00:00
}
2020-02-28 17:14:55 +00:00
UInt64 IMergeTreeDataPart : : calculateTotalSizeOnDisk ( const DiskPtr & disk_ , const String & from )
2019-10-10 16:30:30 +00:00
{
2020-02-28 17:14:55 +00:00
if ( disk_ - > isFile ( from ) )
return disk_ - > getFileSize ( from ) ;
2019-10-10 16:30:30 +00:00
std : : vector < std : : string > files ;
2020-02-28 17:14:55 +00:00
disk_ - > listFiles ( from , files ) ;
2019-10-10 16:30:30 +00:00
UInt64 res = 0 ;
for ( const auto & file : files )
2020-02-28 17:14:55 +00:00
res + = calculateTotalSizeOnDisk ( disk_ , from + file ) ;
2019-10-10 16:30:30 +00:00
return res ;
}
void IMergeTreeDataPart : : renameTo ( const String & new_relative_path , bool remove_new_dir_if_exists ) const
{
2019-10-31 14:44:17 +00:00
assertOnDisk ( ) ;
2020-02-27 16:47:40 +00:00
String from = getFullRelativePath ( ) ;
String to = storage . relative_data_path + new_relative_path + " / " ;
2019-10-10 16:30:30 +00:00
2020-05-09 21:24:15 +00:00
if ( ! volume - > getDisk ( ) - > exists ( from ) )
throw Exception ( " Part directory " + fullPath ( volume - > getDisk ( ) , from ) + " doesn't exist. Most likely it is logical error. " , ErrorCodes : : FILE_DOESNT_EXIST ) ;
2019-10-10 16:30:30 +00:00
2020-05-09 21:24:15 +00:00
if ( volume - > getDisk ( ) - > exists ( to ) )
2019-10-10 16:30:30 +00:00
{
if ( remove_new_dir_if_exists )
{
Names files ;
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > listFiles ( to , files ) ;
2019-10-10 16:30:30 +00:00
2020-05-23 22:24:01 +00:00
LOG_WARNING ( storage . log , " Part directory {} already exists and contains {} files. Removing it. " , fullPath ( volume - > getDisk ( ) , to ) , files . size ( ) ) ;
2019-10-10 16:30:30 +00:00
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > removeRecursive ( to ) ;
2019-10-10 16:30:30 +00:00
}
else
{
2020-05-09 21:24:15 +00:00
throw Exception ( " Part directory " + fullPath ( volume - > getDisk ( ) , to ) + " already exists " , ErrorCodes : : DIRECTORY_ALREADY_EXISTS ) ;
2019-10-10 16:30:30 +00:00
}
}
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > setLastModified ( from , Poco : : Timestamp : : fromEpochTime ( time ( nullptr ) ) ) ;
volume - > getDisk ( ) - > moveFile ( from , to ) ;
2019-10-10 16:30:30 +00:00
relative_path = new_relative_path ;
}
2019-12-03 14:33:56 +00:00
2019-10-31 14:44:17 +00:00
void IMergeTreeDataPart : : remove ( ) const
{
if ( ! isStoredOnDisk ( ) )
return ;
if ( relative_path . empty ( ) )
throw Exception ( " Part relative_path cannot be empty. This is bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
/** Atomic directory removal:
* - rename directory to temporary name ;
* - remove it recursive .
*
* For temporary name we use " delete_tmp_ " prefix .
*
* NOTE : We cannot use " tmp_delete_ " prefix , because there is a second thread ,
* that calls " clearOldTemporaryDirectories " and removes all directories , that begin with " tmp_ " and are old enough .
* But when we removing data part , it can be old enough . And rename doesn ' t change mtime .
* And a race condition can happen that will lead to " File not found " error here .
*/
2020-03-19 16:37:55 +00:00
String from = storage . relative_data_path + relative_path ;
String to = storage . relative_data_path + " delete_tmp_ " + name ;
2019-10-31 14:44:17 +00:00
// TODO directory delete_tmp_<name> is never removed if server crashes before returning from this function
2020-05-09 21:24:15 +00:00
if ( volume - > getDisk ( ) - > exists ( to ) )
2019-10-31 14:44:17 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( storage . log , " Directory {} (to which part must be renamed before removing) already exists. Most likely this is due to unclean restart. Removing it. " , fullPath ( volume - > getDisk ( ) , to ) ) ;
2019-10-31 14:44:17 +00:00
try
{
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > removeRecursive ( to + " / " ) ;
2019-10-31 14:44:17 +00:00
}
catch ( . . . )
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( storage . log , " Cannot recursively remove directory {}. Exception: {} " , fullPath ( volume - > getDisk ( ) , to ) , getCurrentExceptionMessage ( false ) ) ;
2019-10-31 14:44:17 +00:00
throw ;
}
}
try
{
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > moveFile ( from , to ) ;
2019-10-31 14:44:17 +00:00
}
catch ( const Poco : : FileNotFoundException & )
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( storage . 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 ( volume - > getDisk ( ) , to ) ) ;
2019-10-31 14:44:17 +00:00
return ;
}
2020-04-13 00:42:23 +00:00
if ( checksums . empty ( ) )
2019-10-31 14:44:17 +00:00
{
2020-04-13 00:42:23 +00:00
/// If the part is not completely written, we cannot use fast path by listing files.
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > removeRecursive ( to + " / " ) ;
2019-10-31 14:44:17 +00:00
}
2020-04-13 00:42:23 +00:00
else
2019-10-31 14:44:17 +00:00
{
2020-04-13 00:42:23 +00:00
try
{
/// Remove each expected file in directory, then remove directory itself.
# if !__clang__
# pragma GCC diagnostic push
# pragma GCC diagnostic ignored "-Wunused-variable"
# endif
for ( const auto & [ file , _ ] : checksums . files )
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > remove ( to + " / " + file ) ;
2020-04-13 00:42:23 +00:00
# if !__clang__
# pragma GCC diagnostic pop
# endif
for ( const auto & file : { " checksums.txt " , " columns.txt " } )
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > remove ( to + " / " + file ) ;
volume - > getDisk ( ) - > removeIfExists ( to + " / " + DELETE_ON_DESTROY_MARKER_PATH ) ;
2020-04-13 00:42:23 +00:00
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > remove ( to ) ;
2020-04-13 00:42:23 +00:00
}
catch ( . . . )
{
/// Recursive directory removal does many excessive "stat" syscalls under the hood.
2019-10-31 14:44:17 +00:00
2020-05-23 22:24:01 +00:00
LOG_ERROR ( storage . log , " Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {} " , fullPath ( volume - > getDisk ( ) , to ) , getCurrentExceptionMessage ( false ) ) ;
2019-10-31 14:44:17 +00:00
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > removeRecursive ( to + " / " ) ;
2020-04-13 00:42:23 +00:00
}
2019-10-31 14:44:17 +00:00
}
}
2019-10-10 16:30:30 +00:00
String IMergeTreeDataPart : : getRelativePathForDetachedPart ( const String & prefix ) const
{
/// Do not allow underscores in the prefix because they are used as separators.
assert ( prefix . find_first_of ( ' _ ' ) = = String : : npos ) ;
String res ;
/** If you need to detach a part, and directory into which we want to rename it already exists,
* we will rename to the directory with the name to which the suffix is added in the form of " _tryN " .
* This is done only in the case of ` to_detached ` , because it is assumed that in this case the exact name does not matter .
* No more than 10 attempts are made so that there are not too many junk directories left .
*/
for ( int try_no = 0 ; try_no < 10 ; try_no + + )
{
2020-02-27 16:47:40 +00:00
res = " detached/ " + ( prefix . empty ( ) ? " " : prefix + " _ " ) + name + ( try_no ? " _try " + DB : : toString ( try_no ) : " " ) ;
2019-10-10 16:30:30 +00:00
2020-05-09 21:24:15 +00:00
if ( ! volume - > getDisk ( ) - > exists ( getFullRelativePath ( ) + res ) )
2019-10-10 16:30:30 +00:00
return res ;
2020-05-23 22:24:01 +00:00
LOG_WARNING ( storage . log , " Directory {} (to detach to) already exists. Will detach to directory with '_tryN' suffix. " , res ) ;
2019-10-10 16:30:30 +00:00
}
return res ;
}
void IMergeTreeDataPart : : renameToDetached ( const String & prefix ) const
{
assertOnDisk ( ) ;
renameTo ( getRelativePathForDetachedPart ( prefix ) ) ;
}
void IMergeTreeDataPart : : makeCloneInDetached ( const String & prefix ) const
{
assertOnDisk ( ) ;
2020-05-23 22:24:01 +00:00
LOG_INFO ( storage . log , " Detaching {} " , relative_path ) ;
2019-10-10 16:30:30 +00:00
2020-02-27 08:37:52 +00:00
String destination_path = storage . relative_data_path + getRelativePathForDetachedPart ( prefix ) ;
2019-10-10 16:30:30 +00:00
/// Backup is not recursive (max_level is 0), so do not copy inner directories
2020-05-09 21:24:15 +00:00
localBackup ( volume - > getDisk ( ) , getFullRelativePath ( ) , destination_path , 0 ) ;
volume - > getDisk ( ) - > removeIfExists ( destination_path + " / " + DELETE_ON_DESTROY_MARKER_PATH ) ;
2019-10-10 16:30:30 +00:00
}
2019-12-19 13:10:57 +00:00
void IMergeTreeDataPart : : makeCloneOnDiskDetached ( const ReservationPtr & reservation ) const
2019-10-10 16:30:30 +00:00
{
assertOnDisk ( ) ;
2019-12-19 13:10:57 +00:00
auto reserved_disk = reservation - > getDisk ( ) ;
2020-05-09 21:24:15 +00:00
if ( reserved_disk - > getName ( ) = = volume - > getDisk ( ) - > getName ( ) )
throw Exception ( " Can not clone data part " + name + " to same disk " + volume - > getDisk ( ) - > getName ( ) , ErrorCodes : : LOGICAL_ERROR ) ;
2019-10-10 16:30:30 +00:00
2020-03-19 16:37:55 +00:00
String path_to_clone = storage . relative_data_path + " detached/ " ;
2019-10-10 16:30:30 +00:00
2020-03-19 16:37:55 +00:00
if ( reserved_disk - > exists ( path_to_clone + relative_path ) )
throw Exception ( " Path " + fullPath ( reserved_disk , path_to_clone + relative_path ) + " already exists. Can not clone " , ErrorCodes : : DIRECTORY_ALREADY_EXISTS ) ;
reserved_disk - > createDirectory ( path_to_clone ) ;
2019-10-10 16:30:30 +00:00
2020-05-09 21:24:15 +00:00
volume - > getDisk ( ) - > copy ( getFullRelativePath ( ) , reserved_disk , path_to_clone ) ;
volume - > getDisk ( ) - > removeIfExists ( path_to_clone + " / " + DELETE_ON_DESTROY_MARKER_PATH ) ;
2019-10-10 16:30:30 +00:00
}
2020-01-15 19:16:56 +00:00
void IMergeTreeDataPart : : checkConsistencyBase ( ) const
{
2020-02-27 16:47:40 +00:00
String path = getFullRelativePath ( ) ;
2020-01-15 19:16:56 +00:00
if ( ! checksums . empty ( ) )
{
2020-05-21 19:46:03 +00:00
if ( storage . hasPrimaryKey ( ) & & ! checksums . files . count ( " primary.idx " ) )
2020-01-15 19:16:56 +00:00
throw Exception ( " No checksum for primary.idx " , ErrorCodes : : NO_FILE_IN_DATA_PART ) ;
if ( storage . format_version > = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING )
{
if ( ! checksums . files . count ( " count.txt " ) )
throw Exception ( " No checksum for count.txt " , ErrorCodes : : NO_FILE_IN_DATA_PART ) ;
2020-05-20 12:16:55 +00:00
if ( storage . hasPartitionKey ( ) & & ! checksums . files . count ( " partition.dat " ) )
2020-01-15 19:16:56 +00:00
throw Exception ( " No checksum for partition.dat " , ErrorCodes : : NO_FILE_IN_DATA_PART ) ;
if ( ! isEmpty ( ) )
{
for ( const String & col_name : storage . minmax_idx_columns )
{
if ( ! checksums . files . count ( " minmax_ " + escapeForFileName ( col_name ) + " .idx " ) )
throw Exception ( " No minmax idx file checksum for column " + col_name , ErrorCodes : : NO_FILE_IN_DATA_PART ) ;
}
}
}
2020-05-09 21:24:15 +00:00
checksums . checkSizes ( volume - > getDisk ( ) , path ) ;
2020-01-15 19:16:56 +00:00
}
else
{
2020-02-27 17:57:49 +00:00
auto check_file_not_empty = [ & path ] ( const DiskPtr & disk_ , const String & file_path )
{
2020-02-27 16:47:40 +00:00
UInt64 file_size ;
if ( ! disk_ - > exists ( file_path ) | | ( file_size = disk_ - > getFileSize ( file_path ) ) = = 0 )
throw Exception ( " Part " + fullPath ( disk_ , path ) + " is broken: " + fullPath ( disk_ , file_path ) + " is empty " , ErrorCodes : : BAD_SIZE_OF_FILE_IN_DATA_PART ) ;
return file_size ;
2020-01-15 19:16:56 +00:00
} ;
/// Check that the primary key index is not empty.
2020-05-21 19:46:03 +00:00
if ( storage . hasPrimaryKey ( ) )
2020-05-09 21:24:15 +00:00
check_file_not_empty ( volume - > getDisk ( ) , path + " primary.idx " ) ;
2020-01-15 19:16:56 +00:00
if ( storage . format_version > = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING )
{
2020-05-09 21:24:15 +00:00
check_file_not_empty ( volume - > getDisk ( ) , path + " count.txt " ) ;
2020-01-15 19:16:56 +00:00
2020-05-20 12:16:55 +00:00
if ( storage . hasPartitionKey ( ) )
2020-05-09 21:24:15 +00:00
check_file_not_empty ( volume - > getDisk ( ) , path + " partition.dat " ) ;
2020-01-15 19:16:56 +00:00
for ( const String & col_name : storage . minmax_idx_columns )
2020-05-09 21:24:15 +00:00
check_file_not_empty ( volume - > getDisk ( ) , path + " minmax_ " + escapeForFileName ( col_name ) + " .idx " ) ;
2020-01-15 19:16:56 +00:00
}
}
}
2020-03-23 12:19:43 +00:00
void IMergeTreeDataPart : : calculateColumnsSizesOnDisk ( )
{
if ( getColumns ( ) . empty ( ) | | checksums . empty ( ) )
throw Exception ( " Cannot calculate columns sizes when columns or checksums are not initialized " , ErrorCodes : : LOGICAL_ERROR ) ;
calculateEachColumnSizesOnDisk ( columns_sizes , total_columns_size ) ;
}
ColumnSize IMergeTreeDataPart : : getColumnSize ( const String & column_name , const IDataType & /* type */ ) const
{
/// For some types of parts columns_size maybe not calculated
auto it = columns_sizes . find ( column_name ) ;
if ( it ! = columns_sizes . end ( ) )
return it - > second ;
return ColumnSize { } ;
}
void IMergeTreeDataPart : : accumulateColumnSizes ( ColumnToSize & column_to_size ) const
{
2020-03-23 15:43:20 +00:00
for ( const auto & [ column_name , size ] : columns_sizes )
column_to_size [ column_name ] = size . data_compressed ;
2020-03-23 12:19:43 +00:00
}
2019-12-18 13:09:58 +00:00
bool isCompactPart ( const MergeTreeDataPartPtr & data_part )
{
return ( data_part & & data_part - > getType ( ) = = MergeTreeDataPartType : : COMPACT ) ;
}
bool isWidePart ( const MergeTreeDataPartPtr & data_part )
{
return ( data_part & & data_part - > getType ( ) = = MergeTreeDataPartType : : WIDE ) ;
}
2019-10-10 16:30:30 +00:00
}