2019-10-21 00:28:29 +00:00
# include <Storages/MergeTree/MergeTreeDataPartWriterWide.h>
2020-05-20 20:16:32 +00:00
# include <Interpreters/Context.h>
2020-09-21 07:17:58 +00:00
# include <Compression/CompressionFactory.h>
2020-12-14 07:28:42 +00:00
# include <Compression/CompressedReadBufferFromFile.h>
2019-10-21 00:28:29 +00:00
namespace DB
{
2020-02-25 18:10:48 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR ;
}
2019-10-21 00:28:29 +00:00
2019-10-21 15:33:59 +00:00
namespace
{
constexpr auto DATA_FILE_EXTENSION = " .bin " ;
}
2020-12-14 12:03:49 +00:00
namespace
{
Granules getGranulesToWrite ( const MergeTreeIndexGranularity & index_granularity , size_t block_rows , size_t current_mark , size_t rows_written_in_last_mark )
{
if ( current_mark > = index_granularity . getMarksCount ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Request to get granules from mark {} but index granularity size is {} " , current_mark , index_granularity . getMarksCount ( ) ) ;
Granules result ;
size_t current_row = 0 ;
if ( rows_written_in_last_mark > 0 )
{
size_t rows_left_in_last_mark = index_granularity . getMarkRows ( current_mark ) - rows_written_in_last_mark ;
size_t rest_rows = block_rows - current_row ;
if ( rest_rows < rows_left_in_last_mark )
2020-12-15 08:04:37 +00:00
result . emplace_back ( Granule {
. start_row = current_row ,
. granularity_rows = rows_left_in_last_mark ,
. block_rows = rest_rows ,
. mark_number = current_mark ,
. mark_on_start = false ,
} ) ;
2020-12-14 12:03:49 +00:00
else
2020-12-15 08:04:37 +00:00
result . emplace_back ( Granule {
. start_row = current_row ,
. granularity_rows = rows_left_in_last_mark ,
. block_rows = rows_left_in_last_mark ,
. mark_number = current_mark ,
. mark_on_start = false ,
} ) ;
2020-12-14 12:03:49 +00:00
current_row + = rows_left_in_last_mark ;
current_mark + + ;
}
while ( current_row < block_rows )
{
size_t expected_rows = index_granularity . getMarkRows ( current_mark ) ;
size_t rest_rows = block_rows - current_row ;
if ( rest_rows < expected_rows )
2020-12-15 08:04:37 +00:00
result . emplace_back ( Granule {
. start_row = current_row ,
. granularity_rows = expected_rows ,
. block_rows = rest_rows ,
. mark_number = current_mark ,
. mark_on_start = true ,
} ) ;
2020-12-14 12:03:49 +00:00
else
2020-12-15 08:04:37 +00:00
result . emplace_back ( Granule {
. start_row = current_row ,
. granularity_rows = expected_rows ,
. block_rows = expected_rows ,
. mark_number = current_mark ,
. mark_on_start = true ,
} ) ;
2020-12-14 12:03:49 +00:00
current_row + = expected_rows ;
current_mark + + ;
}
return result ;
}
}
2019-10-21 15:33:59 +00:00
MergeTreeDataPartWriterWide : : MergeTreeDataPartWriterWide (
2020-05-10 13:33:27 +00:00
const MergeTreeData : : DataPartPtr & data_part_ ,
2019-10-21 15:33:59 +00:00
const NamesAndTypesList & columns_list_ ,
2020-06-17 12:39:20 +00:00
const StorageMetadataPtr & metadata_snapshot_ ,
2019-12-18 16:41:11 +00:00
const std : : vector < MergeTreeIndexPtr > & indices_to_recalc_ ,
2019-10-21 15:33:59 +00:00
const String & marks_file_extension_ ,
const CompressionCodecPtr & default_codec_ ,
2019-12-18 15:54:45 +00:00
const MergeTreeWriterSettings & settings_ ,
2019-11-07 11:11:38 +00:00
const MergeTreeIndexGranularity & index_granularity_ )
2020-06-26 11:27:19 +00:00
: MergeTreeDataPartWriterOnDisk ( data_part_ , columns_list_ , metadata_snapshot_ ,
2020-05-10 13:33:27 +00:00
indices_to_recalc_ , marks_file_extension_ ,
default_codec_ , settings_ , index_granularity_ )
2019-10-21 15:33:59 +00:00
{
2020-06-17 12:39:20 +00:00
const auto & columns = metadata_snapshot - > getColumns ( ) ;
2019-10-21 15:33:59 +00:00
for ( const auto & it : columns_list )
2020-09-21 11:24:10 +00:00
addStreams ( it . name , * it . type , columns . getCodecDescOrDefault ( it . name , default_codec ) , settings . estimated_size ) ;
2019-10-21 15:33:59 +00:00
}
void MergeTreeDataPartWriterWide : : addStreams (
const String & name ,
const IDataType & type ,
2020-09-21 07:17:58 +00:00
const ASTPtr & effective_codec_desc ,
2019-12-09 21:21:17 +00:00
size_t estimated_size )
2019-10-21 15:33:59 +00:00
{
2020-09-21 07:17:58 +00:00
IDataType : : StreamCallback callback = [ & ] ( const IDataType : : SubstreamPath & substream_path , const IDataType & substream_type )
2019-10-21 15:33:59 +00:00
{
String stream_name = IDataType : : getFileNameForStream ( name , substream_path ) ;
/// Shared offsets for Nested type.
if ( column_streams . count ( stream_name ) )
return ;
2020-09-21 14:22:13 +00:00
CompressionCodecPtr compression_codec ;
2020-09-29 22:23:42 +00:00
/// If we can use special codec then just get it
2020-09-21 14:22:13 +00:00
if ( IDataType : : isSpecialCompressionAllowed ( substream_path ) )
compression_codec = CompressionCodecFactory : : instance ( ) . get ( effective_codec_desc , & substream_type , default_codec ) ;
2020-09-29 22:23:42 +00:00
else /// otherwise return only generic codecs and don't use info about the data_type
2020-09-21 14:47:10 +00:00
compression_codec = CompressionCodecFactory : : instance ( ) . get ( effective_codec_desc , nullptr , default_codec , true ) ;
2020-09-21 07:17:58 +00:00
2020-02-19 14:07:36 +00:00
column_streams [ stream_name ] = std : : make_unique < Stream > (
2019-10-21 15:33:59 +00:00
stream_name ,
2020-05-10 13:33:27 +00:00
data_part - > volume - > getDisk ( ) ,
2019-10-21 15:33:59 +00:00
part_path + stream_name , DATA_FILE_EXTENSION ,
part_path + stream_name , marks_file_extension ,
2020-09-21 11:24:10 +00:00
compression_codec ,
2019-10-21 15:33:59 +00:00
settings . max_compress_block_size ,
estimated_size ,
settings . aio_threshold ) ;
} ;
IDataType : : SubstreamPath stream_path ;
type . enumerateStreams ( callback , stream_path ) ;
}
IDataType : : OutputStreamGetter MergeTreeDataPartWriterWide : : createStreamGetter (
2020-12-09 18:10:09 +00:00
const String & name , WrittenOffsetColumns & offset_columns ) const
2019-10-21 15:33:59 +00:00
{
2019-12-09 21:21:17 +00:00
return [ & , this ] ( const IDataType : : SubstreamPath & substream_path ) - > WriteBuffer *
2019-10-21 15:33:59 +00:00
{
bool is_offsets = ! substream_path . empty ( ) & & substream_path . back ( ) . type = = IDataType : : Substream : : ArraySizes ;
String stream_name = IDataType : : getFileNameForStream ( name , substream_path ) ;
/// Don't write offsets more than one time for Nested type.
if ( is_offsets & & offset_columns . count ( stream_name ) )
return nullptr ;
2020-12-09 18:10:09 +00:00
return & column_streams . at ( stream_name ) - > compressed ;
2019-10-21 15:33:59 +00:00
} ;
}
2020-12-15 09:54:48 +00:00
void MergeTreeDataPartWriterWide : : shiftCurrentMark ( const Granules & granules_written )
{
auto last_granule = granules_written . back ( ) ;
if ( ! last_granule . isCompleted ( ) )
{
setCurrentMark ( getCurrentMark ( ) + granules_written . size ( ) - 1 ) ;
bool still_in_the_same_granule = granules_written . size ( ) = = 1 ;
if ( still_in_the_same_granule )
rows_written_in_last_mark + = last_granule . block_rows ;
else
rows_written_in_last_mark = last_granule . block_rows ;
}
else
{
setCurrentMark ( getCurrentMark ( ) + granules_written . size ( ) ) ;
rows_written_in_last_mark = 0 ;
}
}
2020-12-10 08:57:52 +00:00
void MergeTreeDataPartWriterWide : : write ( const Block & block , const IColumn : : Permutation * permutation )
2019-10-21 00:28:29 +00:00
{
2019-11-07 11:11:38 +00:00
/// Fill index granularity for this block
/// if it's unknown (in case of insert data or horizontal merge,
/// but not in case of vertical merge)
if ( compute_granularity )
2020-04-26 21:19:25 +00:00
{
size_t index_granularity_for_block = computeIndexGranularity ( block ) ;
fillIndexGranularity ( index_granularity_for_block , block . rows ( ) ) ;
}
2019-11-27 19:57:07 +00:00
2020-12-14 12:51:14 +00:00
auto granules_to_write = getGranulesToWrite ( index_granularity , block . rows ( ) , getCurrentMark ( ) , rows_written_in_last_mark ) ;
2020-12-11 13:20:19 +00:00
2019-12-09 21:21:17 +00:00
auto offset_columns = written_offset_columns ? * written_offset_columns : WrittenOffsetColumns { } ;
2020-12-10 08:57:52 +00:00
Block primary_key_block ;
if ( settings . rewrite_primary_key )
primary_key_block = getBlockAndPermute ( block , metadata_snapshot - > getPrimaryKeyColumns ( ) , permutation ) ;
2020-12-11 08:41:02 +00:00
Block skip_indexes_block = getBlockAndPermute ( block , getSkipIndicesColumns ( ) , permutation ) ;
2019-10-21 00:28:29 +00:00
auto it = columns_list . begin ( ) ;
for ( size_t i = 0 ; i < columns_list . size ( ) ; + + i , + + it )
{
const ColumnWithTypeAndName & column = block . getByName ( it - > name ) ;
if ( permutation )
{
if ( primary_key_block . has ( it - > name ) )
{
const auto & primary_column = * primary_key_block . getByName ( it - > name ) . column ;
2020-12-11 15:00:58 +00:00
writeColumn ( column . name , * column . type , primary_column , offset_columns , granules_to_write ) ;
2019-10-21 00:28:29 +00:00
}
else if ( skip_indexes_block . has ( it - > name ) )
{
const auto & index_column = * skip_indexes_block . getByName ( it - > name ) . column ;
2020-12-11 13:20:19 +00:00
writeColumn ( column . name , * column . type , index_column , offset_columns , granules_to_write ) ;
2019-10-21 00:28:29 +00:00
}
else
{
/// We rearrange the columns that are not included in the primary key here; Then the result is released - to save RAM.
ColumnPtr permuted_column = column . column - > permute ( * permutation , 0 ) ;
2020-12-11 13:20:19 +00:00
writeColumn ( column . name , * column . type , * permuted_column , offset_columns , granules_to_write ) ;
2019-10-21 00:28:29 +00:00
}
}
else
{
2020-12-11 13:20:19 +00:00
writeColumn ( column . name , * column . type , * column . column , offset_columns , granules_to_write ) ;
2019-10-21 00:28:29 +00:00
}
}
2020-12-10 08:57:52 +00:00
if ( settings . rewrite_primary_key )
2020-12-11 13:20:19 +00:00
calculateAndSerializePrimaryIndex ( primary_key_block , granules_to_write ) ;
calculateAndSerializeSkipIndices ( skip_indexes_block , granules_to_write ) ;
2020-12-10 08:57:52 +00:00
2020-12-15 09:54:48 +00:00
shiftCurrentMark ( granules_to_write ) ;
2019-10-21 00:28:29 +00:00
}
void MergeTreeDataPartWriterWide : : writeSingleMark (
const String & name ,
const IDataType & type ,
WrittenOffsetColumns & offset_columns ,
size_t number_of_rows ,
DB : : IDataType : : SubstreamPath & path )
{
2020-12-10 16:29:10 +00:00
StreamsWithMarks marks = getCurrentMarksForColumn ( name , type , offset_columns , path ) ;
for ( const auto & mark : marks )
flushMarkToFile ( mark , number_of_rows ) ;
}
void MergeTreeDataPartWriterWide : : flushMarkToFile ( const StreamNameAndMark & stream_with_mark , size_t rows_in_mark )
{
Stream & stream = * column_streams [ stream_with_mark . stream_name ] ;
writeIntBinary ( stream_with_mark . mark . offset_in_compressed_file , stream . marks ) ;
writeIntBinary ( stream_with_mark . mark . offset_in_decompressed_block , stream . marks ) ;
if ( settings . can_use_adaptive_granularity )
writeIntBinary ( rows_in_mark , stream . marks ) ;
}
StreamsWithMarks MergeTreeDataPartWriterWide : : getCurrentMarksForColumn (
const String & name ,
const IDataType & type ,
WrittenOffsetColumns & offset_columns ,
DB : : IDataType : : SubstreamPath & path )
{
StreamsWithMarks result ;
2020-09-18 11:37:58 +00:00
type . enumerateStreams ( [ & ] ( const IDataType : : SubstreamPath & substream_path , const IDataType & /* substream_type */ )
2020-12-10 16:29:10 +00:00
{
bool is_offsets = ! substream_path . empty ( ) & & substream_path . back ( ) . type = = IDataType : : Substream : : ArraySizes ;
String stream_name = IDataType : : getFileNameForStream ( name , substream_path ) ;
/// Don't write offsets more than one time for Nested type.
if ( is_offsets & & offset_columns . count ( stream_name ) )
return ;
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
Stream & stream = * column_streams [ stream_name ] ;
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
/// There could already be enough data to compress into the new block.
if ( stream . compressed . offset ( ) > = settings . min_compress_block_size )
stream . compressed . next ( ) ;
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
StreamNameAndMark stream_with_mark ;
stream_with_mark . stream_name = stream_name ;
stream_with_mark . mark . offset_in_compressed_file = stream . plain_hashing . count ( ) ;
stream_with_mark . mark . offset_in_decompressed_block = stream . compressed . offset ( ) ;
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
result . push_back ( stream_with_mark ) ;
} , path ) ;
2019-10-21 00:28:29 +00:00
2020-12-10 16:29:10 +00:00
return result ;
2019-10-21 00:28:29 +00:00
}
2020-12-11 13:20:19 +00:00
void MergeTreeDataPartWriterWide : : writeSingleGranule (
2019-10-21 00:28:29 +00:00
const String & name ,
const IDataType & type ,
const IColumn & column ,
WrittenOffsetColumns & offset_columns ,
IDataType : : SerializeBinaryBulkStatePtr & serialization_state ,
IDataType : : SerializeBinaryBulkSettings & serialize_settings ,
size_t from_row ,
size_t number_of_rows ,
bool write_marks )
{
if ( write_marks )
2019-12-09 21:21:17 +00:00
writeSingleMark ( name , type , offset_columns , number_of_rows , serialize_settings . path ) ;
2019-10-21 00:28:29 +00:00
type . serializeBinaryBulkWithMultipleStreams ( column , from_row , number_of_rows , serialize_settings , serialization_state ) ;
/// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one.
2020-09-18 11:37:58 +00:00
type . enumerateStreams ( [ & ] ( const IDataType : : SubstreamPath & substream_path , const IDataType & /* substream_type */ )
2019-10-21 00:28:29 +00:00
{
bool is_offsets = ! substream_path . empty ( ) & & substream_path . back ( ) . type = = IDataType : : Substream : : ArraySizes ;
String stream_name = IDataType : : getFileNameForStream ( name , substream_path ) ;
/// Don't write offsets more than one time for Nested type.
if ( is_offsets & & offset_columns . count ( stream_name ) )
return ;
column_streams [ stream_name ] - > compressed . nextIfAtEnd ( ) ;
} , serialize_settings . path ) ;
}
2019-12-18 15:54:45 +00:00
/// Column must not be empty. (column.size() !== 0)
void MergeTreeDataPartWriterWide : : writeColumn (
2019-10-21 00:28:29 +00:00
const String & name ,
const IDataType & type ,
const IColumn & column ,
2020-12-11 13:20:19 +00:00
WrittenOffsetColumns & offset_columns ,
const Granules & granules )
2019-10-21 00:28:29 +00:00
{
2019-11-07 11:11:38 +00:00
auto [ it , inserted ] = serialization_states . emplace ( name , nullptr ) ;
if ( inserted )
{
IDataType : : SerializeBinaryBulkSettings serialize_settings ;
2019-12-09 21:21:17 +00:00
serialize_settings . getter = createStreamGetter ( name , offset_columns ) ;
2019-11-07 11:11:38 +00:00
type . serializeBinaryBulkStatePrefix ( serialize_settings , it - > second ) ;
}
2019-12-12 18:55:19 +00:00
const auto & global_settings = storage . global_context . getSettingsRef ( ) ;
2019-10-21 00:28:29 +00:00
IDataType : : SerializeBinaryBulkSettings serialize_settings ;
2019-12-09 21:21:17 +00:00
serialize_settings . getter = createStreamGetter ( name , offset_columns ) ;
2019-12-12 18:55:19 +00:00
serialize_settings . low_cardinality_max_dictionary_size = global_settings . low_cardinality_max_dictionary_size ;
serialize_settings . low_cardinality_use_single_dictionary_for_part = global_settings . low_cardinality_use_single_dictionary_for_part ! = 0 ;
2019-10-21 00:28:29 +00:00
2020-12-11 13:20:19 +00:00
for ( const auto & granule : granules )
2019-10-21 00:28:29 +00:00
{
2020-12-14 12:51:14 +00:00
if ( granule . granularity_rows > 0 )
2020-12-14 11:06:02 +00:00
data_written = true ;
writeSingleGranule (
name ,
type ,
column ,
offset_columns ,
it - > second ,
serialize_settings ,
2020-12-15 08:04:37 +00:00
granule . start_row ,
2020-12-14 12:51:14 +00:00
granule . granularity_rows ,
2020-12-14 11:06:02 +00:00
granule . mark_on_start
2019-10-21 00:28:29 +00:00
) ;
}
2020-09-18 11:37:58 +00:00
type . enumerateStreams ( [ & ] ( const IDataType : : SubstreamPath & substream_path , const IDataType & /* substream_type */ )
2019-10-21 00:28:29 +00:00
{
bool is_offsets = ! substream_path . empty ( ) & & substream_path . back ( ) . type = = IDataType : : Substream : : ArraySizes ;
if ( is_offsets )
{
String stream_name = IDataType : : getFileNameForStream ( name , substream_path ) ;
offset_columns . insert ( stream_name ) ;
}
} , serialize_settings . path ) ;
}
2020-12-14 07:28:42 +00:00
void MergeTreeDataPartWriterWide : : validateColumnOfFixedSize ( const String & name , const IDataType & type )
{
if ( ! type . isValueRepresentedByNumber ( ) | | type . haveSubtypes ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Cannot validate column of non fixed type {} " , type . getName ( ) ) ;
auto disk = data_part - > volume - > getDisk ( ) ;
String mrk_path = fullPath ( disk , part_path + name + marks_file_extension ) ;
String bin_path = fullPath ( disk , part_path + name + DATA_FILE_EXTENSION ) ;
DB : : ReadBufferFromFile mrk_in ( mrk_path ) ;
DB : : CompressedReadBufferFromFile bin_in ( bin_path , 0 , 0 , 0 ) ;
bool must_be_last = false ;
2020-12-14 12:51:14 +00:00
//auto * log = &Poco::Logger::get(storage.getLogName());
2020-12-14 07:28:42 +00:00
UInt64 offset_in_compressed_file = 0 ;
UInt64 offset_in_decompressed_block = 0 ;
UInt64 index_granularity_rows = 0 ;
2020-12-14 11:06:02 +00:00
size_t total_rows = 0 ;
2020-12-14 07:28:42 +00:00
size_t mark_num ;
2020-12-14 11:06:02 +00:00
2020-12-14 07:28:42 +00:00
for ( mark_num = 0 ; ! mrk_in . eof ( ) ; + + mark_num )
{
2020-12-14 11:06:02 +00:00
if ( mark_num > index_granularity . getMarksCount ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Incorrect number of marks in memory {}, on disk (at least) { } " , index_granularity.getMarksCount(), mark_num + 1);
2020-12-14 07:28:42 +00:00
DB : : readBinary ( offset_in_compressed_file , mrk_in ) ;
DB : : readBinary ( offset_in_decompressed_block , mrk_in ) ;
if ( settings . can_use_adaptive_granularity )
DB : : readBinary ( index_granularity_rows , mrk_in ) ;
else
index_granularity_rows = storage . getSettings ( ) - > index_granularity ;
if ( must_be_last )
{
if ( index_granularity_rows ! = 0 )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " We ran out of binary data but still have non empty mark #{} with rows number {} " , mark_num , index_granularity_rows ) ;
if ( ! mrk_in . eof ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Mark #{} must be last, but we still have some to read " , mark_num ) ;
2020-12-14 11:06:02 +00:00
break ;
}
if ( index_granularity_rows = = 0 )
{
auto column = type . createColumn ( ) ;
type . deserializeBinaryBulk ( * column , bin_in , 1000000000 , 0.0 ) ;
throw Exception ( ErrorCodes : : LOGICAL_ERROR ,
" Still have {} rows in bin stream, last mark #{} index granularity size {}, last rows {} " , column - > size ( ) , mark_num , index_granularity . getMarksCount ( ) , index_granularity_rows ) ;
2020-12-14 07:28:42 +00:00
}
if ( index_granularity_rows ! = index_granularity . getMarkRows ( mark_num ) )
throw Exception (
ErrorCodes : : LOGICAL_ERROR , " Incorrect mark rows for mark #{} (compressed offset {}, decompressed offset {}), in-memory {}, on disk {} " ,
mark_num , offset_in_compressed_file , offset_in_decompressed_block , index_granularity . getMarkRows ( mark_num ) , index_granularity_rows ) ;
auto column = type . createColumn ( ) ;
type . deserializeBinaryBulk ( * column , bin_in , index_granularity_rows , 0.0 ) ;
2020-12-14 11:06:02 +00:00
total_rows + = column - > size ( ) ;
2020-12-14 07:28:42 +00:00
if ( bin_in . eof ( ) )
{
must_be_last = true ;
}
else if ( column - > size ( ) ! = index_granularity_rows )
{
throw Exception (
ErrorCodes : : LOGICAL_ERROR , " Incorrect mark rows for mark #{} (compressed offset {}, decompressed offset {}), actually in bin file {}, in mrk file {} " ,
mark_num , offset_in_compressed_file , offset_in_decompressed_block , column - > size ( ) , index_granularity . getMarkRows ( mark_num ) ) ;
}
}
if ( ! mrk_in . eof ( ) )
2020-12-14 11:06:02 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR ,
" Still have something in marks stream, last mark #{} index granularity size {}, last rows {} " , mark_num , index_granularity . getMarksCount ( ) , index_granularity_rows ) ;
if ( ! bin_in . eof ( ) )
2020-12-14 07:28:42 +00:00
{
2020-12-14 11:06:02 +00:00
auto column = type . createColumn ( ) ;
type . deserializeBinaryBulk ( * column , bin_in , 1000000000 , 0.0 ) ;
throw Exception ( ErrorCodes : : LOGICAL_ERROR ,
" Still have {} rows in bin stream, last mark #{} index granularity size {}, last rows {} " , column - > size ( ) , mark_num , index_granularity . getMarksCount ( ) , index_granularity_rows ) ;
2020-12-14 07:28:42 +00:00
}
}
2020-06-25 16:55:45 +00:00
void MergeTreeDataPartWriterWide : : finishDataSerialization ( IMergeTreeDataPart : : Checksums & checksums , bool sync )
2019-10-21 17:23:06 +00:00
{
2019-12-12 18:55:19 +00:00
const auto & global_settings = storage . global_context . getSettingsRef ( ) ;
2019-10-21 17:23:06 +00:00
IDataType : : SerializeBinaryBulkSettings serialize_settings ;
2019-12-12 18:55:19 +00:00
serialize_settings . low_cardinality_max_dictionary_size = global_settings . low_cardinality_max_dictionary_size ;
serialize_settings . low_cardinality_use_single_dictionary_for_part = global_settings . low_cardinality_use_single_dictionary_for_part ! = 0 ;
2019-10-21 17:23:06 +00:00
WrittenOffsetColumns offset_columns ;
2019-11-18 15:18:50 +00:00
bool write_final_mark = ( with_final_mark & & data_written ) ;
2019-10-21 17:23:06 +00:00
{
auto it = columns_list . begin ( ) ;
for ( size_t i = 0 ; i < columns_list . size ( ) ; + + i , + + it )
{
if ( ! serialization_states . empty ( ) )
{
2019-12-09 21:21:17 +00:00
serialize_settings . getter = createStreamGetter ( it - > name , written_offset_columns ? * written_offset_columns : offset_columns ) ;
2019-11-07 11:11:38 +00:00
it - > type - > serializeBinaryBulkStateSuffix ( serialize_settings , serialization_states [ it - > name ] ) ;
2019-10-21 17:23:06 +00:00
}
if ( write_final_mark )
2019-12-09 21:21:17 +00:00
writeFinalMark ( it - > name , it - > type , offset_columns , serialize_settings . path ) ;
2019-10-21 17:23:06 +00:00
}
}
2020-03-09 01:59:08 +00:00
for ( auto & stream : column_streams )
2019-10-21 17:23:06 +00:00
{
2020-03-09 01:59:08 +00:00
stream . second - > finalize ( ) ;
stream . second - > addToChecksums ( checksums ) ;
2020-06-25 16:55:45 +00:00
if ( sync )
stream . second - > sync ( ) ;
2019-10-21 17:23:06 +00:00
}
column_streams . clear ( ) ;
2019-10-22 17:42:59 +00:00
serialization_states . clear ( ) ;
2020-12-14 07:28:42 +00:00
# ifndef NDEBUG
for ( const auto & column : columns_list )
{
if ( column . type - > isValueRepresentedByNumber ( ) & & ! column . type - > haveSubtypes ( ) )
validateColumnOfFixedSize ( column . name , * column . type ) ;
}
# endif
2019-10-21 17:23:06 +00:00
}
2020-12-10 08:57:52 +00:00
void MergeTreeDataPartWriterWide : : finish ( IMergeTreeDataPart : : Checksums & checksums , bool sync )
{
finishDataSerialization ( checksums , sync ) ;
if ( settings . rewrite_primary_key )
finishPrimaryIndexSerialization ( checksums , sync ) ;
finishSkipIndicesSerialization ( checksums , sync ) ;
}
2019-10-21 17:23:06 +00:00
void MergeTreeDataPartWriterWide : : writeFinalMark (
const std : : string & column_name ,
const DataTypePtr column_type ,
WrittenOffsetColumns & offset_columns ,
DB : : IDataType : : SubstreamPath & path )
{
2019-12-09 21:21:17 +00:00
writeSingleMark ( column_name , * column_type , offset_columns , 0 , path ) ;
2019-10-21 17:23:06 +00:00
/// Memoize information about offsets
2020-09-18 11:37:58 +00:00
column_type - > enumerateStreams ( [ & ] ( const IDataType : : SubstreamPath & substream_path , const IDataType & /* substream_type */ )
2019-10-21 17:23:06 +00:00
{
bool is_offsets = ! substream_path . empty ( ) & & substream_path . back ( ) . type = = IDataType : : Substream : : ArraySizes ;
if ( is_offsets )
{
String stream_name = IDataType : : getFileNameForStream ( column_name , substream_path ) ;
offset_columns . insert ( stream_name ) ;
}
} , path ) ;
}
2020-12-09 18:10:09 +00:00
static void fillIndexGranularityImpl (
MergeTreeIndexGranularity & index_granularity ,
2020-12-14 13:01:01 +00:00
size_t index_offset ,
2020-12-09 18:10:09 +00:00
size_t index_granularity_for_block ,
size_t rows_in_block )
{
2020-12-14 13:01:01 +00:00
for ( size_t current_row = index_offset ; current_row < rows_in_block ; current_row + = index_granularity_for_block )
2020-12-09 18:10:09 +00:00
index_granularity . appendMark ( index_granularity_for_block ) ;
}
void MergeTreeDataPartWriterWide : : fillIndexGranularity ( size_t index_granularity_for_block , size_t rows_in_block )
{
2020-12-14 11:06:02 +00:00
if ( getCurrentMark ( ) < index_granularity . getMarksCount ( ) & & getCurrentMark ( ) ! = index_granularity . getMarksCount ( ) - 1 )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Trying to add marks, while current mark {}, but total marks {} " , getCurrentMark ( ) , index_granularity . getMarksCount ( ) ) ;
2020-12-14 13:01:01 +00:00
size_t index_offset = 0 ;
if ( rows_written_in_last_mark ! = 0 )
index_offset = index_granularity . getLastMarkRows ( ) - rows_written_in_last_mark ;
2020-12-09 18:10:09 +00:00
fillIndexGranularityImpl (
index_granularity ,
2020-12-14 13:01:01 +00:00
index_offset ,
2020-12-09 18:10:09 +00:00
index_granularity_for_block ,
rows_in_block ) ;
}
2019-10-21 00:28:29 +00:00
}