2020-03-19 16:37:55 +00:00
# include <Compression/CompressedReadBuffer.h>
2017-04-01 09:19:00 +00:00
# include <DataStreams/ExpressionBlockInputStream.h>
# include <DataStreams/copyData.h>
2020-03-19 16:37:55 +00:00
# include <DataTypes/DataTypeArray.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeDate.h>
# include <DataTypes/DataTypeDateTime.h>
# include <DataTypes/DataTypeEnum.h>
# include <DataTypes/DataTypeNullable.h>
2020-03-19 16:37:55 +00:00
# include <DataTypes/NestedUtils.h>
# include <Formats/FormatFactory.h>
2017-04-01 09:19:00 +00:00
# include <Functions/FunctionFactory.h>
# include <Functions/IFunction.h>
2020-03-19 16:37:55 +00:00
# include <IO/ConcatReadBuffer.h>
# include <IO/HexWriteBuffer.h>
# include <IO/Operators.h>
# include <IO/ReadBufferFromMemory.h>
# include <IO/WriteBufferFromString.h>
# include <Interpreters/ExpressionAnalyzer.h>
# include <Interpreters/PartLog.h>
# include <Interpreters/SyntaxAnalyzer.h>
2020-05-20 20:16:32 +00:00
# include <Interpreters/Context.h>
2020-03-19 16:37:55 +00:00
# include <Parsers/ASTFunction.h>
# include <Parsers/ASTLiteral.h>
# include <Parsers/ASTNameTypePair.h>
# include <Parsers/ASTPartition.h>
# include <Parsers/ASTSetQuery.h>
# include <Parsers/ExpressionListParsers.h>
# include <Parsers/parseQuery.h>
# include <Parsers/queryToString.h>
# include <Storages/AlterCommands.h>
# include <Storages/MergeTree/MergeTreeData.h>
# include <Storages/MergeTree/MergeTreeDataPartCompact.h>
# include <Storages/MergeTree/MergeTreeDataPartWide.h>
2020-04-02 16:28:50 +00:00
# include <Storages/MergeTree/MergeTreeSequentialSource.h>
2020-03-19 16:37:55 +00:00
# include <Storages/MergeTree/MergedBlockOutputStream.h>
# include <Storages/MergeTree/MergedColumnOnlyOutputStream.h>
# include <Storages/MergeTree/checkDataPart.h>
# include <Storages/MergeTree/localBackup.h>
# include <Storages/StorageMergeTree.h>
# include <Storages/StorageReplicatedMergeTree.h>
2017-05-10 08:08:32 +00:00
# include <Common/Increment.h>
2017-05-10 06:49:19 +00:00
# include <Common/SimpleIncrement.h>
2020-03-19 16:37:55 +00:00
# include <Common/Stopwatch.h>
# include <Common/StringUtils/StringUtils.h>
2017-04-01 09:19:00 +00:00
# include <Common/escapeForFileName.h>
2019-11-28 10:13:53 +00:00
# include <Common/quoteString.h>
2017-07-13 20:58:19 +00:00
# include <Common/typeid_cast.h>
2017-09-05 12:12:55 +00:00
# include <Poco/DirectoryIterator.h>
2014-10-16 01:21:03 +00:00
2017-09-11 22:40:51 +00:00
# include <boost/range/adaptor/filtered.hpp>
2014-06-10 14:24:33 +00:00
# include <algorithm>
2014-10-16 01:21:03 +00:00
# include <iomanip>
2019-12-10 09:31:26 +00:00
# include <optional>
2019-02-05 14:50:25 +00:00
# include <set>
2015-04-16 07:22:29 +00:00
# include <thread>
2017-02-09 17:29:36 +00:00
# include <typeinfo>
# include <typeindex>
2019-12-10 09:31:26 +00:00
# include <unordered_set>
2014-03-09 17:36:01 +00:00
2016-10-24 02:02:37 +00:00
namespace ProfileEvents
{
2017-04-01 07:20:54 +00:00
extern const Event RejectedInserts ;
extern const Event DelayedInserts ;
extern const Event DelayedInsertsMilliseconds ;
2016-10-24 02:02:37 +00:00
}
2016-10-27 22:50:02 +00:00
namespace CurrentMetrics
{
2017-04-01 07:20:54 +00:00
extern const Metric DelayedInserts ;
2016-10-27 22:50:02 +00:00
}
2014-03-09 17:36:01 +00:00
2019-10-31 10:40:11 +00:00
namespace
{
constexpr UInt64 RESERVATION_MIN_ESTIMATION_SIZE = 1u * 1024u * 1024u ; /// 1MB
}
2014-03-09 17:36:01 +00:00
namespace DB
{
2016-11-20 12:43:20 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int NO_SUCH_DATA_PART ;
extern const int NOT_IMPLEMENTED ;
extern const int DIRECTORY_ALREADY_EXISTS ;
extern const int TOO_MANY_UNEXPECTED_DATA_PARTS ;
extern const int DUPLICATE_DATA_PART ;
extern const int NO_SUCH_COLUMN_IN_TABLE ;
extern const int LOGICAL_ERROR ;
extern const int ILLEGAL_COLUMN ;
extern const int CORRUPTED_DATA ;
extern const int BAD_TYPE_OF_FIELD ;
2019-02-12 17:54:52 +00:00
extern const int BAD_ARGUMENTS ;
2017-09-06 20:34:26 +00:00
extern const int INVALID_PARTITION_VALUE ;
2017-09-08 18:11:09 +00:00
extern const int METADATA_MISMATCH ;
2017-11-20 19:33:12 +00:00
extern const int PART_IS_TEMPORARILY_LOCKED ;
2018-03-09 23:23:15 +00:00
extern const int TOO_MANY_PARTS ;
2018-05-21 13:49:54 +00:00
extern const int INCOMPATIBLE_COLUMNS ;
2019-04-15 09:30:45 +00:00
extern const int BAD_TTL_EXPRESSION ;
2019-07-26 20:04:45 +00:00
extern const int INCORRECT_FILE_NAME ;
extern const int BAD_DATA_PART_NAME ;
2019-07-24 12:56:39 +00:00
extern const int UNKNOWN_SETTING ;
2019-08-30 20:12:26 +00:00
extern const int READONLY_SETTING ;
2019-09-05 13:12:29 +00:00
extern const int ABORTED ;
2020-01-17 12:24:27 +00:00
extern const int UNKNOWN_PART_TYPE ;
2020-01-22 19:52:55 +00:00
extern const int UNEXPECTED_AST_STRUCTURE ;
2020-02-25 13:57:33 +00:00
extern const int UNKNOWN_DISK ;
extern const int NOT_ENOUGH_SPACE ;
2020-04-27 13:54:31 +00:00
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN ;
2016-11-20 12:43:20 +00:00
}
2020-02-27 08:37:52 +00:00
const char * DELETE_ON_DESTROY_MARKER_PATH = " delete-on-destroy.txt " ;
2019-11-18 08:42:46 +00:00
2014-03-09 17:36:01 +00:00
MergeTreeData : : MergeTreeData (
2019-12-04 16:06:55 +00:00
const StorageID & table_id_ ,
2019-10-28 20:12:14 +00:00
const String & relative_data_path_ ,
2019-12-27 16:34:50 +00:00
const StorageInMemoryMetadata & metadata ,
2017-04-01 07:20:54 +00:00
Context & context_ ,
2017-09-08 18:11:09 +00:00
const String & date_column_name ,
2017-04-01 07:20:54 +00:00
const MergingParams & merging_params_ ,
2019-08-26 14:24:29 +00:00
std : : unique_ptr < MergeTreeSettings > storage_settings_ ,
2017-04-01 07:20:54 +00:00
bool require_part_metadata_ ,
bool attach ,
2017-12-01 21:40:58 +00:00
BrokenPartCallback broken_part_callback_ )
2020-04-27 13:55:30 +00:00
: IStorage ( table_id_ )
2019-12-03 16:25:32 +00:00
, global_context ( context_ )
2019-08-29 16:17:47 +00:00
, merging_params ( merging_params_ )
2019-12-28 09:21:09 +00:00
, partition_by_ast ( metadata . partition_by_ast )
, sample_by_ast ( metadata . sample_by_ast )
, settings_ast ( metadata . settings_ast )
2019-08-29 16:17:47 +00:00
, require_part_metadata ( require_part_metadata_ )
2019-10-28 20:12:14 +00:00
, relative_data_path ( relative_data_path_ )
2019-08-29 16:17:47 +00:00
, broken_part_callback ( broken_part_callback_ )
2019-12-04 16:06:55 +00:00
, log_name ( table_id_ . getNameForLogs ( ) )
2019-08-29 16:17:47 +00:00
, log ( & Logger : : get ( log_name ) )
, storage_settings ( std : : move ( storage_settings_ ) )
, data_parts_by_info ( data_parts_indexes . get < TagByInfo > ( ) )
, data_parts_by_state_and_info ( data_parts_indexes . get < TagByStateAndInfo > ( ) )
2019-09-05 13:12:29 +00:00
, parts_mover ( this )
2014-03-09 17:36:01 +00:00
{
2020-01-21 11:11:11 +00:00
if ( relative_data_path . empty ( ) )
throw Exception ( " MergeTree storages require data path " , ErrorCodes : : INCORRECT_FILE_NAME ) ;
2019-08-26 14:24:29 +00:00
const auto settings = getSettings ( ) ;
2020-04-29 02:21:43 +00:00
setProperties ( metadata , /*only_check*/ false , attach ) ;
2018-11-13 12:51:55 +00:00
2018-03-06 14:49:27 +00:00
/// NOTE: using the same columns list as is read when performing actual merges.
2018-03-13 15:00:28 +00:00
merging_params . check ( getColumns ( ) . getAllPhysical ( ) ) ;
2017-04-01 07:20:54 +00:00
2018-11-09 19:01:39 +00:00
if ( sample_by_ast )
2018-11-08 16:39:43 +00:00
{
2018-11-09 19:01:39 +00:00
sampling_expr_column_name = sample_by_ast - > getColumnName ( ) ;
2017-09-01 20:33:17 +00:00
2018-11-09 19:01:39 +00:00
if ( ! primary_key_sample . has ( sampling_expr_column_name )
2019-08-13 08:35:49 +00:00
& & ! attach & & ! settings - > compatibility_allow_sampling_expression_not_in_primary_key ) /// This is for backward compatibility.
2018-11-08 16:39:43 +00:00
throw Exception ( " Sampling expression must be present in the primary key " , ErrorCodes : : BAD_ARGUMENTS ) ;
2017-09-18 19:24:27 +00:00
2019-01-09 16:16:59 +00:00
auto syntax = SyntaxAnalyzer ( global_context ) . analyze ( sample_by_ast , getColumns ( ) . getAllPhysical ( ) ) ;
2019-08-09 14:50:04 +00:00
columns_required_for_sampling = syntax - > requiredSourceColumns ( ) ;
2018-11-08 16:39:43 +00:00
}
2017-09-18 19:24:27 +00:00
2017-09-08 18:11:09 +00:00
MergeTreeDataFormatVersion min_format_version ( 0 ) ;
if ( ! date_column_name . empty ( ) )
2017-09-01 20:33:17 +00:00
{
2017-09-08 18:11:09 +00:00
try
{
2018-11-06 18:25:36 +00:00
partition_by_ast = makeASTFunction ( " toYYYYMM " , std : : make_shared < ASTIdentifier > ( date_column_name ) ) ;
2017-09-08 18:11:09 +00:00
initPartitionKey ( ) ;
2017-09-08 13:17:38 +00:00
2017-09-08 18:11:09 +00:00
if ( minmax_idx_date_column_pos = = - 1 )
throw Exception ( " Could not find Date column " , ErrorCodes : : BAD_TYPE_OF_FIELD ) ;
}
catch ( Exception & e )
{
/// Better error message.
2019-06-15 12:06:22 +00:00
e . addMessage ( " (while initializing MergeTree partition key from date column " + backQuote ( date_column_name ) + " ) " ) ;
2017-09-08 18:11:09 +00:00
throw ;
}
2017-09-08 13:17:38 +00:00
}
2017-09-08 18:11:09 +00:00
else
2017-09-08 13:17:38 +00:00
{
2018-11-13 12:51:55 +00:00
is_custom_partitioned = true ;
2017-09-08 18:11:09 +00:00
initPartitionKey ( ) ;
min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING ;
2017-09-01 20:33:17 +00:00
}
2017-04-01 07:20:54 +00:00
2020-04-02 16:11:10 +00:00
setTTLExpressions ( metadata . columns , metadata . ttl_for_table_ast ) ;
2019-04-15 09:30:45 +00:00
2020-02-27 16:47:40 +00:00
/// format_file always contained on any data path
PathWithDisk version_file ;
2017-04-01 07:20:54 +00:00
/// Creating directories, if not exist.
2020-02-27 16:47:40 +00:00
for ( const auto & [ path , disk ] : getRelativeDataPathsWithDisks ( ) )
2019-04-05 17:37:27 +00:00
{
2020-02-27 16:47:40 +00:00
disk - > createDirectories ( path ) ;
disk - > createDirectories ( path + " detached " ) ;
2020-02-18 11:44:16 +00:00
auto current_version_file_path = path + " format_version.txt " ;
2020-02-27 16:47:40 +00:00
if ( disk - > exists ( current_version_file_path ) )
2019-04-21 20:23:02 +00:00
{
2020-02-27 16:47:40 +00:00
if ( ! version_file . first . empty ( ) )
2019-04-21 18:38:44 +00:00
{
2020-05-23 18:59:49 +00:00
LOG_ERROR_FORMATTED ( log , " Duplication of version file {} and {} " , fullPath ( version_file . second , version_file . first ) , current_version_file_path ) ;
2019-04-21 18:38:44 +00:00
throw Exception ( " Multiple format_version.txt file " , ErrorCodes : : CORRUPTED_DATA ) ;
}
2020-02-27 16:47:40 +00:00
version_file = { current_version_file_path , disk } ;
2019-04-21 18:38:44 +00:00
}
2019-04-01 18:41:19 +00:00
}
2019-04-21 18:38:44 +00:00
/// If not choose any
2020-02-27 16:47:40 +00:00
if ( version_file . first . empty ( ) )
2020-03-06 07:31:00 +00:00
version_file = { relative_data_path + " format_version.txt " , getStoragePolicy ( ) - > getAnyDisk ( ) } ;
2018-04-16 10:04:59 +00:00
2020-02-27 16:47:40 +00:00
bool version_file_exists = version_file . second - > exists ( version_file . first ) ;
2017-08-25 20:41:45 +00:00
2018-07-04 15:23:25 +00:00
// When data path or file not exists, ignore the format_version check
2019-04-21 18:38:44 +00:00
if ( ! attach | | ! version_file_exists )
2017-08-25 20:41:45 +00:00
{
2017-09-08 18:11:09 +00:00
format_version = min_format_version ;
2020-02-27 16:47:40 +00:00
auto buf = version_file . second - > writeFile ( version_file . first ) ;
writeIntText ( format_version . toUnderType ( ) , * buf ) ;
2017-08-25 20:41:45 +00:00
}
2019-05-22 19:20:10 +00:00
else
2017-08-25 20:41:45 +00:00
{
2020-02-27 16:47:40 +00:00
auto buf = version_file . second - > readFile ( version_file . first ) ;
2018-12-26 01:12:13 +00:00
UInt32 read_format_version ;
2020-02-27 16:47:40 +00:00
readIntText ( read_format_version , * buf ) ;
2018-12-26 01:12:13 +00:00
format_version = read_format_version ;
2020-02-27 16:47:40 +00:00
if ( ! buf - > eof ( ) )
throw Exception ( " Bad version file: " + fullPath ( version_file . second , version_file . first ) , ErrorCodes : : CORRUPTED_DATA ) ;
2017-08-25 20:41:45 +00:00
}
2017-09-08 18:11:09 +00:00
if ( format_version < min_format_version )
2018-11-12 16:14:37 +00:00
{
if ( min_format_version = = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING . toUnderType ( ) )
throw Exception (
" MergeTree data format version on disk doesn't support custom partitioning " ,
ErrorCodes : : METADATA_MISMATCH ) ;
}
2020-01-13 16:28:29 +00:00
2020-02-14 21:33:02 +00:00
String reason ;
if ( ! canUsePolymorphicParts ( * settings , & reason ) & & ! reason . empty ( ) )
LOG_WARNING ( log , reason + " Settings 'min_bytes_for_wide_part' and 'min_bytes_for_wide_part' will be ignored. " ) ;
2019-03-25 13:55:24 +00:00
}
2019-12-26 18:17:05 +00:00
StorageInMemoryMetadata MergeTreeData : : getInMemoryMetadata ( ) const
{
2020-02-14 13:17:50 +00:00
StorageInMemoryMetadata metadata ( getColumns ( ) , getIndices ( ) , getConstraints ( ) ) ;
2019-12-27 15:01:41 +00:00
if ( partition_by_ast )
metadata . partition_by_ast = partition_by_ast - > clone ( ) ;
2019-12-27 14:36:59 +00:00
if ( order_by_ast )
2019-12-27 14:46:11 +00:00
metadata . order_by_ast = order_by_ast - > clone ( ) ;
2019-12-27 14:36:59 +00:00
if ( primary_key_ast )
2019-12-27 14:46:11 +00:00
metadata . primary_key_ast = primary_key_ast - > clone ( ) ;
2019-12-27 14:36:59 +00:00
if ( ttl_table_ast )
2019-12-27 14:46:11 +00:00
metadata . ttl_for_table_ast = ttl_table_ast - > clone ( ) ;
2019-12-27 15:01:41 +00:00
if ( sample_by_ast )
metadata . sample_by_ast = sample_by_ast - > clone ( ) ;
2019-12-27 14:36:59 +00:00
if ( settings_ast )
metadata . settings_ast = settings_ast - > clone ( ) ;
return metadata ;
2019-12-26 18:17:05 +00:00
}
2020-01-09 14:50:34 +00:00
StoragePolicyPtr MergeTreeData : : getStoragePolicy ( ) const
{
return global_context . getStoragePolicy ( getSettings ( ) - > storage_policy ) ;
}
2018-02-21 17:05:21 +00:00
static void checkKeyExpression ( const ExpressionActions & expr , const Block & sample_block , const String & key_name )
2017-12-09 10:14:45 +00:00
{
2018-02-21 17:05:21 +00:00
for ( const ExpressionAction & action : expr . getActions ( ) )
{
if ( action . type = = ExpressionAction : : ARRAY_JOIN )
2018-11-22 21:19:58 +00:00
throw Exception ( key_name + " key cannot contain array joins " , ErrorCodes : : ILLEGAL_COLUMN ) ;
2018-02-21 17:05:21 +00:00
if ( action . type = = ExpressionAction : : APPLY_FUNCTION )
{
2018-09-21 14:15:21 +00:00
IFunctionBase & func = * action . function_base ;
2018-02-21 17:05:21 +00:00
if ( ! func . isDeterministic ( ) )
throw Exception ( key_name + " key cannot contain non-deterministic functions, "
" but contains function " + func . getName ( ) ,
ErrorCodes : : BAD_ARGUMENTS ) ;
}
}
2017-12-09 10:14:45 +00:00
2018-02-21 17:05:21 +00:00
for ( const ColumnWithTypeAndName & element : sample_block )
{
const ColumnPtr & column = element . column ;
2019-06-27 19:28:52 +00:00
if ( column & & ( isColumnConst ( * column ) | | column - > isDummy ( ) ) )
2018-02-21 17:05:21 +00:00
throw Exception { key_name + " key cannot contain constants " , ErrorCodes : : ILLEGAL_COLUMN } ;
if ( element . type - > isNullable ( ) )
throw Exception { key_name + " key cannot contain nullable columns " , ErrorCodes : : ILLEGAL_COLUMN } ;
}
2017-12-09 10:14:45 +00:00
}
2020-04-29 02:21:43 +00:00
void MergeTreeData : : setProperties ( const StorageInMemoryMetadata & metadata , bool only_check , bool attach )
2016-05-16 23:04:03 +00:00
{
2019-12-27 16:34:50 +00:00
if ( ! metadata . order_by_ast )
2018-11-06 18:25:36 +00:00
throw Exception ( " ORDER BY cannot be empty " , ErrorCodes : : BAD_ARGUMENTS ) ;
2018-10-11 17:06:55 +00:00
2019-12-27 16:34:50 +00:00
ASTPtr new_sorting_key_expr_list = extractKeyExpressionList ( metadata . order_by_ast ) ;
ASTPtr new_primary_key_expr_list = metadata . primary_key_ast
? extractKeyExpressionList ( metadata . primary_key_ast ) : new_sorting_key_expr_list - > clone ( ) ;
2018-10-23 13:34:04 +00:00
2018-11-06 18:25:36 +00:00
if ( merging_params . mode = = MergeTreeData : : MergingParams : : VersionedCollapsing )
new_sorting_key_expr_list - > children . push_back ( std : : make_shared < ASTIdentifier > ( merging_params . version_column ) ) ;
2018-02-09 10:53:50 +00:00
2018-11-06 18:25:36 +00:00
size_t primary_key_size = new_primary_key_expr_list - > children . size ( ) ;
size_t sorting_key_size = new_sorting_key_expr_list - > children . size ( ) ;
2018-10-12 19:00:43 +00:00
if ( primary_key_size > sorting_key_size )
2018-10-11 17:06:55 +00:00
throw Exception ( " Primary key must be a prefix of the sorting key, but its length: "
2018-10-12 19:00:43 +00:00
+ toString ( primary_key_size ) + " is greater than the sorting key length: " + toString ( sorting_key_size ) ,
2018-10-11 17:06:55 +00:00
ErrorCodes : : BAD_ARGUMENTS ) ;
2017-04-01 07:20:54 +00:00
2018-10-11 17:06:55 +00:00
Names new_primary_key_columns ;
2018-10-12 19:00:43 +00:00
Names new_sorting_key_columns ;
2019-12-19 14:05:26 +00:00
NameSet primary_key_columns_set ;
2017-04-01 07:20:54 +00:00
2018-10-12 19:00:43 +00:00
for ( size_t i = 0 ; i < sorting_key_size ; + + i )
2017-04-01 07:20:54 +00:00
{
2018-11-06 18:25:36 +00:00
String sorting_key_column = new_sorting_key_expr_list - > children [ i ] - > getColumnName ( ) ;
2018-10-12 19:00:43 +00:00
new_sorting_key_columns . push_back ( sorting_key_column ) ;
2018-10-11 17:06:55 +00:00
if ( i < primary_key_size )
{
2018-11-06 18:25:36 +00:00
String pk_column = new_primary_key_expr_list - > children [ i ] - > getColumnName ( ) ;
2018-10-12 19:00:43 +00:00
if ( pk_column ! = sorting_key_column )
2018-10-11 17:06:55 +00:00
throw Exception ( " Primary key must be a prefix of the sorting key, but in position "
2018-10-12 19:00:43 +00:00
+ toString ( i ) + " its column is " + pk_column + " , not " + sorting_key_column ,
2018-10-11 17:06:55 +00:00
ErrorCodes : : BAD_ARGUMENTS ) ;
2019-12-19 14:05:26 +00:00
if ( ! primary_key_columns_set . emplace ( pk_column ) . second )
throw Exception ( " Primary key contains duplicate columns " , ErrorCodes : : BAD_ARGUMENTS ) ;
2018-10-11 17:06:55 +00:00
new_primary_key_columns . push_back ( pk_column ) ;
}
2018-02-09 10:53:50 +00:00
}
2019-12-27 16:34:50 +00:00
auto all_columns = metadata . columns . getAllPhysical ( ) ;
2017-04-01 07:20:54 +00:00
2019-12-26 18:17:05 +00:00
/// Order by check AST
2018-11-13 12:51:55 +00:00
if ( order_by_ast & & only_check )
2018-02-09 10:53:50 +00:00
{
2018-11-13 12:51:55 +00:00
/// This is ALTER, not CREATE/ATTACH TABLE. Let us check that all new columns used in the sorting key
/// expression have just been added (so that the sorting order is guaranteed to be valid with the new key).
ASTPtr added_key_column_expr_list = std : : make_shared < ASTExpressionList > ( ) ;
for ( size_t new_i = 0 , old_i = 0 ; new_i < sorting_key_size ; + + new_i )
{
if ( old_i < sorting_key_columns . size ( ) )
{
if ( new_sorting_key_columns [ new_i ] ! = sorting_key_columns [ old_i ] )
added_key_column_expr_list - > children . push_back ( new_sorting_key_expr_list - > children [ new_i ] ) ;
else
+ + old_i ;
}
else
added_key_column_expr_list - > children . push_back ( new_sorting_key_expr_list - > children [ new_i ] ) ;
}
if ( ! added_key_column_expr_list - > children . empty ( ) )
{
2019-01-09 16:16:59 +00:00
auto syntax = SyntaxAnalyzer ( global_context ) . analyze ( added_key_column_expr_list , all_columns ) ;
2019-08-09 14:50:04 +00:00
Names used_columns = syntax - > requiredSourceColumns ( ) ;
2018-11-13 12:51:55 +00:00
NamesAndTypesList deleted_columns ;
NamesAndTypesList added_columns ;
getColumns ( ) . getAllPhysical ( ) . getDifference ( all_columns , deleted_columns , added_columns ) ;
2018-11-08 15:43:14 +00:00
2018-11-13 12:51:55 +00:00
for ( const String & col : used_columns )
{
if ( ! added_columns . contains ( col ) | | deleted_columns . contains ( col ) )
throw Exception ( " Existing column " + col + " is used in the expression that was "
" added to the sorting key. You can add expressions that use only the newly added columns " ,
ErrorCodes : : BAD_ARGUMENTS ) ;
2019-12-27 16:34:50 +00:00
if ( metadata . columns . getDefaults ( ) . count ( col ) )
2018-11-13 12:51:55 +00:00
throw Exception ( " Newly added column " + col + " has a default expression, so adding "
" expressions that use it to the sorting key is forbidden " ,
ErrorCodes : : BAD_ARGUMENTS ) ;
}
}
2018-02-09 10:53:50 +00:00
}
2017-04-01 07:20:54 +00:00
2019-01-09 16:16:59 +00:00
auto new_sorting_key_syntax = SyntaxAnalyzer ( global_context ) . analyze ( new_sorting_key_expr_list , all_columns ) ;
2019-01-04 12:10:00 +00:00
auto new_sorting_key_expr = ExpressionAnalyzer ( new_sorting_key_expr_list , new_sorting_key_syntax , global_context )
2018-10-11 17:06:55 +00:00
. getActions ( false ) ;
2018-10-12 19:00:43 +00:00
auto new_sorting_key_sample =
2019-01-04 12:10:00 +00:00
ExpressionAnalyzer ( new_sorting_key_expr_list , new_sorting_key_syntax , global_context )
2018-10-11 17:06:55 +00:00
. getActions ( true ) - > getSampleBlock ( ) ;
2018-02-09 10:53:50 +00:00
2018-10-12 19:00:43 +00:00
checkKeyExpression ( * new_sorting_key_expr , new_sorting_key_sample , " Sorting " ) ;
2017-04-01 07:20:54 +00:00
2019-01-09 16:16:59 +00:00
auto new_primary_key_syntax = SyntaxAnalyzer ( global_context ) . analyze ( new_primary_key_expr_list , all_columns ) ;
2019-01-04 12:10:00 +00:00
auto new_primary_key_expr = ExpressionAnalyzer ( new_primary_key_expr_list , new_primary_key_syntax , global_context )
2018-10-11 17:06:55 +00:00
. getActions ( false ) ;
2018-02-09 10:53:50 +00:00
2018-10-11 17:06:55 +00:00
Block new_primary_key_sample ;
DataTypes new_primary_key_data_types ;
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < primary_key_size ; + + i )
2018-10-11 17:06:55 +00:00
{
2018-10-12 19:00:43 +00:00
const auto & elem = new_sorting_key_sample . getByPosition ( i ) ;
2018-10-11 17:06:55 +00:00
new_primary_key_sample . insert ( elem ) ;
new_primary_key_data_types . push_back ( elem . type ) ;
2018-02-09 10:53:50 +00:00
}
2019-02-05 14:50:25 +00:00
ASTPtr skip_indices_with_primary_key_expr_list = new_primary_key_expr_list - > clone ( ) ;
ASTPtr skip_indices_with_sorting_key_expr_list = new_sorting_key_expr_list - > clone ( ) ;
MergeTreeIndices new_indices ;
2019-12-27 16:34:50 +00:00
if ( ! metadata . indices . indices . empty ( ) )
2019-02-05 14:50:25 +00:00
{
std : : set < String > indices_names ;
2019-12-27 16:34:50 +00:00
for ( const auto & index_ast : metadata . indices . indices )
2019-02-05 14:50:25 +00:00
{
const auto & index_decl = std : : dynamic_pointer_cast < ASTIndexDeclaration > ( index_ast ) ;
new_indices . push_back (
2019-08-29 10:56:43 +00:00
MergeTreeIndexFactory : : instance ( ) . get (
all_columns ,
2020-04-29 02:21:43 +00:00
std : : dynamic_pointer_cast < ASTIndexDeclaration > ( index_decl ) ,
global_context ,
attach ) ) ;
2019-02-05 14:50:25 +00:00
if ( indices_names . find ( new_indices . back ( ) - > name ) ! = indices_names . end ( ) )
throw Exception (
2019-06-15 12:06:22 +00:00
" Index with name " + backQuote ( new_indices . back ( ) - > name ) + " already exsists " ,
2019-02-05 14:50:25 +00:00
ErrorCodes : : LOGICAL_ERROR ) ;
ASTPtr expr_list = MergeTreeData : : extractKeyExpressionList ( index_decl - > expr - > clone ( ) ) ;
for ( const auto & expr : expr_list - > children )
{
skip_indices_with_primary_key_expr_list - > children . push_back ( expr - > clone ( ) ) ;
skip_indices_with_sorting_key_expr_list - > children . push_back ( expr - > clone ( ) ) ;
}
indices_names . insert ( new_indices . back ( ) - > name ) ;
}
}
2020-02-26 19:33:09 +00:00
auto syntax_primary = SyntaxAnalyzer ( global_context ) . analyze (
2019-02-05 14:50:25 +00:00
skip_indices_with_primary_key_expr_list , all_columns ) ;
auto new_indices_with_primary_key_expr = ExpressionAnalyzer (
skip_indices_with_primary_key_expr_list , syntax_primary , global_context ) . getActions ( false ) ;
2020-02-26 19:33:09 +00:00
auto syntax_sorting = SyntaxAnalyzer ( global_context ) . analyze (
2019-02-05 14:50:25 +00:00
skip_indices_with_sorting_key_expr_list , all_columns ) ;
auto new_indices_with_sorting_key_expr = ExpressionAnalyzer (
skip_indices_with_sorting_key_expr_list , syntax_sorting , global_context ) . getActions ( false ) ;
2018-11-13 12:51:55 +00:00
if ( ! only_check )
2018-02-09 10:53:50 +00:00
{
2019-12-27 16:34:50 +00:00
setColumns ( std : : move ( metadata . columns ) ) ;
2018-02-09 10:53:50 +00:00
2019-12-27 16:34:50 +00:00
order_by_ast = metadata . order_by_ast ;
2018-11-13 12:51:55 +00:00
sorting_key_columns = std : : move ( new_sorting_key_columns ) ;
2018-11-27 17:07:10 +00:00
sorting_key_expr_ast = std : : move ( new_sorting_key_expr_list ) ;
2018-11-13 12:51:55 +00:00
sorting_key_expr = std : : move ( new_sorting_key_expr ) ;
2018-02-09 10:53:50 +00:00
2019-12-27 16:34:50 +00:00
primary_key_ast = metadata . primary_key_ast ;
2018-11-13 12:51:55 +00:00
primary_key_columns = std : : move ( new_primary_key_columns ) ;
2018-11-27 17:07:10 +00:00
primary_key_expr_ast = std : : move ( new_primary_key_expr_list ) ;
2018-11-13 12:51:55 +00:00
primary_key_expr = std : : move ( new_primary_key_expr ) ;
primary_key_sample = std : : move ( new_primary_key_sample ) ;
primary_key_data_types = std : : move ( new_primary_key_data_types ) ;
2019-02-05 14:50:25 +00:00
2019-12-27 16:34:50 +00:00
setIndices ( metadata . indices ) ;
2019-02-05 14:50:25 +00:00
skip_indices = std : : move ( new_indices ) ;
2019-12-27 16:34:50 +00:00
setConstraints ( metadata . constraints ) ;
2019-06-02 14:41:12 +00:00
2019-02-05 14:50:25 +00:00
primary_key_and_skip_indices_expr = new_indices_with_primary_key_expr ;
sorting_key_and_skip_indices_expr = new_indices_with_sorting_key_expr ;
2018-11-13 12:51:55 +00:00
}
2014-03-13 12:48:07 +00:00
}
2014-03-09 17:36:01 +00:00
2018-11-13 12:51:55 +00:00
2018-11-06 18:25:36 +00:00
ASTPtr MergeTreeData : : extractKeyExpressionList ( const ASTPtr & node )
{
if ( ! node )
return std : : make_shared < ASTExpressionList > ( ) ;
2019-03-11 13:22:51 +00:00
const auto * expr_func = node - > as < ASTFunction > ( ) ;
2018-11-06 18:25:36 +00:00
if ( expr_func & & expr_func - > name = = " tuple " )
{
2018-12-05 14:02:34 +00:00
/// Primary key is specified in tuple, extract its arguments.
return expr_func - > arguments - > clone ( ) ;
2018-11-06 18:25:36 +00:00
}
else
{
/// Primary key consists of one column.
auto res = std : : make_shared < ASTExpressionList > ( ) ;
res - > children . push_back ( node ) ;
return res ;
2018-02-09 10:53:50 +00:00
}
2014-03-13 12:48:07 +00:00
}
2014-03-09 17:36:01 +00:00
2016-04-15 17:42:51 +00:00
2017-09-08 18:11:09 +00:00
void MergeTreeData : : initPartitionKey ( )
2017-08-19 18:11:20 +00:00
{
2018-11-06 18:25:36 +00:00
ASTPtr partition_key_expr_list = extractKeyExpressionList ( partition_by_ast ) ;
if ( partition_key_expr_list - > children . empty ( ) )
2017-09-01 20:33:17 +00:00
return ;
2018-11-08 15:43:14 +00:00
{
2019-01-09 16:16:59 +00:00
auto syntax_result = SyntaxAnalyzer ( global_context ) . analyze ( partition_key_expr_list , getColumns ( ) . getAllPhysical ( ) ) ;
2019-01-04 12:10:00 +00:00
partition_key_expr = ExpressionAnalyzer ( partition_key_expr_list , syntax_result , global_context ) . getActions ( false ) ;
2018-11-08 15:43:14 +00:00
}
2018-11-06 18:25:36 +00:00
for ( const ASTPtr & ast : partition_key_expr_list - > children )
2017-08-30 19:03:19 +00:00
{
String col_name = ast - > getColumnName ( ) ;
2018-11-06 18:25:36 +00:00
partition_key_sample . insert ( partition_key_expr - > getSampleBlock ( ) . getByName ( col_name ) ) ;
2017-08-30 19:03:19 +00:00
}
2017-08-21 15:35:29 +00:00
2018-11-06 18:25:36 +00:00
checkKeyExpression ( * partition_key_expr , partition_key_sample , " Partition " ) ;
2018-02-21 17:05:21 +00:00
2017-09-01 20:33:17 +00:00
/// Add all columns used in the partition key to the min-max index.
2018-11-06 18:25:36 +00:00
const NamesAndTypesList & minmax_idx_columns_with_types = partition_key_expr - > getRequiredColumnsWithTypes ( ) ;
2019-01-04 12:10:00 +00:00
minmax_idx_expr = std : : make_shared < ExpressionActions > ( minmax_idx_columns_with_types , global_context ) ;
2017-12-25 21:57:29 +00:00
for ( const NameAndTypePair & column : minmax_idx_columns_with_types )
2017-08-21 15:35:29 +00:00
{
minmax_idx_columns . emplace_back ( column . name ) ;
minmax_idx_column_types . emplace_back ( column . type ) ;
}
2017-09-01 20:33:17 +00:00
/// Try to find the date column in columns used by the partition key (a common case).
2017-08-21 15:35:29 +00:00
bool encountered_date_column = false ;
for ( size_t i = 0 ; i < minmax_idx_column_types . size ( ) ; + + i )
{
if ( typeid_cast < const DataTypeDate * > ( minmax_idx_column_types [ i ] . get ( ) ) )
{
if ( ! encountered_date_column )
{
minmax_idx_date_column_pos = i ;
encountered_date_column = true ;
}
else
{
/// There is more than one Date column in partition key and we don't know which one to choose.
minmax_idx_date_column_pos = - 1 ;
}
}
}
2018-12-10 10:20:19 +00:00
if ( ! encountered_date_column )
{
for ( size_t i = 0 ; i < minmax_idx_column_types . size ( ) ; + + i )
{
if ( typeid_cast < const DataTypeDateTime * > ( minmax_idx_column_types [ i ] . get ( ) ) )
{
if ( ! encountered_date_column )
{
minmax_idx_time_column_pos = i ;
encountered_date_column = true ;
}
else
{
/// There is more than one DateTime column in partition key and we don't know which one to choose.
minmax_idx_time_column_pos = - 1 ;
}
}
}
}
2017-08-19 18:11:20 +00:00
}
2019-04-15 09:30:45 +00:00
namespace
{
void checkTTLExpression ( const ExpressionActionsPtr & ttl_expression , const String & result_column_name )
{
for ( const auto & action : ttl_expression - > getActions ( ) )
{
if ( action . type = = ExpressionAction : : APPLY_FUNCTION )
{
IFunctionBase & func = * action . function_base ;
if ( ! func . isDeterministic ( ) )
throw Exception ( " TTL expression cannot contain non-deterministic functions, "
" but contains function " + func . getName ( ) , ErrorCodes : : BAD_ARGUMENTS ) ;
}
}
const auto & result_column = ttl_expression - > getSampleBlock ( ) . getByName ( result_column_name ) ;
if ( ! typeid_cast < const DataTypeDateTime * > ( result_column . type . get ( ) )
& & ! typeid_cast < const DataTypeDate * > ( result_column . type . get ( ) ) )
{
throw Exception ( " TTL expression result column should have DateTime or Date type, but has "
+ result_column . type - > getName ( ) , ErrorCodes : : BAD_TTL_EXPRESSION ) ;
}
}
}
2020-04-03 08:59:47 +00:00
void MergeTreeData : : setTTLExpressions ( const ColumnsDescription & new_columns ,
2019-04-15 09:30:45 +00:00
const ASTPtr & new_ttl_table_ast , bool only_check )
{
2020-04-02 16:11:10 +00:00
2020-04-03 08:59:47 +00:00
auto new_column_ttls = new_columns . getColumnTTLs ( ) ;
2020-04-02 16:11:10 +00:00
2020-04-03 08:59:47 +00:00
auto create_ttl_entry = [ this , & new_columns ] ( ASTPtr ttl_ast )
2019-04-15 09:30:45 +00:00
{
2019-12-05 08:05:07 +00:00
TTLEntry result ;
2019-04-15 09:30:45 +00:00
2020-04-03 08:59:47 +00:00
auto syntax_result = SyntaxAnalyzer ( global_context ) . analyze ( ttl_ast , new_columns . getAllPhysical ( ) ) ;
2019-12-05 08:05:07 +00:00
result . expression = ExpressionAnalyzer ( ttl_ast , syntax_result , global_context ) . getActions ( false ) ;
result . destination_type = PartDestinationType : : DELETE ;
result . result_column = ttl_ast - > getColumnName ( ) ;
2019-04-15 09:30:45 +00:00
2019-12-05 08:05:07 +00:00
checkTTLExpression ( result . expression , result . result_column ) ;
return result ;
2019-04-15 09:30:45 +00:00
} ;
if ( ! new_column_ttls . empty ( ) )
{
NameSet columns_ttl_forbidden ;
if ( partition_key_expr )
for ( const auto & col : partition_key_expr - > getRequiredColumns ( ) )
columns_ttl_forbidden . insert ( col ) ;
if ( sorting_key_expr )
for ( const auto & col : sorting_key_expr - > getRequiredColumns ( ) )
columns_ttl_forbidden . insert ( col ) ;
for ( const auto & [ name , ast ] : new_column_ttls )
{
if ( columns_ttl_forbidden . count ( name ) )
2019-11-26 08:02:48 +00:00
throw Exception ( " Trying to set TTL for key column " + name , ErrorCodes : : ILLEGAL_COLUMN ) ;
2019-04-15 09:30:45 +00:00
else
{
auto new_ttl_entry = create_ttl_entry ( ast ) ;
if ( ! only_check )
2020-01-22 19:52:55 +00:00
column_ttl_entries_by_name [ name ] = new_ttl_entry ;
2019-04-15 09:30:45 +00:00
}
}
}
if ( new_ttl_table_ast )
{
2020-01-17 13:16:39 +00:00
std : : vector < TTLEntry > update_move_ttl_entries ;
2020-01-22 19:52:55 +00:00
TTLEntry update_rows_ttl_entry ;
2020-01-07 14:48:14 +00:00
2019-10-22 07:55:36 +00:00
bool seen_delete_ttl = false ;
2020-03-09 01:50:33 +00:00
for ( const auto & ttl_element_ptr : new_ttl_table_ast - > children )
2019-04-15 09:30:45 +00:00
{
2020-01-22 19:52:55 +00:00
const auto * ttl_element = ttl_element_ptr - > as < ASTTTLElement > ( ) ;
if ( ! ttl_element )
throw Exception ( " Unexpected AST element in TTL expression " , ErrorCodes : : UNEXPECTED_AST_STRUCTURE ) ;
if ( ttl_element - > destination_type = = PartDestinationType : : DELETE )
2019-10-16 07:32:37 +00:00
{
2019-10-22 07:55:36 +00:00
if ( seen_delete_ttl )
2019-10-16 07:32:37 +00:00
{
2019-10-31 10:40:11 +00:00
throw Exception ( " More than one DELETE TTL expression is not allowed " , ErrorCodes : : BAD_TTL_EXPRESSION ) ;
2019-10-16 07:32:37 +00:00
}
2019-10-22 07:55:36 +00:00
2020-01-22 19:52:55 +00:00
auto new_rows_ttl_entry = create_ttl_entry ( ttl_element - > children [ 0 ] ) ;
2019-10-22 07:55:36 +00:00
if ( ! only_check )
2020-01-22 19:52:55 +00:00
update_rows_ttl_entry = new_rows_ttl_entry ;
2019-10-22 07:55:36 +00:00
seen_delete_ttl = true ;
2019-10-16 07:32:37 +00:00
}
else
{
2020-01-22 19:52:55 +00:00
auto new_ttl_entry = create_ttl_entry ( ttl_element - > children [ 0 ] ) ;
2019-12-26 14:53:31 +00:00
new_ttl_entry . entry_ast = ttl_element_ptr ;
2020-01-22 19:52:55 +00:00
new_ttl_entry . destination_type = ttl_element - > destination_type ;
new_ttl_entry . destination_name = ttl_element - > destination_name ;
2019-12-26 14:53:31 +00:00
if ( ! new_ttl_entry . getDestination ( getStoragePolicy ( ) ) )
{
String message ;
if ( new_ttl_entry . destination_type = = PartDestinationType : : DISK )
message = " No such disk " + backQuote ( new_ttl_entry . destination_name ) + " for given storage policy. " ;
else
message = " No such volume " + backQuote ( new_ttl_entry . destination_name ) + " for given storage policy. " ;
throw Exception ( message , ErrorCodes : : BAD_TTL_EXPRESSION ) ;
}
2019-10-17 16:01:28 +00:00
if ( ! only_check )
2020-01-17 13:16:39 +00:00
update_move_ttl_entries . emplace_back ( std : : move ( new_ttl_entry ) ) ;
2019-10-16 07:32:37 +00:00
}
2019-04-15 09:30:45 +00:00
}
2020-01-17 13:16:39 +00:00
if ( ! only_check )
{
2020-01-22 19:52:55 +00:00
rows_ttl_entry = update_rows_ttl_entry ;
ttl_table_ast = new_ttl_table_ast ;
2020-01-17 13:16:39 +00:00
auto move_ttl_entries_lock = std : : lock_guard < std : : mutex > ( move_ttl_entries_mutex ) ;
move_ttl_entries = update_move_ttl_entries ;
}
2019-04-15 09:30:45 +00:00
}
}
2017-08-19 18:11:20 +00:00
2020-04-22 06:22:14 +00:00
void MergeTreeData : : checkStoragePolicy ( const StoragePolicyPtr & new_storage_policy ) const
2020-01-09 13:52:37 +00:00
{
const auto old_storage_policy = getStoragePolicy ( ) ;
2020-01-09 14:50:34 +00:00
old_storage_policy - > checkCompatibleWith ( new_storage_policy ) ;
2020-01-09 13:52:37 +00:00
}
2017-12-25 21:57:29 +00:00
void MergeTreeData : : MergingParams : : check ( const NamesAndTypesList & columns ) const
2016-04-15 17:42:51 +00:00
{
2018-02-02 09:46:54 +00:00
if ( ! sign_column . empty ( ) & & mode ! = MergingParams : : Collapsing & & mode ! = MergingParams : : VersionedCollapsing )
throw Exception ( " Sign column for MergeTree cannot be specified in modes except Collapsing or VersionedCollapsing. " ,
2018-01-29 17:42:19 +00:00
ErrorCodes : : LOGICAL_ERROR ) ;
2018-02-02 09:46:54 +00:00
if ( ! version_column . empty ( ) & & mode ! = MergingParams : : Replacing & & mode ! = MergingParams : : VersionedCollapsing )
throw Exception ( " Version column for MergeTree cannot be specified in modes except Replacing or VersionedCollapsing. " ,
2018-01-29 17:42:19 +00:00
ErrorCodes : : LOGICAL_ERROR ) ;
if ( ! columns_to_sum . empty ( ) & & mode ! = MergingParams : : Summing )
throw Exception ( " List of columns to sum for MergeTree cannot be specified in all modes except Summing. " ,
ErrorCodes : : LOGICAL_ERROR ) ;
2017-04-01 07:20:54 +00:00
/// Check that if the sign column is needed, it exists and is of type Int8.
2018-01-29 17:42:19 +00:00
auto check_sign_column = [ this , & columns ] ( bool is_optional , const std : : string & storage )
2017-04-01 07:20:54 +00:00
{
if ( sign_column . empty ( ) )
2018-01-29 17:42:19 +00:00
{
if ( is_optional )
return ;
throw Exception ( " Logical error: Sign column for storage " + storage + " is empty " , ErrorCodes : : LOGICAL_ERROR ) ;
}
2017-04-01 07:20:54 +00:00
2018-01-01 12:00:05 +00:00
bool miss_column = true ;
2017-04-01 07:20:54 +00:00
for ( const auto & column : columns )
{
if ( column . name = = sign_column )
{
if ( ! typeid_cast < const DataTypeInt8 * > ( column . type . get ( ) ) )
2018-01-29 17:42:19 +00:00
throw Exception ( " Sign column ( " + sign_column + " ) for storage " + storage + " must have type Int8. "
" Provided column of type " + column . type - > getName ( ) + " . " , ErrorCodes : : BAD_TYPE_OF_FIELD ) ;
2018-01-01 12:00:05 +00:00
miss_column = false ;
2017-04-01 07:20:54 +00:00
break ;
}
}
2018-01-09 18:00:19 +00:00
if ( miss_column )
2018-11-22 21:19:58 +00:00
throw Exception ( " Sign column " + sign_column + " does not exist in table declaration. " , ErrorCodes : : NO_SUCH_COLUMN_IN_TABLE ) ;
2018-01-29 17:42:19 +00:00
} ;
2017-04-01 07:20:54 +00:00
2018-01-29 17:42:19 +00:00
/// that if the version_column column is needed, it exists and is of unsigned integer type.
auto check_version_column = [ this , & columns ] ( bool is_optional , const std : : string & storage )
2017-04-01 07:20:54 +00:00
{
2018-01-29 17:42:19 +00:00
if ( version_column . empty ( ) )
{
if ( is_optional )
return ;
2017-04-01 07:20:54 +00:00
2018-01-29 17:42:19 +00:00
throw Exception ( " Logical error: Version column for storage " + storage + " is empty " , ErrorCodes : : LOGICAL_ERROR ) ;
}
2017-04-01 07:20:54 +00:00
2018-01-01 12:00:05 +00:00
bool miss_column = true ;
2017-04-01 07:20:54 +00:00
for ( const auto & column : columns )
{
if ( column . name = = version_column )
{
2018-03-06 14:49:27 +00:00
if ( ! column . type - > canBeUsedAsVersion ( ) )
throw Exception ( " The column " + version_column +
" cannot be used as a version column for storage " + storage +
" because it is of type " + column . type - > getName ( ) +
2018-03-06 19:01:45 +00:00
" (must be of an integer type or of type Date or DateTime) " , ErrorCodes : : BAD_TYPE_OF_FIELD ) ;
2018-01-01 12:00:05 +00:00
miss_column = false ;
2017-04-01 07:20:54 +00:00
break ;
}
}
2018-01-09 18:00:19 +00:00
if ( miss_column )
2018-11-22 21:19:58 +00:00
throw Exception ( " Version column " + version_column + " does not exist in table declaration. " , ErrorCodes : : NO_SUCH_COLUMN_IN_TABLE ) ;
2018-01-29 17:42:19 +00:00
} ;
if ( mode = = MergingParams : : Collapsing )
check_sign_column ( false , " CollapsingMergeTree " ) ;
if ( mode = = MergingParams : : Summing )
{
/// If columns_to_sum are set, then check that such columns exist.
for ( const auto & column_to_sum : columns_to_sum )
2018-02-02 12:14:30 +00:00
{
auto check_column_to_sum_exists = [ & column_to_sum ] ( const NameAndTypePair & name_and_type )
{
return column_to_sum = = Nested : : extractTableName ( name_and_type . name ) ;
} ;
if ( columns . end ( ) = = std : : find_if ( columns . begin ( ) , columns . end ( ) , check_column_to_sum_exists ) )
throw Exception (
2018-11-22 21:19:58 +00:00
" Column " + column_to_sum + " listed in columns to sum does not exist in table declaration. " , ErrorCodes : : NO_SUCH_COLUMN_IN_TABLE ) ;
2018-02-02 12:14:30 +00:00
}
2018-01-29 17:42:19 +00:00
}
if ( mode = = MergingParams : : Replacing )
check_version_column ( true , " ReplacingMergeTree " ) ;
2018-02-02 09:46:54 +00:00
if ( mode = = MergingParams : : VersionedCollapsing )
2018-01-29 17:42:19 +00:00
{
2018-02-02 09:46:54 +00:00
check_sign_column ( false , " VersionedCollapsingMergeTree " ) ;
check_version_column ( false , " VersionedCollapsingMergeTree " ) ;
2017-04-01 07:20:54 +00:00
}
/// TODO Checks for Graphite mode.
2016-04-24 09:44:47 +00:00
}
String MergeTreeData : : MergingParams : : getModeName ( ) const
{
2017-04-01 07:20:54 +00:00
switch ( mode )
{
2017-09-11 22:40:51 +00:00
case Ordinary : return " " ;
case Collapsing : return " Collapsing " ;
case Summing : return " Summing " ;
case Aggregating : return " Aggregating " ;
2017-04-01 07:20:54 +00:00
case Replacing : return " Replacing " ;
2017-09-11 22:40:51 +00:00
case Graphite : return " Graphite " ;
2018-11-26 00:56:50 +00:00
case VersionedCollapsing : return " VersionedCollapsing " ;
2017-04-01 07:20:54 +00:00
}
2019-01-05 03:33:22 +00:00
__builtin_unreachable ( ) ;
2016-04-15 17:42:51 +00:00
}
2019-05-04 03:45:58 +00:00
Int64 MergeTreeData : : getMaxBlockNumber ( ) const
2014-03-13 12:48:07 +00:00
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2016-01-30 00:57:35 +00:00
2018-05-13 00:24:23 +00:00
Int64 max_block_num = 0 ;
2018-02-19 15:31:43 +00:00
for ( const DataPartPtr & part : data_parts_by_info )
2018-07-16 03:14:46 +00:00
max_block_num = std : : max ( { max_block_num , part - > info . max_block , part - > info . mutation } ) ;
2014-09-29 05:03:03 +00:00
2018-05-13 00:24:23 +00:00
return max_block_num ;
2014-03-09 17:36:01 +00:00
}
2014-08-13 08:07:52 +00:00
void MergeTreeData : : loadDataParts ( bool skip_sanity_checks )
2014-03-09 17:36:01 +00:00
{
2020-05-23 16:42:39 +00:00
LOG_DEBUG_FORMATTED ( log , " Loading data parts " ) ;
2017-04-01 07:20:54 +00:00
2019-08-26 14:24:29 +00:00
const auto settings = getSettings ( ) ;
2019-11-27 09:39:44 +00:00
std : : vector < std : : pair < String , DiskPtr > > part_names_with_disks ;
2017-04-01 07:20:54 +00:00
Strings part_file_names ;
2019-04-28 14:49:41 +00:00
2020-01-09 14:50:34 +00:00
auto disks = getStoragePolicy ( ) - > getDisks ( ) ;
2019-04-28 14:49:41 +00:00
2019-12-27 18:58:59 +00:00
/// Only check if user did touch storage configuration for this table.
2019-12-28 07:30:12 +00:00
if ( ! getStoragePolicy ( ) - > isDefaultPolicy ( ) & & ! skip_sanity_checks )
2019-12-10 09:31:26 +00:00
{
/// Check extra parts at different disks, in order to not allow to miss data parts at undefined disks.
2019-12-10 14:15:42 +00:00
std : : unordered_set < String > defined_disk_names ;
for ( const auto & disk_ptr : disks )
defined_disk_names . insert ( disk_ptr - > getName ( ) ) ;
2020-05-21 14:11:56 +00:00
for ( const auto & [ disk_name , disk ] : global_context . getDisksMap ( ) )
2019-12-10 09:31:26 +00:00
{
2020-02-27 16:47:40 +00:00
if ( defined_disk_names . count ( disk_name ) = = 0 & & disk - > exists ( relative_data_path ) )
2019-12-10 09:31:26 +00:00
{
2020-02-27 16:47:40 +00:00
for ( const auto it = disk - > iterateDirectory ( relative_data_path ) ; it - > isValid ( ) ; it - > next ( ) )
2019-12-10 09:31:26 +00:00
{
2019-12-25 22:12:17 +00:00
MergeTreePartInfo part_info ;
2020-02-27 16:47:40 +00:00
if ( MergeTreePartInfo : : tryParsePartName ( it - > name ( ) , & part_info , format_version ) )
throw Exception ( " Part " + backQuote ( it - > name ( ) ) + " was found on disk " + backQuote ( disk_name ) + " which is not defined in the storage policy " , ErrorCodes : : UNKNOWN_DISK ) ;
2019-12-10 09:31:26 +00:00
}
}
}
}
2019-07-16 13:06:23 +00:00
/// Reversed order to load part from low priority disks firstly.
/// Used for keep part on low priority disk if duplication found
for ( auto disk_it = disks . rbegin ( ) ; disk_it ! = disks . rend ( ) ; + + disk_it )
2017-04-01 07:20:54 +00:00
{
2019-07-16 13:06:23 +00:00
auto disk_ptr = * disk_it ;
2020-02-27 16:47:40 +00:00
for ( auto it = disk_ptr - > iterateDirectory ( relative_data_path ) ; it - > isValid ( ) ; it - > next ( ) )
2019-04-01 18:41:19 +00:00
{
/// Skip temporary directories.
2020-02-27 16:47:40 +00:00
if ( startsWith ( it - > name ( ) , " tmp " ) )
2019-04-01 18:41:19 +00:00
continue ;
2017-04-01 07:20:54 +00:00
2020-02-27 16:47:40 +00:00
part_names_with_disks . emplace_back ( it - > name ( ) , disk_ptr ) ;
2019-04-01 18:41:19 +00:00
}
2017-04-01 07:20:54 +00:00
}
2019-08-11 20:02:51 +00:00
auto part_lock = lockParts ( ) ;
data_parts_indexes . clear ( ) ;
2019-08-14 09:46:30 +00:00
if ( part_names_with_disks . empty ( ) )
2019-08-11 20:02:51 +00:00
{
2020-05-23 16:42:39 +00:00
LOG_DEBUG_FORMATTED ( log , " There is no data parts " ) ;
2019-08-11 20:02:51 +00:00
return ;
}
/// Parallel loading of data parts.
2019-08-29 16:17:47 +00:00
size_t num_threads = std : : min ( size_t ( settings - > max_part_loading_threads ) , part_names_with_disks . size ( ) ) ;
2019-08-11 20:02:51 +00:00
std : : mutex mutex ;
2017-04-01 07:20:54 +00:00
DataPartsVector broken_parts_to_remove ;
DataPartsVector broken_parts_to_detach ;
size_t suspicious_broken_parts = 0 ;
2019-08-11 20:02:51 +00:00
std : : atomic < bool > has_adaptive_parts = false ;
std : : atomic < bool > has_non_adaptive_parts = false ;
ThreadPool pool ( num_threads ) ;
2017-11-20 19:33:12 +00:00
2019-08-14 09:46:30 +00:00
for ( size_t i = 0 ; i < part_names_with_disks . size ( ) ; + + i )
2017-04-01 07:20:54 +00:00
{
2019-10-17 14:41:27 +00:00
pool . scheduleOrThrowOnError ( [ & , i ]
2017-04-01 07:20:54 +00:00
{
2019-08-14 11:59:45 +00:00
const auto & part_name = part_names_with_disks [ i ] . first ;
const auto part_disk_ptr = part_names_with_disks [ i ] . second ;
2019-10-31 14:44:17 +00:00
2019-08-11 20:02:51 +00:00
MergeTreePartInfo part_info ;
2019-08-14 09:46:30 +00:00
if ( ! MergeTreePartInfo : : tryParsePartName ( part_name , & part_info , format_version ) )
2019-08-11 20:02:51 +00:00
return ;
2017-04-01 07:20:54 +00:00
2020-05-09 21:24:15 +00:00
auto single_disk_volume = std : : make_shared < SingleDiskVolume > ( " volume_ " + part_name , part_disk_ptr ) ;
auto part = createPart ( part_name , part_info , single_disk_volume , part_name ) ;
2019-08-11 20:02:51 +00:00
bool broken = false ;
2020-02-27 16:47:40 +00:00
String part_path = relative_data_path + " / " + part_name ;
String marker_path = part_path + " / " + DELETE_ON_DESTROY_MARKER_PATH ;
if ( part_disk_ptr - > exists ( marker_path ) )
2019-12-09 13:44:11 +00:00
{
2019-12-09 16:20:56 +00:00
LOG_WARNING ( log , " Detaching stale part " < < getFullPathOnDisk ( part_disk_ptr ) < < part_name < < " , which should have been deleted after a move. That can only happen after unclean restart of ClickHouse after move of a part having an operation blocking that stale copy of part. " ) ;
2019-12-09 13:44:11 +00:00
std : : lock_guard loading_lock ( mutex ) ;
broken_parts_to_detach . push_back ( part ) ;
+ + suspicious_broken_parts ;
2019-12-09 16:20:56 +00:00
return ;
2019-12-09 13:44:11 +00:00
}
2019-08-11 20:02:51 +00:00
try
2017-04-01 07:20:54 +00:00
{
2019-08-11 20:02:51 +00:00
part - > loadColumnsChecksumsIndexes ( require_part_metadata , true ) ;
2017-04-01 07:20:54 +00:00
}
2019-08-11 20:02:51 +00:00
catch ( const Exception & e )
2017-04-01 07:20:54 +00:00
{
2019-08-11 20:02:51 +00:00
/// Don't count the part as broken if there is not enough memory to load it.
/// In fact, there can be many similar situations.
/// But it is OK, because there is a safety guard against deleting too many parts.
2020-04-20 01:44:24 +00:00
if ( isNotEnoughMemoryErrorCode ( e . code ( ) ) )
2019-08-11 20:02:51 +00:00
throw ;
broken = true ;
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
catch ( . . . )
{
broken = true ;
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
2017-04-01 07:20:54 +00:00
2019-08-11 20:02:51 +00:00
/// Ignore and possibly delete broken parts that can appear as a result of hard server restart.
if ( broken )
{
if ( part - > info . level = = 0 )
2017-04-01 07:20:54 +00:00
{
2019-08-11 20:02:51 +00:00
/// It is impossible to restore level 0 parts.
2019-08-14 09:46:30 +00:00
LOG_ERROR ( log , " Considering to remove broken part " < < getFullPathOnDisk ( part_disk_ptr ) < < part_name < < " because it's impossible to repair. " ) ;
2019-08-11 20:02:51 +00:00
std : : lock_guard loading_lock ( mutex ) ;
broken_parts_to_remove . push_back ( part ) ;
}
else
{
/// Count the number of parts covered by the broken part. If it is at least two, assume that
/// the broken part was created as a result of merging them and we won't lose data if we
/// delete it.
size_t contained_parts = 0 ;
2017-04-18 20:38:07 +00:00
2019-08-14 09:46:30 +00:00
LOG_ERROR ( log , " Part " < < getFullPathOnDisk ( part_disk_ptr ) < < part_name < < " is broken. Looking for parts to replace it. " ) ;
2017-04-18 20:38:07 +00:00
2019-08-14 09:46:30 +00:00
for ( const auto & [ contained_name , contained_disk_ptr ] : part_names_with_disks )
2017-04-01 07:20:54 +00:00
{
2019-08-14 09:46:30 +00:00
if ( contained_name = = part_name )
2019-08-11 20:02:51 +00:00
continue ;
MergeTreePartInfo contained_part_info ;
if ( ! MergeTreePartInfo : : tryParsePartName ( contained_name , & contained_part_info , format_version ) )
continue ;
if ( part - > info . contains ( contained_part_info ) )
{
2019-08-14 09:46:30 +00:00
LOG_ERROR ( log , " Found part " < < getFullPathOnDisk ( contained_disk_ptr ) < < contained_name ) ;
2019-08-11 20:02:51 +00:00
+ + contained_parts ;
}
2017-04-01 07:20:54 +00:00
}
2019-08-11 20:02:51 +00:00
if ( contained_parts > = 2 )
{
2019-08-14 09:46:30 +00:00
LOG_ERROR ( log , " Considering to remove broken part " < < getFullPathOnDisk ( part_disk_ptr ) < < part_name < < " because it covers at least 2 other parts " ) ;
2019-08-11 20:02:51 +00:00
std : : lock_guard loading_lock ( mutex ) ;
broken_parts_to_remove . push_back ( part ) ;
}
else
{
2020-05-23 18:53:58 +00:00
LOG_ERROR ( log , " Detaching broken part " < < getFullPathOnDisk ( part_disk_ptr ) < < part_name < < " because it covers less than 2 parts. You need to resolve this manually " ) ;
2019-08-11 20:02:51 +00:00
std : : lock_guard loading_lock ( mutex ) ;
broken_parts_to_detach . push_back ( part ) ;
+ + suspicious_broken_parts ;
}
2017-04-01 07:20:54 +00:00
}
2019-08-11 20:02:51 +00:00
return ;
}
if ( ! part - > index_granularity_info . is_adaptive )
has_non_adaptive_parts . store ( true , std : : memory_order_relaxed ) ;
else
has_adaptive_parts . store ( true , std : : memory_order_relaxed ) ;
2017-04-01 07:20:54 +00:00
2020-03-05 14:02:15 +00:00
part - > modification_time = part_disk_ptr - > getLastModified ( relative_data_path + part_name ) . epochTime ( ) ;
2019-08-11 20:02:51 +00:00
/// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later
part - > state = DataPartState : : Committed ;
2017-04-01 07:20:54 +00:00
2019-08-11 20:02:51 +00:00
std : : lock_guard loading_lock ( mutex ) ;
if ( ! data_parts_indexes . insert ( part ) . second )
throw Exception ( " Part " + part - > name + " already exists " , ErrorCodes : : DUPLICATE_DATA_PART ) ;
} ) ;
2017-04-01 07:20:54 +00:00
}
2019-08-11 20:02:51 +00:00
pool . wait ( ) ;
2019-08-13 10:29:31 +00:00
if ( has_non_adaptive_parts & & has_adaptive_parts & & ! settings - > enable_mixed_granularity_parts )
2019-06-19 14:46:06 +00:00
throw Exception ( " Table contains parts with adaptive and non adaptive marks, but `setting enable_mixed_granularity_parts` is disabled " , ErrorCodes : : LOGICAL_ERROR ) ;
has_non_adaptive_index_granularity_parts = has_non_adaptive_parts ;
2019-08-13 10:29:31 +00:00
if ( suspicious_broken_parts > settings - > max_suspicious_broken_parts & & ! skip_sanity_checks )
2017-04-01 07:20:54 +00:00
throw Exception ( " Suspiciously many ( " + toString ( suspicious_broken_parts ) + " ) broken parts to remove. " ,
ErrorCodes : : TOO_MANY_UNEXPECTED_DATA_PARTS ) ;
2017-05-16 15:40:32 +00:00
for ( auto & part : broken_parts_to_remove )
2017-04-01 07:20:54 +00:00
part - > remove ( ) ;
2017-05-16 15:40:32 +00:00
for ( auto & part : broken_parts_to_detach )
2018-05-21 13:49:54 +00:00
part - > renameToDetached ( " " ) ;
2017-04-01 07:20:54 +00:00
/// Delete from the set of current parts those parts that are covered by another part (those parts that
2017-06-21 19:07:08 +00:00
/// were merged), but that for some reason are still not deleted from the filesystem.
2017-04-01 07:20:54 +00:00
/// Deletion of files will be performed later in the clearOldParts() method.
2017-11-20 19:33:12 +00:00
if ( data_parts_indexes . size ( ) > = 2 )
2017-04-01 07:20:54 +00:00
{
2018-02-19 15:31:43 +00:00
/// Now all parts are committed, so data_parts_by_state_and_info == committed_parts_range
auto prev_jt = data_parts_by_state_and_info . begin ( ) ;
2017-09-21 21:51:17 +00:00
auto curr_jt = std : : next ( prev_jt ) ;
2018-02-19 15:31:43 +00:00
auto deactivate_part = [ & ] ( DataPartIteratorByStateAndInfo it )
2017-11-20 19:33:12 +00:00
{
2018-03-03 17:44:53 +00:00
( * it ) - > remove_time . store ( ( * it ) - > modification_time , std : : memory_order_relaxed ) ;
2017-11-20 19:33:12 +00:00
modifyPartState ( it , DataPartState : : Outdated ) ;
} ;
( * prev_jt ) - > assertState ( { DataPartState : : Committed } ) ;
2018-02-19 15:31:43 +00:00
while ( curr_jt ! = data_parts_by_state_and_info . end ( ) & & ( * curr_jt ) - > state = = DataPartState : : Committed )
2017-04-01 07:20:54 +00:00
{
2017-08-14 18:16:11 +00:00
/// Don't consider data parts belonging to different partitions.
if ( ( * curr_jt ) - > info . partition_id ! = ( * prev_jt ) - > info . partition_id )
2017-04-01 07:20:54 +00:00
{
+ + prev_jt ;
+ + curr_jt ;
continue ;
}
if ( ( * curr_jt ) - > contains ( * * prev_jt ) )
{
2017-11-20 19:33:12 +00:00
deactivate_part ( prev_jt ) ;
2017-04-01 07:20:54 +00:00
prev_jt = curr_jt ;
+ + curr_jt ;
}
else if ( ( * prev_jt ) - > contains ( * * curr_jt ) )
{
2017-11-20 19:33:12 +00:00
auto next = std : : next ( curr_jt ) ;
deactivate_part ( curr_jt ) ;
curr_jt = next ;
2017-04-01 07:20:54 +00:00
}
else
{
+ + prev_jt ;
+ + curr_jt ;
}
}
}
2017-05-14 23:14:21 +00:00
calculateColumnSizesImpl ( ) ;
2017-04-01 07:20:54 +00:00
2020-05-23 16:56:05 +00:00
LOG_DEBUG_FORMATTED ( log , " Loaded data parts ({} items) " , data_parts_indexes . size ( ) ) ;
2014-03-09 17:36:01 +00:00
}
2017-02-07 17:52:41 +00:00
/// Is the part directory old.
/// True if its modification time and the modification time of all files inside it is less then threshold.
/// (Only files on the first level of nesting are considered).
2020-03-05 14:02:15 +00:00
static bool isOldPartDirectory ( const DiskPtr & disk , const String & directory_path , time_t threshold )
2016-06-06 19:16:34 +00:00
{
2020-03-05 14:02:15 +00:00
if ( disk - > getLastModified ( directory_path ) . epochTime ( ) > = threshold )
2017-04-01 07:20:54 +00:00
return false ;
2016-06-06 19:16:34 +00:00
2020-03-05 14:02:15 +00:00
for ( auto it = disk - > iterateDirectory ( directory_path ) ; it - > isValid ( ) ; it - > next ( ) )
if ( disk - > getLastModified ( it - > path ( ) ) . epochTime ( ) > = threshold )
2017-04-01 07:20:54 +00:00
return false ;
2016-06-06 19:16:34 +00:00
2017-04-01 07:20:54 +00:00
return true ;
2016-06-06 19:16:34 +00:00
}
2017-05-31 15:01:25 +00:00
void MergeTreeData : : clearOldTemporaryDirectories ( ssize_t custom_directories_lifetime_seconds )
2014-03-09 17:36:01 +00:00
{
2017-04-01 07:20:54 +00:00
/// If the method is already called from another thread, then we don't need to do anything.
2019-01-02 06:44:36 +00:00
std : : unique_lock lock ( clear_old_temporary_directories_mutex , std : : defer_lock ) ;
2017-04-01 07:20:54 +00:00
if ( ! lock . try_lock ( ) )
return ;
2019-08-26 14:24:29 +00:00
const auto settings = getSettings ( ) ;
2017-08-04 14:00:26 +00:00
time_t current_time = time ( nullptr ) ;
2017-05-31 15:01:25 +00:00
ssize_t deadline = ( custom_directories_lifetime_seconds > = 0 )
? current_time - custom_directories_lifetime_seconds
2019-08-13 10:29:31 +00:00
: current_time - settings - > temporary_directories_lifetime . totalSeconds ( ) ;
2017-04-01 07:20:54 +00:00
/// Delete temporary directories older than a day.
2020-02-27 16:47:40 +00:00
for ( const auto & [ path , disk ] : getRelativeDataPathsWithDisks ( ) )
2017-04-01 07:20:54 +00:00
{
2020-02-27 16:47:40 +00:00
for ( auto it = disk - > iterateDirectory ( path ) ; it - > isValid ( ) ; it - > next ( ) )
2017-04-01 07:20:54 +00:00
{
2020-02-27 16:47:40 +00:00
if ( startsWith ( it - > name ( ) , " tmp_ " ) )
2017-04-01 07:20:54 +00:00
{
2019-04-05 17:37:27 +00:00
try
2017-04-01 07:20:54 +00:00
{
2020-03-05 14:02:15 +00:00
if ( disk - > isDirectory ( it - > path ( ) ) & & isOldPartDirectory ( disk , it - > path ( ) , deadline ) )
2019-04-05 17:37:27 +00:00
{
2020-05-23 16:47:56 +00:00
LOG_WARNING_FORMATTED ( log , " Removing temporary directory {} " , fullPath ( disk , it - > path ( ) ) ) ;
2020-03-05 14:02:15 +00:00
disk - > removeRecursive ( it - > path ( ) ) ;
2019-04-01 18:41:19 +00:00
}
}
2019-04-05 17:37:27 +00:00
catch ( const Poco : : FileNotFoundException & )
{
2019-04-01 18:41:19 +00:00
/// If the file is already deleted, do nothing.
2017-04-01 07:20:54 +00:00
}
}
}
}
2016-02-14 11:02:47 +00:00
}
2020-01-10 09:46:24 +00:00
MergeTreeData : : DataPartsVector MergeTreeData : : grabOldParts ( bool force )
2016-02-14 11:02:47 +00:00
{
2017-04-01 07:20:54 +00:00
DataPartsVector res ;
/// If the method is already called from another thread, then we don't need to do anything.
2019-01-02 06:44:36 +00:00
std : : unique_lock lock ( grab_old_parts_mutex , std : : defer_lock ) ;
2017-04-01 07:20:54 +00:00
if ( ! lock . try_lock ( ) )
return res ;
2017-08-04 14:00:26 +00:00
time_t now = time ( nullptr ) ;
2018-02-19 15:31:43 +00:00
std : : vector < DataPartIteratorByStateAndInfo > parts_to_delete ;
2017-04-01 07:20:54 +00:00
{
2019-03-28 19:58:41 +00:00
auto parts_lock = lockParts ( ) ;
2017-04-01 07:20:54 +00:00
2017-11-20 19:33:12 +00:00
auto outdated_parts_range = getDataPartsStateRange ( DataPartState : : Outdated ) ;
for ( auto it = outdated_parts_range . begin ( ) ; it ! = outdated_parts_range . end ( ) ; + + it )
2017-04-01 07:20:54 +00:00
{
2017-11-20 19:33:12 +00:00
const DataPartPtr & part = * it ;
2018-03-03 17:44:53 +00:00
auto part_remove_time = part - > remove_time . load ( std : : memory_order_relaxed ) ;
2018-02-19 15:31:43 +00:00
if ( part . unique ( ) & & /// Grab only parts that are not used by anyone (SELECTs for example).
2020-01-10 09:46:24 +00:00
( ( part_remove_time < now & &
now - part_remove_time > getSettings ( ) - > old_parts_lifetime . totalSeconds ( ) ) | | force ) )
2017-04-01 07:20:54 +00:00
{
2017-11-20 19:33:12 +00:00
parts_to_delete . emplace_back ( it ) ;
2017-04-01 07:20:54 +00:00
}
}
2017-11-20 19:33:12 +00:00
res . reserve ( parts_to_delete . size ( ) ) ;
for ( const auto & it_to_delete : parts_to_delete )
{
res . emplace_back ( * it_to_delete ) ;
modifyPartState ( it_to_delete , DataPartState : : Deleting ) ;
}
2017-04-01 07:20:54 +00:00
}
if ( ! res . empty ( ) )
2020-05-23 16:56:05 +00:00
LOG_TRACE_FORMATTED ( log , " Found {} old parts to remove. " , res . size ( ) ) ;
2017-04-01 07:20:54 +00:00
return res ;
2014-03-09 17:36:01 +00:00
}
2016-02-14 11:02:47 +00:00
2017-09-11 22:40:51 +00:00
void MergeTreeData : : rollbackDeletingParts ( const MergeTreeData : : DataPartsVector & parts )
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2020-04-22 06:22:14 +00:00
for ( const auto & part : parts )
2017-09-11 22:40:51 +00:00
{
/// We should modify it under data_parts_mutex
2017-09-21 21:51:17 +00:00
part - > assertState ( { DataPartState : : Deleting } ) ;
2017-11-20 19:33:12 +00:00
modifyPartState ( part , DataPartState : : Outdated ) ;
2017-09-11 22:40:51 +00:00
}
}
void MergeTreeData : : removePartsFinally ( const MergeTreeData : : DataPartsVector & parts )
2014-07-25 11:15:11 +00:00
{
2018-01-23 22:56:46 +00:00
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2017-09-11 22:40:51 +00:00
2018-01-23 22:56:46 +00:00
/// TODO: use data_parts iterators instead of pointers
2020-04-22 06:22:14 +00:00
for ( const auto & part : parts )
2018-01-23 22:56:46 +00:00
{
2018-02-19 15:31:43 +00:00
auto it = data_parts_by_info . find ( part - > info ) ;
if ( it = = data_parts_by_info . end ( ) )
throw Exception ( " Deleting data part " + part - > name + " doesn't exist " , ErrorCodes : : LOGICAL_ERROR ) ;
2018-01-23 22:56:46 +00:00
( * it ) - > assertState ( { DataPartState : : Deleting } ) ;
data_parts_indexes . erase ( it ) ;
}
}
/// Data parts is still alive (since DataPartsVector holds shared_ptrs) and contain useful metainformation for logging
/// NOTE: There is no need to log parts deletion somewhere else, all deleting parts pass through this function and pass away
2019-12-03 16:25:32 +00:00
auto table_id = getStorageID ( ) ;
2019-12-12 12:30:31 +00:00
if ( auto part_log = global_context . getPartLog ( table_id . database_name ) )
2017-09-11 22:40:51 +00:00
{
2018-01-23 22:56:46 +00:00
PartLogElement part_log_elem ;
part_log_elem . event_type = PartLogElement : : REMOVE_PART ;
part_log_elem . event_time = time ( nullptr ) ;
part_log_elem . duration_ms = 0 ;
2017-09-11 22:40:51 +00:00
2019-12-03 16:25:32 +00:00
part_log_elem . database_name = table_id . database_name ;
part_log_elem . table_name = table_id . table_name ;
2017-11-20 19:33:12 +00:00
2020-04-22 06:22:14 +00:00
for ( const auto & part : parts )
2018-01-23 22:56:46 +00:00
{
2019-01-31 17:30:56 +00:00
part_log_elem . partition_id = part - > info . partition_id ;
2018-01-23 22:56:46 +00:00
part_log_elem . part_name = part - > name ;
2020-03-23 13:32:02 +00:00
part_log_elem . bytes_compressed_on_disk = part - > getBytesOnDisk ( ) ;
2018-01-23 22:56:46 +00:00
part_log_elem . rows = part - > rows_count ;
part_log - > add ( part_log_elem ) ;
}
2017-09-11 22:40:51 +00:00
}
2014-07-25 11:15:11 +00:00
}
2020-01-10 09:46:24 +00:00
void MergeTreeData : : clearOldPartsFromFilesystem ( bool force )
2014-07-25 11:15:11 +00:00
{
2020-01-10 09:46:24 +00:00
DataPartsVector parts_to_remove = grabOldParts ( force ) ;
2019-08-11 19:14:42 +00:00
clearPartsFromFilesystem ( parts_to_remove ) ;
removePartsFinally ( parts_to_remove ) ;
}
2014-07-25 11:15:11 +00:00
2019-08-11 19:14:42 +00:00
void MergeTreeData : : clearPartsFromFilesystem ( const DataPartsVector & parts_to_remove )
{
2019-08-26 14:24:29 +00:00
const auto settings = getSettings ( ) ;
2019-08-13 10:29:31 +00:00
if ( parts_to_remove . size ( ) > 1 & & settings - > max_part_removal_threads > 1 & & parts_to_remove . size ( ) > settings - > concurrent_part_removal_threshold )
2017-04-01 07:20:54 +00:00
{
2019-08-11 19:14:42 +00:00
/// Parallel parts removal.
2017-11-20 19:33:12 +00:00
2019-08-13 10:29:31 +00:00
size_t num_threads = std : : min ( size_t ( settings - > max_part_removal_threads ) , parts_to_remove . size ( ) ) ;
2019-08-11 19:14:42 +00:00
ThreadPool pool ( num_threads ) ;
/// NOTE: Under heavy system load you may get "Cannot schedule a task" from ThreadPool.
2019-08-11 19:30:39 +00:00
for ( const DataPartPtr & part : parts_to_remove )
2019-08-11 19:14:42 +00:00
{
2019-10-17 14:41:27 +00:00
pool . scheduleOrThrowOnError ( [ & ]
2019-08-11 19:14:42 +00:00
{
2020-05-23 16:47:56 +00:00
LOG_DEBUG_FORMATTED ( log , " Removing part from filesystem {} " , part - > name ) ;
2019-08-11 19:30:39 +00:00
part - > remove ( ) ;
2019-08-11 19:14:42 +00:00
} ) ;
}
pool . wait ( ) ;
}
else
{
for ( const DataPartPtr & part : parts_to_remove )
{
2020-05-23 16:47:56 +00:00
LOG_DEBUG_FORMATTED ( log , " Removing part from filesystem {} " , part - > name ) ;
2019-08-11 19:14:42 +00:00
part - > remove ( ) ;
}
}
2014-07-25 11:15:11 +00:00
}
2020-04-07 14:05:51 +00:00
void MergeTreeData : : rename ( const String & new_table_path , const StorageID & new_table_id )
2014-03-09 17:36:01 +00:00
{
2020-01-09 14:50:34 +00:00
auto disks = getStoragePolicy ( ) - > getDisks ( ) ;
2019-04-28 14:49:41 +00:00
for ( const auto & disk : disks )
2019-04-05 19:58:59 +00:00
{
2019-12-03 13:37:40 +00:00
if ( disk - > exists ( new_table_path ) )
throw Exception { " Target path already exists: " + fullPath ( disk , new_table_path ) , ErrorCodes : : DIRECTORY_ALREADY_EXISTS } ;
2019-04-05 19:45:59 +00:00
}
2019-04-28 14:49:41 +00:00
for ( const auto & disk : disks )
2019-04-06 15:21:29 +00:00
{
2020-03-19 16:37:55 +00:00
auto new_table_path_parent = parentPath ( new_table_path ) ;
2020-01-16 18:13:18 +00:00
disk - > createDirectories ( new_table_path_parent ) ;
2020-01-02 14:37:31 +00:00
disk - > moveDirectory ( relative_data_path , new_table_path ) ;
2019-04-06 15:21:29 +00:00
}
2017-12-03 02:15:35 +00:00
2019-01-04 12:10:00 +00:00
global_context . dropCaches ( ) ;
2019-04-21 18:38:44 +00:00
2019-10-28 20:12:14 +00:00
relative_data_path = new_table_path ;
2020-04-07 14:05:51 +00:00
renameInMemory ( new_table_id ) ;
2014-03-09 17:36:01 +00:00
}
2014-03-13 12:48:07 +00:00
void MergeTreeData : : dropAllData ( )
2014-03-09 17:36:01 +00:00
{
2020-05-23 16:42:39 +00:00
LOG_TRACE_FORMATTED ( log , " dropAllData: waiting for locks. " ) ;
2015-09-17 21:31:26 +00:00
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2015-09-17 21:31:26 +00:00
2020-05-23 16:42:39 +00:00
LOG_TRACE_FORMATTED ( log , " dropAllData: removing data from memory. " ) ;
2015-09-17 21:31:26 +00:00
2019-08-11 19:14:42 +00:00
DataPartsVector all_parts ( data_parts_by_info . begin ( ) , data_parts_by_info . end ( ) ) ;
2017-11-20 19:33:12 +00:00
data_parts_indexes . clear ( ) ;
2017-04-01 07:20:54 +00:00
column_sizes . clear ( ) ;
2014-03-09 17:36:01 +00:00
2019-01-04 12:10:00 +00:00
global_context . dropCaches ( ) ;
2014-03-13 19:14:25 +00:00
2020-05-23 16:42:39 +00:00
LOG_TRACE_FORMATTED ( log , " dropAllData: removing data from filesystem. " ) ;
2015-09-17 21:31:26 +00:00
2019-06-16 19:59:30 +00:00
/// Removing of each data part before recursive removal of directory is to speed-up removal, because there will be less number of syscalls.
2019-08-11 19:14:42 +00:00
clearPartsFromFilesystem ( all_parts ) ;
2019-06-16 19:59:30 +00:00
2020-02-27 16:47:40 +00:00
for ( const auto & [ path , disk ] : getRelativeDataPathsWithDisks ( ) )
disk - > removeRecursive ( path ) ;
2015-09-17 21:31:26 +00:00
2020-05-23 16:42:39 +00:00
LOG_TRACE_FORMATTED ( log , " dropAllData: done. " ) ;
2014-03-09 17:36:01 +00:00
}
2017-02-08 18:43:35 +00:00
namespace
{
/// If true, then in order to ALTER the type of the column from the type from to the type to
/// we don't need to rewrite the data, we only need to update metadata and columns.txt in part directories.
2017-02-09 17:29:36 +00:00
/// The function works for Arrays and Nullables of the same structure.
2017-02-08 18:43:35 +00:00
bool isMetadataOnlyConversion ( const IDataType * from , const IDataType * to )
{
2017-04-01 07:20:54 +00:00
if ( from - > getName ( ) = = to - > getName ( ) )
return true ;
static const std : : unordered_multimap < std : : type_index , const std : : type_info & > ALLOWED_CONVERSIONS =
{
{ typeid ( DataTypeEnum8 ) , typeid ( DataTypeEnum8 ) } ,
{ typeid ( DataTypeEnum8 ) , typeid ( DataTypeInt8 ) } ,
{ typeid ( DataTypeEnum16 ) , typeid ( DataTypeEnum16 ) } ,
{ typeid ( DataTypeEnum16 ) , typeid ( DataTypeInt16 ) } ,
{ typeid ( DataTypeDateTime ) , typeid ( DataTypeUInt32 ) } ,
{ typeid ( DataTypeUInt32 ) , typeid ( DataTypeDateTime ) } ,
{ typeid ( DataTypeDate ) , typeid ( DataTypeUInt16 ) } ,
{ typeid ( DataTypeUInt16 ) , typeid ( DataTypeDate ) } ,
} ;
while ( true )
{
auto it_range = ALLOWED_CONVERSIONS . equal_range ( typeid ( * from ) ) ;
for ( auto it = it_range . first ; it ! = it_range . second ; + + it )
{
if ( it - > second = = typeid ( * to ) )
return true ;
}
const auto * arr_from = typeid_cast < const DataTypeArray * > ( from ) ;
const auto * arr_to = typeid_cast < const DataTypeArray * > ( to ) ;
if ( arr_from & & arr_to )
{
from = arr_from - > getNestedType ( ) . get ( ) ;
to = arr_to - > getNestedType ( ) . get ( ) ;
continue ;
}
const auto * nullable_from = typeid_cast < const DataTypeNullable * > ( from ) ;
const auto * nullable_to = typeid_cast < const DataTypeNullable * > ( to ) ;
if ( nullable_from & & nullable_to )
{
from = nullable_from - > getNestedType ( ) . get ( ) ;
to = nullable_to - > getNestedType ( ) . get ( ) ;
continue ;
}
return false ;
}
2017-02-08 18:43:35 +00:00
}
}
2019-12-26 18:17:05 +00:00
void MergeTreeData : : checkAlterIsPossible ( const AlterCommands & commands , const Settings & settings )
2014-03-09 17:36:01 +00:00
{
2017-04-01 07:20:54 +00:00
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
2019-12-26 18:17:05 +00:00
StorageInMemoryMetadata metadata = getInMemoryMetadata ( ) ;
commands . apply ( metadata ) ;
if ( getIndices ( ) . empty ( ) & & ! metadata . indices . empty ( ) & &
! settings . allow_experimental_data_skipping_indices )
2019-02-12 17:54:52 +00:00
throw Exception ( " You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
" before using data skipping indices. " , ErrorCodes : : BAD_ARGUMENTS ) ;
2017-04-01 07:20:54 +00:00
/// Set of columns that shouldn't be altered.
2019-12-24 18:07:51 +00:00
NameSet columns_alter_type_forbidden ;
2017-04-01 07:20:54 +00:00
2017-09-12 19:20:56 +00:00
/// Primary key columns can be ALTERed only if they are used in the key as-is
2017-04-01 07:20:54 +00:00
/// (and not as a part of some expression) and if the ALTER only affects column metadata.
2019-12-24 18:07:51 +00:00
NameSet columns_alter_type_metadata_only ;
2017-04-01 07:20:54 +00:00
2018-11-06 18:25:36 +00:00
if ( partition_key_expr )
2017-04-01 07:20:54 +00:00
{
2017-09-12 19:20:56 +00:00
/// Forbid altering partition key columns because it can change partition ID format.
/// TODO: in some cases (e.g. adding an Enum value) a partition key column can still be ALTERed.
/// We should allow it.
2018-11-06 18:25:36 +00:00
for ( const String & col : partition_key_expr - > getRequiredColumns ( ) )
2019-12-24 18:07:51 +00:00
columns_alter_type_forbidden . insert ( col ) ;
2017-09-12 19:20:56 +00:00
}
2017-08-31 13:33:32 +00:00
2019-02-05 14:50:25 +00:00
for ( const auto & index : skip_indices )
{
for ( const String & col : index - > expr - > getRequiredColumns ( ) )
2019-12-24 18:07:51 +00:00
columns_alter_type_forbidden . insert ( col ) ;
2019-02-05 14:50:25 +00:00
}
2018-10-12 19:00:43 +00:00
if ( sorting_key_expr )
2017-09-12 19:20:56 +00:00
{
2018-10-12 19:00:43 +00:00
for ( const ExpressionAction & action : sorting_key_expr - > getActions ( ) )
2017-04-01 07:20:54 +00:00
{
auto action_columns = action . getNeededColumns ( ) ;
2019-12-24 18:07:51 +00:00
columns_alter_type_forbidden . insert ( action_columns . begin ( ) , action_columns . end ( ) ) ;
2017-04-01 07:20:54 +00:00
}
2018-10-12 19:00:43 +00:00
for ( const String & col : sorting_key_expr - > getRequiredColumns ( ) )
2019-12-24 18:07:51 +00:00
columns_alter_type_metadata_only . insert ( col ) ;
2017-08-31 13:33:32 +00:00
2018-11-09 19:01:39 +00:00
/// We don't process sample_by_ast separately because it must be among the primary key columns
2018-10-12 19:00:43 +00:00
/// and we don't process primary_key_expr separately because it is a prefix of sorting_key_expr.
2018-10-11 14:53:23 +00:00
}
2017-08-31 13:33:32 +00:00
if ( ! merging_params . sign_column . empty ( ) )
2019-12-24 18:07:51 +00:00
columns_alter_type_forbidden . insert ( merging_params . sign_column ) ;
2017-04-01 07:20:54 +00:00
std : : map < String , const IDataType * > old_types ;
2018-03-13 15:00:28 +00:00
for ( const auto & column : getColumns ( ) . getAllPhysical ( ) )
2017-04-01 07:20:54 +00:00
old_types . emplace ( column . name , column . type . get ( ) ) ;
2020-04-01 18:21:27 +00:00
2017-04-01 07:20:54 +00:00
for ( const AlterCommand & command : commands )
{
2019-12-24 18:07:51 +00:00
if ( command . type = = AlterCommand : : MODIFY_ORDER_BY & & ! is_custom_partitioned )
2018-10-14 15:30:06 +00:00
{
2019-12-24 18:07:51 +00:00
throw Exception (
" ALTER MODIFY ORDER BY is not supported for default-partitioned tables created with the old syntax " ,
ErrorCodes : : BAD_ARGUMENTS ) ;
2018-10-14 15:30:06 +00:00
}
2020-01-30 07:13:09 +00:00
if ( command . type = = AlterCommand : : ADD_INDEX & & ! is_custom_partitioned )
2020-01-24 16:47:05 +00:00
{
throw Exception (
" ALTER ADD INDEX is not supported for tables with the old syntax " ,
ErrorCodes : : BAD_ARGUMENTS ) ;
}
2020-04-01 18:21:27 +00:00
if ( command . type = = AlterCommand : : RENAME_COLUMN )
{
if ( columns_alter_type_forbidden . count ( command . column_name ) | | columns_alter_type_metadata_only . count ( command . column_name ) )
{
throw Exception (
" Trying to ALTER RENAME key " + backQuoteIfNeed ( command . column_name ) + " column which is a part of key expression " ,
2020-05-17 07:47:53 +00:00
ErrorCodes : : ALTER_OF_COLUMN_IS_FORBIDDEN ) ;
2020-04-01 18:21:27 +00:00
}
}
2020-04-20 11:15:06 +00:00
else if ( command . isModifyingData ( getInMemoryMetadata ( ) ) )
2017-04-01 07:20:54 +00:00
{
2019-12-24 18:07:51 +00:00
if ( columns_alter_type_forbidden . count ( command . column_name ) )
2020-04-27 13:54:31 +00:00
throw Exception ( " ALTER of key column " + command . column_name + " is forbidden " , ErrorCodes : : ALTER_OF_COLUMN_IS_FORBIDDEN ) ;
2019-12-24 18:07:51 +00:00
if ( columns_alter_type_metadata_only . count ( command . column_name ) )
2017-04-01 07:20:54 +00:00
{
2019-12-24 18:07:51 +00:00
if ( command . type = = AlterCommand : : MODIFY_COLUMN )
{
auto it = old_types . find ( command . column_name ) ;
if ( it = = old_types . end ( ) | | ! isMetadataOnlyConversion ( it - > second , command . data_type . get ( ) ) )
2020-04-27 13:54:31 +00:00
throw Exception ( " ALTER of key column " + command . column_name + " must be metadata-only " ,
ErrorCodes : : ALTER_OF_COLUMN_IS_FORBIDDEN ) ;
2019-12-24 18:07:51 +00:00
}
2017-04-01 07:20:54 +00:00
}
2018-11-13 12:51:55 +00:00
}
2017-04-01 07:20:54 +00:00
}
2019-12-27 19:04:30 +00:00
setProperties ( metadata , /* only_check = */ true ) ;
2018-11-13 12:51:55 +00:00
2020-04-02 16:11:10 +00:00
setTTLExpressions ( metadata . columns , metadata . ttl_for_table_ast , /* only_check = */ true ) ;
2019-04-15 09:30:45 +00:00
2019-12-27 14:36:59 +00:00
if ( settings_ast )
2019-12-26 18:17:05 +00:00
{
2019-12-27 14:36:59 +00:00
const auto & current_changes = settings_ast - > as < const ASTSetQuery & > ( ) . changes ;
2020-02-14 21:33:02 +00:00
const auto & new_changes = metadata . settings_ast - > as < const ASTSetQuery & > ( ) . changes ;
for ( const auto & changed_setting : new_changes )
2019-12-27 14:36:59 +00:00
{
if ( MergeTreeSettings : : findIndex ( changed_setting . name ) = = MergeTreeSettings : : npos )
throw Exception { " Storage ' " + getName ( ) + " ' doesn't have setting ' " + changed_setting . name + " ' " ,
ErrorCodes : : UNKNOWN_SETTING } ;
auto comparator = [ & changed_setting ] ( const auto & change ) { return change . name = = changed_setting . name ; } ;
auto current_setting_it
= std : : find_if ( current_changes . begin ( ) , current_changes . end ( ) , comparator ) ;
if ( ( current_setting_it = = current_changes . end ( ) | | * current_setting_it ! = changed_setting )
& & MergeTreeSettings : : isReadonlySetting ( changed_setting . name ) )
{
throw Exception { " Setting ' " + changed_setting . name + " ' is readonly for storage ' " + getName ( ) + " ' " ,
ErrorCodes : : READONLY_SETTING } ;
}
2020-01-09 13:52:37 +00:00
2020-02-14 21:33:02 +00:00
if ( current_setting_it = = current_changes . end ( )
& & MergeTreeSettings : : isPartFormatSetting ( changed_setting . name ) )
{
MergeTreeSettings copy = * getSettings ( ) ;
copy . applyChange ( changed_setting ) ;
String reason ;
if ( ! canUsePolymorphicParts ( copy , & reason ) & & ! reason . empty ( ) )
throw Exception ( " Can't change settings. Reason: " + reason , ErrorCodes : : NOT_IMPLEMENTED ) ;
}
2020-01-09 13:52:37 +00:00
if ( changed_setting . name = = " storage_policy " )
2020-01-09 14:50:34 +00:00
checkStoragePolicy ( global_context . getStoragePolicy ( changed_setting . value . safeGet < String > ( ) ) ) ;
2019-12-27 14:36:59 +00:00
}
2019-12-26 18:17:05 +00:00
}
2014-03-09 17:36:01 +00:00
}
2019-11-25 20:19:43 +00:00
MergeTreeDataPartType MergeTreeData : : choosePartType ( size_t bytes_uncompressed , size_t rows_count ) const
2019-11-21 16:10:22 +00:00
{
2020-01-13 16:28:29 +00:00
if ( ! canUseAdaptiveGranularity ( ) )
return MergeTreeDataPartType : : WIDE ;
2017-04-01 07:20:54 +00:00
2019-11-21 16:10:22 +00:00
const auto settings = getSettings ( ) ;
2019-11-25 20:19:43 +00:00
if ( bytes_uncompressed < settings - > min_bytes_for_wide_part | | rows_count < settings - > min_rows_for_wide_part )
2019-11-21 16:10:22 +00:00
return MergeTreeDataPartType : : COMPACT ;
2017-04-01 07:20:54 +00:00
2019-11-21 16:10:22 +00:00
return MergeTreeDataPartType : : WIDE ;
}
2017-04-01 07:20:54 +00:00
2019-11-22 12:51:00 +00:00
MergeTreeData : : MutableDataPartPtr MergeTreeData : : createPart ( const String & name ,
MergeTreeDataPartType type , const MergeTreePartInfo & part_info ,
2020-05-09 21:24:15 +00:00
const VolumePtr & volume , const String & relative_path ) const
2019-11-21 16:10:22 +00:00
{
if ( type = = MergeTreeDataPartType : : COMPACT )
2020-05-09 21:24:15 +00:00
return std : : make_shared < MergeTreeDataPartCompact > ( * this , name , part_info , volume , relative_path ) ;
2019-11-21 16:10:22 +00:00
else if ( type = = MergeTreeDataPartType : : WIDE )
2020-05-09 21:24:15 +00:00
return std : : make_shared < MergeTreeDataPartWide > ( * this , name , part_info , volume , relative_path ) ;
2019-11-21 16:10:22 +00:00
else
2020-01-13 16:28:29 +00:00
throw Exception ( " Unknown type in part " + relative_path , ErrorCodes : : UNKNOWN_PART_TYPE ) ;
2019-11-21 16:10:22 +00:00
}
2019-03-28 08:52:09 +00:00
2019-11-21 16:10:22 +00:00
static MergeTreeDataPartType getPartTypeFromMarkExtension ( const String & mrk_ext )
{
if ( mrk_ext = = getNonAdaptiveMrkExtension ( ) )
return MergeTreeDataPartType : : WIDE ;
if ( mrk_ext = = getAdaptiveMrkExtension ( MergeTreeDataPartType : : WIDE ) )
return MergeTreeDataPartType : : WIDE ;
if ( mrk_ext = = getAdaptiveMrkExtension ( MergeTreeDataPartType : : COMPACT ) )
return MergeTreeDataPartType : : COMPACT ;
2019-03-28 08:52:09 +00:00
2020-02-19 14:07:36 +00:00
throw Exception ( " Can't determine part type, because of unknown mark extension " + mrk_ext , ErrorCodes : : UNKNOWN_PART_TYPE ) ;
2019-11-21 16:10:22 +00:00
}
2017-04-01 07:20:54 +00:00
2019-11-22 12:51:00 +00:00
MergeTreeData : : MutableDataPartPtr MergeTreeData : : createPart (
2020-05-09 21:24:15 +00:00
const String & name , const VolumePtr & volume , const String & relative_path ) const
2019-11-21 16:10:22 +00:00
{
2020-05-09 21:24:15 +00:00
return createPart ( name , MergeTreePartInfo : : fromPartName ( name , format_version ) , volume , relative_path ) ;
2019-11-21 16:10:22 +00:00
}
2019-11-22 12:51:00 +00:00
MergeTreeData : : MutableDataPartPtr MergeTreeData : : createPart (
const String & name , const MergeTreePartInfo & part_info ,
2020-05-09 21:24:15 +00:00
const VolumePtr & volume , const String & relative_path ) const
2019-11-21 16:10:22 +00:00
{
2020-02-11 13:41:26 +00:00
MergeTreeDataPartType type ;
2020-02-27 16:47:40 +00:00
auto full_path = relative_data_path + relative_path + " / " ;
2020-05-09 21:24:15 +00:00
auto mrk_ext = MergeTreeIndexGranularityInfo : : getMarksExtensionFromFilesystem ( volume - > getDisk ( ) , full_path ) ;
2017-04-01 07:20:54 +00:00
2019-11-21 16:10:22 +00:00
if ( mrk_ext )
type = getPartTypeFromMarkExtension ( * mrk_ext ) ;
else
2017-04-01 07:20:54 +00:00
{
2019-11-21 16:10:22 +00:00
/// Didn't find any mark file, suppose that part is empty.
type = choosePartType ( 0 , 0 ) ;
2017-04-01 07:20:54 +00:00
}
2020-05-09 21:24:15 +00:00
return createPart ( name , type , part_info , volume , relative_path ) ;
2014-03-20 13:00:42 +00:00
}
2014-03-09 17:36:01 +00:00
2019-08-27 09:34:53 +00:00
void MergeTreeData : : changeSettings (
2019-12-27 14:36:59 +00:00
const ASTPtr & new_settings ,
2019-08-27 09:34:53 +00:00
TableStructureWriteLockHolder & /* table_lock_holder */ )
2019-08-06 13:04:29 +00:00
{
2019-12-27 14:36:59 +00:00
if ( new_settings )
2019-08-27 09:34:53 +00:00
{
2019-12-27 14:36:59 +00:00
const auto & new_changes = new_settings - > as < const ASTSetQuery & > ( ) . changes ;
2020-01-09 14:50:34 +00:00
for ( const auto & change : new_changes )
if ( change . name = = " storage_policy " )
{
StoragePolicyPtr new_storage_policy = global_context . getStoragePolicy ( change . value . safeGet < String > ( ) ) ;
StoragePolicyPtr old_storage_policy = getStoragePolicy ( ) ;
checkStoragePolicy ( new_storage_policy ) ;
std : : unordered_set < String > all_diff_disk_names ;
for ( const auto & disk : new_storage_policy - > getDisks ( ) )
all_diff_disk_names . insert ( disk - > getName ( ) ) ;
for ( const auto & disk : old_storage_policy - > getDisks ( ) )
all_diff_disk_names . erase ( disk - > getName ( ) ) ;
for ( const String & disk_name : all_diff_disk_names )
{
2020-03-19 16:37:55 +00:00
auto disk = new_storage_policy - > getDiskByName ( disk_name ) ;
if ( disk - > exists ( relative_data_path ) )
2020-01-09 14:50:34 +00:00
throw Exception ( " New storage policy contain disks which already contain data of a table with the same name " , ErrorCodes : : LOGICAL_ERROR ) ;
}
for ( const String & disk_name : all_diff_disk_names )
{
2020-03-19 16:37:55 +00:00
auto disk = new_storage_policy - > getDiskByName ( disk_name ) ;
disk - > createDirectories ( relative_data_path ) ;
disk - > createDirectories ( relative_data_path + " detached " ) ;
2020-01-09 14:50:34 +00:00
}
/// FIXME how would that be done while reloading configuration???
}
2019-08-27 09:34:53 +00:00
MergeTreeSettings copy = * getSettings ( ) ;
2019-08-27 13:14:19 +00:00
copy . applyChanges ( new_changes ) ;
2019-08-27 09:34:53 +00:00
storage_settings . set ( std : : make_unique < const MergeTreeSettings > ( copy ) ) ;
2019-12-27 14:36:59 +00:00
settings_ast = new_settings ;
2019-08-27 09:34:53 +00:00
}
2019-08-06 13:04:29 +00:00
}
2019-08-27 20:43:08 +00:00
void MergeTreeData : : freezeAll ( const String & with_name , const Context & context , TableStructureReadLockHolder & )
2018-11-01 10:35:50 +00:00
{
2018-11-20 17:05:22 +00:00
freezePartitionsByMatcher ( [ ] ( const DataPartPtr & ) { return true ; } , with_name , context ) ;
2018-11-01 10:35:50 +00:00
}
2019-07-30 17:24:40 +00:00
void MergeTreeData : : PartsTemporaryRename : : addPart ( const String & old_name , const String & new_name )
{
old_and_new_names . push_back ( { old_name , new_name } ) ;
2020-03-19 16:37:55 +00:00
for ( const auto & [ path , disk ] : storage . getRelativeDataPathsWithDisks ( ) )
2019-08-29 16:17:47 +00:00
{
2020-03-19 16:37:55 +00:00
for ( auto it = disk - > iterateDirectory ( path + source_dir ) ; it - > isValid ( ) ; it - > next ( ) )
2019-08-29 16:17:47 +00:00
{
2020-03-19 16:37:55 +00:00
if ( it - > name ( ) = = old_name )
2019-08-29 16:17:47 +00:00
{
2020-03-19 16:37:55 +00:00
old_part_name_to_path_and_disk [ old_name ] = { path , disk } ;
2019-08-29 16:17:47 +00:00
break ;
}
}
}
2019-07-30 17:24:40 +00:00
}
2019-07-31 14:44:55 +00:00
void MergeTreeData : : PartsTemporaryRename : : tryRenameAll ( )
{
renamed = true ;
for ( size_t i = 0 ; i < old_and_new_names . size ( ) ; + + i )
{
try
{
2020-03-19 16:37:55 +00:00
const auto & [ old_name , new_name ] = old_and_new_names [ i ] ;
if ( old_name . empty ( ) | | new_name . empty ( ) )
2019-07-31 14:44:55 +00:00
throw DB : : Exception ( " Empty part name. Most likely it's a bug. " , ErrorCodes : : INCORRECT_FILE_NAME ) ;
2020-03-19 16:37:55 +00:00
const auto & [ path , disk ] = old_part_name_to_path_and_disk [ old_name ] ;
const auto full_path = path + source_dir ; /// for old_name
disk - > moveFile ( full_path + old_name , full_path + new_name ) ;
2019-07-31 14:44:55 +00:00
}
catch ( . . . )
{
old_and_new_names . resize ( i ) ;
2020-05-23 16:47:56 +00:00
LOG_WARNING_FORMATTED ( storage . log , " Cannot rename parts to perform operation on them: {} " , getCurrentExceptionMessage ( false ) ) ;
2019-07-31 14:44:55 +00:00
throw ;
}
}
}
2019-07-30 17:24:40 +00:00
MergeTreeData : : PartsTemporaryRename : : ~ PartsTemporaryRename ( )
{
2019-07-31 14:44:55 +00:00
// TODO what if server had crashed before this destructor was called?
if ( ! renamed )
return ;
2020-03-19 16:37:55 +00:00
for ( const auto & [ old_name , new_name ] : old_and_new_names )
2019-07-30 17:24:40 +00:00
{
2020-03-19 16:37:55 +00:00
if ( old_name . empty ( ) )
2019-07-30 17:24:40 +00:00
continue ;
2019-08-29 16:17:47 +00:00
2019-07-30 17:24:40 +00:00
try
{
2020-03-19 16:37:55 +00:00
const auto & [ path , disk ] = old_part_name_to_path_and_disk [ old_name ] ;
const auto full_path = path + source_dir ; /// for old_name
disk - > moveFile ( full_path + new_name , full_path + old_name ) ;
2019-07-30 17:24:40 +00:00
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
}
}
2014-03-09 17:36:01 +00:00
2018-02-19 15:31:43 +00:00
MergeTreeData : : DataPartsVector MergeTreeData : : getActivePartsToReplace (
const MergeTreePartInfo & new_part_info ,
2018-02-19 16:12:16 +00:00
const String & new_part_name ,
2018-02-19 15:31:43 +00:00
DataPartPtr & out_covering_part ,
2018-05-21 13:49:54 +00:00
DataPartsLock & /* data_parts_lock */ ) const
2014-03-13 12:48:07 +00:00
{
2018-02-19 15:31:43 +00:00
/// Parts contained in the part are consecutive in data_parts, intersecting the insertion place for the part itself.
2018-05-21 13:49:54 +00:00
auto it_middle = data_parts_by_state_and_info . lower_bound ( DataPartStateAndInfo { DataPartState : : Committed , new_part_info } ) ;
2018-02-19 15:31:43 +00:00
auto committed_parts_range = getDataPartsStateRange ( DataPartState : : Committed ) ;
2017-11-20 19:33:12 +00:00
2018-02-19 15:31:43 +00:00
/// Go to the left.
DataPartIteratorByStateAndInfo begin = it_middle ;
while ( begin ! = committed_parts_range . begin ( ) )
2017-04-01 07:20:54 +00:00
{
2018-02-19 15:31:43 +00:00
auto prev = std : : prev ( begin ) ;
2017-04-01 07:20:54 +00:00
2018-02-19 15:31:43 +00:00
if ( ! new_part_info . contains ( ( * prev ) - > info ) )
2017-09-11 17:55:41 +00:00
{
2018-02-19 15:31:43 +00:00
if ( ( * prev ) - > info . contains ( new_part_info ) )
{
out_covering_part = * prev ;
return { } ;
}
2017-09-11 17:55:41 +00:00
2018-02-19 16:12:16 +00:00
if ( ! new_part_info . isDisjoint ( ( * prev ) - > info ) )
throw Exception ( " Part " + new_part_name + " intersects previous part " + ( * prev ) - > getNameWithState ( ) +
" . It is a bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-04-01 07:20:54 +00:00
2018-02-19 15:31:43 +00:00
break ;
2017-09-11 17:55:41 +00:00
}
2017-04-01 07:20:54 +00:00
2018-02-19 15:31:43 +00:00
begin = prev ;
}
2017-06-25 02:22:10 +00:00
2018-02-19 15:31:43 +00:00
/// Go to the right.
DataPartIteratorByStateAndInfo end = it_middle ;
while ( end ! = committed_parts_range . end ( ) )
{
if ( ( * end ) - > info = = new_part_info )
throw Exception ( " Unexpected duplicate part " + ( * end ) - > getNameWithState ( ) + " . It is a bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-11-20 19:33:12 +00:00
2018-02-19 15:31:43 +00:00
if ( ! new_part_info . contains ( ( * end ) - > info ) )
2017-05-24 20:19:29 +00:00
{
2018-02-19 15:31:43 +00:00
if ( ( * end ) - > info . contains ( new_part_info ) )
2017-11-20 19:33:12 +00:00
{
2018-02-19 15:31:43 +00:00
out_covering_part = * end ;
return { } ;
2017-11-20 19:33:12 +00:00
}
2017-09-11 22:40:51 +00:00
2018-02-19 16:12:16 +00:00
if ( ! new_part_info . isDisjoint ( ( * end ) - > info ) )
throw Exception ( " Part " + new_part_name + " intersects next part " + ( * end ) - > getNameWithState ( ) +
" . It is a bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-05-24 20:19:29 +00:00
2018-02-19 15:31:43 +00:00
break ;
2017-05-24 20:19:29 +00:00
}
2017-11-20 19:33:12 +00:00
2018-02-19 15:31:43 +00:00
+ + end ;
}
2017-11-20 19:33:12 +00:00
2018-02-19 15:31:43 +00:00
return DataPartsVector { begin , end } ;
}
2017-11-20 19:33:12 +00:00
2017-04-01 07:20:54 +00:00
2018-02-19 15:31:43 +00:00
void MergeTreeData : : renameTempPartAndAdd ( MutableDataPartPtr & part , SimpleIncrement * increment , Transaction * out_transaction )
{
auto removed = renameTempPartAndReplace ( part , increment , out_transaction ) ;
if ( ! removed . empty ( ) )
throw Exception ( " Added part " + part - > name + " covers " + toString ( removed . size ( ) )
+ " existing part(s) (including " + removed [ 0 ] - > name + " ) " , ErrorCodes : : LOGICAL_ERROR ) ;
}
2017-09-11 22:40:51 +00:00
2018-05-21 13:49:54 +00:00
void MergeTreeData : : renameTempPartAndReplace (
2018-09-20 14:30:52 +00:00
MutableDataPartPtr & part , SimpleIncrement * increment , Transaction * out_transaction ,
2018-05-21 13:49:54 +00:00
std : : unique_lock < std : : mutex > & lock , DataPartsVector * out_covered_parts )
2018-02-19 15:31:43 +00:00
{
2018-09-20 14:30:52 +00:00
if ( out_transaction & & & out_transaction - > data ! = this )
throw Exception ( " MergeTreeData::Transaction for one table cannot be used with another. It is a bug. " ,
ErrorCodes : : LOGICAL_ERROR ) ;
2018-02-19 15:31:43 +00:00
part - > assertState ( { DataPartState : : Temporary } ) ;
2017-09-11 22:40:51 +00:00
2018-02-19 15:31:43 +00:00
MergeTreePartInfo part_info = part - > info ;
String part_name ;
2017-09-11 22:40:51 +00:00
2018-02-19 15:31:43 +00:00
if ( DataPartPtr existing_part_in_partition = getAnyPartInPartition ( part - > info . partition_id , lock ) )
{
if ( part - > partition . value ! = existing_part_in_partition - > partition . value )
throw Exception (
" Partition value mismatch between two parts with the same partition ID. Existing part: "
+ existing_part_in_partition - > name + " , newly added part: " + part - > name ,
ErrorCodes : : CORRUPTED_DATA ) ;
}
2017-09-11 22:40:51 +00:00
2018-02-19 15:31:43 +00:00
/** It is important that obtaining new block number and adding that block to parts set is done atomically.
* Otherwise there is race condition - merge of blocks could happen in interval that doesn ' t yet contain new part .
*/
if ( increment )
2018-05-23 19:34:37 +00:00
{
2019-12-17 12:23:17 +00:00
part_info . min_block = part_info . max_block = increment - > get ( ) ;
part_info . mutation = 0 ; /// it's equal to min_block by default
2018-05-23 19:34:37 +00:00
part_name = part - > getNewName ( part_info ) ;
}
2018-02-19 15:31:43 +00:00
else
2018-05-23 19:34:37 +00:00
part_name = part - > name ;
2017-04-01 07:20:54 +00:00
2020-05-23 17:00:41 +00:00
LOG_TRACE_FORMATTED ( log , " Renaming temporary part {} to {}. " , part - > relative_path , part_name ) ;
2017-11-20 19:33:12 +00:00
2018-02-19 15:31:43 +00:00
auto it_duplicate = data_parts_by_info . find ( part_info ) ;
if ( it_duplicate ! = data_parts_by_info . end ( ) )
{
String message = " Part " + ( * it_duplicate ) - > getNameWithState ( ) + " already exists " ;
2017-09-11 22:40:51 +00:00
2018-02-19 15:31:43 +00:00
if ( ( * it_duplicate ) - > checkState ( { DataPartState : : Outdated , DataPartState : : Deleting } ) )
throw Exception ( message + " , but it will be deleted soon " , ErrorCodes : : PART_IS_TEMPORARILY_LOCKED ) ;
2017-09-11 22:40:51 +00:00
2018-02-19 15:31:43 +00:00
throw Exception ( message , ErrorCodes : : DUPLICATE_DATA_PART ) ;
}
2017-11-20 19:33:12 +00:00
2018-02-19 15:31:43 +00:00
DataPartPtr covering_part ;
2018-02-19 16:12:16 +00:00
DataPartsVector covered_parts = getActivePartsToReplace ( part_info , part_name , covering_part , lock ) ;
2017-11-20 19:33:12 +00:00
2018-02-19 15:31:43 +00:00
if ( covering_part )
{
2020-05-23 16:58:15 +00:00
LOG_WARNING_FORMATTED ( log , " Tried to add obsolete part {} covered by {} " , part_name , covering_part - > getNameWithState ( ) ) ;
2018-05-21 13:49:54 +00:00
return ;
2018-02-19 15:31:43 +00:00
}
2017-11-20 19:33:12 +00:00
2018-02-19 15:31:43 +00:00
/// All checks are passed. Now we can rename the part on disk.
/// So, we maintain invariant: if a non-temporary part in filesystem then it is in data_parts
///
/// If out_transaction is null, we commit the part to the active set immediately, else add it to the transaction.
part - > name = part_name ;
part - > info = part_info ;
part - > is_temp = false ;
part - > state = DataPartState : : PreCommitted ;
part - > renameTo ( part_name ) ;
2017-11-20 19:33:12 +00:00
2018-02-19 15:31:43 +00:00
auto part_it = data_parts_indexes . insert ( part ) . first ;
2017-04-01 07:20:54 +00:00
2018-02-19 15:31:43 +00:00
if ( out_transaction )
{
out_transaction - > precommitted_parts . insert ( part ) ;
}
else
{
auto current_time = time ( nullptr ) ;
for ( const DataPartPtr & covered_part : covered_parts )
2017-11-20 19:33:12 +00:00
{
2018-03-03 17:44:53 +00:00
covered_part - > remove_time . store ( current_time , std : : memory_order_relaxed ) ;
2018-02-19 15:31:43 +00:00
modifyPartState ( covered_part , DataPartState : : Outdated ) ;
removePartContributionToColumnSizes ( covered_part ) ;
2017-11-20 19:33:12 +00:00
}
2018-02-19 15:31:43 +00:00
modifyPartState ( part_it , DataPartState : : Committed ) ;
addPartContributionToColumnSizes ( part ) ;
2017-04-01 07:20:54 +00:00
}
2018-05-21 13:49:54 +00:00
if ( out_covered_parts )
{
for ( DataPartPtr & covered_part : covered_parts )
out_covered_parts - > emplace_back ( std : : move ( covered_part ) ) ;
}
}
MergeTreeData : : DataPartsVector MergeTreeData : : renameTempPartAndReplace (
MutableDataPartPtr & part , SimpleIncrement * increment , Transaction * out_transaction )
{
2018-09-20 14:30:52 +00:00
if ( out_transaction & & & out_transaction - > data ! = this )
throw Exception ( " MergeTreeData::Transaction for one table cannot be used with another. It is a bug. " ,
ErrorCodes : : LOGICAL_ERROR ) ;
2018-05-21 13:49:54 +00:00
DataPartsVector covered_parts ;
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2018-05-21 13:49:54 +00:00
renameTempPartAndReplace ( part , increment , out_transaction , lock , & covered_parts ) ;
}
2018-02-19 15:31:43 +00:00
return covered_parts ;
2014-03-13 17:44:00 +00:00
}
2018-05-21 13:49:54 +00:00
void MergeTreeData : : removePartsFromWorkingSet ( const MergeTreeData : : DataPartsVector & remove , bool clear_without_timeout , DataPartsLock & /*acquired_lock*/ )
2014-07-01 15:58:25 +00:00
{
2018-05-21 13:49:54 +00:00
auto remove_time = clear_without_timeout ? 0 : time ( nullptr ) ;
for ( const DataPartPtr & part : remove )
{
2020-01-17 12:24:27 +00:00
if ( part - > state = = IMergeTreeDataPart : : State : : Committed )
2018-05-21 13:49:54 +00:00
removePartContributionToColumnSizes ( part ) ;
2020-01-17 12:24:27 +00:00
if ( part - > state = = IMergeTreeDataPart : : State : : Committed | | clear_without_timeout )
2018-05-21 13:49:54 +00:00
part - > remove_time . store ( remove_time , std : : memory_order_relaxed ) ;
2020-01-17 12:24:27 +00:00
if ( part - > state ! = IMergeTreeDataPart : : State : : Outdated )
2019-10-10 16:30:30 +00:00
modifyPartState ( part , IMergeTreeDataPart : : State : : Outdated ) ;
2018-05-21 13:49:54 +00:00
}
}
void MergeTreeData : : removePartsFromWorkingSet ( const DataPartsVector & remove , bool clear_without_timeout , DataPartsLock * acquired_lock )
{
auto lock = ( acquired_lock ) ? DataPartsLock ( ) : lockParts ( ) ;
2014-07-01 15:58:25 +00:00
2020-04-22 06:22:14 +00:00
for ( const auto & part : remove )
2017-04-01 07:20:54 +00:00
{
2018-02-19 15:31:43 +00:00
if ( ! data_parts_by_info . count ( part - > info ) )
2017-09-11 22:40:51 +00:00
throw Exception ( " Part " + part - > getNameWithState ( ) + " not found in data_parts " , ErrorCodes : : LOGICAL_ERROR ) ;
2016-01-30 02:29:20 +00:00
2017-09-21 21:51:17 +00:00
part - > assertState ( { DataPartState : : PreCommitted , DataPartState : : Committed , DataPartState : : Outdated } ) ;
2017-04-01 07:20:54 +00:00
}
2015-09-16 04:18:16 +00:00
2018-05-21 13:49:54 +00:00
removePartsFromWorkingSet ( remove , clear_without_timeout , lock ) ;
}
MergeTreeData : : DataPartsVector MergeTreeData : : removePartsInRangeFromWorkingSet ( const MergeTreePartInfo & drop_range , bool clear_without_timeout ,
bool skip_intersecting_parts , DataPartsLock & lock )
{
DataPartsVector parts_to_remove ;
if ( drop_range . min_block > drop_range . max_block )
return parts_to_remove ;
auto partition_range = getDataPartsPartitionRange ( drop_range . partition_id ) ;
for ( const DataPartPtr & part : partition_range )
2017-04-01 07:20:54 +00:00
{
2018-05-21 13:49:54 +00:00
if ( part - > info . partition_id ! = drop_range . partition_id )
throw Exception ( " Unexpected partition_id of part " + part - > name + " . This is a bug. " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-11-20 19:33:12 +00:00
2018-05-21 13:49:54 +00:00
if ( part - > info . min_block < drop_range . min_block )
{
if ( drop_range . min_block < = part - > info . max_block )
{
/// Intersect left border
String error = " Unexpected merged part " + part - > name + " intersecting drop range " + drop_range . getPartName ( ) ;
if ( ! skip_intersecting_parts )
throw Exception ( error , ErrorCodes : : LOGICAL_ERROR ) ;
LOG_WARNING ( log , error ) ;
}
continue ;
}
/// Stop on new parts
if ( part - > info . min_block > drop_range . max_block )
break ;
if ( part - > info . min_block < = drop_range . max_block & & drop_range . max_block < part - > info . max_block )
{
/// Intersect right border
String error = " Unexpected merged part " + part - > name + " intersecting drop range " + drop_range . getPartName ( ) ;
if ( ! skip_intersecting_parts )
throw Exception ( error , ErrorCodes : : LOGICAL_ERROR ) ;
LOG_WARNING ( log , error ) ;
continue ;
}
if ( part - > state ! = DataPartState : : Deleting )
parts_to_remove . emplace_back ( part ) ;
2017-04-01 07:20:54 +00:00
}
2014-07-01 15:58:25 +00:00
2018-05-21 13:49:54 +00:00
removePartsFromWorkingSet ( parts_to_remove , clear_without_timeout , lock ) ;
2017-09-05 19:03:51 +00:00
2018-05-21 13:49:54 +00:00
return parts_to_remove ;
}
2017-09-05 19:03:51 +00:00
2018-05-21 13:49:54 +00:00
void MergeTreeData : : forgetPartAndMoveToDetached ( const MergeTreeData : : DataPartPtr & part_to_detach , const String & prefix , bool
restore_covered )
2014-04-02 07:59:43 +00:00
{
2018-05-21 13:49:54 +00:00
LOG_INFO ( log , " Renaming " < < part_to_detach - > relative_path < < " to " < < prefix < < part_to_detach - > name < < " and forgiving it. " ) ;
2017-04-01 07:20:54 +00:00
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2017-04-01 07:20:54 +00:00
2018-02-19 15:31:43 +00:00
auto it_part = data_parts_by_info . find ( part_to_detach - > info ) ;
if ( it_part = = data_parts_by_info . end ( ) )
2017-09-11 22:40:51 +00:00
throw Exception ( " No such data part " + part_to_detach - > getNameWithState ( ) , ErrorCodes : : NO_SUCH_DATA_PART ) ;
2017-04-01 07:20:54 +00:00
2018-05-21 13:49:54 +00:00
/// What if part_to_detach is a reference to *it_part? Make a new owner just in case.
2017-11-20 19:33:12 +00:00
DataPartPtr part = * it_part ;
2017-04-01 07:20:54 +00:00
2017-11-20 19:33:12 +00:00
if ( part - > state = = DataPartState : : Committed )
removePartContributionToColumnSizes ( part ) ;
modifyPartState ( it_part , DataPartState : : Deleting ) ;
2018-05-21 13:49:54 +00:00
part - > renameToDetached ( prefix ) ;
2017-11-20 19:33:12 +00:00
data_parts_indexes . erase ( it_part ) ;
2017-04-01 07:20:54 +00:00
2017-11-20 19:33:12 +00:00
if ( restore_covered & & part - > info . level = = 0 )
2017-04-01 07:20:54 +00:00
{
2020-05-23 16:47:56 +00:00
LOG_WARNING_FORMATTED ( log , " Will not recover parts covered by zero-level part {} " , part - > name ) ;
2017-11-20 19:33:12 +00:00
return ;
}
2017-09-11 22:40:51 +00:00
2017-11-20 19:33:12 +00:00
if ( restore_covered )
{
2017-04-01 07:20:54 +00:00
Strings restored ;
bool error = false ;
2017-11-20 19:33:12 +00:00
String error_parts ;
2017-04-01 07:20:54 +00:00
2017-08-14 18:16:11 +00:00
Int64 pos = part - > info . min_block ;
2017-04-01 07:20:54 +00:00
2017-11-20 19:33:12 +00:00
auto is_appropriate_state = [ ] ( DataPartState state )
2017-04-01 07:20:54 +00:00
{
2017-11-20 19:33:12 +00:00
return state = = DataPartState : : Committed | | state = = DataPartState : : Outdated ;
} ;
2018-02-19 15:31:43 +00:00
auto update_error = [ & ] ( DataPartIteratorByInfo it )
2017-11-20 19:33:12 +00:00
{
error = true ;
error_parts + = ( * it ) - > getNameWithState ( ) + " " ;
} ;
2018-02-19 15:31:43 +00:00
auto it_middle = data_parts_by_info . lower_bound ( part - > info ) ;
2017-11-20 19:33:12 +00:00
/// Restore the leftmost part covered by the part
2018-02-19 15:31:43 +00:00
if ( it_middle ! = data_parts_by_info . begin ( ) )
2017-11-20 19:33:12 +00:00
{
auto it = std : : prev ( it_middle ) ;
if ( part - > contains ( * * it ) & & is_appropriate_state ( ( * it ) - > state ) )
2017-04-01 07:20:54 +00:00
{
2017-11-20 19:33:12 +00:00
/// Maybe, we must consider part level somehow
2017-08-14 18:16:11 +00:00
if ( ( * it ) - > info . min_block ! = part - > info . min_block )
2017-11-20 19:33:12 +00:00
update_error ( it ) ;
2017-09-11 22:40:51 +00:00
if ( ( * it ) - > state ! = DataPartState : : Committed )
{
addPartContributionToColumnSizes ( * it ) ;
2017-11-20 19:33:12 +00:00
modifyPartState ( it , DataPartState : : Committed ) ; // iterator is not invalidated here
2017-09-11 22:40:51 +00:00
}
2017-08-14 18:16:11 +00:00
pos = ( * it ) - > info . max_block + 1 ;
2017-04-01 07:20:54 +00:00
restored . push_back ( ( * it ) - > name ) ;
}
else
2017-11-20 19:33:12 +00:00
update_error ( it ) ;
2017-04-01 07:20:54 +00:00
}
else
error = true ;
2017-11-20 19:33:12 +00:00
/// Restore "right" parts
2018-02-19 15:31:43 +00:00
for ( auto it = it_middle ; it ! = data_parts_by_info . end ( ) & & part - > contains ( * * it ) ; + + it )
2017-04-01 07:20:54 +00:00
{
2017-08-14 18:16:11 +00:00
if ( ( * it ) - > info . min_block < pos )
2017-04-01 07:20:54 +00:00
continue ;
2017-11-20 19:33:12 +00:00
if ( ! is_appropriate_state ( ( * it ) - > state ) )
{
update_error ( it ) ;
continue ;
}
2017-08-14 18:16:11 +00:00
if ( ( * it ) - > info . min_block > pos )
2017-11-20 19:33:12 +00:00
update_error ( it ) ;
2017-09-11 22:40:51 +00:00
if ( ( * it ) - > state ! = DataPartState : : Committed )
{
addPartContributionToColumnSizes ( * it ) ;
2017-11-20 19:33:12 +00:00
modifyPartState ( it , DataPartState : : Committed ) ;
2017-09-11 22:40:51 +00:00
}
2017-08-14 18:16:11 +00:00
pos = ( * it ) - > info . max_block + 1 ;
2017-04-01 07:20:54 +00:00
restored . push_back ( ( * it ) - > name ) ;
}
2017-08-14 18:16:11 +00:00
if ( pos ! = part - > info . max_block + 1 )
2017-04-01 07:20:54 +00:00
error = true ;
for ( const String & name : restored )
{
2020-05-23 16:47:56 +00:00
LOG_INFO_FORMATTED ( log , " Activated part {} " , name ) ;
2017-04-01 07:20:54 +00:00
}
if ( error )
2017-11-20 19:33:12 +00:00
{
2020-05-23 18:53:58 +00:00
LOG_ERROR ( log , " The set of parts restored in place of " < < part - > name < < " looks incomplete. " < < " There might or might not be a data loss. " < < ( error_parts . empty ( ) ? " " : " Suspicious parts: " + error_parts ) ) ;
2017-11-20 19:33:12 +00:00
}
2017-04-01 07:20:54 +00:00
}
2014-03-13 17:44:00 +00:00
}
2014-09-19 11:44:29 +00:00
2018-09-20 14:30:52 +00:00
void MergeTreeData : : tryRemovePartImmediately ( DataPartPtr & & part )
{
DataPartPtr part_to_delete ;
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2018-09-20 14:30:52 +00:00
2020-05-23 16:47:56 +00:00
LOG_TRACE_FORMATTED ( log , " Trying to immediately remove part {} " , part - > getNameWithState ( ) ) ;
2018-09-20 14:30:52 +00:00
auto it = data_parts_by_info . find ( part - > info ) ;
if ( it = = data_parts_by_info . end ( ) | | ( * it ) . get ( ) ! = part . get ( ) )
throw Exception ( " Part " + part - > name + " doesn't exist " , ErrorCodes : : LOGICAL_ERROR ) ;
part . reset ( ) ;
if ( ! ( ( * it ) - > state = = DataPartState : : Outdated & & it - > unique ( ) ) )
return ;
modifyPartState ( it , DataPartState : : Deleting ) ;
part_to_delete = * it ;
}
try
{
part_to_delete - > remove ( ) ;
}
catch ( . . . )
{
rollbackDeletingParts ( { part_to_delete } ) ;
throw ;
}
removePartsFinally ( { part_to_delete } ) ;
2020-05-23 16:47:56 +00:00
LOG_TRACE_FORMATTED ( log , " Removed part {} " , part_to_delete - > name ) ;
2018-09-20 14:30:52 +00:00
}
2015-11-18 21:37:28 +00:00
size_t MergeTreeData : : getTotalActiveSizeInBytes ( ) const
2015-04-17 05:35:53 +00:00
{
2017-04-01 07:20:54 +00:00
size_t res = 0 ;
2017-11-20 19:33:12 +00:00
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2017-11-20 19:33:12 +00:00
2020-04-22 06:22:14 +00:00
for ( const auto & part : getDataPartsStateRange ( DataPartState : : Committed ) )
2020-03-23 13:32:02 +00:00
res + = part - > getBytesOnDisk ( ) ;
2017-11-20 19:33:12 +00:00
}
2015-04-17 05:35:53 +00:00
2017-04-01 07:20:54 +00:00
return res ;
2015-04-17 05:35:53 +00:00
}
2014-04-09 16:32:32 +00:00
2019-10-28 17:27:43 +00:00
size_t MergeTreeData : : getTotalActiveSizeInRows ( ) const
{
size_t res = 0 ;
{
auto lock = lockParts ( ) ;
2020-04-22 06:22:14 +00:00
for ( const auto & part : getDataPartsStateRange ( DataPartState : : Committed ) )
2019-10-28 17:27:43 +00:00
res + = part - > rows_count ;
}
return res ;
}
2019-05-02 14:48:54 +00:00
size_t MergeTreeData : : getPartsCount ( ) const
{
auto lock = lockParts ( ) ;
size_t res = 0 ;
for ( const auto & part [[maybe_unused]] : getDataPartsStateRange ( DataPartState : : Committed ) )
+ + res ;
return res ;
}
2017-08-14 18:16:11 +00:00
size_t MergeTreeData : : getMaxPartsCountForPartition ( ) const
2014-04-11 16:56:49 +00:00
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2017-04-01 07:20:54 +00:00
size_t res = 0 ;
size_t cur_count = 0 ;
2017-08-14 18:16:11 +00:00
const String * cur_partition_id = nullptr ;
2017-04-01 07:20:54 +00:00
2017-11-20 19:33:12 +00:00
for ( const auto & part : getDataPartsStateRange ( DataPartState : : Committed ) )
2017-04-01 07:20:54 +00:00
{
2017-08-14 18:16:11 +00:00
if ( cur_partition_id & & part - > info . partition_id = = * cur_partition_id )
2017-04-01 07:20:54 +00:00
{
+ + cur_count ;
}
else
{
2017-08-14 18:16:11 +00:00
cur_partition_id = & part - > info . partition_id ;
2017-04-01 07:20:54 +00:00
cur_count = 1 ;
}
res = std : : max ( res , cur_count ) ;
}
return res ;
2014-04-11 16:56:49 +00:00
}
2016-01-30 00:57:35 +00:00
2018-07-31 12:34:34 +00:00
std : : optional < Int64 > MergeTreeData : : getMinPartDataVersion ( ) const
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2018-07-31 12:34:34 +00:00
std : : optional < Int64 > result ;
2019-05-02 14:48:54 +00:00
for ( const auto & part : getDataPartsStateRange ( DataPartState : : Committed ) )
2018-07-31 12:34:34 +00:00
{
if ( ! result | | * result > part - > info . getDataVersion ( ) )
result = part - > info . getDataVersion ( ) ;
}
return result ;
}
2019-08-07 15:21:45 +00:00
void MergeTreeData : : delayInsertOrThrowIfNeeded ( Poco : : Event * until ) const
2014-05-27 08:43:01 +00:00
{
2019-08-26 14:24:29 +00:00
const auto settings = getSettings ( ) ;
2019-05-02 14:48:54 +00:00
const size_t parts_count_in_total = getPartsCount ( ) ;
2019-08-13 10:29:31 +00:00
if ( parts_count_in_total > = settings - > max_parts_in_total )
2019-05-02 14:48:54 +00:00
{
ProfileEvents : : increment ( ProfileEvents : : RejectedInserts ) ;
throw Exception ( " Too many parts ( " + toString ( parts_count_in_total ) + " ) in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified with 'max_parts_in_total' setting in <merge_tree> element in config.xml or with per-table setting. " , ErrorCodes : : TOO_MANY_PARTS ) ;
}
const size_t parts_count_in_partition = getMaxPartsCountForPartition ( ) ;
2019-08-13 10:29:31 +00:00
if ( parts_count_in_partition < settings - > parts_to_delay_insert )
2017-04-01 07:20:54 +00:00
return ;
2016-10-27 22:50:02 +00:00
2019-08-13 10:29:31 +00:00
if ( parts_count_in_partition > = settings - > parts_to_throw_insert )
2017-04-01 07:20:54 +00:00
{
ProfileEvents : : increment ( ProfileEvents : : RejectedInserts ) ;
2019-05-02 14:48:54 +00:00
throw Exception ( " Too many parts ( " + toString ( parts_count_in_partition ) + " ). Merges are processing significantly slower than inserts. " , ErrorCodes : : TOO_MANY_PARTS ) ;
2017-04-01 07:20:54 +00:00
}
2014-09-12 20:29:29 +00:00
2019-08-13 10:29:31 +00:00
const size_t max_k = settings - > parts_to_throw_insert - settings - > parts_to_delay_insert ; /// always > 0
const size_t k = 1 + parts_count_in_partition - settings - > parts_to_delay_insert ; /// from 1 to max_k
const double delay_milliseconds = : : pow ( settings - > max_delay_to_insert * 1000 , static_cast < double > ( k ) / max_k ) ;
2014-09-13 18:34:08 +00:00
2017-04-01 07:20:54 +00:00
ProfileEvents : : increment ( ProfileEvents : : DelayedInserts ) ;
2017-06-22 16:17:01 +00:00
ProfileEvents : : increment ( ProfileEvents : : DelayedInsertsMilliseconds , delay_milliseconds ) ;
2014-06-20 18:45:19 +00:00
2017-04-01 07:20:54 +00:00
CurrentMetrics : : Increment metric_increment ( CurrentMetrics : : DelayedInserts ) ;
2016-10-27 22:50:02 +00:00
2020-05-23 18:53:58 +00:00
LOG_INFO ( log , " Delaying inserting block by " < < std : : fixed < < std : : setprecision ( 4 ) < < delay_milliseconds < < " ms. because there are " < < parts_count_in_partition < < " parts " ) ;
2014-09-03 02:32:23 +00:00
2017-04-01 07:20:54 +00:00
if ( until )
2017-06-22 16:17:01 +00:00
until - > tryWait ( delay_milliseconds ) ;
2017-04-01 07:20:54 +00:00
else
2017-06-22 16:29:15 +00:00
std : : this_thread : : sleep_for ( std : : chrono : : milliseconds ( static_cast < size_t > ( delay_milliseconds ) ) ) ;
2014-05-27 08:43:01 +00:00
}
2018-05-21 23:17:57 +00:00
void MergeTreeData : : throwInsertIfNeeded ( ) const
2014-04-03 11:48:28 +00:00
{
2019-08-26 14:24:29 +00:00
const auto settings = getSettings ( ) ;
2019-05-02 14:48:54 +00:00
const size_t parts_count_in_total = getPartsCount ( ) ;
2019-08-13 10:29:31 +00:00
if ( parts_count_in_total > = settings - > max_parts_in_total )
2019-05-02 14:48:54 +00:00
{
ProfileEvents : : increment ( ProfileEvents : : RejectedInserts ) ;
throw Exception ( " Too many parts ( " + toString ( parts_count_in_total ) + " ) in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified with 'max_parts_in_total' setting in <merge_tree> element in config.xml or with per-table setting. " , ErrorCodes : : TOO_MANY_PARTS ) ;
}
const size_t parts_count_in_partition = getMaxPartsCountForPartition ( ) ;
2017-04-01 07:20:54 +00:00
2019-08-13 10:29:31 +00:00
if ( parts_count_in_partition > = settings - > parts_to_throw_insert )
2018-05-21 23:17:57 +00:00
{
ProfileEvents : : increment ( ProfileEvents : : RejectedInserts ) ;
2019-05-02 14:48:54 +00:00
throw Exception ( " Too many parts ( " + toString ( parts_count_in_partition ) + " ). Merges are processing significantly slower than inserts. " , ErrorCodes : : TOO_MANY_PARTS ) ;
2018-05-21 23:17:57 +00:00
}
}
2017-04-01 07:20:54 +00:00
2018-05-21 13:49:54 +00:00
MergeTreeData : : DataPartPtr MergeTreeData : : getActiveContainingPart (
2019-10-28 17:27:43 +00:00
const MergeTreePartInfo & part_info , MergeTreeData : : DataPartState state , DataPartsLock & /*lock*/ ) const
2014-04-03 11:48:28 +00:00
{
2019-10-16 19:01:50 +00:00
auto current_state_parts_range = getDataPartsStateRange ( state ) ;
2017-11-20 19:33:12 +00:00
2017-04-01 07:20:54 +00:00
/// The part can be covered only by the previous or the next one in data_parts.
2018-05-21 13:49:54 +00:00
auto it = data_parts_by_state_and_info . lower_bound ( DataPartStateAndInfo { state , part_info } ) ;
2017-04-01 07:20:54 +00:00
2019-10-16 19:01:50 +00:00
if ( it ! = current_state_parts_range . end ( ) )
2017-04-01 07:20:54 +00:00
{
2018-05-21 13:49:54 +00:00
if ( ( * it ) - > info = = part_info )
2017-04-01 07:20:54 +00:00
return * it ;
2017-08-15 11:59:08 +00:00
if ( ( * it ) - > info . contains ( part_info ) )
2017-04-01 07:20:54 +00:00
return * it ;
}
2019-10-16 19:01:50 +00:00
if ( it ! = current_state_parts_range . begin ( ) )
2017-04-01 07:20:54 +00:00
{
- - it ;
2017-08-15 11:59:08 +00:00
if ( ( * it ) - > info . contains ( part_info ) )
2017-04-01 07:20:54 +00:00
return * it ;
}
return nullptr ;
2014-04-03 11:48:28 +00:00
}
2019-08-19 14:40:12 +00:00
void MergeTreeData : : swapActivePart ( MergeTreeData : : DataPartPtr part_copy )
2019-06-07 19:16:42 +00:00
{
2019-08-21 12:32:48 +00:00
auto lock = lockParts ( ) ;
2020-03-12 09:09:44 +00:00
for ( auto original_active_part : getDataPartsStateRange ( DataPartState : : Committed ) ) // NOLINT (copy is intended)
2019-06-07 19:16:42 +00:00
{
2019-08-16 15:57:19 +00:00
if ( part_copy - > name = = original_active_part - > name )
2019-06-07 19:16:42 +00:00
{
2019-08-16 15:57:19 +00:00
auto active_part_it = data_parts_by_info . find ( original_active_part - > info ) ;
if ( active_part_it = = data_parts_by_info . end ( ) )
2019-09-04 18:26:18 +00:00
throw Exception ( " Cannot swap part ' " + part_copy - > name + " ', no such active part. " , ErrorCodes : : NO_SUCH_DATA_PART ) ;
2019-06-07 19:16:42 +00:00
2019-09-04 18:26:18 +00:00
modifyPartState ( original_active_part , DataPartState : : DeleteOnDestroy ) ;
2019-08-16 15:57:19 +00:00
data_parts_indexes . erase ( active_part_it ) ;
2019-07-30 16:15:57 +00:00
2019-08-16 15:57:19 +00:00
auto part_it = data_parts_indexes . insert ( part_copy ) . first ;
2019-06-07 19:16:42 +00:00
modifyPartState ( part_it , DataPartState : : Committed ) ;
2019-11-17 21:41:40 +00:00
2020-05-09 21:24:15 +00:00
auto disk = original_active_part - > volume - > getDisk ( ) ;
2020-03-19 16:37:55 +00:00
String marker_path = original_active_part - > getFullRelativePath ( ) + DELETE_ON_DESTROY_MARKER_PATH ;
2019-11-17 21:41:40 +00:00
try
{
2020-03-19 16:37:55 +00:00
disk - > createFile ( marker_path ) ;
2019-11-17 21:41:40 +00:00
}
2019-12-09 13:44:11 +00:00
catch ( Poco : : Exception & e )
2019-11-17 21:41:40 +00:00
{
2020-03-19 16:37:55 +00:00
LOG_ERROR ( log , e . what ( ) < < " (while creating DeleteOnDestroy marker: " + backQuote ( fullPath ( disk , marker_path ) ) + " ) " ) ;
2019-11-17 21:41:40 +00:00
}
2019-08-15 09:43:31 +00:00
return ;
2019-06-07 19:16:42 +00:00
}
}
2019-09-04 18:26:18 +00:00
throw Exception ( " Cannot swap part ' " + part_copy - > name + " ', no such active part. " , ErrorCodes : : NO_SUCH_DATA_PART ) ;
2019-08-16 15:57:19 +00:00
}
2019-10-28 17:27:43 +00:00
MergeTreeData : : DataPartPtr MergeTreeData : : getActiveContainingPart ( const MergeTreePartInfo & part_info ) const
2014-07-25 11:38:46 +00:00
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
return getActiveContainingPart ( part_info , DataPartState : : Committed , lock ) ;
2018-05-21 13:49:54 +00:00
}
2019-10-28 17:27:43 +00:00
MergeTreeData : : DataPartPtr MergeTreeData : : getActiveContainingPart ( const String & part_name ) const
2018-09-11 14:41:04 +00:00
{
auto part_info = MergeTreePartInfo : : fromPartName ( part_name , format_version ) ;
return getActiveContainingPart ( part_info ) ;
}
2018-05-21 13:49:54 +00:00
MergeTreeData : : DataPartsVector MergeTreeData : : getDataPartsVectorInPartition ( MergeTreeData : : DataPartState state , const String & partition_id )
{
DataPartStateAndPartitionID state_with_partition { state , partition_id } ;
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2018-05-21 13:49:54 +00:00
return DataPartsVector (
data_parts_by_state_and_info . lower_bound ( state_with_partition ) ,
data_parts_by_state_and_info . upper_bound ( state_with_partition ) ) ;
}
MergeTreeData : : DataPartPtr MergeTreeData : : getPartIfExists ( const MergeTreePartInfo & part_info , const MergeTreeData : : DataPartStates & valid_states )
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2017-10-03 19:04:56 +00:00
2018-02-19 15:31:43 +00:00
auto it = data_parts_by_info . find ( part_info ) ;
if ( it = = data_parts_by_info . end ( ) )
2017-11-20 19:33:12 +00:00
return nullptr ;
for ( auto state : valid_states )
{
if ( ( * it ) - > state = = state )
2017-12-18 17:26:46 +00:00
return * it ;
2017-11-20 19:33:12 +00:00
}
2014-07-25 11:38:46 +00:00
2017-04-01 07:20:54 +00:00
return nullptr ;
2014-07-25 11:38:46 +00:00
}
2018-05-21 13:49:54 +00:00
MergeTreeData : : DataPartPtr MergeTreeData : : getPartIfExists ( const String & part_name , const MergeTreeData : : DataPartStates & valid_states )
{
return getPartIfExists ( MergeTreePartInfo : : fromPartName ( part_name , format_version ) , valid_states ) ;
}
2016-01-28 01:00:27 +00:00
2020-03-18 00:57:00 +00:00
static void loadPartAndFixMetadataImpl ( MergeTreeData : : MutableDataPartPtr part )
2019-07-25 10:46:07 +00:00
{
2020-05-09 21:24:15 +00:00
auto disk = part - > volume - > getDisk ( ) ;
2020-03-19 16:37:55 +00:00
String full_part_path = part - > getFullRelativePath ( ) ;
2017-04-01 07:20:54 +00:00
2019-12-18 13:09:58 +00:00
/// Earlier the list of columns was written incorrectly. Delete it and re-create.
2020-01-17 12:24:27 +00:00
/// But in compact parts we can't get list of columns without this file.
2020-03-19 16:37:55 +00:00
if ( isWidePart ( part ) )
disk - > removeIfExists ( full_part_path + " columns.txt " ) ;
2017-04-01 07:20:54 +00:00
2017-08-16 19:24:50 +00:00
part - > loadColumnsChecksumsIndexes ( false , true ) ;
2020-03-19 16:37:55 +00:00
part - > modification_time = disk - > getLastModified ( full_part_path ) . epochTime ( ) ;
2017-04-01 07:20:54 +00:00
/// If the checksums file is not present, calculate the checksums and write them to disk.
/// Check the data while we are at it.
if ( part - > checksums . empty ( ) )
{
2020-01-13 14:53:32 +00:00
part - > checksums = checkDataPart ( part , false ) ;
2017-04-01 07:20:54 +00:00
{
2020-03-19 16:37:55 +00:00
auto out = disk - > writeFile ( full_part_path + " checksums.txt.tmp " , 4096 ) ;
part - > checksums . write ( * out ) ;
2017-04-01 07:20:54 +00:00
}
2020-03-19 16:37:55 +00:00
disk - > moveFile ( full_part_path + " checksums.txt.tmp " , full_part_path + " checksums.txt " ) ;
2017-04-01 07:20:54 +00:00
}
2014-08-08 08:28:13 +00:00
}
2020-05-09 21:24:15 +00:00
MergeTreeData : : MutableDataPartPtr MergeTreeData : : loadPartAndFixMetadata ( const VolumePtr & volume , const String & relative_path ) const
2020-03-18 00:57:00 +00:00
{
2020-05-09 21:24:15 +00:00
MutableDataPartPtr part = createPart ( Poco : : Path ( relative_path ) . getFileName ( ) , volume , relative_path ) ;
2020-03-18 00:57:00 +00:00
loadPartAndFixMetadataImpl ( part ) ;
return part ;
}
2014-03-27 11:29:40 +00:00
2017-05-14 23:14:21 +00:00
void MergeTreeData : : calculateColumnSizesImpl ( )
2014-09-19 11:44:29 +00:00
{
2017-04-01 07:20:54 +00:00
column_sizes . clear ( ) ;
2014-09-19 11:44:29 +00:00
2017-09-21 21:51:17 +00:00
/// Take into account only committed parts
2017-11-20 19:33:12 +00:00
auto committed_parts_range = getDataPartsStateRange ( DataPartState : : Committed ) ;
for ( const auto & part : committed_parts_range )
2017-04-01 07:20:54 +00:00
addPartContributionToColumnSizes ( part ) ;
2014-09-19 11:44:29 +00:00
}
void MergeTreeData : : addPartContributionToColumnSizes ( const DataPartPtr & part )
{
2020-01-16 16:15:01 +00:00
for ( const auto & column : part - > getColumns ( ) )
2017-04-01 07:20:54 +00:00
{
2019-07-16 17:13:12 +00:00
ColumnSize & total_column_size = column_sizes [ column . name ] ;
ColumnSize part_column_size = part - > getColumnSize ( column . name , * column . type ) ;
2018-03-26 14:18:04 +00:00
total_column_size . add ( part_column_size ) ;
2017-04-01 07:20:54 +00:00
}
2014-09-19 11:44:29 +00:00
}
void MergeTreeData : : removePartContributionToColumnSizes ( const DataPartPtr & part )
{
2020-01-16 16:15:01 +00:00
for ( const auto & column : part - > getColumns ( ) )
2017-04-01 07:20:54 +00:00
{
2019-07-16 17:13:12 +00:00
ColumnSize & total_column_size = column_sizes [ column . name ] ;
ColumnSize part_column_size = part - > getColumnSize ( column . name , * column . type ) ;
2017-04-01 07:20:54 +00:00
2018-03-26 14:18:04 +00:00
auto log_subtract = [ & ] ( size_t & from , size_t value , const char * field )
2017-04-01 07:20:54 +00:00
{
2018-03-26 14:18:04 +00:00
if ( value > from )
2020-05-23 18:53:58 +00:00
LOG_ERROR ( log , " Possibly incorrect column size subtraction: " < < from < < " - " < < value < < " = " < < from - value < < " , column: " < < column . name < < " , field: " < < field ) ;
2018-03-26 14:18:04 +00:00
from - = value ;
} ;
2017-04-01 07:20:54 +00:00
2018-03-26 14:18:04 +00:00
log_subtract ( total_column_size . data_compressed , part_column_size . data_compressed , " .data_compressed " ) ;
log_subtract ( total_column_size . data_uncompressed , part_column_size . data_uncompressed , " .data_uncompressed " ) ;
log_subtract ( total_column_size . marks , part_column_size . marks , " .marks " ) ;
2017-04-01 07:20:54 +00:00
}
2014-09-19 11:44:29 +00:00
}
2014-10-03 17:57:01 +00:00
2019-08-27 20:43:08 +00:00
void MergeTreeData : : freezePartition ( const ASTPtr & partition_ast , const String & with_name , const Context & context , TableStructureReadLockHolder & )
2014-11-11 04:11:07 +00:00
{
2017-11-20 04:15:43 +00:00
std : : optional < String > prefix ;
2017-09-18 20:49:21 +00:00
String partition_id ;
2017-10-12 18:21:17 +00:00
2017-09-07 16:21:06 +00:00
if ( format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING )
2017-09-06 20:34:26 +00:00
{
2017-09-18 20:49:21 +00:00
/// Month-partitioning specific - partition value can represent a prefix of the partition to freeze.
2019-03-15 16:14:13 +00:00
if ( const auto * partition_lit = partition_ast - > as < ASTPartition & > ( ) . value - > as < ASTLiteral > ( ) )
2017-09-06 20:34:26 +00:00
prefix = partition_lit - > value . getType ( ) = = Field : : Types : : UInt64
? toString ( partition_lit - > value . get < UInt64 > ( ) )
: partition_lit - > value . safeGet < String > ( ) ;
2017-09-18 20:49:21 +00:00
else
partition_id = getPartitionIDFromQuery ( partition_ast , context ) ;
2017-09-06 20:34:26 +00:00
}
else
2017-09-18 20:49:21 +00:00
partition_id = getPartitionIDFromQuery ( partition_ast , context ) ;
2017-09-06 20:34:26 +00:00
2017-09-18 20:49:21 +00:00
if ( prefix )
2020-05-23 17:09:37 +00:00
LOG_DEBUG_FORMATTED ( log , " Freezing parts with prefix {} " , * prefix ) ;
2017-09-18 20:49:21 +00:00
else
2020-05-23 17:09:37 +00:00
LOG_DEBUG_FORMATTED ( log , " Freezing parts with partition ID {} " , partition_id ) ;
2017-04-01 07:20:54 +00:00
2018-11-01 17:13:01 +00:00
freezePartitionsByMatcher (
2018-11-20 17:05:22 +00:00
[ & prefix , & partition_id ] ( const DataPartPtr & part )
2018-11-01 17:47:19 +00:00
{
2018-11-01 17:13:01 +00:00
if ( prefix )
return startsWith ( part - > info . partition_id , * prefix ) ;
else
return part - > info . partition_id = = partition_id ;
} ,
with_name ,
context ) ;
2014-11-11 04:11:07 +00:00
}
2019-08-20 09:59:19 +00:00
void MergeTreeData : : movePartitionToDisk ( const ASTPtr & partition , const String & name , bool moving_part , const Context & context )
2019-07-18 15:19:03 +00:00
{
2019-08-20 09:59:19 +00:00
String partition_id ;
2019-07-23 13:34:17 +00:00
2019-08-20 09:59:19 +00:00
if ( moving_part )
partition_id = partition - > as < ASTLiteral & > ( ) . value . safeGet < String > ( ) ;
else
partition_id = getPartitionIDFromQuery ( partition , context ) ;
DataPartsVector parts ;
if ( moving_part )
{
2019-12-09 21:38:41 +00:00
auto part_info = MergeTreePartInfo : : fromPartName ( partition_id , format_version ) ;
parts . push_back ( getActiveContainingPart ( part_info ) ) ;
if ( ! parts . back ( ) | | parts . back ( ) - > name ! = part_info . getPartName ( ) )
2019-08-20 19:04:58 +00:00
throw Exception ( " Part " + partition_id + " is not exists or not active " , ErrorCodes : : NO_SUCH_DATA_PART ) ;
2019-08-20 09:59:19 +00:00
}
else
parts = getDataPartsVectorInPartition ( MergeTreeDataPartState : : Committed , partition_id ) ;
2019-07-23 13:34:17 +00:00
2020-01-09 14:50:34 +00:00
auto disk = getStoragePolicy ( ) - > getDiskByName ( name ) ;
2019-07-23 13:34:17 +00:00
if ( ! disk )
2020-01-09 14:50:34 +00:00
throw Exception ( " Disk " + name + " does not exists on policy " + getStoragePolicy ( ) - > getName ( ) , ErrorCodes : : UNKNOWN_DISK ) ;
2019-07-23 13:34:17 +00:00
2019-10-24 08:52:33 +00:00
parts . erase ( std : : remove_if ( parts . begin ( ) , parts . end ( ) , [ & ] ( auto part_ptr )
{
2020-05-09 21:24:15 +00:00
return part_ptr - > volume - > getDisk ( ) - > getName ( ) = = disk - > getName ( ) ;
2019-10-24 08:52:33 +00:00
} ) , parts . end ( ) ) ;
2019-10-22 14:45:01 +00:00
2019-12-09 21:38:41 +00:00
if ( parts . empty ( ) )
throw Exception ( " Nothing to move " , ErrorCodes : : NO_SUCH_DATA_PART ) ;
2019-10-24 08:52:33 +00:00
if ( parts . empty ( ) )
2019-08-20 09:59:19 +00:00
{
2019-10-24 10:56:32 +00:00
String no_parts_to_move_message ;
if ( moving_part )
no_parts_to_move_message = " Part ' " + partition_id + " ' is already on disk ' " + disk - > getName ( ) + " ' " ;
else
no_parts_to_move_message = " All parts of partition ' " + partition_id + " ' are already on disk ' " + disk - > getName ( ) + " ' " ;
throw Exception ( no_parts_to_move_message , ErrorCodes : : UNKNOWN_DISK ) ;
2019-08-20 09:59:19 +00:00
}
2019-07-23 13:34:17 +00:00
2019-12-03 13:37:40 +00:00
if ( ! movePartsToSpace ( parts , std : : static_pointer_cast < Space > ( disk ) ) )
2019-12-09 21:38:41 +00:00
throw Exception ( " Cannot move parts because moves are manually disabled " , ErrorCodes : : ABORTED ) ;
2019-07-23 13:34:17 +00:00
}
2019-08-20 09:59:19 +00:00
void MergeTreeData : : movePartitionToVolume ( const ASTPtr & partition , const String & name , bool moving_part , const Context & context )
2019-07-23 13:34:17 +00:00
{
2019-08-20 09:59:19 +00:00
String partition_id ;
if ( moving_part )
partition_id = partition - > as < ASTLiteral & > ( ) . value . safeGet < String > ( ) ;
else
partition_id = getPartitionIDFromQuery ( partition , context ) ;
DataPartsVector parts ;
if ( moving_part )
{
2019-12-09 21:38:41 +00:00
auto part_info = MergeTreePartInfo : : fromPartName ( partition_id , format_version ) ;
parts . emplace_back ( getActiveContainingPart ( part_info ) ) ;
if ( ! parts . back ( ) | | parts . back ( ) - > name ! = part_info . getPartName ( ) )
2019-08-20 19:04:58 +00:00
throw Exception ( " Part " + partition_id + " is not exists or not active " , ErrorCodes : : NO_SUCH_DATA_PART ) ;
2019-08-20 09:59:19 +00:00
}
else
parts = getDataPartsVectorInPartition ( MergeTreeDataPartState : : Committed , partition_id ) ;
2019-07-23 13:34:17 +00:00
2020-01-09 14:50:34 +00:00
auto volume = getStoragePolicy ( ) - > getVolumeByName ( name ) ;
2019-07-23 13:34:17 +00:00
if ( ! volume )
2020-01-09 14:50:34 +00:00
throw Exception ( " Volume " + name + " does not exists on policy " + getStoragePolicy ( ) - > getName ( ) , ErrorCodes : : UNKNOWN_DISK ) ;
2019-07-23 13:34:17 +00:00
2019-12-09 21:38:41 +00:00
if ( parts . empty ( ) )
throw Exception ( " Nothing to move " , ErrorCodes : : NO_SUCH_DATA_PART ) ;
2019-10-24 08:52:33 +00:00
parts . erase ( std : : remove_if ( parts . begin ( ) , parts . end ( ) , [ & ] ( auto part_ptr )
2019-10-22 14:45:01 +00:00
{
2020-05-09 21:24:15 +00:00
for ( const auto & disk : volume - > getDisks ( ) )
2019-10-22 14:45:01 +00:00
{
2020-05-09 21:24:15 +00:00
if ( part_ptr - > volume - > getDisk ( ) - > getName ( ) = = disk - > getName ( ) )
2019-10-24 08:52:33 +00:00
{
return true ;
}
2019-10-22 14:45:01 +00:00
}
2019-10-24 08:52:33 +00:00
return false ;
} ) , parts . end ( ) ) ;
2019-10-22 14:45:01 +00:00
2019-10-24 08:52:33 +00:00
if ( parts . empty ( ) )
2019-10-22 14:45:01 +00:00
{
2019-10-24 10:56:32 +00:00
String no_parts_to_move_message ;
if ( moving_part )
no_parts_to_move_message = " Part ' " + partition_id + " ' is already on volume ' " + volume - > getName ( ) + " ' " ;
else
no_parts_to_move_message = " All parts of partition ' " + partition_id + " ' are already on volume ' " + volume - > getName ( ) + " ' " ;
throw Exception ( no_parts_to_move_message , ErrorCodes : : UNKNOWN_DISK ) ;
2019-10-22 14:45:01 +00:00
}
2019-07-23 13:34:17 +00:00
2019-12-03 13:37:40 +00:00
if ( ! movePartsToSpace ( parts , std : : static_pointer_cast < Space > ( volume ) ) )
2019-12-09 21:38:41 +00:00
throw Exception ( " Cannot move parts because moves are manually disabled " , ErrorCodes : : ABORTED ) ;
2019-07-18 15:19:03 +00:00
}
2017-09-11 17:55:41 +00:00
String MergeTreeData : : getPartitionIDFromQuery ( const ASTPtr & ast , const Context & context )
2014-10-03 17:57:01 +00:00
{
2019-03-15 16:14:13 +00:00
const auto & partition_ast = ast - > as < ASTPartition & > ( ) ;
2017-09-06 20:34:26 +00:00
2017-09-11 17:55:41 +00:00
if ( ! partition_ast . value )
return partition_ast . id ;
2017-09-06 20:34:26 +00:00
2017-09-07 16:21:06 +00:00
if ( format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING )
2017-09-06 20:34:26 +00:00
{
2017-09-11 17:55:41 +00:00
/// Month-partitioning specific - partition ID can be passed in the partition value.
2019-03-15 16:14:13 +00:00
const auto * partition_lit = partition_ast . value - > as < ASTLiteral > ( ) ;
2017-09-06 20:34:26 +00:00
if ( partition_lit & & partition_lit - > value . getType ( ) = = Field : : Types : : String )
{
String partition_id = partition_lit - > value . get < String > ( ) ;
if ( partition_id . size ( ) ! = 6 | | ! std : : all_of ( partition_id . begin ( ) , partition_id . end ( ) , isNumericASCII ) )
throw Exception (
" Invalid partition format: " + partition_id + " . Partition should consist of 6 digits: YYYYMM " ,
ErrorCodes : : INVALID_PARTITION_VALUE ) ;
return partition_id ;
}
}
2014-10-03 17:57:01 +00:00
2017-09-06 20:34:26 +00:00
/// Re-parse partition key fields using the information about expected field types.
2018-02-21 17:05:21 +00:00
size_t fields_count = partition_key_sample . columns ( ) ;
2017-09-11 17:55:41 +00:00
if ( partition_ast . fields_count ! = fields_count )
2017-09-06 20:34:26 +00:00
throw Exception (
2017-09-11 17:55:41 +00:00
" Wrong number of fields in the partition expression: " + toString ( partition_ast . fields_count ) +
2017-09-06 20:34:26 +00:00
" , must be: " + toString ( fields_count ) ,
ErrorCodes : : INVALID_PARTITION_VALUE ) ;
2018-06-08 01:51:55 +00:00
const FormatSettings format_settings ;
2017-09-06 20:34:26 +00:00
Row partition_row ( fields_count ) ;
if ( fields_count )
{
ReadBufferFromMemory left_paren_buf ( " ( " , 1 ) ;
2019-06-25 13:10:09 +00:00
ReadBufferFromMemory fields_buf ( partition_ast . fields_str . data ( ) , partition_ast . fields_str . size ( ) ) ;
2017-09-06 20:34:26 +00:00
ReadBufferFromMemory right_paren_buf ( " ) " , 1) ;
ConcatReadBuffer buf ( { & left_paren_buf , & fields_buf , & right_paren_buf } ) ;
2019-08-02 17:16:58 +00:00
auto input_stream = FormatFactory : : instance ( ) . getInput ( " Values " , buf , partition_key_sample , context , context . getSettingsRef ( ) . max_block_size ) ;
2017-12-15 21:11:24 +00:00
2019-08-02 17:16:58 +00:00
auto block = input_stream - > read ( ) ;
if ( ! block | | ! block . rows ( ) )
2017-09-06 20:34:26 +00:00
throw Exception (
2019-06-25 13:10:09 +00:00
" Could not parse partition value: ` " + partition_ast . fields_str + " ` " ,
2017-09-06 20:34:26 +00:00
ErrorCodes : : INVALID_PARTITION_VALUE ) ;
for ( size_t i = 0 ; i < fields_count ; + + i )
2019-08-02 17:16:58 +00:00
block . getByPosition ( i ) . column - > get ( 0 , partition_row [ i ] ) ;
2017-09-06 20:34:26 +00:00
}
2014-10-03 17:57:01 +00:00
2017-09-11 17:55:41 +00:00
MergeTreePartition partition ( std : : move ( partition_row ) ) ;
String partition_id = partition . getID ( * this ) ;
{
2019-03-28 19:58:41 +00:00
auto data_parts_lock = lockParts ( ) ;
2017-09-11 17:55:41 +00:00
DataPartPtr existing_part_in_partition = getAnyPartInPartition ( partition_id , data_parts_lock ) ;
if ( existing_part_in_partition & & existing_part_in_partition - > partition . value ! = partition . value )
{
WriteBufferFromOwnString buf ;
writeCString ( " Parsed partition value: " , buf ) ;
2018-10-09 18:32:44 +00:00
partition . serializeText ( * this , buf , format_settings ) ;
2017-09-11 17:55:41 +00:00
writeCString ( " doesn't match partition value for an existing part with the same partition ID: " , buf ) ;
writeString ( existing_part_in_partition - > name , buf ) ;
throw Exception ( buf . str ( ) , ErrorCodes : : INVALID_PARTITION_VALUE ) ;
}
}
return partition_id ;
2016-01-30 00:57:35 +00:00
}
2017-11-20 19:33:12 +00:00
MergeTreeData : : DataPartsVector MergeTreeData : : getDataPartsVector ( const DataPartStates & affordable_states , DataPartStateVector * out_states ) const
2017-09-11 22:40:51 +00:00
{
DataPartsVector res ;
2017-11-20 19:33:12 +00:00
DataPartsVector buf ;
2017-09-11 22:40:51 +00:00
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2017-11-20 19:33:12 +00:00
for ( auto state : affordable_states )
{
2019-01-09 15:44:20 +00:00
std : : swap ( buf , res ) ;
2017-11-20 19:33:12 +00:00
res . clear ( ) ;
auto range = getDataPartsStateRange ( state ) ;
std : : merge ( range . begin ( ) , range . end ( ) , buf . begin ( ) , buf . end ( ) , std : : back_inserter ( res ) , LessDataPart ( ) ) ;
}
if ( out_states ! = nullptr )
{
out_states - > resize ( res . size ( ) ) ;
for ( size_t i = 0 ; i < res . size ( ) ; + + i )
( * out_states ) [ i ] = res [ i ] - > state ;
}
2017-09-11 22:40:51 +00:00
}
2017-11-20 19:33:12 +00:00
2017-09-11 22:40:51 +00:00
return res ;
}
2017-11-20 19:33:12 +00:00
MergeTreeData : : DataPartsVector MergeTreeData : : getAllDataPartsVector ( MergeTreeData : : DataPartStateVector * out_states ) const
2017-10-06 16:48:41 +00:00
{
DataPartsVector res ;
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2018-02-19 15:31:43 +00:00
res . assign ( data_parts_by_info . begin ( ) , data_parts_by_info . end ( ) ) ;
2017-10-06 16:48:41 +00:00
2017-11-20 19:33:12 +00:00
if ( out_states ! = nullptr )
{
out_states - > resize ( res . size ( ) ) ;
for ( size_t i = 0 ; i < res . size ( ) ; + + i )
( * out_states ) [ i ] = res [ i ] - > state ;
}
2017-10-06 16:48:41 +00:00
}
2017-11-20 19:33:12 +00:00
2017-10-06 16:48:41 +00:00
return res ;
}
2019-05-20 16:24:36 +00:00
std : : vector < DetachedPartInfo >
MergeTreeData : : getDetachedParts ( ) const
{
std : : vector < DetachedPartInfo > res ;
2020-03-19 16:37:55 +00:00
for ( const auto & [ path , disk ] : getRelativeDataPathsWithDisks ( ) )
2019-05-20 16:24:36 +00:00
{
2020-03-19 16:37:55 +00:00
for ( auto it = disk - > iterateDirectory ( path + " detached " ) ; it - > isValid ( ) ; it - > next ( ) )
2019-06-09 12:38:54 +00:00
{
res . emplace_back ( ) ;
2019-08-29 16:17:47 +00:00
auto & part = res . back ( ) ;
2019-05-20 16:24:36 +00:00
2020-03-19 16:37:55 +00:00
DetachedPartInfo : : tryParseDetachedPartName ( it - > name ( ) , part , format_version ) ;
2019-11-19 06:44:10 +00:00
part . disk = disk - > getName ( ) ;
2019-08-29 16:17:47 +00:00
}
2019-05-20 16:24:36 +00:00
}
return res ;
}
2019-07-26 20:04:45 +00:00
void MergeTreeData : : validateDetachedPartName ( const String & name ) const
{
if ( name . find ( ' / ' ) ! = std : : string : : npos | | name = = " . " | | name = = " .. " )
2019-09-11 17:17:10 +00:00
throw DB : : Exception ( " Invalid part name ' " + name + " ' " , ErrorCodes : : INCORRECT_FILE_NAME ) ;
2019-05-20 16:24:36 +00:00
2020-03-19 16:37:55 +00:00
auto full_path = getFullRelativePathForPart ( name , " detached/ " ) ;
2019-08-29 16:17:47 +00:00
2020-03-19 16:37:55 +00:00
if ( ! full_path )
2019-07-26 20:04:45 +00:00
throw DB : : Exception ( " Detached part \" " + name + " \" not found " , ErrorCodes : : BAD_DATA_PART_NAME ) ;
2019-05-20 16:24:36 +00:00
2019-07-31 14:44:55 +00:00
if ( startsWith ( name , " attaching_ " ) | | startsWith ( name , " deleting_ " ) )
throw DB : : Exception ( " Cannot drop part " + name + " : "
" most likely it is used by another DROP or ATTACH query. " ,
ErrorCodes : : BAD_DATA_PART_NAME ) ;
2019-07-26 20:04:45 +00:00
}
2019-07-31 14:44:55 +00:00
void MergeTreeData : : dropDetached ( const ASTPtr & partition , bool part , const Context & context )
2019-07-30 17:24:40 +00:00
{
2019-08-29 16:17:47 +00:00
PartsTemporaryRename renamed_parts ( * this , " detached/ " ) ;
2019-07-31 14:44:55 +00:00
if ( part )
{
String part_name = partition - > as < ASTLiteral & > ( ) . value . safeGet < String > ( ) ;
validateDetachedPartName ( part_name ) ;
renamed_parts . addPart ( part_name , " deleting_ " + part_name ) ;
2019-05-20 16:24:36 +00:00
}
2019-07-31 14:44:55 +00:00
else
{
String partition_id = getPartitionIDFromQuery ( partition , context ) ;
DetachedPartsInfo detached_parts = getDetachedParts ( ) ;
for ( const auto & part_info : detached_parts )
if ( part_info . valid_name & & part_info . partition_id = = partition_id
& & part_info . prefix ! = " attaching " & & part_info . prefix ! = " deleting " )
renamed_parts . addPart ( part_info . dir_name , " deleting_ " + part_info . dir_name ) ;
}
2019-05-20 16:24:36 +00:00
2020-05-23 16:56:05 +00:00
LOG_DEBUG_FORMATTED ( log , " Will drop {} detached parts. " , renamed_parts . old_and_new_names . size ( ) ) ;
2019-07-30 17:24:40 +00:00
2019-07-31 14:44:55 +00:00
renamed_parts . tryRenameAll ( ) ;
2019-09-06 15:09:20 +00:00
for ( auto & [ old_name , new_name ] : renamed_parts . old_and_new_names )
2019-07-31 14:44:55 +00:00
{
2020-03-19 16:37:55 +00:00
const auto & [ path , disk ] = renamed_parts . old_part_name_to_path_and_disk [ old_name ] ;
disk - > removeRecursive ( path + " detached/ " + new_name + " / " ) ;
2020-05-23 16:47:56 +00:00
LOG_DEBUG_FORMATTED ( log , " Dropped detached part {} " , old_name ) ;
2019-09-06 15:09:20 +00:00
old_name . clear ( ) ;
2019-07-31 14:44:55 +00:00
}
2019-07-30 17:24:40 +00:00
}
2019-07-30 19:11:15 +00:00
MergeTreeData : : MutableDataPartsVector MergeTreeData : : tryLoadPartsToAttach ( const ASTPtr & partition , bool attach_part ,
const Context & context , PartsTemporaryRename & renamed_parts )
{
String source_dir = " detached/ " ;
2019-11-27 09:39:44 +00:00
std : : map < String , DiskPtr > name_to_disk ;
2019-07-30 19:11:15 +00:00
/// Let's compose a list of parts that should be added.
if ( attach_part )
{
2019-07-31 14:44:55 +00:00
String part_id = partition - > as < ASTLiteral & > ( ) . value . safeGet < String > ( ) ;
validateDetachedPartName ( part_id ) ;
renamed_parts . addPart ( part_id , " attaching_ " + part_id ) ;
2019-10-03 18:07:47 +00:00
if ( MergeTreePartInfo : : tryParsePartName ( part_id , nullptr , format_version ) )
name_to_disk [ part_id ] = getDiskForPart ( part_id , source_dir ) ;
2019-07-30 19:11:15 +00:00
}
else
{
2019-07-31 14:44:55 +00:00
String partition_id = getPartitionIDFromQuery ( partition , context ) ;
2020-05-23 16:58:15 +00:00
LOG_DEBUG_FORMATTED ( log , " Looking for parts for partition {} in {} " , partition_id , source_dir ) ;
2019-07-30 19:11:15 +00:00
ActiveDataPartSet active_parts ( format_version ) ;
2020-01-09 14:50:34 +00:00
const auto disks = getStoragePolicy ( ) - > getDisks ( ) ;
2020-04-22 06:22:14 +00:00
for ( const auto & disk : disks )
2019-07-30 19:11:15 +00:00
{
2020-03-19 16:37:55 +00:00
for ( auto it = disk - > iterateDirectory ( relative_data_path + source_dir ) ; it - > isValid ( ) ; it - > next ( ) )
2019-08-29 16:17:47 +00:00
{
2020-03-19 16:37:55 +00:00
const String & name = it - > name ( ) ;
2019-08-29 16:17:47 +00:00
MergeTreePartInfo part_info ;
// TODO what if name contains "_tryN" suffix?
/// Parts with prefix in name (e.g. attaching_1_3_3_0, deleting_1_3_3_0) will be ignored
if ( ! MergeTreePartInfo : : tryParsePartName ( name , & part_info , format_version )
| | part_info . partition_id ! = partition_id )
{
continue ;
}
2020-05-23 16:47:56 +00:00
LOG_DEBUG_FORMATTED ( log , " Found part {} " , name ) ;
2019-08-29 16:17:47 +00:00
active_parts . add ( name ) ;
name_to_disk [ name ] = disk ;
}
2019-07-30 19:11:15 +00:00
}
LOG_DEBUG ( log , active_parts . size ( ) < < " of them are active " ) ;
/// Inactive parts rename so they can not be attached in case of repeated ATTACH.
2019-09-03 08:30:59 +00:00
for ( const auto & [ name , disk ] : name_to_disk )
2019-07-30 19:11:15 +00:00
{
String containing_part = active_parts . getContainingPart ( name ) ;
if ( ! containing_part . empty ( ) & & containing_part ! = name )
2019-08-29 16:17:47 +00:00
{
2019-07-31 14:44:55 +00:00
// TODO maybe use PartsTemporaryRename here?
2020-03-19 16:37:55 +00:00
disk - > moveDirectory ( relative_data_path + source_dir + name , relative_data_path + source_dir + " inactive_ " + name ) ;
2019-08-29 16:17:47 +00:00
}
2019-07-31 14:44:55 +00:00
else
renamed_parts . addPart ( name , " attaching_ " + name ) ;
2019-06-09 12:38:54 +00:00
}
2019-05-20 16:24:36 +00:00
}
2019-07-30 19:11:15 +00:00
2019-08-29 16:17:47 +00:00
2019-07-30 19:11:15 +00:00
/// Try to rename all parts before attaching to prevent race with DROP DETACHED and another ATTACH.
2019-07-31 14:44:55 +00:00
renamed_parts . tryRenameAll ( ) ;
2019-07-30 19:11:15 +00:00
/// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist.
2020-05-23 16:42:39 +00:00
LOG_DEBUG_FORMATTED ( log , " Checking parts " ) ;
2019-07-30 19:11:15 +00:00
MutableDataPartsVector loaded_parts ;
2019-07-31 14:44:55 +00:00
loaded_parts . reserve ( renamed_parts . old_and_new_names . size ( ) ) ;
2019-07-30 19:11:15 +00:00
for ( const auto & part_names : renamed_parts . old_and_new_names )
{
2020-05-23 16:47:56 +00:00
LOG_DEBUG_FORMATTED ( log , " Checking part {} " , part_names . second ) ;
2020-05-09 21:24:15 +00:00
auto single_disk_volume = std : : make_shared < SingleDiskVolume > ( " volume_ " + part_names . first , name_to_disk [ part_names . first ] ) ;
MutableDataPartPtr part = createPart ( part_names . first , single_disk_volume , source_dir + part_names . second ) ;
2020-03-18 00:57:00 +00:00
loadPartAndFixMetadataImpl ( part ) ;
2019-07-30 19:11:15 +00:00
loaded_parts . push_back ( part ) ;
}
return loaded_parts ;
2019-05-20 16:24:36 +00:00
}
2019-10-31 10:40:11 +00:00
namespace
2019-04-01 18:41:19 +00:00
{
2019-04-05 17:37:27 +00:00
2019-12-12 11:10:18 +00:00
inline ReservationPtr checkAndReturnReservation ( UInt64 expected_size , ReservationPtr reservation )
2019-10-31 10:40:11 +00:00
{
2019-04-05 17:37:27 +00:00
if ( reservation )
2019-04-03 12:52:09 +00:00
return reservation ;
2019-10-31 10:40:11 +00:00
throw Exception ( " Cannot reserve " + formatReadableSizeWithBinarySuffix ( expected_size ) + " , not enough space " ,
2019-04-03 12:52:09 +00:00
ErrorCodes : : NOT_ENOUGH_SPACE ) ;
2019-04-01 18:41:19 +00:00
}
2019-10-31 10:40:11 +00:00
}
2019-12-12 11:10:18 +00:00
ReservationPtr MergeTreeData : : reserveSpace ( UInt64 expected_size ) const
2019-10-31 10:40:11 +00:00
{
2019-09-04 17:26:53 +00:00
expected_size = std : : max ( RESERVATION_MIN_ESTIMATION_SIZE , expected_size ) ;
2020-01-09 14:50:34 +00:00
auto reservation = getStoragePolicy ( ) - > reserve ( expected_size ) ;
2019-12-05 08:05:07 +00:00
return checkAndReturnReservation ( expected_size , std : : move ( reservation ) ) ;
}
2020-03-18 00:57:00 +00:00
ReservationPtr MergeTreeData : : reserveSpace ( UInt64 expected_size , SpacePtr space )
2019-12-05 08:05:07 +00:00
{
expected_size = std : : max ( RESERVATION_MIN_ESTIMATION_SIZE , expected_size ) ;
auto reservation = tryReserveSpace ( expected_size , space ) ;
return checkAndReturnReservation ( expected_size , std : : move ( reservation ) ) ;
}
2020-03-18 00:57:00 +00:00
ReservationPtr MergeTreeData : : tryReserveSpace ( UInt64 expected_size , SpacePtr space )
2019-12-05 08:05:07 +00:00
{
expected_size = std : : max ( RESERVATION_MIN_ESTIMATION_SIZE , expected_size ) ;
return space - > reserve ( expected_size ) ;
2019-10-31 10:40:11 +00:00
}
2019-12-12 11:10:18 +00:00
ReservationPtr MergeTreeData : : reserveSpacePreferringTTLRules ( UInt64 expected_size ,
2019-12-19 13:10:57 +00:00
const IMergeTreeDataPart : : TTLInfos & ttl_infos ,
2020-01-07 00:02:57 +00:00
time_t time_of_move ,
size_t min_volume_index ) const
2019-10-31 10:40:11 +00:00
{
expected_size = std : : max ( RESERVATION_MIN_ESTIMATION_SIZE , expected_size ) ;
2020-01-07 00:02:57 +00:00
ReservationPtr reservation = tryReserveSpacePreferringTTLRules ( expected_size , ttl_infos , time_of_move , min_volume_index ) ;
2019-11-30 19:22:01 +00:00
2019-12-05 08:05:07 +00:00
return checkAndReturnReservation ( expected_size , std : : move ( reservation ) ) ;
2019-11-30 19:22:01 +00:00
}
2019-12-12 11:10:18 +00:00
ReservationPtr MergeTreeData : : tryReserveSpacePreferringTTLRules ( UInt64 expected_size ,
2019-12-19 13:10:57 +00:00
const IMergeTreeDataPart : : TTLInfos & ttl_infos ,
2020-01-07 00:02:57 +00:00
time_t time_of_move ,
size_t min_volume_index ) const
2019-11-30 19:22:01 +00:00
{
expected_size = std : : max ( RESERVATION_MIN_ESTIMATION_SIZE , expected_size ) ;
2019-12-12 11:10:18 +00:00
ReservationPtr reservation ;
2019-10-31 10:40:11 +00:00
2019-12-05 11:15:47 +00:00
auto ttl_entry = selectTTLEntryForTTLInfos ( ttl_infos , time_of_move ) ;
2020-01-15 08:28:48 +00:00
if ( ttl_entry )
2019-10-31 10:40:11 +00:00
{
2020-01-09 14:50:34 +00:00
SpacePtr destination_ptr = ttl_entry - > getDestination ( getStoragePolicy ( ) ) ;
2019-11-29 07:00:43 +00:00
if ( ! destination_ptr )
2019-10-31 10:40:11 +00:00
{
2019-11-29 07:00:43 +00:00
if ( ttl_entry - > destination_type = = PartDestinationType : : VOLUME )
2020-05-23 19:31:54 +00:00
LOG_WARNING ( log , " Would like to reserve space on volume ' " < < ttl_entry - > destination_name < < " ' by TTL rule of table ' " < < log_name < < " ' but volume was not found " ) ;
2019-11-29 07:00:43 +00:00
else if ( ttl_entry - > destination_type = = PartDestinationType : : DISK )
2020-05-23 19:31:54 +00:00
LOG_WARNING ( log , " Would like to reserve space on disk ' " < < ttl_entry - > destination_name < < " ' by TTL rule of table ' " < < log_name < < " ' but disk was not found " ) ;
2019-10-31 10:40:11 +00:00
}
2019-11-29 07:00:43 +00:00
else
{
reservation = destination_ptr - > reserve ( expected_size ) ;
if ( reservation )
return reservation ;
2020-01-07 18:45:38 +00:00
else
if ( ttl_entry - > destination_type = = PartDestinationType : : VOLUME )
2020-05-23 19:31:54 +00:00
LOG_WARNING ( log , " Would like to reserve space on volume ' " < < ttl_entry - > destination_name < < " ' by TTL rule of table ' " < < log_name < < " ' but there is not enough space " ) ;
2020-01-07 18:45:38 +00:00
else if ( ttl_entry - > destination_type = = PartDestinationType : : DISK )
2020-05-23 19:31:54 +00:00
LOG_WARNING ( log , " Would like to reserve space on disk ' " < < ttl_entry - > destination_name < < " ' by TTL rule of table ' " < < log_name < < " ' but there is not enough space " ) ;
2019-11-29 07:00:43 +00:00
}
2019-10-31 10:40:11 +00:00
}
2020-01-09 14:50:34 +00:00
reservation = getStoragePolicy ( ) - > reserve ( expected_size , min_volume_index ) ;
2019-11-29 07:00:43 +00:00
2019-11-30 19:22:01 +00:00
return reservation ;
2019-10-31 10:40:11 +00:00
}
2020-01-09 14:50:34 +00:00
SpacePtr MergeTreeData : : TTLEntry : : getDestination ( StoragePolicyPtr policy ) const
2019-11-29 07:00:43 +00:00
{
if ( destination_type = = PartDestinationType : : VOLUME )
2019-12-01 05:52:53 +00:00
return policy - > getVolumeByName ( destination_name ) ;
2019-11-29 07:00:43 +00:00
else if ( destination_type = = PartDestinationType : : DISK )
2019-12-01 05:52:53 +00:00
return policy - > getDiskByName ( destination_name ) ;
2019-11-29 07:00:43 +00:00
else
return { } ;
}
2020-01-09 14:50:34 +00:00
bool MergeTreeData : : TTLEntry : : isPartInDestination ( StoragePolicyPtr policy , const IMergeTreeDataPart & part ) const
2019-11-29 07:00:43 +00:00
{
if ( destination_type = = PartDestinationType : : VOLUME )
{
2020-05-09 21:24:15 +00:00
for ( const auto & disk : policy - > getVolumeByName ( destination_name ) - > getDisks ( ) )
if ( disk - > getName ( ) = = part . volume - > getDisk ( ) - > getName ( ) )
2019-11-29 07:00:43 +00:00
return true ;
}
else if ( destination_type = = PartDestinationType : : DISK )
2020-05-09 21:24:15 +00:00
return policy - > getDiskByName ( destination_name ) - > getName ( ) = = part . volume - > getDisk ( ) - > getName ( ) ;
2019-11-29 07:00:43 +00:00
return false ;
}
2020-01-15 08:28:48 +00:00
std : : optional < MergeTreeData : : TTLEntry > MergeTreeData : : selectTTLEntryForTTLInfos (
2019-12-19 13:10:57 +00:00
const IMergeTreeDataPart : : TTLInfos & ttl_infos ,
2019-11-29 07:00:43 +00:00
time_t time_of_move ) const
{
time_t max_max_ttl = 0 ;
2020-01-15 08:28:48 +00:00
std : : vector < DB : : MergeTreeData : : TTLEntry > : : const_iterator best_entry_it ;
2019-11-29 07:00:43 +00:00
2020-01-15 08:28:48 +00:00
auto lock = std : : lock_guard ( move_ttl_entries_mutex ) ;
for ( auto ttl_entry_it = move_ttl_entries . begin ( ) ; ttl_entry_it ! = move_ttl_entries . end ( ) ; + + ttl_entry_it )
2019-11-29 07:00:43 +00:00
{
2020-01-15 08:28:48 +00:00
auto ttl_info_it = ttl_infos . moves_ttl . find ( ttl_entry_it - > result_column ) ;
/// Prefer TTL rule which went into action last.
2019-11-29 07:00:43 +00:00
if ( ttl_info_it ! = ttl_infos . moves_ttl . end ( )
& & ttl_info_it - > second . max < = time_of_move
2019-11-29 07:21:44 +00:00
& & max_max_ttl < = ttl_info_it - > second . max )
2019-11-29 07:00:43 +00:00
{
2020-01-15 08:28:48 +00:00
best_entry_it = ttl_entry_it ;
2019-11-29 07:00:43 +00:00
max_max_ttl = ttl_info_it - > second . max ;
}
}
2020-01-15 08:28:48 +00:00
return max_max_ttl ? * best_entry_it : std : : optional < MergeTreeData : : TTLEntry > ( ) ;
2019-11-29 07:00:43 +00:00
}
2017-10-03 19:04:56 +00:00
MergeTreeData : : DataParts MergeTreeData : : getDataParts ( const DataPartStates & affordable_states ) const
2017-09-11 22:40:51 +00:00
{
DataParts res ;
{
2019-03-28 19:58:41 +00:00
auto lock = lockParts ( ) ;
2017-11-20 19:33:12 +00:00
for ( auto state : affordable_states )
{
auto range = getDataPartsStateRange ( state ) ;
res . insert ( range . begin ( ) , range . end ( ) ) ;
}
2017-09-11 22:40:51 +00:00
}
return res ;
}
MergeTreeData : : DataParts MergeTreeData : : getDataParts ( ) const
{
return getDataParts ( { DataPartState : : Committed } ) ;
}
MergeTreeData : : DataPartsVector MergeTreeData : : getDataPartsVector ( ) const
{
return getDataPartsVector ( { DataPartState : : Committed } ) ;
}
MergeTreeData : : DataPartPtr MergeTreeData : : getAnyPartInPartition (
2018-05-21 13:49:54 +00:00
const String & partition_id , DataPartsLock & /*data_parts_lock*/ )
2017-09-11 22:40:51 +00:00
{
2018-05-21 13:49:54 +00:00
auto it = data_parts_by_state_and_info . lower_bound ( DataPartStateAndPartitionID { DataPartState : : Committed , partition_id } ) ;
2017-09-11 22:40:51 +00:00
2018-02-19 15:31:43 +00:00
if ( it ! = data_parts_by_state_and_info . end ( ) & & ( * it ) - > state = = DataPartState : : Committed & & ( * it ) - > info . partition_id = = partition_id )
2017-09-11 22:40:51 +00:00
return * it ;
2017-11-20 19:33:12 +00:00
return nullptr ;
2017-09-11 22:40:51 +00:00
}
2017-05-24 20:19:29 +00:00
void MergeTreeData : : Transaction : : rollback ( )
{
2017-11-20 19:33:12 +00:00
if ( ! isEmpty ( ) )
2017-05-24 20:19:29 +00:00
{
std : : stringstream ss ;
2018-02-19 15:31:43 +00:00
ss < < " Removing parts: " ;
for ( const auto & part : precommitted_parts )
ss < < " " < < part - > relative_path ;
ss < < " . " ;
2020-05-23 16:47:56 +00:00
LOG_DEBUG_FORMATTED ( data . log , " Undoing transaction.{} " , ss . str ( ) ) ;
2017-05-24 20:19:29 +00:00
2018-09-20 14:30:52 +00:00
data . removePartsFromWorkingSet (
2018-02-19 15:31:43 +00:00
DataPartsVector ( precommitted_parts . begin ( ) , precommitted_parts . end ( ) ) ,
/* clear_without_timeout = */ true ) ;
2017-05-24 20:19:29 +00:00
}
2017-11-20 19:33:12 +00:00
clear ( ) ;
2017-05-24 20:19:29 +00:00
}
2018-05-21 13:49:54 +00:00
MergeTreeData : : DataPartsVector MergeTreeData : : Transaction : : commit ( MergeTreeData : : DataPartsLock * acquired_parts_lock )
2017-09-11 17:55:41 +00:00
{
2018-02-19 15:31:43 +00:00
DataPartsVector total_covered_parts ;
2017-09-11 22:40:51 +00:00
2018-02-19 15:31:43 +00:00
if ( ! isEmpty ( ) )
2017-09-11 22:40:51 +00:00
{
2018-09-20 14:30:52 +00:00
auto parts_lock = acquired_parts_lock ? MergeTreeData : : DataPartsLock ( ) : data . lockParts ( ) ;
2020-04-22 06:22:14 +00:00
auto * owing_parts_lock = acquired_parts_lock ? acquired_parts_lock : & parts_lock ;
2017-09-11 22:40:51 +00:00
2018-02-19 15:31:43 +00:00
auto current_time = time ( nullptr ) ;
for ( const DataPartPtr & part : precommitted_parts )
2017-09-11 22:40:51 +00:00
{
2018-02-19 15:31:43 +00:00
DataPartPtr covering_part ;
2018-09-20 14:30:52 +00:00
DataPartsVector covered_parts = data . getActivePartsToReplace ( part - > info , part - > name , covering_part , * owing_parts_lock ) ;
2018-02-19 15:31:43 +00:00
if ( covering_part )
2017-09-11 22:40:51 +00:00
{
2020-05-23 19:31:54 +00:00
LOG_WARNING ( data . log , " Tried to commit obsolete part " < < part - > name < < " covered by " < < covering_part - > getNameWithState ( ) ) ;
2018-02-19 15:31:43 +00:00
2018-03-03 17:44:53 +00:00
part - > remove_time . store ( 0 , std : : memory_order_relaxed ) ; /// The part will be removed without waiting for old_parts_lifetime seconds.
2018-09-20 14:30:52 +00:00
data . modifyPartState ( part , DataPartState : : Outdated ) ;
2017-09-11 22:40:51 +00:00
}
2018-02-19 15:31:43 +00:00
else
{
total_covered_parts . insert ( total_covered_parts . end ( ) , covered_parts . begin ( ) , covered_parts . end ( ) ) ;
for ( const DataPartPtr & covered_part : covered_parts )
{
2018-03-03 17:44:53 +00:00
covered_part - > remove_time . store ( current_time , std : : memory_order_relaxed ) ;
2018-09-20 14:30:52 +00:00
data . modifyPartState ( covered_part , DataPartState : : Outdated ) ;
data . removePartContributionToColumnSizes ( covered_part ) ;
2018-02-19 15:31:43 +00:00
}
2017-09-11 22:40:51 +00:00
2018-09-20 14:30:52 +00:00
data . modifyPartState ( part , DataPartState : : Committed ) ;
data . addPartContributionToColumnSizes ( part ) ;
2018-02-19 15:31:43 +00:00
}
2017-09-11 22:40:51 +00:00
}
}
2018-02-19 15:31:43 +00:00
clear ( ) ;
return total_covered_parts ;
2017-09-11 17:55:41 +00:00
}
2017-05-24 20:19:29 +00:00
2018-04-19 20:34:02 +00:00
bool MergeTreeData : : isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions ( const ASTPtr & node ) const
2018-01-21 07:30:07 +00:00
{
2018-06-30 21:35:01 +00:00
const String column_name = node - > getColumnName ( ) ;
2018-01-21 07:30:07 +00:00
2018-10-11 14:53:23 +00:00
for ( const auto & name : primary_key_columns )
2018-06-30 21:35:01 +00:00
if ( column_name = = name )
2018-01-21 07:30:07 +00:00
return true ;
2018-06-30 21:35:01 +00:00
for ( const auto & name : minmax_idx_columns )
if ( column_name = = name )
2018-04-19 20:34:02 +00:00
return true ;
2018-04-04 20:37:28 +00:00
2019-03-11 13:22:51 +00:00
if ( const auto * func = node - > as < ASTFunction > ( ) )
2018-03-16 06:51:37 +00:00
if ( func - > arguments - > children . size ( ) = = 1 )
2018-04-19 20:34:02 +00:00
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions ( func - > arguments - > children . front ( ) ) ;
2018-03-16 06:51:37 +00:00
2018-01-21 07:30:07 +00:00
return false ;
}
2019-05-03 02:00:57 +00:00
bool MergeTreeData : : mayBenefitFromIndexForIn ( const ASTPtr & left_in_operand , const Context & ) const
2018-01-21 07:30:07 +00:00
{
2018-04-19 20:34:02 +00:00
/// Make sure that the left side of the IN operator contain part of the key.
/// If there is a tuple on the left side of the IN operator, at least one item of the tuple
/// must be part of the key (probably wrapped by a chain of some acceptable functions).
2019-03-11 13:22:51 +00:00
const auto * left_in_operand_tuple = left_in_operand - > as < ASTFunction > ( ) ;
2018-01-21 07:30:07 +00:00
if ( left_in_operand_tuple & & left_in_operand_tuple - > name = = " tuple " )
{
for ( const auto & item : left_in_operand_tuple - > arguments - > children )
2019-02-25 08:43:19 +00:00
{
2018-04-19 20:34:02 +00:00
if ( isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions ( item ) )
2018-04-03 18:37:35 +00:00
return true ;
2019-02-25 08:43:19 +00:00
for ( const auto & index : skip_indices )
if ( index - > mayBenefitFromIndexForIn ( item ) )
return true ;
}
2018-04-03 18:37:35 +00:00
/// The tuple itself may be part of the primary key, so check that as a last resort.
2018-04-19 20:34:02 +00:00
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions ( left_in_operand ) ;
2018-01-21 07:30:07 +00:00
}
else
{
2019-02-25 08:43:19 +00:00
for ( const auto & index : skip_indices )
if ( index - > mayBenefitFromIndexForIn ( left_in_operand ) )
return true ;
2018-04-19 20:34:02 +00:00
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions ( left_in_operand ) ;
2018-01-21 07:30:07 +00:00
}
}
2017-09-11 22:40:51 +00:00
2020-03-19 23:48:53 +00:00
MergeTreeData & MergeTreeData : : checkStructureAndGetMergeTreeData ( IStorage & source_table ) const
2018-05-21 13:49:54 +00:00
{
2020-03-19 23:48:53 +00:00
MergeTreeData * src_data = dynamic_cast < MergeTreeData * > ( & source_table ) ;
2019-05-03 02:00:57 +00:00
if ( ! src_data )
2020-03-19 23:48:53 +00:00
throw Exception ( " Table " + source_table . getStorageID ( ) . getNameForLogs ( ) +
2019-12-03 16:25:32 +00:00
" supports attachPartitionFrom only for MergeTree family of table engines. "
2020-03-19 23:48:53 +00:00
" Got " + source_table . getName ( ) , ErrorCodes : : NOT_IMPLEMENTED ) ;
2018-05-21 13:49:54 +00:00
if ( getColumns ( ) . getAllPhysical ( ) . sizeOfDifference ( src_data - > getColumns ( ) . getAllPhysical ( ) ) )
throw Exception ( " Tables have different structure " , ErrorCodes : : INCOMPATIBLE_COLUMNS ) ;
auto query_to_string = [ ] ( const ASTPtr & ast )
{
return ast ? queryToString ( ast ) : " " ;
} ;
2018-11-06 18:25:36 +00:00
if ( query_to_string ( order_by_ast ) ! = query_to_string ( src_data - > order_by_ast ) )
2018-05-21 13:49:54 +00:00
throw Exception ( " Tables have different ordering " , ErrorCodes : : BAD_ARGUMENTS ) ;
2018-11-06 18:25:36 +00:00
if ( query_to_string ( partition_by_ast ) ! = query_to_string ( src_data - > partition_by_ast ) )
2018-05-21 13:49:54 +00:00
throw Exception ( " Tables have different partition key " , ErrorCodes : : BAD_ARGUMENTS ) ;
if ( format_version ! = src_data - > format_version )
throw Exception ( " Tables have different format_version " , ErrorCodes : : BAD_ARGUMENTS ) ;
2019-05-03 02:00:57 +00:00
return * src_data ;
2018-05-21 13:49:54 +00:00
}
2019-07-26 08:42:17 +00:00
MergeTreeData & MergeTreeData : : checkStructureAndGetMergeTreeData ( const StoragePtr & source_table ) const
{
2020-03-19 23:48:53 +00:00
return checkStructureAndGetMergeTreeData ( * source_table ) ;
2019-07-26 08:42:17 +00:00
}
2019-11-03 22:27:05 +00:00
MergeTreeData : : MutableDataPartPtr MergeTreeData : : cloneAndLoadDataPartOnSameDisk ( const MergeTreeData : : DataPartPtr & src_part ,
const String & tmp_part_prefix ,
const MergeTreePartInfo & dst_part_info )
2018-05-21 13:49:54 +00:00
{
2020-02-26 19:22:58 +00:00
/// Check that the storage policy contains the disk where the src_part is located.
2020-02-26 14:51:01 +00:00
bool does_storage_policy_allow_same_disk = false ;
for ( const DiskPtr & disk : getStoragePolicy ( ) - > getDisks ( ) )
2020-02-26 19:22:58 +00:00
{
2020-05-09 21:24:15 +00:00
if ( disk - > getName ( ) = = src_part - > volume - > getDisk ( ) - > getName ( ) )
2020-02-26 19:22:58 +00:00
{
2020-02-26 14:51:01 +00:00
does_storage_policy_allow_same_disk = true ;
2020-02-26 19:22:58 +00:00
break ;
}
}
2020-02-26 14:51:01 +00:00
if ( ! does_storage_policy_allow_same_disk )
throw Exception (
2020-02-27 06:57:10 +00:00
" Could not clone and load part " + quoteString ( src_part - > getFullPath ( ) ) + " because disk does not belong to storage policy " , ErrorCodes : : BAD_ARGUMENTS ) ;
2020-02-26 14:51:01 +00:00
2018-07-20 23:56:25 +00:00
String dst_part_name = src_part - > getNewName ( dst_part_info ) ;
2018-05-21 13:49:54 +00:00
String tmp_dst_part_name = tmp_part_prefix + dst_part_name ;
2020-05-09 21:24:15 +00:00
auto reservation = reserveSpace ( src_part - > getBytesOnDisk ( ) , src_part - > volume - > getDisk ( ) ) ;
2020-03-19 16:37:55 +00:00
auto disk = reservation - > getDisk ( ) ;
String src_part_path = src_part - > getFullRelativePath ( ) ;
String dst_part_path = relative_data_path + tmp_dst_part_name ;
2018-05-21 13:49:54 +00:00
2020-03-19 16:37:55 +00:00
if ( disk - > exists ( dst_part_path ) )
throw Exception ( " Part in " + fullPath ( disk , dst_part_path ) + " already exists " , ErrorCodes : : DIRECTORY_ALREADY_EXISTS ) ;
2018-05-21 13:49:54 +00:00
2020-05-23 16:58:15 +00:00
LOG_DEBUG_FORMATTED ( log , " Cloning part {} to {} " , fullPath ( disk , src_part_path ) , fullPath ( disk , dst_part_path ) ) ;
2020-03-19 16:37:55 +00:00
localBackup ( disk , src_part_path , dst_part_path ) ;
2020-02-27 08:37:52 +00:00
disk - > removeIfExists ( dst_part_path + " / " + DELETE_ON_DESTROY_MARKER_PATH ) ;
2018-05-21 13:49:54 +00:00
2020-05-09 21:24:15 +00:00
auto single_disk_volume = std : : make_shared < SingleDiskVolume > ( disk - > getName ( ) , disk ) ;
auto dst_data_part = createPart ( dst_part_name , dst_part_info , single_disk_volume , tmp_dst_part_name ) ;
2019-09-11 10:57:32 +00:00
2018-05-21 13:49:54 +00:00
dst_data_part - > is_temp = true ;
dst_data_part - > loadColumnsChecksumsIndexes ( require_part_metadata , true ) ;
2020-03-19 16:37:55 +00:00
dst_data_part - > modification_time = disk - > getLastModified ( dst_part_path ) . epochTime ( ) ;
2018-05-21 13:49:54 +00:00
return dst_data_part ;
}
2019-11-27 09:39:44 +00:00
String MergeTreeData : : getFullPathOnDisk ( const DiskPtr & disk ) const
2019-04-21 20:23:02 +00:00
{
2019-10-28 20:12:14 +00:00
return disk - > getPath ( ) + relative_data_path ;
2019-04-21 18:38:44 +00:00
}
2019-08-29 16:17:47 +00:00
2020-03-19 16:37:55 +00:00
DiskPtr MergeTreeData : : getDiskForPart ( const String & part_name , const String & additional_path ) const
2019-08-29 16:17:47 +00:00
{
2020-01-09 14:50:34 +00:00
const auto disks = getStoragePolicy ( ) - > getDisks ( ) ;
2020-03-19 16:37:55 +00:00
2019-11-27 09:39:44 +00:00
for ( const DiskPtr & disk : disks )
2020-03-19 16:37:55 +00:00
for ( auto it = disk - > iterateDirectory ( relative_data_path + additional_path ) ; it - > isValid ( ) ; it - > next ( ) )
if ( it - > name ( ) = = part_name )
2019-08-29 16:17:47 +00:00
return disk ;
2020-03-19 16:37:55 +00:00
2019-08-29 16:17:47 +00:00
return nullptr ;
}
2020-03-19 16:37:55 +00:00
std : : optional < String > MergeTreeData : : getFullRelativePathForPart ( const String & part_name , const String & additional_path ) const
2019-08-29 16:17:47 +00:00
{
2020-03-19 16:37:55 +00:00
auto disk = getDiskForPart ( part_name , additional_path ) ;
2019-08-29 16:17:47 +00:00
if ( disk )
2020-03-19 16:37:55 +00:00
return relative_data_path + additional_path ;
return { } ;
2019-08-29 16:17:47 +00:00
}
2019-05-11 18:50:29 +00:00
Strings MergeTreeData : : getDataPaths ( ) const
2019-04-21 20:23:02 +00:00
{
2019-04-21 18:38:44 +00:00
Strings res ;
2020-01-09 14:50:34 +00:00
auto disks = getStoragePolicy ( ) - > getDisks ( ) ;
2019-04-28 14:49:41 +00:00
for ( const auto & disk : disks )
2019-04-21 18:38:44 +00:00
res . push_back ( getFullPathOnDisk ( disk ) ) ;
return res ;
}
2020-02-27 16:47:40 +00:00
MergeTreeData : : PathsWithDisks MergeTreeData : : getRelativeDataPathsWithDisks ( ) const
{
PathsWithDisks res ;
2020-03-06 08:29:38 +00:00
auto disks = getStoragePolicy ( ) - > getDisks ( ) ;
2020-02-27 16:47:40 +00:00
for ( const auto & disk : disks )
res . emplace_back ( relative_data_path , disk ) ;
return res ;
}
2018-11-01 17:13:01 +00:00
void MergeTreeData : : freezePartitionsByMatcher ( MatcherFn matcher , const String & with_name , const Context & context )
{
String clickhouse_path = Poco : : Path ( context . getPath ( ) ) . makeAbsolute ( ) . toString ( ) ;
2019-09-11 10:57:32 +00:00
String default_shadow_path = clickhouse_path + " shadow/ " ;
Poco : : File ( default_shadow_path ) . createDirectories ( ) ;
auto increment = Increment ( default_shadow_path + " increment.txt " ) . get ( true ) ;
2018-11-01 17:13:01 +00:00
2020-03-19 16:37:55 +00:00
const String shadow_path = " shadow/ " ;
2018-11-01 17:13:01 +00:00
/// Acquire a snapshot of active data parts to prevent removing while doing backup.
const auto data_parts = getDataParts ( ) ;
size_t parts_processed = 0 ;
for ( const auto & part : data_parts )
{
if ( ! matcher ( part ) )
continue ;
2020-05-09 21:24:15 +00:00
part - > volume - > getDisk ( ) - > createDirectories ( shadow_path ) ;
2018-11-01 17:13:01 +00:00
2019-09-10 13:06:26 +00:00
String backup_path = shadow_path
+ ( ! with_name . empty ( )
? escapeForFileName ( with_name )
: toString ( increment ) )
+ " / " ;
2018-11-01 17:13:01 +00:00
2019-09-10 13:06:26 +00:00
LOG_DEBUG ( log , " Freezing part " < < part - > name < < " snapshot will be placed at " + backup_path ) ;
2020-03-19 16:37:55 +00:00
String backup_part_path = backup_path + relative_data_path + part - > relative_path ;
2020-05-09 21:24:15 +00:00
localBackup ( part - > volume - > getDisk ( ) , part - > getFullRelativePath ( ) , backup_part_path ) ;
part - > volume - > getDisk ( ) - > removeIfExists ( backup_part_path + " / " + DELETE_ON_DESTROY_MARKER_PATH ) ;
2020-02-27 08:37:52 +00:00
2019-08-21 03:16:59 +00:00
part - > is_frozen . store ( true , std : : memory_order_relaxed ) ;
2018-11-01 17:13:01 +00:00
+ + parts_processed ;
}
2020-05-23 16:56:05 +00:00
LOG_DEBUG_FORMATTED ( log , " Freezed {} parts " , parts_processed ) ;
2018-11-01 17:13:01 +00:00
}
2019-06-19 16:16:13 +00:00
bool MergeTreeData : : canReplacePartition ( const DataPartPtr & src_part ) const
{
2019-08-26 14:24:29 +00:00
const auto settings = getSettings ( ) ;
2019-08-13 08:35:49 +00:00
2019-08-13 10:29:31 +00:00
if ( ! settings - > enable_mixed_granularity_parts | | settings - > index_granularity_bytes = = 0 )
2019-06-19 16:16:13 +00:00
{
if ( ! canUseAdaptiveGranularity ( ) & & src_part - > index_granularity_info . is_adaptive )
return false ;
if ( canUseAdaptiveGranularity ( ) & & ! src_part - > index_granularity_info . is_adaptive )
return false ;
}
return true ;
}
2019-09-03 11:32:25 +00:00
void MergeTreeData : : writePartLog (
PartLogElement : : Type type ,
const ExecutionStatus & execution_status ,
UInt64 elapsed_ns ,
const String & new_part_name ,
const DataPartPtr & result_part ,
const DataPartsVector & source_parts ,
const MergeListEntry * merge_entry )
try
{
2019-12-03 16:25:32 +00:00
auto table_id = getStorageID ( ) ;
auto part_log = global_context . getPartLog ( table_id . database_name ) ;
2019-09-03 11:32:25 +00:00
if ( ! part_log )
return ;
PartLogElement part_log_elem ;
part_log_elem . event_type = type ;
part_log_elem . error = static_cast < UInt16 > ( execution_status . code ) ;
part_log_elem . exception = execution_status . message ;
part_log_elem . event_time = time ( nullptr ) ;
/// TODO: Stop stopwatch in outer code to exclude ZK timings and so on
2019-10-02 07:46:53 +00:00
part_log_elem . duration_ms = elapsed_ns / 1000000 ;
2019-09-03 11:32:25 +00:00
2019-12-03 16:25:32 +00:00
part_log_elem . database_name = table_id . database_name ;
part_log_elem . table_name = table_id . table_name ;
2019-09-03 11:32:25 +00:00
part_log_elem . partition_id = MergeTreePartInfo : : fromPartName ( new_part_name , format_version ) . partition_id ;
part_log_elem . part_name = new_part_name ;
if ( result_part )
{
part_log_elem . path_on_disk = result_part - > getFullPath ( ) ;
2020-03-23 13:32:02 +00:00
part_log_elem . bytes_compressed_on_disk = result_part - > getBytesOnDisk ( ) ;
2019-09-03 11:32:25 +00:00
part_log_elem . rows = result_part - > rows_count ;
}
part_log_elem . source_part_names . reserve ( source_parts . size ( ) ) ;
for ( const auto & source_part : source_parts )
part_log_elem . source_part_names . push_back ( source_part - > name ) ;
if ( merge_entry )
{
part_log_elem . rows_read = ( * merge_entry ) - > rows_read ;
part_log_elem . bytes_read_uncompressed = ( * merge_entry ) - > bytes_read_uncompressed ;
part_log_elem . rows = ( * merge_entry ) - > rows_written ;
part_log_elem . bytes_uncompressed = ( * merge_entry ) - > bytes_written_uncompressed ;
2020-03-19 11:31:21 +00:00
part_log_elem . peak_memory_usage = ( * merge_entry ) - > memory_tracker . getPeak ( ) ;
2019-09-03 11:32:25 +00:00
}
part_log - > add ( part_log_elem ) ;
}
catch ( . . . )
{
tryLogCurrentException ( log , __PRETTY_FUNCTION__ ) ;
}
2019-09-05 15:53:23 +00:00
MergeTreeData : : CurrentlyMovingPartsTagger : : CurrentlyMovingPartsTagger ( MergeTreeMovingParts & & moving_parts_ , MergeTreeData & data_ )
: parts_to_move ( std : : move ( moving_parts_ ) ) , data ( data_ )
2019-09-05 13:12:29 +00:00
{
2019-09-05 15:53:23 +00:00
for ( const auto & moving_part : parts_to_move )
if ( ! data . currently_moving_parts . emplace ( moving_part . part ) . second )
throw Exception ( " Cannot move part ' " + moving_part . part - > name + " '. It's already moving. " , ErrorCodes : : LOGICAL_ERROR ) ;
}
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
MergeTreeData : : CurrentlyMovingPartsTagger : : ~ CurrentlyMovingPartsTagger ( )
2019-09-05 13:12:29 +00:00
{
2019-09-05 15:53:23 +00:00
std : : lock_guard lock ( data . moving_parts_mutex ) ;
for ( const auto & moving_part : parts_to_move )
2019-09-05 13:12:29 +00:00
{
2019-09-05 15:53:23 +00:00
/// Something went completely wrong
if ( ! data . currently_moving_parts . count ( moving_part . part ) )
std : : terminate ( ) ;
data . currently_moving_parts . erase ( moving_part . part ) ;
2019-09-05 13:12:29 +00:00
}
2019-09-05 15:53:23 +00:00
}
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
bool MergeTreeData : : selectPartsAndMove ( )
{
if ( parts_mover . moves_blocker . isCancelled ( ) )
return false ;
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
auto moving_tagger = selectPartsForMove ( ) ;
if ( moving_tagger . parts_to_move . empty ( ) )
return false ;
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
return moveParts ( std : : move ( moving_tagger ) ) ;
2019-09-05 13:12:29 +00:00
}
2019-11-14 11:10:17 +00:00
bool MergeTreeData : : areBackgroundMovesNeeded ( ) const
{
2020-01-09 14:50:34 +00:00
auto policy = getStoragePolicy ( ) ;
2020-01-17 13:30:54 +00:00
if ( policy - > getVolumes ( ) . size ( ) > 1 )
return true ;
2020-05-09 21:24:15 +00:00
return policy - > getVolumes ( ) . size ( ) = = 1 & & policy - > getVolumes ( ) [ 0 ] - > getDisks ( ) . size ( ) > 1 & & ! move_ttl_entries . empty ( ) ;
2019-11-14 11:10:17 +00:00
}
2019-11-27 09:39:44 +00:00
bool MergeTreeData : : movePartsToSpace ( const DataPartsVector & parts , SpacePtr space )
2019-09-05 13:12:29 +00:00
{
if ( parts_mover . moves_blocker . isCancelled ( ) )
return false ;
2019-09-05 15:53:23 +00:00
auto moving_tagger = checkPartsForMove ( parts , space ) ;
if ( moving_tagger . parts_to_move . empty ( ) )
return false ;
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
return moveParts ( std : : move ( moving_tagger ) ) ;
}
MergeTreeData : : CurrentlyMovingPartsTagger MergeTreeData : : selectPartsForMove ( )
{
MergeTreeMovingParts parts_to_move ;
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
auto can_move = [ this ] ( const DataPartPtr & part , String * reason ) - > bool
{
if ( partIsAssignedToBackgroundOperation ( part ) )
2019-09-05 13:12:29 +00:00
{
2019-09-09 17:01:19 +00:00
* reason = " part already assigned to background operation. " ;
2019-09-05 15:53:23 +00:00
return false ;
}
if ( currently_moving_parts . count ( part ) )
{
* reason = " part is already moving. " ;
return false ;
}
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
return true ;
} ;
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
std : : lock_guard moving_lock ( moving_parts_mutex ) ;
2019-09-05 13:12:29 +00:00
2019-09-06 15:09:20 +00:00
parts_mover . selectPartsForMove ( parts_to_move , can_move , moving_lock ) ;
2019-09-05 15:53:23 +00:00
return CurrentlyMovingPartsTagger ( std : : move ( parts_to_move ) , * this ) ;
}
2019-09-05 13:12:29 +00:00
2019-11-27 09:39:44 +00:00
MergeTreeData : : CurrentlyMovingPartsTagger MergeTreeData : : checkPartsForMove ( const DataPartsVector & parts , SpacePtr space )
2019-09-05 15:53:23 +00:00
{
std : : lock_guard moving_lock ( moving_parts_mutex ) ;
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
MergeTreeMovingParts parts_to_move ;
for ( const auto & part : parts )
{
2020-03-23 13:32:02 +00:00
auto reservation = space - > reserve ( part - > getBytesOnDisk ( ) ) ;
2019-09-05 15:53:23 +00:00
if ( ! reservation )
2019-09-09 17:01:19 +00:00
throw Exception ( " Move is not possible. Not enough space on ' " + space - > getName ( ) + " ' " , ErrorCodes : : NOT_ENOUGH_SPACE ) ;
2019-09-05 13:12:29 +00:00
2019-12-03 13:37:40 +00:00
auto reserved_disk = reservation - > getDisk ( ) ;
2019-09-05 13:12:29 +00:00
2020-03-19 16:37:55 +00:00
if ( reserved_disk - > exists ( relative_data_path + part - > name ) )
2019-09-05 15:53:23 +00:00
throw Exception (
2020-03-19 16:37:55 +00:00
" Move is not possible: " + fullPath ( reserved_disk , relative_data_path + part - > name ) + " already exists " ,
2019-09-05 15:53:23 +00:00
ErrorCodes : : DIRECTORY_ALREADY_EXISTS ) ;
if ( currently_moving_parts . count ( part ) | | partIsAssignedToBackgroundOperation ( part ) )
throw Exception (
2019-09-09 17:01:19 +00:00
" Cannot move part ' " + part - > name + " ' because it's participating in background process " ,
2019-09-05 15:53:23 +00:00
ErrorCodes : : PART_IS_TEMPORARILY_LOCKED ) ;
parts_to_move . emplace_back ( part , std : : move ( reservation ) ) ;
2019-09-05 13:12:29 +00:00
}
2019-09-05 15:53:23 +00:00
return CurrentlyMovingPartsTagger ( std : : move ( parts_to_move ) , * this ) ;
}
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
bool MergeTreeData : : moveParts ( CurrentlyMovingPartsTagger & & moving_tagger )
{
2020-05-23 16:56:05 +00:00
LOG_INFO_FORMATTED ( log , " Got {} parts to move. " , moving_tagger . parts_to_move . size ( ) ) ;
2019-09-05 13:12:29 +00:00
2019-09-05 15:53:23 +00:00
for ( const auto & moving_part : moving_tagger . parts_to_move )
2019-09-05 13:12:29 +00:00
{
Stopwatch stopwatch ;
DataPartPtr cloned_part ;
auto write_part_log = [ & ] ( const ExecutionStatus & execution_status )
{
writePartLog (
PartLogElement : : Type : : MOVE_PART ,
execution_status ,
stopwatch . elapsed ( ) ,
moving_part . part - > name ,
cloned_part ,
{ moving_part . part } ,
nullptr ) ;
} ;
try
{
cloned_part = parts_mover . clonePart ( moving_part ) ;
parts_mover . swapClonedPart ( cloned_part ) ;
write_part_log ( { } ) ;
}
catch ( . . . )
{
write_part_log ( ExecutionStatus : : fromCurrentException ( ) ) ;
if ( cloned_part )
cloned_part - > remove ( ) ;
throw ;
}
}
return true ;
}
2019-09-03 11:32:25 +00:00
2020-02-17 20:39:24 +00:00
ColumnDependencies MergeTreeData : : getColumnDependencies ( const NameSet & updated_columns ) const
2020-01-22 13:24:20 +00:00
{
2020-02-17 20:39:24 +00:00
if ( updated_columns . empty ( ) )
return { } ;
ColumnDependencies res ;
NameSet indices_columns ;
NameSet required_ttl_columns ;
NameSet updated_ttl_columns ;
auto add_dependent_columns = [ & updated_columns ] ( const auto & expression , auto & to_set )
2020-01-22 13:24:20 +00:00
{
2020-02-17 20:39:24 +00:00
auto requiered_columns = expression - > getRequiredColumns ( ) ;
for ( const auto & dependency : requiered_columns )
{
if ( updated_columns . count ( dependency ) )
{
to_set . insert ( requiered_columns . begin ( ) , requiered_columns . end ( ) ) ;
return true ;
}
}
return false ;
2020-01-22 13:24:20 +00:00
} ;
2020-02-17 20:39:24 +00:00
for ( const auto & index : skip_indices )
add_dependent_columns ( index - > expr , indices_columns ) ;
2020-01-27 11:53:50 +00:00
if ( hasRowsTTL ( ) )
2020-02-17 20:39:24 +00:00
{
if ( add_dependent_columns ( rows_ttl_entry . expression , required_ttl_columns ) )
{
/// Filter all columns, if rows TTL expression have to be recalculated.
for ( const auto & column : getColumns ( ) . getAllPhysical ( ) )
updated_ttl_columns . insert ( column . name ) ;
}
}
2020-01-22 13:24:20 +00:00
2020-02-17 20:39:24 +00:00
for ( const auto & [ name , entry ] : column_ttl_entries_by_name )
{
if ( add_dependent_columns ( entry . expression , required_ttl_columns ) )
updated_ttl_columns . insert ( name ) ;
}
2020-01-22 13:24:20 +00:00
for ( const auto & entry : move_ttl_entries )
2020-02-17 20:39:24 +00:00
add_dependent_columns ( entry . expression , required_ttl_columns ) ;
2020-01-22 13:24:20 +00:00
2020-02-17 20:39:24 +00:00
for ( const auto & column : indices_columns )
res . emplace ( column , ColumnDependency : : SKIP_INDEX ) ;
for ( const auto & column : required_ttl_columns )
res . emplace ( column , ColumnDependency : : TTL_EXPRESSION ) ;
for ( const auto & column : updated_ttl_columns )
res . emplace ( column , ColumnDependency : : TTL_TARGET ) ;
2020-01-22 13:24:20 +00:00
return res ;
}
2020-04-22 06:22:14 +00:00
bool MergeTreeData : : canUsePolymorphicParts ( const MergeTreeSettings & settings , String * out_reason ) const
2020-01-13 16:28:29 +00:00
{
2020-02-14 21:33:02 +00:00
if ( ! canUseAdaptiveGranularity ( ) )
2020-01-13 16:28:29 +00:00
{
2020-02-14 21:33:02 +00:00
if ( ( settings . min_rows_for_wide_part ! = 0 | | settings . min_bytes_for_wide_part ! = 0 ) & & out_reason )
2020-01-13 16:28:29 +00:00
{
2020-02-14 21:33:02 +00:00
std : : ostringstream message ;
message < < " Table can't create parts with adaptive granularity, but settings min_rows_for_wide_part = "
< < settings . min_rows_for_wide_part < < " , min_bytes_for_wide_part = " < < settings . min_bytes_for_wide_part
< < " . Parts with non-adaptive granularity can be stored only in Wide (default) format. " ;
* out_reason = message . str ( ) ;
2020-01-13 16:28:29 +00:00
}
2020-02-14 21:33:02 +00:00
return false ;
2020-01-13 16:28:29 +00:00
}
2020-02-14 21:33:02 +00:00
return true ;
2020-01-13 16:28:29 +00:00
}
2020-03-25 18:44:08 +00:00
MergeTreeData : : AlterConversions MergeTreeData : : getAlterConversionsForPart ( const MergeTreeDataPartPtr part ) const
2020-03-24 17:05:38 +00:00
{
2020-03-25 18:44:08 +00:00
MutationCommands commands = getFirtsAlterMutationCommandsForPart ( part ) ;
AlterConversions result { } ;
for ( const auto & command : commands )
2020-04-03 11:23:14 +00:00
/// Currently we need explicit conversions only for RENAME alter
/// all other conversions can be deduced from diff between part columns
/// and columns in storage.
2020-03-25 18:44:08 +00:00
if ( command . type = = MutationCommand : : Type : : RENAME_COLUMN )
2020-03-30 16:03:08 +00:00
result . rename_map [ command . rename_to ] = command . column_name ;
2020-03-25 18:44:08 +00:00
return result ;
2020-03-24 17:05:38 +00:00
}
2020-04-27 13:55:30 +00:00
2020-04-28 10:38:57 +00:00
NamesAndTypesList MergeTreeData : : getVirtuals ( ) const
2020-04-27 13:55:30 +00:00
{
2020-04-28 10:38:57 +00:00
return NamesAndTypesList {
2020-04-27 13:55:30 +00:00
NameAndTypePair ( " _part " , std : : make_shared < DataTypeString > ( ) ) ,
NameAndTypePair ( " _part_index " , std : : make_shared < DataTypeUInt64 > ( ) ) ,
NameAndTypePair ( " _partition_id " , std : : make_shared < DataTypeString > ( ) ) ,
NameAndTypePair ( " _sample_factor " , std : : make_shared < DataTypeFloat64 > ( ) ) ,
} ;
}
2014-03-09 17:36:01 +00:00
}