2019-10-19 16:49:36 +00:00
# include <Storages/MergeTree/IMergeTreeDataPartWriter.h>
2020-02-27 16:47:40 +00:00
# include <utility>
2019-10-19 16:49:36 +00:00
namespace DB
{
2020-02-25 18:10:48 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR ;
}
2019-10-21 15:33:59 +00:00
2019-11-05 11:53:22 +00:00
namespace
{
constexpr auto INDEX_FILE_EXTENSION = " .idx " ;
}
2019-10-19 16:49:36 +00:00
2020-02-19 14:07:36 +00:00
void IMergeTreeDataPartWriter : : Stream : : finalize ( )
2019-10-19 16:49:36 +00:00
{
compressed . next ( ) ;
plain_file - > next ( ) ;
marks . next ( ) ;
}
2020-02-19 14:07:36 +00:00
void IMergeTreeDataPartWriter : : Stream : : sync ( )
2019-10-19 16:49:36 +00:00
{
plain_file - > sync ( ) ;
2020-02-27 16:47:40 +00:00
marks_file - > sync ( ) ;
2019-10-19 16:49:36 +00:00
}
2020-02-19 14:07:36 +00:00
IMergeTreeDataPartWriter : : Stream : : Stream (
2019-10-19 16:49:36 +00:00
const String & escaped_column_name_ ,
2020-02-27 16:47:40 +00:00
DiskPtr disk_ ,
2019-10-19 16:49:36 +00:00
const String & data_path_ ,
const std : : string & data_file_extension_ ,
const std : : string & marks_path_ ,
const std : : string & marks_file_extension_ ,
const CompressionCodecPtr & compression_codec_ ,
size_t max_compress_block_size_ ,
size_t estimated_size_ ,
size_t aio_threshold_ ) :
escaped_column_name ( escaped_column_name_ ) ,
data_file_extension { data_file_extension_ } ,
marks_file_extension { marks_file_extension_ } ,
2020-02-27 16:47:40 +00:00
plain_file ( disk_ - > writeFile ( data_path_ + data_file_extension , max_compress_block_size_ , WriteMode : : Rewrite , estimated_size_ , aio_threshold_ ) ) ,
2019-10-19 16:49:36 +00:00
plain_hashing ( * plain_file ) , compressed_buf ( plain_hashing , compression_codec_ ) , compressed ( compressed_buf ) ,
2020-02-27 16:47:40 +00:00
marks_file ( disk_ - > writeFile ( marks_path_ + marks_file_extension , 4096 , WriteMode : : Rewrite ) ) , marks ( * marks_file )
2019-10-19 16:49:36 +00:00
{
}
2020-02-19 14:07:36 +00:00
void IMergeTreeDataPartWriter : : Stream : : addToChecksums ( MergeTreeData : : DataPart : : Checksums & checksums )
2019-10-21 15:33:59 +00:00
{
String name = escaped_column_name ;
2019-10-19 16:49:36 +00:00
2019-10-21 15:33:59 +00:00
checksums . files [ name + data_file_extension ] . is_compressed = true ;
checksums . files [ name + data_file_extension ] . uncompressed_size = compressed . count ( ) ;
checksums . files [ name + data_file_extension ] . uncompressed_hash = compressed . getHash ( ) ;
checksums . files [ name + data_file_extension ] . file_size = plain_hashing . count ( ) ;
checksums . files [ name + data_file_extension ] . file_hash = plain_hashing . getHash ( ) ;
2019-10-19 16:49:36 +00:00
2019-10-21 15:33:59 +00:00
checksums . files [ name + marks_file_extension ] . file_size = marks . count ( ) ;
checksums . files [ name + marks_file_extension ] . file_hash = marks . getHash ( ) ;
}
2019-10-19 16:49:36 +00:00
2019-11-05 11:53:22 +00:00
IMergeTreeDataPartWriter : : IMergeTreeDataPartWriter (
2020-02-27 16:47:40 +00:00
DiskPtr disk_ ,
2019-11-05 11:53:22 +00:00
const String & part_path_ ,
const MergeTreeData & storage_ ,
const NamesAndTypesList & columns_list_ ,
2019-12-18 16:41:11 +00:00
const std : : vector < MergeTreeIndexPtr > & indices_to_recalc_ ,
2019-11-05 11:53:22 +00:00
const String & marks_file_extension_ ,
const CompressionCodecPtr & default_codec_ ,
2019-12-18 15:54:45 +00:00
const MergeTreeWriterSettings & settings_ ,
2019-11-26 09:48:22 +00:00
const MergeTreeIndexGranularity & index_granularity_ ,
bool need_finish_last_granule_ )
2020-02-27 16:47:40 +00:00
: disk ( std : : move ( disk_ ) )
, part_path ( part_path_ )
2019-11-05 11:53:22 +00:00
, storage ( storage_ )
, columns_list ( columns_list_ )
, marks_file_extension ( marks_file_extension_ )
2019-11-07 11:11:38 +00:00
, index_granularity ( index_granularity_ )
2019-11-05 11:53:22 +00:00
, default_codec ( default_codec_ )
2019-11-07 11:11:38 +00:00
, skip_indices ( indices_to_recalc_ )
2019-11-05 11:53:22 +00:00
, settings ( settings_ )
2019-11-07 11:11:38 +00:00
, compute_granularity ( index_granularity . empty ( ) )
, with_final_mark ( storage . getSettings ( ) - > write_final_mark & & settings . can_use_adaptive_granularity )
2019-11-26 09:48:22 +00:00
, need_finish_last_granule ( need_finish_last_granule_ )
2019-11-05 11:53:22 +00:00
{
if ( settings . blocks_are_granules_size & & ! index_granularity . empty ( ) )
throw Exception ( " Can't take information about index granularity from blocks, when non empty index_granularity array specified " , ErrorCodes : : LOGICAL_ERROR ) ;
2019-12-18 16:41:11 +00:00
2020-02-27 16:47:40 +00:00
if ( ! disk - > exists ( part_path ) )
disk - > createDirectories ( part_path ) ;
2019-11-05 11:53:22 +00:00
}
IMergeTreeDataPartWriter : : ~ IMergeTreeDataPartWriter ( ) = default ;
2019-12-19 14:05:26 +00:00
static void fillIndexGranularityImpl (
2019-11-05 11:53:22 +00:00
const Block & block ,
size_t index_granularity_bytes ,
size_t fixed_index_granularity_rows ,
bool blocks_are_granules ,
size_t index_offset ,
MergeTreeIndexGranularity & index_granularity ,
2019-11-26 09:48:22 +00:00
bool can_use_adaptive_index_granularity ,
2019-12-19 14:05:26 +00:00
bool need_finish_last_granule = false )
2019-11-05 11:53:22 +00:00
{
size_t rows_in_block = block . rows ( ) ;
size_t index_granularity_for_block ;
if ( ! can_use_adaptive_index_granularity )
index_granularity_for_block = fixed_index_granularity_rows ;
else
{
size_t block_size_in_memory = block . bytes ( ) ;
if ( blocks_are_granules )
index_granularity_for_block = rows_in_block ;
else if ( block_size_in_memory > = index_granularity_bytes )
{
size_t granules_in_block = block_size_in_memory / index_granularity_bytes ;
index_granularity_for_block = rows_in_block / granules_in_block ;
}
else
{
size_t size_of_row_in_bytes = block_size_in_memory / rows_in_block ;
index_granularity_for_block = index_granularity_bytes / size_of_row_in_bytes ;
}
}
if ( index_granularity_for_block = = 0 ) /// very rare case when index granularity bytes less then single row
index_granularity_for_block = 1 ;
/// We should be less or equal than fixed index granularity
index_granularity_for_block = std : : min ( fixed_index_granularity_rows , index_granularity_for_block ) ;
2019-12-18 16:41:11 +00:00
2019-11-26 09:48:22 +00:00
size_t current_row ;
for ( current_row = index_offset ; current_row < rows_in_block ; current_row + = index_granularity_for_block )
{
2019-11-27 19:57:07 +00:00
size_t rows_left_in_block = rows_in_block - current_row ;
2019-12-27 21:32:55 +00:00
2020-01-21 11:56:01 +00:00
/// Try to extend last granule if it's needed and block is large enough
/// or it shouldn't be first in granule (index_offset != 0).
2019-12-27 21:17:53 +00:00
if ( need_finish_last_granule & & rows_left_in_block < index_granularity_for_block
& & ( rows_in_block > = index_granularity_for_block | | index_offset ! = 0 ) )
{
// If enough rows are left, create a new granule. Otherwise, extend previous granule.
// So, real size of granule differs from index_granularity_for_block not more than 50%.
if ( rows_left_in_block * 2 > = index_granularity_for_block )
index_granularity . appendMark ( rows_left_in_block ) ;
else
index_granularity . addRowsToLastMark ( rows_left_in_block ) ;
}
2019-11-26 09:48:22 +00:00
else
2019-12-27 21:17:53 +00:00
{
2019-11-27 11:35:27 +00:00
index_granularity . appendMark ( index_granularity_for_block ) ;
2019-12-27 21:17:53 +00:00
}
2019-11-26 09:48:22 +00:00
}
2019-11-05 11:53:22 +00:00
}
void IMergeTreeDataPartWriter : : fillIndexGranularity ( const Block & block )
{
const auto storage_settings = storage . getSettings ( ) ;
fillIndexGranularityImpl (
block ,
storage_settings - > index_granularity_bytes ,
storage_settings - > index_granularity ,
settings . blocks_are_granules_size ,
index_offset ,
index_granularity ,
2019-11-26 09:48:22 +00:00
settings . can_use_adaptive_granularity ,
2019-12-27 21:32:55 +00:00
need_finish_last_granule ) ;
2019-11-05 11:53:22 +00:00
}
void IMergeTreeDataPartWriter : : initPrimaryIndex ( )
{
if ( storage . hasPrimaryKey ( ) )
{
2020-02-27 16:47:40 +00:00
index_file_stream = disk - > writeFile ( part_path + " primary.idx " , DBMS_DEFAULT_BUFFER_SIZE , WriteMode : : Rewrite ) ;
2019-11-05 11:53:22 +00:00
index_stream = std : : make_unique < HashingWriteBuffer > ( * index_file_stream ) ;
}
2019-11-08 14:36:10 +00:00
primary_index_initialized = true ;
2019-11-05 11:53:22 +00:00
}
void IMergeTreeDataPartWriter : : initSkipIndices ( )
{
for ( const auto & index : skip_indices )
{
String stream_name = index - > getFileName ( ) ;
skip_indices_streams . emplace_back (
2020-02-19 14:07:36 +00:00
std : : make_unique < IMergeTreeDataPartWriter : : Stream > (
2019-11-05 11:53:22 +00:00
stream_name ,
2020-02-27 16:47:40 +00:00
disk ,
2019-11-05 11:53:22 +00:00
part_path + stream_name , INDEX_FILE_EXTENSION ,
part_path + stream_name , marks_file_extension ,
default_codec , settings . max_compress_block_size ,
0 , settings . aio_threshold ) ) ;
skip_indices_aggregators . push_back ( index - > createIndexAggregator ( ) ) ;
skip_index_filling . push_back ( 0 ) ;
}
2019-11-08 14:36:10 +00:00
skip_indices_initialized = true ;
2019-11-05 11:53:22 +00:00
}
2020-03-09 01:50:33 +00:00
void IMergeTreeDataPartWriter : : calculateAndSerializePrimaryIndex ( const Block & primary_index_block , size_t rows )
2019-11-05 11:53:22 +00:00
{
2019-11-08 14:36:10 +00:00
if ( ! primary_index_initialized )
throw Exception ( " Primary index is not initialized " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-03-09 01:50:33 +00:00
size_t primary_columns_num = primary_index_block . columns ( ) ;
2019-11-05 11:53:22 +00:00
if ( index_columns . empty ( ) )
{
2020-03-09 01:50:33 +00:00
index_types = primary_index_block . getDataTypes ( ) ;
2019-11-05 11:53:22 +00:00
index_columns . resize ( primary_columns_num ) ;
last_index_row . resize ( primary_columns_num ) ;
for ( size_t i = 0 ; i < primary_columns_num ; + + i )
2020-03-09 01:50:33 +00:00
index_columns [ i ] = primary_index_block . getByPosition ( i ) . column - > cloneEmpty ( ) ;
2019-11-05 11:53:22 +00:00
}
/** While filling index (index_columns), disable memory tracker.
* Because memory is allocated here ( maybe in context of INSERT query ) ,
* but then freed in completely different place ( while merging parts ) , where query memory_tracker is not available .
* And otherwise it will look like excessively growing memory consumption in context of query .
* ( observed in long INSERT SELECTs )
*/
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock ( ) ;
/// Write index. The index contains Primary Key value for each `index_granularity` row.
2019-11-28 20:14:41 +00:00
2019-11-05 11:53:22 +00:00
for ( size_t i = index_offset ; i < rows ; )
{
if ( storage . hasPrimaryKey ( ) )
{
for ( size_t j = 0 ; j < primary_columns_num ; + + j )
{
2020-03-09 01:50:33 +00:00
const auto & primary_column = primary_index_block . getByPosition ( j ) ;
2019-11-05 11:53:22 +00:00
index_columns [ j ] - > insertFrom ( * primary_column . column , i ) ;
primary_column . type - > serializeBinary ( * primary_column . column , i , * index_stream ) ;
}
}
2019-11-28 20:14:41 +00:00
i + = index_granularity . getMarkRows ( current_mark + + ) ;
if ( current_mark > = index_granularity . getMarksCount ( ) )
2019-11-05 11:53:22 +00:00
break ;
}
/// store last index row to write final mark at the end of column
for ( size_t j = 0 ; j < primary_columns_num ; + + j )
{
2020-03-09 01:50:33 +00:00
const IColumn & primary_column = * primary_index_block . getByPosition ( j ) . column . get ( ) ;
2019-11-05 11:53:22 +00:00
primary_column . get ( rows - 1 , last_index_row [ j ] ) ;
}
}
void IMergeTreeDataPartWriter : : calculateAndSerializeSkipIndices (
const Block & skip_indexes_block , size_t rows )
{
2019-11-08 14:36:10 +00:00
if ( ! skip_indices_initialized )
throw Exception ( " Skip indices are not initialized " , ErrorCodes : : LOGICAL_ERROR ) ;
2019-11-05 11:53:22 +00:00
size_t skip_index_current_data_mark = 0 ;
2019-12-27 21:17:53 +00:00
/// Filling and writing skip indices like in MergeTreeDataPartWriterWide::writeColumn
2019-11-05 11:53:22 +00:00
for ( size_t i = 0 ; i < skip_indices . size ( ) ; + + i )
{
const auto index = skip_indices [ i ] ;
auto & stream = * skip_indices_streams [ i ] ;
size_t prev_pos = 0 ;
skip_index_current_data_mark = skip_index_data_mark ;
while ( prev_pos < rows )
{
UInt64 limit = 0 ;
if ( prev_pos = = 0 & & index_offset ! = 0 )
{
limit = index_offset ;
}
else
{
limit = index_granularity . getMarkRows ( skip_index_current_data_mark ) ;
if ( skip_indices_aggregators [ i ] - > empty ( ) )
{
skip_indices_aggregators [ i ] = index - > createIndexAggregator ( ) ;
skip_index_filling [ i ] = 0 ;
if ( stream . compressed . offset ( ) > = settings . min_compress_block_size )
stream . compressed . next ( ) ;
writeIntBinary ( stream . plain_hashing . count ( ) , stream . marks ) ;
writeIntBinary ( stream . compressed . offset ( ) , stream . marks ) ;
/// Actually this numbers is redundant, but we have to store them
/// to be compatible with normal .mrk2 file format
if ( settings . can_use_adaptive_granularity )
writeIntBinary ( 1UL , stream . marks ) ;
}
/// this mark is aggregated, go to the next one
skip_index_current_data_mark + + ;
}
size_t pos = prev_pos ;
skip_indices_aggregators [ i ] - > update ( skip_indexes_block , & pos , limit ) ;
if ( pos = = prev_pos + limit )
{
+ + skip_index_filling [ i ] ;
/// write index if it is filled
if ( skip_index_filling [ i ] = = index - > granularity )
{
skip_indices_aggregators [ i ] - > getGranuleAndReset ( ) - > serializeBinary ( stream . compressed ) ;
skip_index_filling [ i ] = 0 ;
}
}
prev_pos = pos ;
}
}
skip_index_data_mark = skip_index_current_data_mark ;
}
2019-11-18 15:18:50 +00:00
void IMergeTreeDataPartWriter : : finishPrimaryIndexSerialization ( MergeTreeData : : DataPart : : Checksums & checksums )
2019-11-05 11:53:22 +00:00
{
2019-11-20 13:33:41 +00:00
bool write_final_mark = ( with_final_mark & & data_written ) ;
if ( write_final_mark & & compute_granularity )
index_granularity . appendMark ( 0 ) ;
2019-11-05 11:53:22 +00:00
if ( index_stream )
{
2019-11-20 13:33:41 +00:00
if ( write_final_mark )
2019-11-05 11:53:22 +00:00
{
for ( size_t j = 0 ; j < index_columns . size ( ) ; + + j )
{
index_columns [ j ] - > insert ( last_index_row [ j ] ) ;
index_types [ j ] - > serializeBinary ( last_index_row [ j ] , * index_stream ) ;
}
2019-11-08 14:36:10 +00:00
2019-11-05 11:53:22 +00:00
last_index_row . clear ( ) ;
}
index_stream - > next ( ) ;
checksums . files [ " primary.idx " ] . file_size = index_stream - > count ( ) ;
checksums . files [ " primary.idx " ] . file_hash = index_stream - > getHash ( ) ;
index_stream = nullptr ;
}
}
void IMergeTreeDataPartWriter : : finishSkipIndicesSerialization (
MergeTreeData : : DataPart : : Checksums & checksums )
{
for ( size_t i = 0 ; i < skip_indices . size ( ) ; + + i )
{
auto & stream = * skip_indices_streams [ i ] ;
if ( ! skip_indices_aggregators [ i ] - > empty ( ) )
skip_indices_aggregators [ i ] - > getGranuleAndReset ( ) - > serializeBinary ( stream . compressed ) ;
}
for ( auto & stream : skip_indices_streams )
{
stream - > finalize ( ) ;
stream - > addToChecksums ( checksums ) ;
}
2019-11-07 11:11:38 +00:00
skip_indices_streams . clear ( ) ;
2019-11-05 11:53:22 +00:00
skip_indices_aggregators . clear ( ) ;
skip_index_filling . clear ( ) ;
}
void IMergeTreeDataPartWriter : : next ( )
{
current_mark = next_mark ;
index_offset = next_index_offset ;
}
2019-10-21 15:33:59 +00:00
}