2016-02-14 04:58:47 +00:00
# pragma once
2017-04-01 09:19:00 +00:00
# include <Core/Field.h>
# include <Core/NamesAndTypes.h>
# include <Storages/MergeTree/ActiveDataPartSet.h>
2017-07-13 16:49:09 +00:00
# include <Columns/IColumn.h>
2017-07-28 17:34:02 +00:00
# include <shared_mutex>
2016-02-14 04:58:47 +00:00
class SipHash ;
namespace DB
{
2017-04-16 15:00:33 +00:00
/// Checksum of one file.
2016-02-14 04:58:47 +00:00
struct MergeTreeDataPartChecksum
{
2017-06-21 08:35:38 +00:00
using uint128 = CityHash_v1_0_2 : : uint128 ;
2017-06-21 01:24:05 +00:00
2017-04-01 07:20:54 +00:00
size_t file_size { } ;
uint128 file_hash { } ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
bool is_compressed = false ;
size_t uncompressed_size { } ;
uint128 uncompressed_hash { } ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
MergeTreeDataPartChecksum ( ) { }
MergeTreeDataPartChecksum ( size_t file_size_ , uint128 file_hash_ ) : file_size ( file_size_ ) , file_hash ( file_hash_ ) { }
MergeTreeDataPartChecksum ( size_t file_size_ , uint128 file_hash_ , size_t uncompressed_size_ , uint128 uncompressed_hash_ )
: file_size ( file_size_ ) , file_hash ( file_hash_ ) , is_compressed ( true ) ,
uncompressed_size ( uncompressed_size_ ) , uncompressed_hash ( uncompressed_hash_ ) { }
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
void checkEqual ( const MergeTreeDataPartChecksum & rhs , bool have_uncompressed , const String & name ) const ;
void checkSize ( const String & path ) const ;
2016-02-14 04:58:47 +00:00
} ;
2017-04-16 15:00:33 +00:00
/** Checksums of all non-temporary files.
* For compressed files , the check sum and the size of the decompressed data are stored to not depend on the compression method .
2016-02-14 04:58:47 +00:00
*/
struct MergeTreeDataPartChecksums
{
2017-04-01 07:20:54 +00:00
using Checksum = MergeTreeDataPartChecksum ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
using FileChecksums = std : : map < String , Checksum > ;
FileChecksums files ;
2016-02-14 04:58:47 +00:00
2017-06-21 01:24:05 +00:00
void addFile ( const String & file_name , size_t file_size , Checksum : : uint128 file_hash ) ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
void add ( MergeTreeDataPartChecksums & & rhs_checksums ) ;
2016-11-03 12:00:44 +00:00
2017-04-16 15:00:33 +00:00
/// Checks that the set of columns and their checksums are the same. If not, throws an exception.
/// If have_uncompressed, for compressed files it compares the checksums of the decompressed data. Otherwise, it compares only the checksums of the files.
2017-04-01 07:20:54 +00:00
void checkEqual ( const MergeTreeDataPartChecksums & rhs , bool have_uncompressed ) const ;
2016-02-14 04:58:47 +00:00
2017-04-16 15:00:33 +00:00
/// Checks that the directory contains all the needed files of the correct size. Does not check the checksum.
2017-04-01 07:20:54 +00:00
void checkSizes ( const String & path ) const ;
2016-02-14 04:58:47 +00:00
2017-04-16 15:00:33 +00:00
/// Serializes and deserializes in human readable form.
bool read ( ReadBuffer & in ) ; /// Returns false if the checksum is too old.
2017-04-01 07:20:54 +00:00
bool read_v2 ( ReadBuffer & in ) ;
bool read_v3 ( ReadBuffer & in ) ;
bool read_v4 ( ReadBuffer & in ) ;
void write ( WriteBuffer & out ) const ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
bool empty ( ) const
{
return files . empty ( ) ;
}
2016-02-14 04:58:47 +00:00
2017-04-16 15:00:33 +00:00
/// Checksum from the set of checksums of .bin files.
2017-04-01 07:20:54 +00:00
void summaryDataChecksum ( SipHash & hash ) const ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
String toString ( ) const ;
static MergeTreeDataPartChecksums parse ( const String & s ) ;
2016-02-14 04:58:47 +00:00
} ;
class MergeTreeData ;
2017-04-16 15:00:33 +00:00
/// Description of the data part.
2016-02-14 04:58:47 +00:00
struct MergeTreeDataPart : public ActiveDataPartSet : : Part
{
2017-04-01 07:20:54 +00:00
using Checksums = MergeTreeDataPartChecksums ;
2017-03-24 13:52:50 +00:00
using Checksum = MergeTreeDataPartChecksums : : Checksum ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
MergeTreeDataPart ( MergeTreeData & storage_ ) : storage ( storage_ ) { }
2016-02-14 04:58:47 +00:00
2017-03-24 13:52:50 +00:00
/// Returns checksum of column's binary file.
const Checksum * tryGetBinChecksum ( const String & name ) const ;
2017-04-01 07:20:54 +00:00
/// Returns the size of .bin file for column `name` if found, zero otherwise
size_t getColumnCompressedSize ( const String & name ) const ;
2017-03-24 13:52:50 +00:00
size_t getColumnUncompressedSize ( const String & name ) const ;
2016-02-14 04:58:47 +00:00
2017-03-24 13:52:50 +00:00
/// Returns the name of a column with minimum compressed size (as returned by getColumnSize()).
/// If no checksums are present returns the name of the first physically existing column.
2017-04-01 07:20:54 +00:00
String getColumnNameWithMinumumCompressedSize ( ) const ;
2016-02-14 04:58:47 +00:00
2017-03-24 13:52:50 +00:00
/// If part has column with fixed size, will return exact size of part (in rows)
2017-04-10 14:06:44 +00:00
size_t getExactSizeRows ( ) const ;
2017-03-24 13:52:50 +00:00
/// Returns full path to part dir
String getFullPath ( ) const ;
2017-05-16 15:40:32 +00:00
/// Returns part->name with prefixes like 'tmp_<name>'
String getNameWithPrefix ( ) const ;
2017-04-01 07:20:54 +00:00
MergeTreeData & storage ;
2016-02-14 04:58:47 +00:00
2017-05-16 15:40:32 +00:00
/// A directory path (realative to storage's path) where part data is actually stored
2017-05-26 00:47:06 +00:00
/// Examples: 'detached/tmp_fetch_<name>', 'tmp_<name>', '<name>'
2017-05-16 15:40:32 +00:00
mutable String relative_path ;
size_t size = 0 ; /// in number of marks.
std : : atomic < size_t > size_in_bytes { 0 } ; /// size in bytes, 0 - if not counted;
/// is used from several threads without locks (it is changed with ALTER).
2017-04-01 07:20:54 +00:00
time_t modification_time = 0 ;
2017-04-16 15:00:33 +00:00
mutable time_t remove_time = std : : numeric_limits < time_t > : : max ( ) ; /// When the part is removed from the working set.
2016-02-14 04:58:47 +00:00
2017-04-16 15:00:33 +00:00
/// If true, the destructor will delete the directory with the part.
2017-04-01 07:20:54 +00:00
bool is_temp = false ;
2016-02-14 04:58:47 +00:00
2017-03-24 13:52:50 +00:00
/// For resharding.
2017-04-01 07:20:54 +00:00
bool is_sharded = false ;
size_t shard_no = 0 ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
/// Primary key (correspond to primary.idx file).
/// Always loaded in RAM. Contains each index_granularity-th value of primary key tuple.
/// Note that marks (also correspond to primary key) is not always in RAM, but cached. See MarkCache.h.
using Index = Columns ;
Index index ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
Checksums checksums ;
2016-02-14 04:58:47 +00:00
2017-03-24 13:52:50 +00:00
/// Columns description.
2017-04-01 07:20:54 +00:00
NamesAndTypesList columns ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
using ColumnToSize = std : : map < std : : string , size_t > ;
2016-02-14 04:58:47 +00:00
2017-04-16 15:00:33 +00:00
/** It is blocked for writing when changing columns, checksums or any part files.
* Locked to read when reading columns , checksums or any part files .
2017-04-01 07:20:54 +00:00
*/
2017-07-28 17:34:02 +00:00
mutable std : : shared_mutex columns_lock ;
2016-02-14 04:58:47 +00:00
2017-04-16 15:00:33 +00:00
/** It is taken for the whole time ALTER a part: from the beginning of the recording of the temporary files to their renaming to permanent.
* It is taken with unlocked ` columns_lock ` .
2017-04-01 07:20:54 +00:00
*
2017-04-16 15:00:33 +00:00
* NOTE : " You can " do without this mutex if you could turn ReadRWLock into WriteRWLock without removing the lock .
* This transformation is impossible , because it would create a deadlock , if you do it from two threads at once .
* Taking this mutex means that we want to lock columns_lock on read with intention then , not
* unblocking , block it for writing .
2017-04-01 07:20:54 +00:00
*/
mutable std : : mutex alter_mutex ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
~ MergeTreeDataPart ( ) ;
2016-02-14 04:58:47 +00:00
2017-04-16 15:00:33 +00:00
/// Calculate the total size of the entire directory with all the files
2017-04-01 07:20:54 +00:00
static size_t calcTotalSize ( const String & from ) ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
void remove ( ) const ;
2017-05-16 15:40:32 +00:00
/// Makes checks and move part to new directory
/// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly
void renameTo ( const String & new_relative_path , bool remove_new_dir_if_exists = true ) const ;
2016-02-14 04:58:47 +00:00
2017-06-25 00:17:08 +00:00
/// Renames a part by appending a prefix to the name. To_detached - also moved to the detached directory.
2017-04-01 07:20:54 +00:00
void renameAddPrefix ( bool to_detached , const String & prefix ) const ;
2016-02-14 04:58:47 +00:00
2017-03-24 13:52:50 +00:00
/// Loads index file. Also calculates this->size if size=0
2017-04-01 07:20:54 +00:00
void loadIndex ( ) ;
2016-02-14 04:58:47 +00:00
2017-03-24 13:52:50 +00:00
/// If checksums.txt exists, reads files' checksums (and sizes) from it
2017-04-01 07:20:54 +00:00
void loadChecksums ( bool require ) ;
2016-02-14 04:58:47 +00:00
2017-03-24 13:52:50 +00:00
/// Populates columns_to_size map (compressed size).
2017-04-01 07:20:54 +00:00
void accumulateColumnSizes ( ColumnToSize & column_to_size ) const ;
2016-02-14 04:58:47 +00:00
2017-03-24 13:52:50 +00:00
/// Reads columns names and types from columns.txt
2017-04-01 07:20:54 +00:00
void loadColumns ( bool require ) ;
2016-02-14 04:58:47 +00:00
2017-04-01 07:20:54 +00:00
void checkNotBroken ( bool require_part_metadata ) ;
2016-02-14 04:58:47 +00:00
2017-03-24 13:52:50 +00:00
/// Checks that .bin and .mrk files exist
2017-04-01 07:20:54 +00:00
bool hasColumnFiles ( const String & column ) const ;
2017-01-17 20:49:16 +00:00
2017-03-24 13:52:50 +00:00
/// For data in RAM ('index')
2017-04-01 07:20:54 +00:00
size_t getIndexSizeInBytes ( ) const ;
size_t getIndexSizeInAllocatedBytes ( ) const ;
2016-02-14 04:58:47 +00:00
} ;
}