2022-03-30 20:47:51 +00:00
# include <algorithm>
2014-12-30 12:58:02 +00:00
# include <future>
2022-03-30 20:47:51 +00:00
# include <numeric>
2018-08-10 19:53:49 +00:00
# include <Poco/Util/Application.h>
2021-05-31 08:05:40 +00:00
2022-01-30 19:49:48 +00:00
# include <base/sort.h>
2017-04-01 09:19:00 +00:00
# include <Common/Stopwatch.h>
# include <Common/setThreadName.h>
2020-05-23 20:57:48 +00:00
# include <Common/formatReadable.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeAggregateFunction.h>
2018-01-09 00:19:58 +00:00
# include <DataTypes/DataTypeNullable.h>
2019-01-21 10:39:24 +00:00
# include <DataTypes/DataTypeLowCardinality.h>
2017-04-01 09:19:00 +00:00
# include <Columns/ColumnArray.h>
# include <Columns/ColumnTuple.h>
2021-03-12 16:33:41 +00:00
# include <Columns/ColumnSparse.h>
2021-10-15 20:18:20 +00:00
# include <Formats/NativeWriter.h>
2017-04-01 09:19:00 +00:00
# include <IO/WriteBufferFromFile.h>
2018-12-28 18:15:26 +00:00
# include <Compression/CompressedWriteBuffer.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/Aggregator.h>
2022-04-30 11:53:59 +00:00
# include <Common/CacheBase.h>
2017-04-08 01:32:05 +00:00
# include <Common/MemoryTracker.h>
2018-05-29 18:14:31 +00:00
# include <Common/CurrentThread.h>
2017-07-13 20:58:19 +00:00
# include <Common/typeid_cast.h>
2019-08-21 02:28:04 +00:00
# include <Common/assert_cast.h>
2021-04-14 14:07:56 +00:00
# include <Common/JSONBuilder.h>
2019-11-04 05:29:54 +00:00
# include <AggregateFunctions/AggregateFunctionArray.h>
# include <AggregateFunctions/AggregateFunctionState.h>
2020-06-27 14:02:24 +00:00
# include <IO/Operators.h>
2021-05-31 08:05:40 +00:00
# include <Interpreters/JIT/compileFunction.h>
2021-06-13 12:38:57 +00:00
# include <Interpreters/JIT/CompiledExpressionCache.h>
2021-10-03 09:47:38 +00:00
# include <Core/ProtocolDefines.h>
2019-01-21 10:39:24 +00:00
2022-03-30 20:47:51 +00:00
# include <Parsers/ASTSelectQuery.h>
2011-09-19 01:42:16 +00:00
2016-10-24 02:02:37 +00:00
namespace ProfileEvents
{
2022-08-14 07:35:22 +00:00
extern const Event ExternalAggregationWritePart ;
extern const Event ExternalAggregationCompressedBytes ;
extern const Event ExternalAggregationUncompressedBytes ;
extern const Event AggregationPreallocatedElementsInHashTables ;
extern const Event AggregationHashTablesInitializedAsTwoLevel ;
extern const Event OverflowThrow ;
extern const Event OverflowBreak ;
extern const Event OverflowAny ;
2022-03-30 20:47:51 +00:00
}
namespace
{
/** Collects observed HashMap-s sizes to avoid redundant intermediate resizes.
*/
class HashTablesStatistics
{
public :
struct Entry
{
size_t sum_of_sizes ; // used to determine if it's better to convert aggregation to two-level from the beginning
size_t median_size ; // roughly the size we're going to preallocate on each thread
} ;
2022-04-30 11:53:59 +00:00
using Cache = DB : : CacheBase < UInt64 , Entry > ;
2022-03-30 20:47:51 +00:00
using CachePtr = std : : shared_ptr < Cache > ;
using Params = DB : : Aggregator : : Params : : StatsCollectingParams ;
/// Collection and use of the statistics should be enabled.
std : : optional < Entry > getSizeHint ( const Params & params )
{
if ( ! params . isCollectionAndUseEnabled ( ) )
throw DB : : Exception ( DB : : ErrorCodes : : LOGICAL_ERROR , " Collection and use of the statistics should be enabled. " ) ;
std : : lock_guard lock ( mutex ) ;
const auto cache = getHashTableStatsCache ( params , lock ) ;
if ( const auto hint = cache - > get ( params . key ) )
{
2022-05-01 10:54:54 +00:00
LOG_TRACE (
2022-03-30 20:47:51 +00:00
& Poco : : Logger : : get ( " Aggregator " ) ,
" An entry for key={} found in cache: sum_of_sizes={}, median_size={} " ,
params . key ,
hint - > sum_of_sizes ,
hint - > median_size ) ;
return * hint ;
}
return std : : nullopt ;
}
/// Collection and use of the statistics should be enabled.
void update ( size_t sum_of_sizes , size_t median_size , const Params & params )
{
if ( ! params . isCollectionAndUseEnabled ( ) )
throw DB : : Exception ( DB : : ErrorCodes : : LOGICAL_ERROR , " Collection and use of the statistics should be enabled. " ) ;
std : : lock_guard lock ( mutex ) ;
const auto cache = getHashTableStatsCache ( params , lock ) ;
const auto hint = cache - > get ( params . key ) ;
// We'll maintain the maximum among all the observed values until the next prediction turns out to be too wrong.
if ( ! hint | | sum_of_sizes < hint - > sum_of_sizes / 2 | | hint - > sum_of_sizes < sum_of_sizes | | median_size < hint - > median_size / 2
| | hint - > median_size < median_size )
{
2022-05-01 10:54:54 +00:00
LOG_TRACE (
2022-03-30 20:47:51 +00:00
& Poco : : Logger : : get ( " Aggregator " ) ,
" Statistics updated for key={}: new sum_of_sizes={}, median_size={} " ,
params . key ,
sum_of_sizes ,
median_size ) ;
cache - > set ( params . key , std : : make_shared < Entry > ( Entry { . sum_of_sizes = sum_of_sizes , . median_size = median_size } ) ) ;
}
}
std : : optional < DB : : HashTablesCacheStatistics > getCacheStats ( ) const
{
std : : lock_guard lock ( mutex ) ;
if ( hash_table_stats )
{
size_t hits = 0 , misses = 0 ;
hash_table_stats - > getStats ( hits , misses ) ;
return DB : : HashTablesCacheStatistics { . entries = hash_table_stats - > count ( ) , . hits = hits , . misses = misses } ;
}
return std : : nullopt ;
}
static size_t calculateCacheKey ( const DB : : ASTPtr & select_query )
{
if ( ! select_query )
throw DB : : Exception ( DB : : ErrorCodes : : LOGICAL_ERROR , " Query ptr cannot be null " ) ;
const auto & select = select_query - > as < DB : : ASTSelectQuery & > ( ) ;
// It may happen in some corner cases like `select 1 as num group by num`.
if ( ! select . tables ( ) )
return 0 ;
SipHash hash ;
hash . update ( select . tables ( ) - > getTreeHash ( ) ) ;
if ( const auto where = select . where ( ) )
hash . update ( where - > getTreeHash ( ) ) ;
if ( const auto group_by = select . groupBy ( ) )
hash . update ( group_by - > getTreeHash ( ) ) ;
return hash . get64 ( ) ;
}
private :
CachePtr getHashTableStatsCache ( const Params & params , const std : : lock_guard < std : : mutex > & )
{
if ( ! hash_table_stats | | hash_table_stats - > maxSize ( ) ! = params . max_entries_for_hash_table_stats )
hash_table_stats = std : : make_shared < Cache > ( params . max_entries_for_hash_table_stats ) ;
return hash_table_stats ;
}
mutable std : : mutex mutex ;
CachePtr hash_table_stats ;
} ;
HashTablesStatistics & getHashTablesStatistics ( )
{
static HashTablesStatistics hash_tables_stats ;
return hash_tables_stats ;
}
bool worthConvertToTwoLevel (
size_t group_by_two_level_threshold , size_t result_size , size_t group_by_two_level_threshold_bytes , auto result_size_bytes )
{
// params.group_by_two_level_threshold will be equal to 0 if we have only one thread to execute aggregation (refer to AggregatingStep::transformPipeline).
return ( group_by_two_level_threshold & & result_size > = group_by_two_level_threshold )
| | ( group_by_two_level_threshold_bytes & & result_size_bytes > = static_cast < Int64 > ( group_by_two_level_threshold_bytes ) ) ;
}
DB : : AggregatedDataVariants : : Type convertToTwoLevelTypeIfPossible ( DB : : AggregatedDataVariants : : Type type )
{
using Type = DB : : AggregatedDataVariants : : Type ;
switch ( type )
{
# define M(NAME) \
case Type : : NAME : \
return Type : : NAME # # _two_level ;
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL ( M )
# undef M
default :
return type ;
}
__builtin_unreachable ( ) ;
}
void initDataVariantsWithSizeHint (
DB : : AggregatedDataVariants & result , DB : : AggregatedDataVariants : : Type method_chosen , const DB : : Aggregator : : Params & params )
{
const auto & stats_collecting_params = params . stats_collecting_params ;
if ( stats_collecting_params . isCollectionAndUseEnabled ( ) )
{
if ( auto hint = getHashTablesStatistics ( ) . getSizeHint ( stats_collecting_params ) )
{
const auto max_threads = params . group_by_two_level_threshold ! = 0 ? std : : max ( params . max_threads , 1ul ) : 1 ;
const auto lower_limit = hint - > sum_of_sizes / max_threads ;
const auto upper_limit = stats_collecting_params . max_size_to_preallocate_for_aggregation / max_threads ;
const auto adjusted = std : : min ( std : : max ( lower_limit , hint - > median_size ) , upper_limit ) ;
if ( worthConvertToTwoLevel (
params . group_by_two_level_threshold ,
hint - > sum_of_sizes ,
/*group_by_two_level_threshold_bytes*/ 0 ,
/*result_size_bytes*/ 0 ) )
method_chosen = convertToTwoLevelTypeIfPossible ( method_chosen ) ;
result . init ( method_chosen , adjusted ) ;
ProfileEvents : : increment ( ProfileEvents : : AggregationHashTablesInitializedAsTwoLevel , result . isTwoLevel ( ) ) ;
return ;
}
}
result . init ( method_chosen ) ;
}
/// Collection and use of the statistics should be enabled.
void updateStatistics ( const DB : : ManyAggregatedDataVariants & data_variants , const DB : : Aggregator : : Params : : StatsCollectingParams & params )
{
if ( ! params . isCollectionAndUseEnabled ( ) )
throw DB : : Exception ( DB : : ErrorCodes : : LOGICAL_ERROR , " Collection and use of the statistics should be enabled. " ) ;
std : : vector < size_t > sizes ( data_variants . size ( ) ) ;
for ( size_t i = 0 ; i < data_variants . size ( ) ; + + i )
sizes [ i ] = data_variants [ i ] - > size ( ) ;
const auto median_size = sizes . begin ( ) + sizes . size ( ) / 2 ; // not precisely though...
std : : nth_element ( sizes . begin ( ) , median_size , sizes . end ( ) ) ;
const auto sum_of_sizes = std : : accumulate ( sizes . begin ( ) , sizes . end ( ) , 0ull ) ;
getHashTablesStatistics ( ) . update ( sum_of_sizes , * median_size , params ) ;
}
// The std::is_constructible trait isn't suitable here because some classes have template constructors with semantics different from providing size hints.
// Also string hash table variants are not supported due to the fact that both local perf tests and tests in CI showed slowdowns for them.
template < typename . . . >
struct HasConstructorOfNumberOfElements : std : : false_type
{
} ;
template < typename . . . Ts >
struct HasConstructorOfNumberOfElements < HashMapTable < Ts . . . > > : std : : true_type
{
} ;
template < typename Key , typename Cell , typename Hash , typename Grower , typename Allocator , template < typename . . . > typename ImplTable >
struct HasConstructorOfNumberOfElements < TwoLevelHashMapTable < Key , Cell , Hash , Grower , Allocator , ImplTable > > : std : : true_type
{
} ;
template < typename . . . Ts >
struct HasConstructorOfNumberOfElements < HashTable < Ts . . . > > : std : : true_type
{
} ;
template < typename . . . Ts >
struct HasConstructorOfNumberOfElements < TwoLevelHashTable < Ts . . . > > : std : : true_type
{
} ;
template < template < typename > typename Method , typename Base >
struct HasConstructorOfNumberOfElements < Method < Base > > : HasConstructorOfNumberOfElements < Base >
{
} ;
template < typename Method >
auto constructWithReserveIfPossible ( size_t size_hint )
{
if constexpr ( HasConstructorOfNumberOfElements < typename Method : : Data > : : value )
{
ProfileEvents : : increment ( ProfileEvents : : AggregationPreallocatedElementsInHashTables , size_hint ) ;
return std : : make_unique < Method > ( size_hint ) ;
}
else
return std : : make_unique < Method > ( ) ;
}
2022-06-27 11:16:52 +00:00
DB : : ColumnNumbers calculateKeysPositions ( const DB : : Block & header , const DB : : Aggregator : : Params & params )
{
DB : : ColumnNumbers keys_positions ( params . keys_size ) ;
for ( size_t i = 0 ; i < params . keys_size ; + + i )
keys_positions [ i ] = header . getPositionByName ( params . keys [ i ] ) ;
return keys_positions ;
}
2016-10-24 02:02:37 +00:00
}
2011-09-19 01:42:16 +00:00
namespace DB
{
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int UNKNOWN_AGGREGATED_DATA_VARIANT ;
extern const int NOT_ENOUGH_SPACE ;
2018-03-09 23:23:15 +00:00
extern const int TOO_MANY_ROWS ;
2017-04-01 07:20:54 +00:00
extern const int EMPTY_DATA_PASSED ;
extern const int CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS ;
2018-09-01 03:17:43 +00:00
extern const int LOGICAL_ERROR ;
2016-01-11 21:46:36 +00:00
}
2011-09-26 07:25:22 +00:00
2013-02-16 18:59:05 +00:00
AggregatedDataVariants : : ~ AggregatedDataVariants ( )
{
2017-04-01 07:20:54 +00:00
if ( aggregator & & ! aggregator - > all_aggregates_has_trivial_destructor )
{
try
{
aggregator - > destroyAllAggregateStates ( * this ) ;
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
}
2013-02-16 18:59:05 +00:00
}
2022-03-30 20:47:51 +00:00
std : : optional < HashTablesCacheStatistics > getHashTablesCacheStatistics ( )
{
return getHashTablesStatistics ( ) . getCacheStats ( ) ;
}
2013-02-16 18:59:05 +00:00
2018-09-12 13:27:00 +00:00
void AggregatedDataVariants : : convertToTwoLevel ( )
2014-12-30 12:58:02 +00:00
{
2017-04-01 07:20:54 +00:00
if ( aggregator )
2020-05-23 22:24:01 +00:00
LOG_TRACE ( aggregator - > log , " Converting aggregation data to two-level. " ) ;
2014-12-30 12:58:02 +00:00
2017-04-01 07:20:54 +00:00
switch ( type )
{
# define M(NAME) \
case Type : : NAME : \
2020-03-18 02:02:24 +00:00
NAME # # _two_level = std : : make_unique < decltype ( NAME # # _two_level ) : : element_type > ( * ( NAME ) ) ; \
( NAME ) . reset ( ) ; \
2017-04-01 07:20:54 +00:00
type = Type : : NAME # # _two_level ; \
break ;
2014-12-30 12:58:02 +00:00
2017-04-01 07:20:54 +00:00
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL ( M )
2014-12-30 12:58:02 +00:00
2017-04-01 07:20:54 +00:00
# undef M
2014-12-30 12:58:02 +00:00
2017-04-01 07:20:54 +00:00
default :
throw Exception ( " Wrong data variant passed. " , ErrorCodes : : LOGICAL_ERROR ) ;
}
2014-12-30 12:58:02 +00:00
}
2022-03-30 20:47:51 +00:00
void AggregatedDataVariants : : init ( Type type_ , std : : optional < size_t > size_hint )
{
switch ( type_ )
{
case Type : : EMPTY :
case Type : : without_key :
break ;
# define M(NAME, IS_TWO_LEVEL) \
case Type : : NAME : \
if ( size_hint ) \
( NAME ) = constructWithReserveIfPossible < decltype ( NAME ) : : element_type > ( * size_hint ) ; \
else \
( NAME ) = std : : make_unique < decltype ( NAME ) : : element_type > ( ) ; \
break ;
APPLY_FOR_AGGREGATED_VARIANTS ( M )
# undef M
}
type = type_ ;
}
Aggregator : : Params : : StatsCollectingParams : : StatsCollectingParams ( ) = default ;
Aggregator : : Params : : StatsCollectingParams : : StatsCollectingParams (
const ASTPtr & select_query_ ,
bool collect_hash_table_stats_during_aggregation_ ,
size_t max_entries_for_hash_table_stats_ ,
size_t max_size_to_preallocate_for_aggregation_ )
: key ( collect_hash_table_stats_during_aggregation_ ? HashTablesStatistics : : calculateCacheKey ( select_query_ ) : 0 )
, max_entries_for_hash_table_stats ( max_entries_for_hash_table_stats_ )
, max_size_to_preallocate_for_aggregation ( max_size_to_preallocate_for_aggregation_ )
{
}
2018-01-06 18:10:44 +00:00
Block Aggregator : : getHeader ( bool final ) const
2020-06-18 18:29:33 +00:00
{
2022-06-27 11:16:52 +00:00
return params . getHeader ( header , final ) ;
2020-06-18 18:29:33 +00:00
}
Block Aggregator : : Params : : getHeader (
2022-06-27 11:16:52 +00:00
const Block & header , bool only_merge , const Names & keys , const AggregateDescriptions & aggregates , bool final )
2015-11-30 16:57:05 +00:00
{
2018-01-06 18:10:44 +00:00
Block res ;
2015-11-30 16:57:05 +00:00
2022-06-27 11:16:52 +00:00
if ( only_merge )
2020-11-03 11:28:28 +00:00
{
2022-07-01 08:46:58 +00:00
NameSet needed_columns ( keys . begin ( ) , keys . end ( ) ) ;
for ( const auto & aggregate : aggregates )
needed_columns . emplace ( aggregate . column_name ) ;
for ( const auto & column : header )
{
if ( needed_columns . contains ( column . name ) )
res . insert ( column . cloneEmpty ( ) ) ;
}
2020-11-03 11:28:28 +00:00
if ( final )
{
for ( const auto & aggregate : aggregates )
{
auto & elem = res . getByName ( aggregate . column_name ) ;
elem . type = aggregate . function - > getReturnType ( ) ;
elem . column = elem . type - > createColumn ( ) ;
}
}
}
else
2018-01-06 18:10:44 +00:00
{
2020-06-18 18:29:33 +00:00
for ( const auto & key : keys )
2022-06-27 11:16:52 +00:00
res . insert ( header . getByName ( key ) . cloneEmpty ( ) ) ;
2015-11-30 16:57:05 +00:00
2020-06-18 18:29:33 +00:00
for ( const auto & aggregate : aggregates )
2018-01-06 18:10:44 +00:00
{
2022-06-27 11:16:52 +00:00
size_t arguments_size = aggregate . argument_names . size ( ) ;
2018-01-06 18:10:44 +00:00
DataTypes argument_types ( arguments_size ) ;
for ( size_t j = 0 ; j < arguments_size ; + + j )
2022-06-27 11:16:52 +00:00
argument_types [ j ] = header . getByName ( aggregate . argument_names [ j ] ) . type ;
2017-04-01 07:20:54 +00:00
2018-01-09 00:19:58 +00:00
DataTypePtr type ;
2018-01-06 18:10:44 +00:00
if ( final )
2020-06-18 18:29:33 +00:00
type = aggregate . function - > getReturnType ( ) ;
2018-01-06 18:10:44 +00:00
else
2020-06-18 18:29:33 +00:00
type = std : : make_shared < DataTypeAggregateFunction > ( aggregate . function , argument_types , aggregate . parameters ) ;
2017-04-01 07:20:54 +00:00
2020-06-18 18:29:33 +00:00
res . insert ( { type , aggregate . column_name } ) ;
2018-01-06 18:10:44 +00:00
}
}
2018-02-21 06:25:21 +00:00
return materializeBlock ( res ) ;
2018-01-06 18:10:44 +00:00
}
2022-05-19 07:34:41 +00:00
ColumnRawPtrs Aggregator : : Params : : makeRawKeyColumns ( const Block & block ) const
{
ColumnRawPtrs key_columns ( keys_size ) ;
for ( size_t i = 0 ; i < keys_size ; + + i )
key_columns [ i ] = block . safeGetByPosition ( i ) . column . get ( ) ;
return key_columns ;
}
Aggregator : : AggregateColumnsConstData Aggregator : : Params : : makeAggregateColumnsData ( const Block & block ) const
{
AggregateColumnsConstData aggregate_columns ( aggregates_size ) ;
for ( size_t i = 0 ; i < aggregates_size ; + + i )
{
const auto & aggregate_column_name = aggregates [ i ] . column_name ;
aggregate_columns [ i ] = & typeid_cast < const ColumnAggregateFunction & > ( * block . getByName ( aggregate_column_name ) . column ) . getData ( ) ;
}
return aggregate_columns ;
}
2020-07-07 19:51:32 +00:00
void Aggregator : : Params : : explain ( WriteBuffer & out , size_t indent ) const
2020-06-23 16:06:56 +00:00
{
Strings res ;
2020-07-07 19:51:32 +00:00
String prefix ( indent , ' ' ) ;
2020-06-23 16:06:56 +00:00
{
2020-06-27 14:02:24 +00:00
/// Dump keys.
out < < prefix < < " Keys: " ;
2020-06-24 12:09:01 +00:00
bool first = true ;
2022-06-27 11:16:52 +00:00
for ( const auto & key : keys )
2020-06-24 12:09:01 +00:00
{
2020-06-27 14:02:24 +00:00
if ( ! first )
out < < " , " ;
first = false ;
2020-06-24 12:09:01 +00:00
2022-06-27 11:16:52 +00:00
out < < key ;
2020-06-24 12:09:01 +00:00
}
2020-06-27 14:02:24 +00:00
out < < ' \n ' ;
2020-06-23 16:06:56 +00:00
}
2020-06-27 14:02:24 +00:00
if ( ! aggregates . empty ( ) )
{
out < < prefix < < " Aggregates: \n " ;
for ( const auto & aggregate : aggregates )
2020-07-07 19:51:32 +00:00
aggregate . explain ( out , indent + 4 ) ;
2020-06-27 14:02:24 +00:00
}
2020-06-23 16:06:56 +00:00
}
2018-01-06 18:10:44 +00:00
2021-04-14 14:07:56 +00:00
void Aggregator : : Params : : explain ( JSONBuilder : : JSONMap & map ) const
{
auto keys_array = std : : make_unique < JSONBuilder : : JSONArray > ( ) ;
2022-06-27 11:16:52 +00:00
for ( const auto & key : keys )
keys_array - > add ( key ) ;
2021-04-14 14:07:56 +00:00
map . add ( " Keys " , std : : move ( keys_array ) ) ;
if ( ! aggregates . empty ( ) )
{
auto aggregates_array = std : : make_unique < JSONBuilder : : JSONArray > ( ) ;
for ( const auto & aggregate : aggregates )
{
auto aggregate_map = std : : make_unique < JSONBuilder : : JSONMap > ( ) ;
aggregate . explain ( * aggregate_map ) ;
aggregates_array - > add ( std : : move ( aggregate_map ) ) ;
}
map . add ( " Aggregates " , std : : move ( aggregates_array ) ) ;
}
}
2021-06-04 10:43:11 +00:00
# if USE_EMBEDDED_COMPILER
2021-06-03 19:20:53 +00:00
static CHJIT & getJITInstance ( )
{
static CHJIT jit ;
return jit ;
}
2021-06-13 12:38:57 +00:00
class CompiledAggregateFunctionsHolder final : public CompiledExpressionCacheEntry
{
public :
explicit CompiledAggregateFunctionsHolder ( CompiledAggregateFunctions compiled_function_ )
: CompiledExpressionCacheEntry ( compiled_function_ . compiled_module . size )
, compiled_aggregate_functions ( compiled_function_ )
{ }
~ CompiledAggregateFunctionsHolder ( ) override
{
getJITInstance ( ) . deleteCompiledModule ( compiled_aggregate_functions . compiled_module ) ;
}
CompiledAggregateFunctions compiled_aggregate_functions ;
} ;
2021-06-04 10:43:11 +00:00
# endif
2022-06-27 11:16:52 +00:00
Aggregator : : Aggregator ( const Block & header_ , const Params & params_ )
: header ( header_ ) , keys_positions ( calculateKeysPositions ( header , params_ ) ) , params ( params_ )
2018-01-06 18:10:44 +00:00
{
2018-02-01 17:55:08 +00:00
/// Use query-level memory tracker
2020-04-22 05:39:31 +00:00
if ( auto * memory_tracker_child = CurrentThread : : getMemoryTracker ( ) )
if ( auto * memory_tracker = memory_tracker_child - > getParent ( ) )
2019-03-14 18:03:35 +00:00
memory_usage_before_aggregation = memory_tracker - > get ( ) ;
2017-04-01 07:20:54 +00:00
aggregate_functions . resize ( params . aggregates_size ) ;
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
aggregate_functions [ i ] = params . aggregates [ i ] . function . get ( ) ;
/// Initialize sizes of aggregation states and its offsets.
offsets_of_aggregate_states . resize ( params . aggregates_size ) ;
total_size_of_aggregate_states = 0 ;
all_aggregates_has_trivial_destructor = true ;
2020-01-11 09:50:41 +00:00
// aggregate_states will be aligned as below:
2018-08-05 08:45:15 +00:00
// |<-- state_1 -->|<-- pad_1 -->|<-- state_2 -->|<-- pad_2 -->| .....
//
// pad_N will be used to match alignment requirement for each next state.
// The address of state_1 is aligned based on maximum alignment requirements in states
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
{
offsets_of_aggregate_states [ i ] = total_size_of_aggregate_states ;
2018-08-05 08:45:15 +00:00
2017-04-01 07:20:54 +00:00
total_size_of_aggregate_states + = params . aggregates [ i ] . function - > sizeOfData ( ) ;
2020-01-11 09:50:41 +00:00
// aggregate states are aligned based on maximum requirement
2018-09-01 03:17:43 +00:00
align_aggregate_states = std : : max ( align_aggregate_states , params . aggregates [ i ] . function - > alignOfData ( ) ) ;
2018-08-05 08:45:15 +00:00
2018-09-01 03:17:43 +00:00
// If not the last aggregate_state, we need pad it so that next aggregate_state will be aligned.
2018-08-05 08:45:15 +00:00
if ( i + 1 < params . aggregates_size )
{
2018-09-01 03:17:43 +00:00
size_t alignment_of_next_state = params . aggregates [ i + 1 ] . function - > alignOfData ( ) ;
if ( ( alignment_of_next_state & ( alignment_of_next_state - 1 ) ) ! = 0 )
throw Exception ( " Logical error: alignOfData is not 2^N " , ErrorCodes : : LOGICAL_ERROR ) ;
/// Extend total_size to next alignment requirement
/// Add padding by rounding up 'total_size_of_aggregate_states' to be a multiplier of alignment_of_next_state.
total_size_of_aggregate_states = ( total_size_of_aggregate_states + alignment_of_next_state - 1 ) / alignment_of_next_state * alignment_of_next_state ;
2018-08-05 08:45:15 +00:00
}
2017-04-01 07:20:54 +00:00
if ( ! params . aggregates [ i ] . function - > hasTrivialDestructor ( ) )
all_aggregates_has_trivial_destructor = false ;
}
2018-02-18 04:17:11 +00:00
2018-08-27 17:42:13 +00:00
method_chosen = chooseAggregationMethod ( ) ;
2019-01-21 10:39:24 +00:00
HashMethodContext : : Settings cache_settings ;
2018-09-12 13:27:00 +00:00
cache_settings . max_threads = params . max_threads ;
2018-09-14 09:14:37 +00:00
aggregation_state_cache = AggregatedDataVariants : : createCache ( method_chosen , cache_settings ) ;
2021-06-04 10:43:11 +00:00
# if USE_EMBEDDED_COMPILER
2021-07-22 21:43:31 +00:00
compileAggregateFunctionsIfNeeded ( ) ;
2021-06-04 10:43:11 +00:00
# endif
2015-12-09 02:56:18 +00:00
}
2021-06-04 10:43:11 +00:00
# if USE_EMBEDDED_COMPILER
2021-07-22 21:43:31 +00:00
void Aggregator : : compileAggregateFunctionsIfNeeded ( )
2021-06-03 19:20:53 +00:00
{
2021-06-06 16:40:32 +00:00
static std : : unordered_map < UInt128 , UInt64 , UInt128Hash > aggregate_functions_description_to_count ;
2022-05-02 14:21:10 +00:00
static std : : mutex mutex ;
2021-06-06 16:40:32 +00:00
2021-06-29 10:04:14 +00:00
if ( ! params . compile_aggregate_expressions )
2021-06-03 19:20:53 +00:00
return ;
std : : vector < AggregateFunctionWithOffset > functions_to_compile ;
2021-06-06 15:43:03 +00:00
String functions_description ;
2021-06-03 19:20:53 +00:00
2021-06-27 17:43:47 +00:00
is_aggregate_function_compiled . resize ( aggregate_functions . size ( ) ) ;
2021-06-03 19:20:53 +00:00
/// Add values to the aggregate functions.
for ( size_t i = 0 ; i < aggregate_functions . size ( ) ; + + i )
{
const auto * function = aggregate_functions [ i ] ;
size_t offset_of_aggregate_function = offsets_of_aggregate_states [ i ] ;
2021-06-27 17:43:47 +00:00
if ( function - > isCompilable ( ) )
2021-06-03 19:20:53 +00:00
{
AggregateFunctionWithOffset function_to_compile
{
. function = function ,
. aggregate_data_offset = offset_of_aggregate_function
} ;
2022-06-07 11:53:10 +00:00
functions_to_compile . emplace_back ( std : : move ( function_to_compile ) ) ;
2021-06-06 15:43:03 +00:00
functions_description + = function - > getDescription ( ) ;
functions_description + = ' ' ;
2021-06-27 17:43:47 +00:00
functions_description + = std : : to_string ( offset_of_aggregate_function ) ;
functions_description + = ' ' ;
2021-06-03 19:20:53 +00:00
}
2021-06-27 17:43:47 +00:00
is_aggregate_function_compiled [ i ] = function - > isCompilable ( ) ;
2021-06-03 19:20:53 +00:00
}
2021-06-27 17:43:47 +00:00
if ( functions_to_compile . empty ( ) )
2021-06-03 19:20:53 +00:00
return ;
2021-06-13 12:38:57 +00:00
SipHash aggregate_functions_description_hash ;
aggregate_functions_description_hash . update ( functions_description ) ;
2021-06-06 16:40:32 +00:00
2021-06-13 12:38:57 +00:00
UInt128 aggregate_functions_description_hash_key ;
aggregate_functions_description_hash . get128 ( aggregate_functions_description_hash_key ) ;
2021-06-03 19:20:53 +00:00
2021-06-06 16:40:32 +00:00
{
2022-05-02 14:21:10 +00:00
std : : lock_guard < std : : mutex > lock ( mutex ) ;
2021-06-03 19:20:53 +00:00
2021-06-13 12:38:57 +00:00
if ( aggregate_functions_description_to_count [ aggregate_functions_description_hash_key ] + + < params . min_count_to_compile_aggregate_expression )
2021-06-06 16:40:32 +00:00
return ;
2022-05-02 14:21:10 +00:00
}
2021-06-06 16:40:32 +00:00
2022-05-02 14:21:10 +00:00
if ( auto * compilation_cache = CompiledExpressionCacheFactory : : instance ( ) . tryGetCache ( ) )
{
auto [ compiled_function_cache_entry , _ ] = compilation_cache - > getOrSet ( aggregate_functions_description_hash_key , [ & ] ( )
2021-06-03 19:20:53 +00:00
{
2021-06-06 15:43:03 +00:00
LOG_TRACE ( log , " Compile expression {} " , functions_description ) ;
2022-05-02 14:21:10 +00:00
2021-07-22 21:43:31 +00:00
auto compiled_aggregate_functions = compileAggregateFunctions ( getJITInstance ( ) , functions_to_compile , functions_description ) ;
2022-05-02 14:21:10 +00:00
return std : : make_shared < CompiledAggregateFunctionsHolder > ( std : : move ( compiled_aggregate_functions ) ) ;
} ) ;
compiled_aggregate_functions_holder = std : : static_pointer_cast < CompiledAggregateFunctionsHolder > ( compiled_function_cache_entry ) ;
}
else
{
LOG_TRACE ( log , " Compile expression {} " , functions_description ) ;
auto compiled_aggregate_functions = compileAggregateFunctions ( getJITInstance ( ) , functions_to_compile , functions_description ) ;
compiled_aggregate_functions_holder = std : : make_shared < CompiledAggregateFunctionsHolder > ( std : : move ( compiled_aggregate_functions ) ) ;
2021-06-03 19:20:53 +00:00
}
}
2015-12-09 02:56:18 +00:00
2021-06-04 10:43:11 +00:00
# endif
2015-12-09 02:56:18 +00:00
2018-02-18 04:17:11 +00:00
AggregatedDataVariants : : Type Aggregator : : chooseAggregationMethod ( )
2012-05-30 01:38:02 +00:00
{
2018-01-09 00:19:58 +00:00
/// If no keys. All aggregating to single row.
if ( params . keys_size = = 0 )
return AggregatedDataVariants : : Type : : without_key ;
2017-04-01 07:20:54 +00:00
/// Check if at least one of the specified keys is nullable.
2018-01-09 00:19:58 +00:00
DataTypes types_removed_nullable ;
types_removed_nullable . reserve ( params . keys . size ( ) ) ;
2017-04-01 07:20:54 +00:00
bool has_nullable_key = false ;
2018-08-21 14:53:51 +00:00
bool has_low_cardinality = false ;
2017-04-01 07:20:54 +00:00
2022-06-27 11:16:52 +00:00
for ( const auto & key : params . keys )
2017-04-01 07:20:54 +00:00
{
2022-06-27 11:16:52 +00:00
DataTypePtr type = header . getByName ( key ) . type ;
2018-08-21 14:53:51 +00:00
2018-09-27 15:55:22 +00:00
if ( type - > lowCardinality ( ) )
2018-08-21 14:53:51 +00:00
{
has_low_cardinality = true ;
type = removeLowCardinality ( type ) ;
}
2018-01-09 00:19:58 +00:00
if ( type - > isNullable ( ) )
2017-04-01 07:20:54 +00:00
{
has_nullable_key = true ;
2018-08-21 14:53:51 +00:00
type = removeNullable ( type ) ;
2017-04-01 07:20:54 +00:00
}
2018-08-21 14:53:51 +00:00
types_removed_nullable . push_back ( type ) ;
2017-04-01 07:20:54 +00:00
}
/** Returns ordinary (not two-level) methods, because we start from them.
* Later , during aggregation process , data may be converted ( partitioned ) to two - level structure , if cardinality is high .
*/
size_t keys_bytes = 0 ;
2018-01-09 00:19:58 +00:00
size_t num_fixed_contiguous_keys = 0 ;
2017-04-01 07:20:54 +00:00
key_sizes . resize ( params . keys_size ) ;
for ( size_t j = 0 ; j < params . keys_size ; + + j )
{
2018-01-09 00:19:58 +00:00
if ( types_removed_nullable [ j ] - > isValueUnambiguouslyRepresentedInContiguousMemoryRegion ( ) )
2017-04-01 07:20:54 +00:00
{
2018-01-09 00:19:58 +00:00
if ( types_removed_nullable [ j ] - > isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion ( ) )
2017-04-01 07:20:54 +00:00
{
2018-01-09 00:19:58 +00:00
+ + num_fixed_contiguous_keys ;
key_sizes [ j ] = types_removed_nullable [ j ] - > getSizeOfValueInMemory ( ) ;
keys_bytes + = key_sizes [ j ] ;
2017-04-01 07:20:54 +00:00
}
}
}
2018-01-09 00:19:58 +00:00
if ( has_nullable_key )
2017-04-01 07:20:54 +00:00
{
2018-09-14 13:02:03 +00:00
if ( params . keys_size = = num_fixed_contiguous_keys & & ! has_low_cardinality )
2017-04-01 07:20:54 +00:00
{
/// Pack if possible all the keys along with information about which key values are nulls
/// into a fixed 16- or 32-byte blob.
2018-01-09 00:19:58 +00:00
if ( std : : tuple_size < KeysNullMap < UInt128 > > : : value + keys_bytes < = 16 )
2017-04-01 07:20:54 +00:00
return AggregatedDataVariants : : Type : : nullable_keys128 ;
2021-01-27 00:54:57 +00:00
if ( std : : tuple_size < KeysNullMap < UInt256 > > : : value + keys_bytes < = 32 )
2017-04-01 07:20:54 +00:00
return AggregatedDataVariants : : Type : : nullable_keys256 ;
}
2018-12-03 13:00:01 +00:00
if ( has_low_cardinality & & params . keys_size = = 1 )
{
if ( types_removed_nullable [ 0 ] - > isValueRepresentedByNumber ( ) )
{
size_t size_of_field = types_removed_nullable [ 0 ] - > getSizeOfValueInMemory ( ) ;
if ( size_of_field = = 1 )
return AggregatedDataVariants : : Type : : low_cardinality_key8 ;
if ( size_of_field = = 2 )
return AggregatedDataVariants : : Type : : low_cardinality_key16 ;
if ( size_of_field = = 4 )
return AggregatedDataVariants : : Type : : low_cardinality_key32 ;
if ( size_of_field = = 8 )
return AggregatedDataVariants : : Type : : low_cardinality_key64 ;
}
else if ( isString ( types_removed_nullable [ 0 ] ) )
return AggregatedDataVariants : : Type : : low_cardinality_key_string ;
else if ( isFixedString ( types_removed_nullable [ 0 ] ) )
return AggregatedDataVariants : : Type : : low_cardinality_key_fixed_string ;
}
2017-04-01 07:20:54 +00:00
/// Fallback case.
return AggregatedDataVariants : : Type : : serialized ;
}
/// No key has been found to be nullable.
/// Single numeric key.
2018-01-09 00:19:58 +00:00
if ( params . keys_size = = 1 & & types_removed_nullable [ 0 ] - > isValueRepresentedByNumber ( ) )
2017-04-01 07:20:54 +00:00
{
2018-01-09 00:19:58 +00:00
size_t size_of_field = types_removed_nullable [ 0 ] - > getSizeOfValueInMemory ( ) ;
2018-08-21 14:53:51 +00:00
if ( has_low_cardinality )
{
if ( size_of_field = = 1 )
return AggregatedDataVariants : : Type : : low_cardinality_key8 ;
if ( size_of_field = = 2 )
return AggregatedDataVariants : : Type : : low_cardinality_key16 ;
if ( size_of_field = = 4 )
return AggregatedDataVariants : : Type : : low_cardinality_key32 ;
if ( size_of_field = = 8 )
return AggregatedDataVariants : : Type : : low_cardinality_key64 ;
}
2017-04-01 07:20:54 +00:00
if ( size_of_field = = 1 )
return AggregatedDataVariants : : Type : : key8 ;
if ( size_of_field = = 2 )
return AggregatedDataVariants : : Type : : key16 ;
if ( size_of_field = = 4 )
return AggregatedDataVariants : : Type : : key32 ;
if ( size_of_field = = 8 )
return AggregatedDataVariants : : Type : : key64 ;
2017-06-15 09:12:32 +00:00
if ( size_of_field = = 16 )
return AggregatedDataVariants : : Type : : keys128 ;
2020-09-04 13:33:02 +00:00
if ( size_of_field = = 32 )
return AggregatedDataVariants : : Type : : keys256 ;
throw Exception ( " Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32. " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-04-01 07:20:54 +00:00
}
2020-09-20 07:16:46 +00:00
if ( params . keys_size = = 1 & & isFixedString ( types_removed_nullable [ 0 ] ) )
{
if ( has_low_cardinality )
return AggregatedDataVariants : : Type : : low_cardinality_key_fixed_string ;
else
return AggregatedDataVariants : : Type : : key_fixed_string ;
}
2017-04-01 07:20:54 +00:00
/// If all keys fits in N bits, will use hash table with all keys packed (placed contiguously) to single N-bit key.
2018-09-14 13:02:03 +00:00
if ( params . keys_size = = num_fixed_contiguous_keys )
2018-01-09 00:19:58 +00:00
{
2018-09-14 13:02:03 +00:00
if ( has_low_cardinality )
{
if ( keys_bytes < = 16 )
return AggregatedDataVariants : : Type : : low_cardinality_keys128 ;
if ( keys_bytes < = 32 )
return AggregatedDataVariants : : Type : : low_cardinality_keys256 ;
}
2020-03-23 13:44:21 +00:00
if ( keys_bytes < = 2 )
return AggregatedDataVariants : : Type : : keys16 ;
if ( keys_bytes < = 4 )
return AggregatedDataVariants : : Type : : keys32 ;
if ( keys_bytes < = 8 )
return AggregatedDataVariants : : Type : : keys64 ;
2018-01-09 00:19:58 +00:00
if ( keys_bytes < = 16 )
return AggregatedDataVariants : : Type : : keys128 ;
if ( keys_bytes < = 32 )
return AggregatedDataVariants : : Type : : keys256 ;
}
2017-04-01 07:20:54 +00:00
/// If single string key - will use hash table with references to it. Strings itself are stored separately in Arena.
2018-09-07 14:37:26 +00:00
if ( params . keys_size = = 1 & & isString ( types_removed_nullable [ 0 ] ) )
2018-08-21 14:53:51 +00:00
{
if ( has_low_cardinality )
return AggregatedDataVariants : : Type : : low_cardinality_key_string ;
else
return AggregatedDataVariants : : Type : : key_string ;
}
2017-04-01 07:20:54 +00:00
return AggregatedDataVariants : : Type : : serialized ;
2012-05-30 01:38:02 +00:00
}
2021-06-27 17:43:47 +00:00
template < bool skip_compiled_aggregate_functions >
2014-05-19 19:41:56 +00:00
void Aggregator : : createAggregateStates ( AggregateDataPtr & aggregate_data ) const
{
2017-04-01 07:20:54 +00:00
for ( size_t j = 0 ; j < params . aggregates_size ; + + j )
{
2021-06-27 17:43:47 +00:00
if constexpr ( skip_compiled_aggregate_functions )
if ( is_aggregate_function_compiled [ j ] )
continue ;
2017-04-01 07:20:54 +00:00
try
{
2017-04-02 17:37:49 +00:00
/** An exception may occur if there is a shortage of memory.
* In order that then everything is properly destroyed , we " roll back " some of the created states .
* The code is not very convenient .
2017-04-01 07:20:54 +00:00
*/
aggregate_functions [ j ] - > create ( aggregate_data + offsets_of_aggregate_states [ j ] ) ;
}
catch ( . . . )
{
for ( size_t rollback_j = 0 ; rollback_j < j ; + + rollback_j )
2021-06-27 17:43:47 +00:00
{
if constexpr ( skip_compiled_aggregate_functions )
if ( is_aggregate_function_compiled [ j ] )
continue ;
2017-04-01 07:20:54 +00:00
aggregate_functions [ rollback_j ] - > destroy ( aggregate_data + offsets_of_aggregate_states [ rollback_j ] ) ;
2021-06-27 17:43:47 +00:00
}
2017-04-01 07:20:54 +00:00
throw ;
}
}
2014-05-19 19:41:56 +00:00
}
2021-07-20 17:02:41 +00:00
bool Aggregator : : hasSparseArguments ( AggregateFunctionInstruction * aggregate_instructions )
{
for ( auto * inst = aggregate_instructions ; inst - > that ; + + inst )
if ( inst - > has_sparse_arguments )
return true ;
return false ;
}
2022-04-22 17:59:36 +00:00
void Aggregator : : executeOnBlockSmall (
AggregatedDataVariants & result ,
size_t row_begin ,
size_t row_end ,
ColumnRawPtrs & key_columns ,
AggregateFunctionInstruction * aggregate_instructions ) const
{
/// `result` will destroy the states of aggregate functions in the destructor
result . aggregator = this ;
/// How to perform the aggregation?
if ( result . empty ( ) )
{
initDataVariantsWithSizeHint ( result , method_chosen , params ) ;
result . keys_size = params . keys_size ;
result . key_sizes = key_sizes ;
}
executeImpl ( result , row_begin , row_end , key_columns , aggregate_instructions ) ;
}
2022-05-19 07:34:41 +00:00
void Aggregator : : mergeOnBlockSmall (
AggregatedDataVariants & result ,
size_t row_begin ,
size_t row_end ,
const AggregateColumnsConstData & aggregate_columns_data ,
const ColumnRawPtrs & key_columns ) const
{
/// `result` will destroy the states of aggregate functions in the destructor
result . aggregator = this ;
/// How to perform the aggregation?
if ( result . empty ( ) )
{
initDataVariantsWithSizeHint ( result , method_chosen , params ) ;
result . keys_size = params . keys_size ;
result . key_sizes = key_sizes ;
}
if ( false ) { } // NOLINT
# define M(NAME, IS_TWO_LEVEL) \
else if ( result . type = = AggregatedDataVariants : : Type : : NAME ) \
mergeStreamsImpl ( result . aggregates_pool , * result . NAME , result . NAME - > data , \
result . without_key , /* no_more_keys= */ false , \
row_begin , row_end , \
2022-08-03 15:56:59 +00:00
aggregate_columns_data , key_columns , result . aggregates_pool ) ;
2022-05-19 07:34:41 +00:00
APPLY_FOR_AGGREGATED_VARIANTS ( M )
# undef M
else
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
}
2022-04-22 17:59:36 +00:00
void Aggregator : : executeImpl (
AggregatedDataVariants & result ,
size_t row_begin ,
size_t row_end ,
ColumnRawPtrs & key_columns ,
AggregateFunctionInstruction * aggregate_instructions ,
bool no_more_keys ,
AggregateDataPtr overflow_row ) const
{
# define M(NAME, IS_TWO_LEVEL) \
else if ( result . type = = AggregatedDataVariants : : Type : : NAME ) \
executeImpl ( * result . NAME , result . aggregates_pool , row_begin , row_end , key_columns , aggregate_instructions , no_more_keys , overflow_row ) ;
if ( false ) { } // NOLINT
APPLY_FOR_AGGREGATED_VARIANTS ( M )
# undef M
}
2017-04-02 17:37:49 +00:00
/** It's interesting - if you remove `noinline`, then gcc for some reason will inline this function, and the performance decreases (~ 10%).
* ( Probably because after the inline of this function , more internal functions no longer be inlined . )
* Inline does not make sense , since the inner loop is entirely inside this function .
2014-10-29 01:18:50 +00:00
*/
2014-05-10 00:31:22 +00:00
template < typename Method >
2014-10-29 01:18:50 +00:00
void NO_INLINE Aggregator : : executeImpl (
2017-04-01 07:20:54 +00:00
Method & method ,
Arena * aggregates_pool ,
2022-03-07 13:48:38 +00:00
size_t row_begin ,
size_t row_end ,
2017-12-13 01:27:53 +00:00
ColumnRawPtrs & key_columns ,
2017-04-01 07:20:54 +00:00
AggregateFunctionInstruction * aggregate_instructions ,
bool no_more_keys ,
AggregateDataPtr overflow_row ) const
2014-05-10 00:31:22 +00:00
{
2019-01-21 10:39:24 +00:00
typename Method : : State state ( key_columns , key_sizes , aggregation_state_cache ) ;
2014-05-10 00:31:22 +00:00
2017-04-01 07:20:54 +00:00
if ( ! no_more_keys )
2021-06-09 07:40:39 +00:00
{
2021-06-13 12:38:57 +00:00
# if USE_EMBEDDED_COMPILER
2021-07-20 17:02:41 +00:00
if ( compiled_aggregate_functions_holder & & ! hasSparseArguments ( aggregate_instructions ) )
2021-06-09 07:40:39 +00:00
{
2022-03-07 13:48:38 +00:00
executeImplBatch < false , true > ( method , state , aggregates_pool , row_begin , row_end , aggregate_instructions , overflow_row ) ;
2021-06-09 07:40:39 +00:00
}
else
2021-06-13 12:38:57 +00:00
# endif
2021-06-09 07:40:39 +00:00
{
2022-03-07 13:48:38 +00:00
executeImplBatch < false , false > ( method , state , aggregates_pool , row_begin , row_end , aggregate_instructions , overflow_row ) ;
2021-06-09 07:40:39 +00:00
}
}
2017-04-01 07:20:54 +00:00
else
2019-08-10 22:36:55 +00:00
{
2022-03-07 13:48:38 +00:00
executeImplBatch < true , false > ( method , state , aggregates_pool , row_begin , row_end , aggregate_instructions , overflow_row ) ;
2021-05-31 08:05:40 +00:00
}
2014-12-30 10:16:23 +00:00
}
2021-06-13 12:38:57 +00:00
template < bool no_more_keys , bool use_compiled_functions , typename Method >
2021-03-18 09:31:14 +00:00
void NO_INLINE Aggregator : : executeImplBatch (
2017-04-01 07:20:54 +00:00
Method & method ,
typename Method : : State & state ,
Arena * aggregates_pool ,
2022-03-07 13:48:38 +00:00
size_t row_begin ,
size_t row_end ,
2017-04-01 07:20:54 +00:00
AggregateFunctionInstruction * aggregate_instructions ,
AggregateDataPtr overflow_row ) const
2019-08-10 22:36:55 +00:00
{
2020-08-03 01:34:43 +00:00
/// Optimization for special case when there are no aggregate functions.
if ( params . aggregates_size = = 0 )
{
2021-03-18 13:23:09 +00:00
if constexpr ( no_more_keys )
return ;
2020-08-03 01:34:43 +00:00
/// For all rows.
AggregateDataPtr place = aggregates_pool - > alloc ( 0 ) ;
2022-03-07 13:48:38 +00:00
for ( size_t i = row_begin ; i < row_end ; + + i )
2020-08-03 01:34:43 +00:00
state . emplaceKey ( method . data , i , * aggregates_pool ) . setMapped ( place ) ;
return ;
}
/// Optimization for special case when aggregating by 8bit key.
2021-03-18 09:31:14 +00:00
if constexpr ( ! no_more_keys & & std : : is_same_v < Method , typename decltype ( AggregatedDataVariants : : key8 ) : : element_type > )
2020-07-29 18:35:52 +00:00
{
2020-08-14 06:52:28 +00:00
/// We use another method if there are aggregate functions with -Array combinator.
bool has_arrays = false ;
2020-07-29 18:35:52 +00:00
for ( AggregateFunctionInstruction * inst = aggregate_instructions ; inst - > that ; + + inst )
{
2020-08-14 06:52:28 +00:00
if ( inst - > offsets )
{
has_arrays = true ;
break ;
}
}
2021-07-20 17:02:41 +00:00
if ( ! has_arrays & & ! hasSparseArguments ( aggregate_instructions ) )
2020-08-14 06:52:28 +00:00
{
for ( AggregateFunctionInstruction * inst = aggregate_instructions ; inst - > that ; + + inst )
{
inst - > batch_that - > addBatchLookupTable8 (
2022-03-07 13:48:38 +00:00
row_begin ,
row_end ,
2020-08-14 06:52:28 +00:00
reinterpret_cast < AggregateDataPtr * > ( method . data . data ( ) ) ,
inst - > state_offset ,
[ & ] ( AggregateDataPtr & aggregate_data )
{
aggregate_data = aggregates_pool - > alignedAlloc ( total_size_of_aggregate_states , align_aggregate_states ) ;
createAggregateStates ( aggregate_data ) ;
} ,
state . getKeyData ( ) ,
inst - > batch_arguments ,
aggregates_pool ) ;
}
return ;
2020-07-29 18:35:52 +00:00
}
}
2022-03-07 13:48:38 +00:00
/// NOTE: only row_end-row_start is required, but:
/// - this affects only optimize_aggregation_in_order,
/// - this is just a pointer, so it should not be significant,
/// - and plus this will require other changes in the interface.
std : : unique_ptr < AggregateDataPtr [ ] > places ( new AggregateDataPtr [ row_end ] ) ;
2019-08-10 22:36:55 +00:00
/// For all rows.
2022-03-07 13:48:38 +00:00
for ( size_t i = row_begin ; i < row_end ; + + i )
2019-08-10 22:36:55 +00:00
{
AggregateDataPtr aggregate_data = nullptr ;
2021-03-18 09:31:14 +00:00
if constexpr ( ! no_more_keys )
2019-08-10 22:36:55 +00:00
{
2021-03-18 09:31:14 +00:00
auto emplace_result = state . emplaceKey ( method . data , i , * aggregates_pool ) ;
2019-08-10 22:36:55 +00:00
2021-03-18 09:31:14 +00:00
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
if ( emplace_result . isInserted ( ) )
{
/// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
emplace_result . setMapped ( nullptr ) ;
2019-08-10 22:36:55 +00:00
2021-03-18 09:31:14 +00:00
aggregate_data = aggregates_pool - > alignedAlloc ( total_size_of_aggregate_states , align_aggregate_states ) ;
2021-06-13 12:38:57 +00:00
# if USE_EMBEDDED_COMPILER
if constexpr ( use_compiled_functions )
2021-06-16 09:12:23 +00:00
{
2021-06-27 17:43:47 +00:00
const auto & compiled_aggregate_functions = compiled_aggregate_functions_holder - > compiled_aggregate_functions ;
compiled_aggregate_functions . create_aggregate_states_function ( aggregate_data ) ;
if ( compiled_aggregate_functions . functions_count ! = aggregate_functions . size ( ) )
{
static constexpr bool skip_compiled_aggregate_functions = true ;
createAggregateStates < skip_compiled_aggregate_functions > ( aggregate_data ) ;
}
2021-06-16 09:12:23 +00:00
# if defined(MEMORY_SANITIZER)
2021-07-03 13:29:32 +00:00
/// We compile only functions that do not allocate some data in Arena. Only store necessary state in AggregateData place.
2021-06-27 20:48:57 +00:00
for ( size_t aggregate_function_index = 0 ; aggregate_function_index < aggregate_functions . size ( ) ; + + aggregate_function_index )
2021-06-27 17:43:47 +00:00
{
2021-06-27 20:48:57 +00:00
if ( ! is_aggregate_function_compiled [ aggregate_function_index ] )
2021-06-27 17:43:47 +00:00
continue ;
2021-06-27 20:48:57 +00:00
auto aggregate_data_with_offset = aggregate_data + offsets_of_aggregate_states [ aggregate_function_index ] ;
auto data_size = params . aggregates [ aggregate_function_index ] . function - > sizeOfData ( ) ;
__msan_unpoison ( aggregate_data_with_offset , data_size ) ;
2021-06-27 17:43:47 +00:00
}
2021-06-13 12:38:57 +00:00
# endif
2021-06-16 09:12:23 +00:00
}
2021-06-27 17:43:47 +00:00
else
2021-06-16 09:12:23 +00:00
# endif
2021-06-27 17:43:47 +00:00
{
createAggregateStates ( aggregate_data ) ;
}
2019-08-10 22:36:55 +00:00
2021-03-18 09:31:14 +00:00
emplace_result . setMapped ( aggregate_data ) ;
}
else
aggregate_data = emplace_result . getMapped ( ) ;
assert ( aggregate_data ! = nullptr ) ;
2019-08-10 22:36:55 +00:00
}
else
2021-03-18 09:31:14 +00:00
{
/// Add only if the key already exists.
auto find_result = state . findKey ( method . data , i , * aggregates_pool ) ;
if ( find_result . isFound ( ) )
aggregate_data = find_result . getMapped ( ) ;
else
aggregate_data = overflow_row ;
}
2019-08-10 22:36:55 +00:00
places [ i ] = aggregate_data ;
}
2021-06-13 12:38:57 +00:00
# if USE_EMBEDDED_COMPILER
if constexpr ( use_compiled_functions )
{
std : : vector < ColumnData > columns_data ;
2021-06-27 17:43:47 +00:00
for ( size_t i = 0 ; i < aggregate_functions . size ( ) ; + + i )
2021-06-13 12:38:57 +00:00
{
2021-06-27 17:43:47 +00:00
if ( ! is_aggregate_function_compiled [ i ] )
continue ;
AggregateFunctionInstruction * inst = aggregate_instructions + i ;
2021-06-13 12:38:57 +00:00
size_t arguments_size = inst - > that - > getArgumentTypes ( ) . size ( ) ;
2021-06-27 17:43:47 +00:00
for ( size_t argument_index = 0 ; argument_index < arguments_size ; + + argument_index )
columns_data . emplace_back ( getColumnData ( inst - > batch_arguments [ argument_index ] ) ) ;
2021-06-13 12:38:57 +00:00
}
auto add_into_aggregate_states_function = compiled_aggregate_functions_holder - > compiled_aggregate_functions . add_into_aggregate_states_function ;
2022-03-07 13:48:38 +00:00
add_into_aggregate_states_function ( row_begin , row_end , columns_data . data ( ) , places . get ( ) ) ;
2021-06-04 10:43:11 +00:00
}
# endif
2021-06-13 12:38:57 +00:00
2019-08-10 22:36:55 +00:00
/// Add values to the aggregate functions.
2021-06-27 17:43:47 +00:00
for ( size_t i = 0 ; i < aggregate_functions . size ( ) ; + + i )
2019-11-04 05:29:54 +00:00
{
2021-06-27 17:43:47 +00:00
# if USE_EMBEDDED_COMPILER
if constexpr ( use_compiled_functions )
if ( is_aggregate_function_compiled [ i ] )
continue ;
# endif
AggregateFunctionInstruction * inst = aggregate_instructions + i ;
2019-11-11 08:36:19 +00:00
if ( inst - > offsets )
2022-03-07 13:48:38 +00:00
inst - > batch_that - > addBatchArray ( row_begin , row_end , places . get ( ) , inst - > state_offset , inst - > batch_arguments , inst - > offsets , aggregates_pool ) ;
2021-03-12 16:33:41 +00:00
else if ( inst - > has_sparse_arguments )
2022-03-07 13:48:38 +00:00
inst - > batch_that - > addBatchSparse ( row_begin , row_end , places . get ( ) , inst - > state_offset , inst - > batch_arguments , aggregates_pool ) ;
2019-11-11 08:36:19 +00:00
else
2022-03-07 13:48:38 +00:00
inst - > batch_that - > addBatch ( row_begin , row_end , places . get ( ) , inst - > state_offset , inst - > batch_arguments , aggregates_pool ) ;
2019-11-04 05:29:54 +00:00
}
2019-08-10 22:36:55 +00:00
}
2021-07-27 16:50:57 +00:00
template < bool use_compiled_functions >
2015-01-13 03:03:45 +00:00
void NO_INLINE Aggregator : : executeWithoutKeyImpl (
2017-04-01 07:20:54 +00:00
AggregatedDataWithoutKey & res ,
2022-03-07 13:48:38 +00:00
size_t row_begin , size_t row_end ,
2017-04-01 07:20:54 +00:00
AggregateFunctionInstruction * aggregate_instructions ,
2021-09-08 15:30:48 +00:00
Arena * arena ) const
2015-01-13 03:03:45 +00:00
{
2022-03-07 13:48:38 +00:00
if ( row_begin = = row_end )
return ;
2021-07-27 16:50:57 +00:00
# if USE_EMBEDDED_COMPILER
if constexpr ( use_compiled_functions )
{
std : : vector < ColumnData > columns_data ;
for ( size_t i = 0 ; i < aggregate_functions . size ( ) ; + + i )
{
if ( ! is_aggregate_function_compiled [ i ] )
continue ;
AggregateFunctionInstruction * inst = aggregate_instructions + i ;
size_t arguments_size = inst - > that - > getArgumentTypes ( ) . size ( ) ;
for ( size_t argument_index = 0 ; argument_index < arguments_size ; + + argument_index )
{
columns_data . emplace_back ( getColumnData ( inst - > batch_arguments [ argument_index ] ) ) ;
}
}
auto add_into_aggregate_states_function_single_place = compiled_aggregate_functions_holder - > compiled_aggregate_functions . add_into_aggregate_states_function_single_place ;
2022-03-07 13:48:38 +00:00
add_into_aggregate_states_function_single_place ( row_begin , row_end , columns_data . data ( ) , res ) ;
2021-07-28 16:47:36 +00:00
# if defined(MEMORY_SANITIZER)
/// We compile only functions that do not allocate some data in Arena. Only store necessary state in AggregateData place.
for ( size_t aggregate_function_index = 0 ; aggregate_function_index < aggregate_functions . size ( ) ; + + aggregate_function_index )
{
if ( ! is_aggregate_function_compiled [ aggregate_function_index ] )
continue ;
auto aggregate_data_with_offset = res + offsets_of_aggregate_states [ aggregate_function_index ] ;
auto data_size = params . aggregates [ aggregate_function_index ] . function - > sizeOfData ( ) ;
__msan_unpoison ( aggregate_data_with_offset , data_size ) ;
}
# endif
2021-07-27 16:50:57 +00:00
}
# endif
2019-08-10 23:40:15 +00:00
/// Adding values
2021-07-27 16:50:57 +00:00
for ( size_t i = 0 ; i < aggregate_functions . size ( ) ; + + i )
2019-11-04 05:29:54 +00:00
{
2021-07-27 16:50:57 +00:00
AggregateFunctionInstruction * inst = aggregate_instructions + i ;
# if USE_EMBEDDED_COMPILER
if constexpr ( use_compiled_functions )
if ( is_aggregate_function_compiled [ i ] )
continue ;
# endif
2019-11-04 05:29:54 +00:00
if ( inst - > offsets )
2019-11-11 08:36:19 +00:00
inst - > batch_that - > addBatchSinglePlace (
2022-05-09 15:55:32 +00:00
inst - > offsets [ static_cast < ssize_t > ( row_begin ) - 1 ] ,
inst - > offsets [ row_end - 1 ] ,
2022-03-07 13:48:38 +00:00
res + inst - > state_offset ,
inst - > batch_arguments ,
arena ) ;
2021-05-16 18:59:43 +00:00
else if ( inst - > has_sparse_arguments )
2022-03-07 13:48:38 +00:00
inst - > batch_that - > addBatchSparseSinglePlace (
row_begin , row_end ,
res + inst - > state_offset ,
inst - > batch_arguments ,
arena ) ;
2019-11-04 05:29:54 +00:00
else
2022-03-07 13:48:38 +00:00
inst - > batch_that - > addBatchSinglePlace (
row_begin , row_end ,
res + inst - > state_offset ,
inst - > batch_arguments ,
arena ) ;
2019-11-04 05:29:54 +00:00
}
2015-01-13 03:03:45 +00:00
}
2020-04-18 09:51:21 +00:00
void NO_INLINE Aggregator : : executeOnIntervalWithoutKeyImpl (
Fix memory leak in AggregatingInOrderTransform
Reproducer:
# NOTE: we need clickhouse from 33957 since right now LSan is broken due to getauxval().
$ url=https://s3.amazonaws.com/clickhouse-builds/33957/e04b862673644d313712607a0078f5d1c48b5377/package_asan/clickhouse
$ wget $url -o clickhouse-asan
$ chmod +x clickhouse-asan
$ ./clickhouse-asan server &
$ ./clickhouse-asan client
:) create table data (key Int, value String) engine=MergeTree() order by key
:) insert into data select number%5, toString(number) from numbers(10e6)
# usually it is enough one query, benchmark is just for stability of the results
# note, that if the exception was not happen from AggregatingInOrderTransform then add --continue_on_errors and wait
$ ./clickhouse-asan benchmark --query 'select key, uniqCombined64(value), groupArray(value) from data group by key' --optimize_aggregation_in_order=1 --memory_tracker_fault_probability=0.01, max_untracked_memory='2Mi'
LSan report:
==24595==ERROR: LeakSanitizer: detected memory leaks
Direct leak of 3932160 byte(s) in 6 object(s) allocated from:
0 0xcadba93 in realloc ()
1 0xcc108d9 in Allocator<false, false>::realloc() obj-x86_64-linux-gnu/../src/Common/Allocator.h:134:30
2 0xde19eae in void DB::PODArrayBase<>::realloc<DB::Arena*&>(unsigned long, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:161:25
3 0xde5f039 in void DB::PODArrayBase<>::reserveForNextSize<DB::Arena*&>(DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h
4 0xde5f039 in void DB::PODArray<>::push_back<>(DB::GroupArrayNodeString*&, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:432:19
5 0xde5f039 in DB::GroupArrayGeneralImpl<>::add() const obj-x86_64-linux-gnu/../src/AggregateFunctions/AggregateFunctionGroupArray.h:465:31
6 0xde5f039 in DB::IAggregateFunctionHelper<>::addBatchSinglePlaceFromInterval() const obj-x86_64-linux-gnu/../src/AggregateFunctions/IAggregateFunction.h:481:53
7 0x299df134 in DB::Aggregator::executeOnIntervalWithoutKeyImpl() obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:869:31
8 0x2ca75f7d in DB::AggregatingInOrderTransform::consume() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingInOrderTransform.cpp:124:13
...
SUMMARY: AddressSanitizer: 4523184 byte(s) leaked in 12 allocation(s).
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-02-01 15:40:48 +00:00
AggregatedDataVariants & data_variants ,
2021-04-25 22:17:24 +00:00
size_t row_begin ,
size_t row_end ,
2022-05-19 07:34:41 +00:00
AggregateFunctionInstruction * aggregate_instructions ) const
2019-08-31 08:58:16 +00:00
{
Fix memory leak in AggregatingInOrderTransform
Reproducer:
# NOTE: we need clickhouse from 33957 since right now LSan is broken due to getauxval().
$ url=https://s3.amazonaws.com/clickhouse-builds/33957/e04b862673644d313712607a0078f5d1c48b5377/package_asan/clickhouse
$ wget $url -o clickhouse-asan
$ chmod +x clickhouse-asan
$ ./clickhouse-asan server &
$ ./clickhouse-asan client
:) create table data (key Int, value String) engine=MergeTree() order by key
:) insert into data select number%5, toString(number) from numbers(10e6)
# usually it is enough one query, benchmark is just for stability of the results
# note, that if the exception was not happen from AggregatingInOrderTransform then add --continue_on_errors and wait
$ ./clickhouse-asan benchmark --query 'select key, uniqCombined64(value), groupArray(value) from data group by key' --optimize_aggregation_in_order=1 --memory_tracker_fault_probability=0.01, max_untracked_memory='2Mi'
LSan report:
==24595==ERROR: LeakSanitizer: detected memory leaks
Direct leak of 3932160 byte(s) in 6 object(s) allocated from:
0 0xcadba93 in realloc ()
1 0xcc108d9 in Allocator<false, false>::realloc() obj-x86_64-linux-gnu/../src/Common/Allocator.h:134:30
2 0xde19eae in void DB::PODArrayBase<>::realloc<DB::Arena*&>(unsigned long, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:161:25
3 0xde5f039 in void DB::PODArrayBase<>::reserveForNextSize<DB::Arena*&>(DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h
4 0xde5f039 in void DB::PODArray<>::push_back<>(DB::GroupArrayNodeString*&, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:432:19
5 0xde5f039 in DB::GroupArrayGeneralImpl<>::add() const obj-x86_64-linux-gnu/../src/AggregateFunctions/AggregateFunctionGroupArray.h:465:31
6 0xde5f039 in DB::IAggregateFunctionHelper<>::addBatchSinglePlaceFromInterval() const obj-x86_64-linux-gnu/../src/AggregateFunctions/IAggregateFunction.h:481:53
7 0x299df134 in DB::Aggregator::executeOnIntervalWithoutKeyImpl() obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:869:31
8 0x2ca75f7d in DB::AggregatingInOrderTransform::consume() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingInOrderTransform.cpp:124:13
...
SUMMARY: AddressSanitizer: 4523184 byte(s) leaked in 12 allocation(s).
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-02-01 15:40:48 +00:00
/// `data_variants` will destroy the states of aggregate functions in the destructor
data_variants . aggregator = this ;
data_variants . init ( AggregatedDataVariants : : Type : : without_key ) ;
AggregatedDataWithoutKey & res = data_variants . without_key ;
2020-04-18 09:51:21 +00:00
/// Adding values
for ( AggregateFunctionInstruction * inst = aggregate_instructions ; inst - > that ; + + inst )
2018-08-21 14:53:51 +00:00
{
2020-04-18 09:51:21 +00:00
if ( inst - > offsets )
2022-05-09 14:27:09 +00:00
inst - > batch_that - > addBatchSinglePlaceFromInterval (
inst - > offsets [ static_cast < ssize_t > ( row_begin ) - 1 ] ,
inst - > offsets [ row_end - 1 ] ,
res + inst - > state_offset ,
2022-05-19 07:34:41 +00:00
inst - > batch_arguments , data_variants . aggregates_pool ) ;
2020-04-18 09:51:21 +00:00
else
2022-05-09 14:27:09 +00:00
inst - > batch_that - > addBatchSinglePlaceFromInterval (
row_begin ,
row_end ,
res + inst - > state_offset ,
inst - > batch_arguments ,
2022-05-19 07:34:41 +00:00
data_variants . aggregates_pool ) ;
2018-08-21 14:53:51 +00:00
}
2020-04-18 09:51:21 +00:00
}
2018-08-21 14:53:51 +00:00
2022-05-19 07:34:41 +00:00
void NO_INLINE Aggregator : : mergeOnIntervalWithoutKeyImpl (
AggregatedDataVariants & data_variants ,
size_t row_begin ,
size_t row_end ,
const AggregateColumnsConstData & aggregate_columns_data ) const
{
/// `data_variants` will destroy the states of aggregate functions in the destructor
data_variants . aggregator = this ;
data_variants . init ( AggregatedDataVariants : : Type : : without_key ) ;
mergeWithoutKeyStreamsImpl ( data_variants , row_begin , row_end , aggregate_columns_data ) ;
}
2018-08-21 14:53:51 +00:00
2022-06-27 11:16:52 +00:00
void Aggregator : : prepareAggregateInstructions (
Columns columns ,
AggregateColumns & aggregate_columns ,
Columns & materialized_columns ,
AggregateFunctionInstructions & aggregate_functions_instructions ,
NestedColumnsHolder & nested_columns_holder ) const
2020-04-18 09:51:21 +00:00
{
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
2022-06-27 11:16:52 +00:00
aggregate_columns [ i ] . resize ( params . aggregates [ i ] . argument_names . size ( ) ) ;
2017-04-01 07:20:54 +00:00
2020-05-07 14:54:15 +00:00
aggregate_functions_instructions . resize ( params . aggregates_size + 1 ) ;
2017-04-01 07:20:54 +00:00
aggregate_functions_instructions [ params . aggregates_size ] . that = nullptr ;
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
{
2021-03-12 16:33:41 +00:00
bool allow_sparse_arguments = aggregate_columns [ i ] . size ( ) = = 1 ;
bool has_sparse_arguments = false ;
2017-04-01 07:20:54 +00:00
for ( size_t j = 0 ; j < aggregate_columns [ i ] . size ( ) ; + + j )
{
2022-06-27 11:16:52 +00:00
const auto pos = header . getPositionByName ( params . aggregates [ i ] . argument_names [ j ] ) ;
materialized_columns . push_back ( columns . at ( pos ) - > convertToFullColumnIfConst ( ) ) ;
2018-12-21 16:00:07 +00:00
aggregate_columns [ i ] [ j ] = materialized_columns . back ( ) . get ( ) ;
2018-08-21 14:53:51 +00:00
2021-03-12 16:33:41 +00:00
auto full_column = allow_sparse_arguments
? aggregate_columns [ i ] [ j ] - > getPtr ( )
2021-04-06 15:59:03 +00:00
: recursiveRemoveSparse ( aggregate_columns [ i ] [ j ] - > getPtr ( ) ) ;
2021-03-12 16:33:41 +00:00
full_column = recursiveRemoveLowCardinality ( full_column ) ;
if ( full_column . get ( ) ! = aggregate_columns [ i ] [ j ] )
2018-08-21 14:53:51 +00:00
{
2021-03-12 16:33:41 +00:00
materialized_columns . emplace_back ( std : : move ( full_column ) ) ;
2018-08-21 14:53:51 +00:00
aggregate_columns [ i ] [ j ] = materialized_columns . back ( ) . get ( ) ;
}
2021-03-12 16:33:41 +00:00
2021-09-16 12:52:35 +00:00
if ( aggregate_columns [ i ] [ j ] - > isSparse ( ) )
2021-03-12 16:33:41 +00:00
has_sparse_arguments = true ;
2017-04-01 07:20:54 +00:00
}
2021-03-12 16:33:41 +00:00
aggregate_functions_instructions [ i ] . has_sparse_arguments = has_sparse_arguments ;
2017-04-01 07:20:54 +00:00
aggregate_functions_instructions [ i ] . arguments = aggregate_columns [ i ] . data ( ) ;
2019-11-04 05:29:54 +00:00
aggregate_functions_instructions [ i ] . state_offset = offsets_of_aggregate_states [ i ] ;
2021-04-25 22:17:24 +00:00
2021-06-09 23:16:01 +00:00
const auto * that = aggregate_functions [ i ] ;
2019-11-04 05:29:54 +00:00
/// Unnest consecutive trailing -State combinators
2020-04-22 05:39:31 +00:00
while ( const auto * func = typeid_cast < const AggregateFunctionState * > ( that ) )
2019-11-04 05:29:54 +00:00
that = func - > getNestedFunction ( ) . get ( ) ;
aggregate_functions_instructions [ i ] . that = that ;
2020-04-22 05:39:31 +00:00
if ( const auto * func = typeid_cast < const AggregateFunctionArray * > ( that ) )
2019-11-04 05:29:54 +00:00
{
/// Unnest consecutive -State combinators before -Array
that = func - > getNestedFunction ( ) . get ( ) ;
2020-04-22 05:39:31 +00:00
while ( const auto * nested_func = typeid_cast < const AggregateFunctionState * > ( that ) )
2019-11-04 05:29:54 +00:00
that = nested_func - > getNestedFunction ( ) . get ( ) ;
auto [ nested_columns , offsets ] = checkAndGetNestedArrayOffset ( aggregate_columns [ i ] . data ( ) , that - > getArgumentTypes ( ) . size ( ) ) ;
nested_columns_holder . push_back ( std : : move ( nested_columns ) ) ;
aggregate_functions_instructions [ i ] . batch_arguments = nested_columns_holder . back ( ) . data ( ) ;
aggregate_functions_instructions [ i ] . offsets = offsets ;
}
else
aggregate_functions_instructions [ i ] . batch_arguments = aggregate_columns [ i ] . data ( ) ;
aggregate_functions_instructions [ i ] . batch_that = that ;
2017-04-01 07:20:54 +00:00
}
2020-04-18 09:51:21 +00:00
}
2020-05-08 13:28:18 +00:00
2022-03-07 13:48:38 +00:00
bool Aggregator : : executeOnBlock ( const Block & block ,
AggregatedDataVariants & result ,
ColumnRawPtrs & key_columns ,
AggregateColumns & aggregate_columns ,
bool & no_more_keys ) const
{
return executeOnBlock ( block . getColumns ( ) ,
/* row_begin= */ 0 , block . rows ( ) ,
result ,
key_columns ,
aggregate_columns ,
no_more_keys ) ;
2020-05-08 13:28:18 +00:00
}
2022-03-07 13:48:38 +00:00
bool Aggregator : : executeOnBlock ( Columns columns ,
size_t row_begin , size_t row_end ,
AggregatedDataVariants & result ,
ColumnRawPtrs & key_columns ,
AggregateColumns & aggregate_columns ,
bool & no_more_keys ) const
2014-05-10 05:16:23 +00:00
{
2017-04-02 17:37:49 +00:00
/// `result` will destroy the states of aggregate functions in the destructor
2017-04-01 07:20:54 +00:00
result . aggregator = this ;
2018-08-21 14:53:51 +00:00
/// How to perform the aggregation?
if ( result . empty ( ) )
{
2022-03-30 20:47:51 +00:00
initDataVariantsWithSizeHint ( result , method_chosen , params ) ;
2018-08-21 14:53:51 +00:00
result . keys_size = params . keys_size ;
result . key_sizes = key_sizes ;
2020-05-26 18:27:50 +00:00
LOG_TRACE ( log , " Aggregation method: {} " , result . getMethodName ( ) ) ;
2018-08-21 14:53:51 +00:00
}
2017-04-02 17:37:49 +00:00
/** Constant columns are not supported directly during aggregation.
* To make them work anyway , we materialize them .
2017-04-01 07:20:54 +00:00
*/
Columns materialized_columns ;
2020-05-07 20:13:51 +00:00
/// Remember the columns we will work with
for ( size_t i = 0 ; i < params . keys_size ; + + i )
{
2022-06-27 11:16:52 +00:00
materialized_columns . push_back ( recursiveRemoveSparse ( columns . at ( keys_positions [ i ] ) ) - > convertToFullColumnIfConst ( ) ) ;
2020-05-07 20:13:51 +00:00
key_columns [ i ] = materialized_columns . back ( ) . get ( ) ;
if ( ! result . isLowCardinality ( ) )
{
auto column_no_lc = recursiveRemoveLowCardinality ( key_columns [ i ] - > getPtr ( ) ) ;
if ( column_no_lc . get ( ) ! = key_columns [ i ] )
{
materialized_columns . emplace_back ( std : : move ( column_no_lc ) ) ;
key_columns [ i ] = materialized_columns . back ( ) . get ( ) ;
}
}
}
2021-06-04 10:43:11 +00:00
2020-05-15 19:27:18 +00:00
NestedColumnsHolder nested_columns_holder ;
2020-05-07 20:13:51 +00:00
AggregateFunctionInstructions aggregate_functions_instructions ;
2020-05-15 19:27:18 +00:00
prepareAggregateInstructions ( columns , aggregate_columns , materialized_columns , aggregate_functions_instructions , nested_columns_holder ) ;
2017-04-01 07:20:54 +00:00
if ( ( params . overflow_row | | result . type = = AggregatedDataVariants : : Type : : without_key ) & & ! result . without_key )
{
2018-09-01 03:17:43 +00:00
AggregateDataPtr place = result . aggregates_pool - > alignedAlloc ( total_size_of_aggregate_states , align_aggregate_states ) ;
2017-04-01 07:20:54 +00:00
createAggregateStates ( place ) ;
result . without_key = place ;
}
2017-04-02 17:37:49 +00:00
/// We select one of the aggregation methods and call it.
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// For the case when there are no keys (all aggregate into one row).
2017-04-01 07:20:54 +00:00
if ( result . type = = AggregatedDataVariants : : Type : : without_key )
{
2021-08-11 16:37:33 +00:00
/// TODO: Enable compilation after investigation
// #if USE_EMBEDDED_COMPILER
// if (compiled_aggregate_functions_holder)
// {
2022-03-07 13:48:38 +00:00
// executeWithoutKeyImpl<true>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
2021-08-11 16:37:33 +00:00
// }
// else
// #endif
2021-07-27 16:50:57 +00:00
{
2022-03-07 13:48:38 +00:00
executeWithoutKeyImpl < false > ( result . without_key , row_begin , row_end , aggregate_functions_instructions . data ( ) , result . aggregates_pool ) ;
2021-07-27 16:50:57 +00:00
}
2017-04-01 07:20:54 +00:00
}
else
{
2017-04-02 17:37:49 +00:00
/// This is where data is written that does not fit in `max_rows_to_group_by` with `group_by_overflow_mode = any`.
2017-04-01 07:20:54 +00:00
AggregateDataPtr overflow_row_ptr = params . overflow_row ? result . without_key : nullptr ;
2022-04-22 17:59:36 +00:00
executeImpl ( result , row_begin , row_end , key_columns , aggregate_functions_instructions . data ( ) , no_more_keys , overflow_row_ptr ) ;
2017-04-01 07:20:54 +00:00
}
size_t result_size = result . sizeWithoutOverflowRow ( ) ;
Int64 current_memory_usage = 0 ;
2020-04-22 05:39:31 +00:00
if ( auto * memory_tracker_child = CurrentThread : : getMemoryTracker ( ) )
if ( auto * memory_tracker = memory_tracker_child - > getParent ( ) )
2019-03-14 18:03:35 +00:00
current_memory_usage = memory_tracker - > get ( ) ;
2017-04-01 07:20:54 +00:00
2020-06-07 21:05:36 +00:00
/// Here all the results in the sum are taken into account, from different threads.
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation ;
2017-04-01 07:20:54 +00:00
2022-03-30 20:47:51 +00:00
bool worth_convert_to_two_level = worthConvertToTwoLevel (
params . group_by_two_level_threshold , result_size , params . group_by_two_level_threshold_bytes , result_size_bytes ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/** Converting to a two-level data structure.
* It allows you to make , in the subsequent , an effective merge - either economical from memory or parallel .
2017-04-01 07:20:54 +00:00
*/
if ( result . isConvertibleToTwoLevel ( ) & & worth_convert_to_two_level )
2018-09-12 13:27:00 +00:00
result . convertToTwoLevel ( ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// Checking the constraints.
2017-04-01 07:20:54 +00:00
if ( ! checkLimits ( result_size , no_more_keys ) )
return false ;
2017-04-02 17:37:49 +00:00
/** Flush data to disk if too much RAM is consumed.
* Data can only be flushed to disk if a two - level aggregation structure is used .
2017-04-01 07:20:54 +00:00
*/
if ( params . max_bytes_before_external_group_by
& & result . isTwoLevel ( )
& & current_memory_usage > static_cast < Int64 > ( params . max_bytes_before_external_group_by )
& & worth_convert_to_two_level )
{
2020-01-19 14:26:28 +00:00
size_t size = current_memory_usage + params . min_free_disk_space ;
2020-07-08 14:25:23 +00:00
2020-07-30 10:04:49 +00:00
std : : string tmp_path = params . tmp_volume - > getDisk ( ) - > getPath ( ) ;
2020-04-20 08:30:01 +00:00
// enoughSpaceInDirectory() is not enough to make it right, since
// another process (or another thread of aggregator) can consume all
// space.
//
// But true reservation (IVolume::reserve()) cannot be used here since
// current_memory_usage does not takes compression into account and
// will reserve way more that actually will be used.
//
// Hence let's do a simple check.
if ( ! enoughSpaceInDirectory ( tmp_path , size ) )
throw Exception ( " Not enough space for external aggregation in " + tmp_path , ErrorCodes : : NOT_ENOUGH_SPACE ) ;
2019-08-27 18:59:21 +00:00
2020-01-19 14:26:28 +00:00
writeToTemporaryFile ( result , tmp_path ) ;
2017-04-01 07:20:54 +00:00
}
return true ;
2015-10-22 01:44:33 +00:00
}
2021-09-08 15:30:48 +00:00
void Aggregator : : writeToTemporaryFile ( AggregatedDataVariants & data_variants , const String & tmp_path ) const
2015-11-30 19:57:46 +00:00
{
2017-04-01 07:20:54 +00:00
Stopwatch watch ;
2017-07-25 13:09:52 +00:00
size_t rows = data_variants . size ( ) ;
2015-12-01 14:09:05 +00:00
2020-01-19 14:26:28 +00:00
auto file = createTemporaryFile ( tmp_path ) ;
2017-04-01 07:20:54 +00:00
const std : : string & path = file - > path ( ) ;
WriteBufferFromFile file_buf ( path ) ;
CompressedWriteBuffer compressed_buf ( file_buf ) ;
2021-10-08 17:21:19 +00:00
NativeWriter block_out ( compressed_buf , DBMS_TCP_PROTOCOL_VERSION , getHeader ( false ) ) ;
2015-11-30 19:57:46 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Writing part of aggregation data into temporary file {}. " , path ) ;
2017-04-01 07:20:54 +00:00
ProfileEvents : : increment ( ProfileEvents : : ExternalAggregationWritePart ) ;
2015-12-01 14:09:05 +00:00
2017-07-25 13:09:52 +00:00
/// Flush only two-level data and possibly overflow data.
2015-11-30 19:57:46 +00:00
2015-12-01 14:09:05 +00:00
# define M(NAME) \
2017-04-01 07:20:54 +00:00
else if ( data_variants . type = = AggregatedDataVariants : : Type : : NAME ) \
2017-12-01 21:13:25 +00:00
writeToTemporaryFileImpl ( data_variants , * data_variants . NAME , block_out ) ;
2015-11-30 19:57:46 +00:00
2020-03-08 23:48:08 +00:00
if ( false ) { } // NOLINT
2017-04-01 07:20:54 +00:00
APPLY_FOR_VARIANTS_TWO_LEVEL ( M )
2015-11-30 19:57:46 +00:00
# undef M
2017-04-01 07:20:54 +00:00
else
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
2017-04-02 17:37:49 +00:00
/// NOTE Instead of freeing up memory and creating new hash tables and arenas, you can re-use the old ones.
2017-04-01 07:20:54 +00:00
data_variants . init ( data_variants . type ) ;
data_variants . aggregates_pools = Arenas ( 1 , std : : make_shared < Arena > ( ) ) ;
data_variants . aggregates_pool = data_variants . aggregates_pools . back ( ) . get ( ) ;
2021-05-08 09:56:42 +00:00
if ( params . overflow_row | | data_variants . type = = AggregatedDataVariants : : Type : : without_key )
{
AggregateDataPtr place = data_variants . aggregates_pool - > alignedAlloc ( total_size_of_aggregate_states , align_aggregate_states ) ;
createAggregateStates ( place ) ;
data_variants . without_key = place ;
}
2017-04-01 07:20:54 +00:00
block_out . flush ( ) ;
compressed_buf . next ( ) ;
file_buf . next ( ) ;
double elapsed_seconds = watch . elapsedSeconds ( ) ;
double compressed_bytes = file_buf . count ( ) ;
double uncompressed_bytes = compressed_buf . count ( ) ;
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( temporary_files . mutex ) ;
2017-04-01 07:20:54 +00:00
temporary_files . files . emplace_back ( std : : move ( file ) ) ;
temporary_files . sum_size_uncompressed + = uncompressed_bytes ;
temporary_files . sum_size_compressed + = compressed_bytes ;
}
ProfileEvents : : increment ( ProfileEvents : : ExternalAggregationCompressedBytes , compressed_bytes ) ;
ProfileEvents : : increment ( ProfileEvents : : ExternalAggregationUncompressedBytes , uncompressed_bytes ) ;
2021-04-15 18:00:16 +00:00
LOG_DEBUG ( log ,
2021-05-08 10:27:48 +00:00
" Written part in {:.3f} sec., {} rows, {} uncompressed, {} compressed, "
" {:.3f} uncompressed bytes per row, {:.3f} compressed bytes per row, compression rate: {:.3f} "
" ({:.3f} rows/sec., {}/sec. uncompressed, {}/sec. compressed) " ,
2020-05-23 21:16:05 +00:00
elapsed_seconds ,
rows ,
2020-05-30 21:35:52 +00:00
ReadableSize ( uncompressed_bytes ) ,
ReadableSize ( compressed_bytes ) ,
2020-05-23 21:16:05 +00:00
uncompressed_bytes / rows ,
compressed_bytes / rows ,
uncompressed_bytes / compressed_bytes ,
rows / elapsed_seconds ,
2020-05-30 21:35:52 +00:00
ReadableSize ( uncompressed_bytes / elapsed_seconds ) ,
ReadableSize ( compressed_bytes / elapsed_seconds ) ) ;
2015-11-30 19:57:46 +00:00
}
2020-07-27 15:59:09 +00:00
2021-09-08 15:30:48 +00:00
void Aggregator : : writeToTemporaryFile ( AggregatedDataVariants & data_variants ) const
2020-01-19 14:26:28 +00:00
{
2020-07-30 10:04:49 +00:00
String tmp_path = params . tmp_volume - > getDisk ( ) - > getPath ( ) ;
2020-07-27 16:07:04 +00:00
return writeToTemporaryFile ( data_variants , tmp_path ) ;
2020-01-19 14:26:28 +00:00
}
2015-11-30 19:57:46 +00:00
2015-12-06 14:27:09 +00:00
template < typename Method >
Block Aggregator : : convertOneBucketToBlock (
2017-04-01 07:20:54 +00:00
AggregatedDataVariants & data_variants ,
Method & method ,
2020-12-22 11:30:29 +00:00
Arena * arena ,
2017-04-01 07:20:54 +00:00
bool final ,
size_t bucket ) const
2015-12-06 14:27:09 +00:00
{
2017-04-01 07:20:54 +00:00
Block block = prepareBlockAndFill ( data_variants , final , method . data . impls [ bucket ] . size ( ) ,
2020-12-22 11:30:29 +00:00
[ bucket , & method , arena , this ] (
2017-12-15 03:47:43 +00:00
MutableColumns & key_columns ,
2017-04-01 07:20:54 +00:00
AggregateColumnsData & aggregate_columns ,
2017-12-15 03:47:43 +00:00
MutableColumns & final_aggregate_columns ,
2018-08-27 18:16:32 +00:00
bool final_ )
2017-04-01 07:20:54 +00:00
{
convertToBlockImpl ( method , method . data . impls [ bucket ] ,
2020-06-17 19:36:27 +00:00
key_columns , aggregate_columns , final_aggregate_columns , arena , final_ ) ;
2017-04-01 07:20:54 +00:00
} ) ;
block . info . bucket_num = bucket ;
return block ;
2015-12-06 14:27:09 +00:00
}
2019-09-06 12:19:59 +00:00
Block Aggregator : : mergeAndConvertOneBucketToBlock (
ManyAggregatedDataVariants & variants ,
Arena * arena ,
bool final ,
2020-01-10 20:24:59 +00:00
size_t bucket ,
std : : atomic < bool > * is_cancelled ) const
2019-09-06 12:19:59 +00:00
{
auto & merged_data = * variants [ 0 ] ;
auto method = merged_data . type ;
Block block ;
2020-03-08 23:48:08 +00:00
if ( false ) { } // NOLINT
2019-09-06 12:19:59 +00:00
# define M(NAME) \
else if ( method = = AggregatedDataVariants : : Type : : NAME ) \
{ \
mergeBucketImpl < decltype ( merged_data . NAME ) : : element_type > ( variants , bucket , arena ) ; \
2020-01-10 20:24:59 +00:00
if ( is_cancelled & & is_cancelled - > load ( std : : memory_order_seq_cst ) ) \
return { } ; \
2020-12-22 11:30:29 +00:00
block = convertOneBucketToBlock ( merged_data , * merged_data . NAME , arena , final , bucket ) ; \
2019-09-06 12:19:59 +00:00
}
APPLY_FOR_VARIANTS_TWO_LEVEL ( M )
# undef M
return block ;
}
2015-12-06 14:27:09 +00:00
2015-11-30 19:57:46 +00:00
template < typename Method >
void Aggregator : : writeToTemporaryFileImpl (
2017-04-01 07:20:54 +00:00
AggregatedDataVariants & data_variants ,
Method & method ,
2021-10-08 17:21:19 +00:00
NativeWriter & out ) const
2015-11-30 19:57:46 +00:00
{
2017-04-01 07:20:54 +00:00
size_t max_temporary_block_size_rows = 0 ;
size_t max_temporary_block_size_bytes = 0 ;
2015-11-30 19:57:46 +00:00
2017-07-25 13:09:52 +00:00
auto update_max_sizes = [ & ] ( const Block & block )
2017-04-01 07:20:54 +00:00
{
size_t block_size_rows = block . rows ( ) ;
size_t block_size_bytes = block . bytes ( ) ;
2015-12-01 14:09:05 +00:00
2017-04-01 07:20:54 +00:00
if ( block_size_rows > max_temporary_block_size_rows )
2017-07-25 13:09:52 +00:00
max_temporary_block_size_rows = block_size_rows ;
2017-04-01 07:20:54 +00:00
if ( block_size_bytes > max_temporary_block_size_bytes )
max_temporary_block_size_bytes = block_size_bytes ;
2017-07-25 13:09:52 +00:00
} ;
for ( size_t bucket = 0 ; bucket < Method : : Data : : NUM_BUCKETS ; + + bucket )
{
2020-12-22 11:30:29 +00:00
Block block = convertOneBucketToBlock ( data_variants , method , data_variants . aggregates_pool , false , bucket ) ;
2017-07-25 13:09:52 +00:00
out . write ( block ) ;
update_max_sizes ( block ) ;
}
if ( params . overflow_row )
{
Block block = prepareBlockAndFillWithoutKey ( data_variants , false , true ) ;
out . write ( block ) ;
update_max_sizes ( block ) ;
2017-04-01 07:20:54 +00:00
}
2015-11-30 19:57:46 +00:00
2017-07-25 13:09:52 +00:00
/// Pass ownership of the aggregate functions states:
/// `data_variants` will not destroy them in the destructor, they are now owned by ColumnAggregateFunction objects.
2017-04-01 07:20:54 +00:00
data_variants . aggregator = nullptr ;
2015-12-01 16:58:15 +00:00
2021-04-15 18:00:16 +00:00
LOG_DEBUG ( log , " Max size of temporary block: {} rows, {}. " , max_temporary_block_size_rows , ReadableSize ( max_temporary_block_size_bytes ) ) ;
2015-11-30 19:57:46 +00:00
}
2015-10-22 01:44:33 +00:00
bool Aggregator : : checkLimits ( size_t result_size , bool & no_more_keys ) const
{
2017-04-01 07:20:54 +00:00
if ( ! no_more_keys & & params . max_rows_to_group_by & & result_size > params . max_rows_to_group_by )
{
2017-09-08 03:47:27 +00:00
switch ( params . group_by_overflow_mode )
{
case OverflowMode : : THROW :
2022-08-14 07:35:22 +00:00
ProfileEvents : : increment ( ProfileEvents : : OverflowThrow ) ;
2017-09-08 03:47:27 +00:00
throw Exception ( " Limit for rows to GROUP BY exceeded: has " + toString ( result_size )
+ " rows, maximum: " + toString ( params . max_rows_to_group_by ) ,
2018-03-09 23:23:15 +00:00
ErrorCodes : : TOO_MANY_ROWS ) ;
2017-09-08 03:47:27 +00:00
case OverflowMode : : BREAK :
2022-08-14 07:35:22 +00:00
ProfileEvents : : increment ( ProfileEvents : : OverflowBreak ) ;
2017-09-08 03:47:27 +00:00
return false ;
case OverflowMode : : ANY :
2022-08-14 07:35:22 +00:00
ProfileEvents : : increment ( ProfileEvents : : OverflowAny ) ;
2017-09-08 03:47:27 +00:00
no_more_keys = true ;
break ;
}
2017-04-01 07:20:54 +00:00
}
2021-08-05 20:44:14 +00:00
/// Some aggregate functions cannot throw exceptions on allocations (e.g. from C malloc)
/// but still tracks memory. Check it here.
CurrentMemoryTracker : : check ( ) ;
2017-04-01 07:20:54 +00:00
return true ;
2014-05-10 05:16:23 +00:00
}
2015-01-03 05:39:21 +00:00
template < typename Method , typename Table >
void Aggregator : : convertToBlockImpl (
2017-04-01 07:20:54 +00:00
Method & method ,
Table & data ,
2017-12-15 03:47:43 +00:00
MutableColumns & key_columns ,
2017-04-01 07:20:54 +00:00
AggregateColumnsData & aggregate_columns ,
2017-12-15 03:47:43 +00:00
MutableColumns & final_aggregate_columns ,
2020-06-17 19:36:27 +00:00
Arena * arena ,
2017-04-01 07:20:54 +00:00
bool final ) const
2015-01-03 05:39:21 +00:00
{
2017-04-01 07:20:54 +00:00
if ( data . empty ( ) )
return ;
2015-12-11 00:34:00 +00:00
2018-10-15 19:14:08 +00:00
if ( key_columns . size ( ) ! = params . keys_size )
throw Exception { " Aggregate. Unexpected key columns size. " , ErrorCodes : : LOGICAL_ERROR } ;
2021-03-10 11:00:24 +00:00
std : : vector < IColumn * > raw_key_columns ;
raw_key_columns . reserve ( key_columns . size ( ) ) ;
for ( auto & column : key_columns )
raw_key_columns . push_back ( column . get ( ) ) ;
2017-04-01 07:20:54 +00:00
if ( final )
2021-06-13 12:38:57 +00:00
{
# if USE_EMBEDDED_COMPILER
if ( compiled_aggregate_functions_holder )
{
2021-06-29 10:04:14 +00:00
static constexpr bool use_compiled_functions = ! Method : : low_cardinality_optimization ;
convertToBlockImplFinal < Method , use_compiled_functions > ( method , data , std : : move ( raw_key_columns ) , final_aggregate_columns , arena ) ;
2021-06-13 12:38:57 +00:00
}
else
# endif
{
convertToBlockImplFinal < Method , false > ( method , data , std : : move ( raw_key_columns ) , final_aggregate_columns , arena ) ;
}
}
2017-04-01 07:20:54 +00:00
else
2021-06-13 12:38:57 +00:00
{
2021-03-10 11:00:24 +00:00
convertToBlockImplNotFinal ( method , data , std : : move ( raw_key_columns ) , aggregate_columns ) ;
2021-06-13 12:38:57 +00:00
}
2017-04-02 17:37:49 +00:00
/// In order to release memory early.
2017-04-01 07:20:54 +00:00
data . clearAndShrink ( ) ;
2015-01-03 05:39:21 +00:00
}
2020-06-08 04:23:56 +00:00
template < typename Mapped >
2022-03-30 20:47:51 +00:00
inline void Aggregator : : insertAggregatesIntoColumns ( Mapped & mapped , MutableColumns & final_aggregate_columns , Arena * arena ) const
2020-06-08 04:23:56 +00:00
{
/** Final values of aggregate functions are inserted to columns.
* Then states of aggregate functions , that are not longer needed , are destroyed .
*
* We mark already destroyed states with " nullptr " in data ,
* so they will not be destroyed in destructor of Aggregator
* ( other values will be destroyed in destructor in case of exception ) .
*
* But it becomes tricky , because we have multiple aggregate states pointed by a single pointer in data .
* So , if exception is thrown in the middle of moving states for different aggregate functions ,
* we have to catch exceptions and destroy all the states that are no longer needed ,
* to keep the data in consistent state .
*
* It is also tricky , because there are aggregate functions with " -State " modifier .
* When we call " insertResultInto " for them , they insert a pointer to the state to ColumnAggregateFunction
* and ColumnAggregateFunction will take ownership of this state .
* So , for aggregate functions with " -State " modifier , the state must not be destroyed
* after it has been transferred to ColumnAggregateFunction .
* But we should mark that the data no longer owns these states .
*/
size_t insert_i = 0 ;
std : : exception_ptr exception ;
try
{
/// Insert final values of aggregate functions into columns.
for ( ; insert_i < params . aggregates_size ; + + insert_i )
aggregate_functions [ insert_i ] - > insertResultInto (
mapped + offsets_of_aggregate_states [ insert_i ] ,
2020-06-17 19:36:27 +00:00
* final_aggregate_columns [ insert_i ] ,
arena ) ;
2020-06-08 04:23:56 +00:00
}
catch ( . . . )
{
exception = std : : current_exception ( ) ;
}
/** Destroy states that are no longer needed. This loop does not throw.
*
* Don ' t destroy states for " -State " aggregate functions ,
* because the ownership of this state is transferred to ColumnAggregateFunction
* and ColumnAggregateFunction will take care .
*
* But it ' s only for states that has been transferred to ColumnAggregateFunction
* before exception has been thrown ;
*/
for ( size_t destroy_i = 0 ; destroy_i < params . aggregates_size ; + + destroy_i )
{
/// If ownership was not transferred to ColumnAggregateFunction.
if ( ! ( destroy_i < insert_i & & aggregate_functions [ destroy_i ] - > isState ( ) ) )
aggregate_functions [ destroy_i ] - > destroy (
mapped + offsets_of_aggregate_states [ destroy_i ] ) ;
}
/// Mark the cell as destroyed so it will not be destroyed in destructor.
mapped = nullptr ;
if ( exception )
std : : rethrow_exception ( exception ) ;
}
2021-06-13 12:38:57 +00:00
template < typename Method , bool use_compiled_functions , typename Table >
2015-01-03 05:39:21 +00:00
void NO_INLINE Aggregator : : convertToBlockImplFinal (
2017-04-01 07:20:54 +00:00
Method & method ,
Table & data ,
2021-03-10 11:00:24 +00:00
std : : vector < IColumn * > key_columns ,
2020-06-17 19:36:27 +00:00
MutableColumns & final_aggregate_columns ,
Arena * arena ) const
2015-01-03 05:39:21 +00:00
{
2018-12-03 13:00:01 +00:00
if constexpr ( Method : : low_cardinality_optimization )
{
if ( data . hasNullKeyData ( ) )
{
2019-02-08 16:54:04 +00:00
key_columns [ 0 ] - > insertDefault ( ) ;
2020-06-17 19:36:27 +00:00
insertAggregatesIntoColumns ( data . getNullKeyData ( ) , final_aggregate_columns , arena ) ;
2018-12-03 13:00:01 +00:00
}
}
2021-03-10 11:00:24 +00:00
auto shuffled_key_sizes = method . shuffleKeyColumns ( key_columns , key_sizes ) ;
const auto & key_sizes_ref = shuffled_key_sizes ? * shuffled_key_sizes : key_sizes ;
2021-06-13 12:38:57 +00:00
PaddedPODArray < AggregateDataPtr > places ;
places . reserve ( data . size ( ) ) ;
2021-06-03 19:20:53 +00:00
2019-08-08 16:16:19 +00:00
data . forEachValue ( [ & ] ( const auto & key , auto & mapped )
2017-04-01 07:20:54 +00:00
{
2021-03-10 11:00:24 +00:00
method . insertKeyIntoColumns ( key , key_columns , key_sizes_ref ) ;
2021-06-13 12:38:57 +00:00
places . emplace_back ( mapped ) ;
2021-06-08 10:32:32 +00:00
2021-06-13 12:38:57 +00:00
/// Mark the cell as destroyed so it will not be destroyed in destructor.
mapped = nullptr ;
2020-06-07 22:46:58 +00:00
} ) ;
2021-06-03 19:20:53 +00:00
2021-06-13 12:38:57 +00:00
std : : exception_ptr exception ;
size_t aggregate_functions_destroy_index = 0 ;
2021-06-03 19:20:53 +00:00
2021-06-13 12:38:57 +00:00
try
{
# if USE_EMBEDDED_COMPILER
if constexpr ( use_compiled_functions )
2021-06-03 19:20:53 +00:00
{
2021-06-08 10:32:32 +00:00
/** For JIT compiled functions we need to resize columns before pass them into compiled code.
* insert_aggregates_into_columns_function function does not throw exception .
*/
std : : vector < ColumnData > columns_data ;
2021-06-13 12:38:57 +00:00
auto compiled_functions = compiled_aggregate_functions_holder - > compiled_aggregate_functions ;
2021-06-27 17:43:47 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
2021-06-08 10:32:32 +00:00
{
2021-06-27 17:43:47 +00:00
if ( ! is_aggregate_function_compiled [ i ] )
continue ;
2021-06-08 10:32:32 +00:00
auto & final_aggregate_column = final_aggregate_columns [ i ] ;
2021-06-10 19:53:51 +00:00
final_aggregate_column = final_aggregate_column - > cloneResized ( places . size ( ) ) ;
2021-06-08 10:32:32 +00:00
columns_data . emplace_back ( getColumnData ( final_aggregate_column . get ( ) ) ) ;
}
2021-06-13 12:38:57 +00:00
auto insert_aggregates_into_columns_function = compiled_functions . insert_aggregates_into_columns_function ;
2022-03-07 13:48:38 +00:00
insert_aggregates_into_columns_function ( 0 , places . size ( ) , columns_data . data ( ) , places . data ( ) ) ;
2021-06-13 12:38:57 +00:00
}
# endif
2021-06-08 10:32:32 +00:00
2021-06-13 12:38:57 +00:00
for ( ; aggregate_functions_destroy_index < params . aggregates_size ; )
{
2021-06-27 17:43:47 +00:00
if constexpr ( use_compiled_functions )
{
if ( is_aggregate_function_compiled [ aggregate_functions_destroy_index ] )
{
+ + aggregate_functions_destroy_index ;
continue ;
}
}
2021-06-13 12:38:57 +00:00
auto & final_aggregate_column = final_aggregate_columns [ aggregate_functions_destroy_index ] ;
size_t offset = offsets_of_aggregate_states [ aggregate_functions_destroy_index ] ;
2021-06-08 10:32:32 +00:00
2021-07-03 13:29:32 +00:00
/** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch
2021-06-27 17:43:47 +00:00
* throws exception , it also must destroy all necessary states .
* Then code need to continue to destroy other aggregate function states with next function index .
*/
2021-06-13 12:38:57 +00:00
size_t destroy_index = aggregate_functions_destroy_index ;
+ + aggregate_functions_destroy_index ;
2021-06-08 10:32:32 +00:00
2021-07-03 13:29:32 +00:00
/// For State AggregateFunction ownership of aggregate place is passed to result column after insert
2021-06-13 12:38:57 +00:00
bool is_state = aggregate_functions [ destroy_index ] - > isState ( ) ;
2021-07-03 13:29:32 +00:00
bool destroy_place_after_insert = ! is_state ;
2021-06-08 10:32:32 +00:00
2022-03-07 13:48:38 +00:00
aggregate_functions [ destroy_index ] - > insertResultIntoBatch ( 0 , places . size ( ) , places . data ( ) , offset , * final_aggregate_column , arena , destroy_place_after_insert ) ;
2021-06-03 19:20:53 +00:00
}
}
2021-06-13 12:38:57 +00:00
catch ( . . . )
2021-06-03 19:20:53 +00:00
{
2021-06-13 12:38:57 +00:00
exception = std : : current_exception ( ) ;
}
for ( ; aggregate_functions_destroy_index < params . aggregates_size ; + + aggregate_functions_destroy_index )
{
2021-06-27 17:43:47 +00:00
if constexpr ( use_compiled_functions )
{
if ( is_aggregate_function_compiled [ aggregate_functions_destroy_index ] )
{
+ + aggregate_functions_destroy_index ;
continue ;
}
}
2021-06-13 12:38:57 +00:00
size_t offset = offsets_of_aggregate_states [ aggregate_functions_destroy_index ] ;
2022-03-07 13:48:38 +00:00
aggregate_functions [ aggregate_functions_destroy_index ] - > destroyBatch ( 0 , places . size ( ) , places . data ( ) , offset ) ;
2021-06-03 19:20:53 +00:00
}
2021-06-13 12:38:57 +00:00
if ( exception )
std : : rethrow_exception ( exception ) ;
2015-01-03 05:39:21 +00:00
}
template < typename Method , typename Table >
void NO_INLINE Aggregator : : convertToBlockImplNotFinal (
2017-04-01 07:20:54 +00:00
Method & method ,
Table & data ,
2021-03-10 11:00:24 +00:00
std : : vector < IColumn * > key_columns ,
2018-08-27 18:05:28 +00:00
AggregateColumnsData & aggregate_columns ) const
2015-01-03 05:39:21 +00:00
{
2018-12-03 13:00:01 +00:00
if constexpr ( Method : : low_cardinality_optimization )
{
if ( data . hasNullKeyData ( ) )
{
2019-02-08 16:54:04 +00:00
key_columns [ 0 ] - > insertDefault ( ) ;
2018-12-03 13:00:01 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
aggregate_columns [ i ] - > push_back ( data . getNullKeyData ( ) + offsets_of_aggregate_states [ i ] ) ;
2020-06-07 22:46:58 +00:00
data . getNullKeyData ( ) = nullptr ;
2018-12-03 13:00:01 +00:00
}
}
2021-03-10 11:00:24 +00:00
auto shuffled_key_sizes = method . shuffleKeyColumns ( key_columns , key_sizes ) ;
const auto & key_sizes_ref = shuffled_key_sizes ? * shuffled_key_sizes : key_sizes ;
2021-02-26 08:52:50 +00:00
2019-08-08 16:16:19 +00:00
data . forEachValue ( [ & ] ( const auto & key , auto & mapped )
2017-04-01 07:20:54 +00:00
{
2021-03-10 11:00:24 +00:00
method . insertKeyIntoColumns ( key , key_columns , key_sizes_ref ) ;
2015-01-03 05:39:21 +00:00
2017-04-02 17:37:49 +00:00
/// reserved, so push_back does not throw exceptions
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
2019-08-08 16:16:19 +00:00
aggregate_columns [ i ] - > push_back ( mapped + offsets_of_aggregate_states [ i ] ) ;
2015-12-23 07:29:20 +00:00
2019-08-08 16:16:19 +00:00
mapped = nullptr ;
} ) ;
2011-09-19 01:42:16 +00:00
}
2022-07-08 19:24:33 +00:00
Aggregator : : OutputBlockColumns Aggregator : : prepareOutputBlockColumns ( Arenas & aggregates_pools , bool final , size_t rows ) const
2012-02-27 06:28:20 +00:00
{
2017-12-15 03:47:43 +00:00
MutableColumns key_columns ( params . keys_size ) ;
MutableColumns aggregate_columns ( params . aggregates_size ) ;
MutableColumns final_aggregate_columns ( params . aggregates_size ) ;
AggregateColumnsData aggregate_columns_data ( params . aggregates_size ) ;
2017-04-01 07:20:54 +00:00
2022-06-27 11:16:52 +00:00
Block res_header = getHeader ( final ) ;
2018-01-06 18:10:44 +00:00
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . keys_size ; + + i )
{
2022-06-27 11:16:52 +00:00
key_columns [ i ] = res_header . safeGetByPosition ( i ) . type - > createColumn ( ) ;
2017-04-01 07:20:54 +00:00
key_columns [ i ] - > reserve ( rows ) ;
}
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
{
if ( ! final )
{
2020-05-02 11:27:25 +00:00
const auto & aggregate_column_name = params . aggregates [ i ] . column_name ;
2022-06-27 11:16:52 +00:00
aggregate_columns [ i ] = res_header . getByName ( aggregate_column_name ) . type - > createColumn ( ) ;
2017-12-15 03:47:43 +00:00
2017-04-02 17:37:49 +00:00
/// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states.
2019-08-21 02:28:04 +00:00
ColumnAggregateFunction & column_aggregate_func = assert_cast < ColumnAggregateFunction & > ( * aggregate_columns [ i ] ) ;
2017-04-01 07:20:54 +00:00
2022-07-08 19:24:33 +00:00
for ( auto & pool : aggregates_pools )
2020-03-08 23:48:08 +00:00
column_aggregate_func . addArena ( pool ) ;
2017-04-01 07:20:54 +00:00
2017-12-15 03:47:43 +00:00
aggregate_columns_data [ i ] = & column_aggregate_func . getData ( ) ;
aggregate_columns_data [ i ] - > reserve ( rows ) ;
2017-04-01 07:20:54 +00:00
}
else
{
2017-12-15 03:47:43 +00:00
final_aggregate_columns [ i ] = aggregate_functions [ i ] - > getReturnType ( ) - > createColumn ( ) ;
final_aggregate_columns [ i ] - > reserve ( rows ) ;
2017-04-01 07:20:54 +00:00
if ( aggregate_functions [ i ] - > isState ( ) )
{
2017-04-02 17:37:49 +00:00
/// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states.
2020-07-05 23:17:46 +00:00
if ( auto * column_aggregate_func = typeid_cast < ColumnAggregateFunction * > ( final_aggregate_columns [ i ] . get ( ) ) )
2022-07-08 19:24:33 +00:00
for ( auto & pool : aggregates_pools )
2020-07-05 23:17:46 +00:00
column_aggregate_func - > addArena ( pool ) ;
2017-04-01 07:20:54 +00:00
2020-07-05 23:17:46 +00:00
/// Aggregate state can be wrapped into array if aggregate function ends with -Resample combinator.
2022-07-08 19:24:33 +00:00
final_aggregate_columns [ i ] - > forEachSubcolumn (
[ & aggregates_pools ] ( auto & subcolumn )
{
if ( auto * column_aggregate_func = typeid_cast < ColumnAggregateFunction * > ( subcolumn . get ( ) ) )
for ( auto & pool : aggregates_pools )
column_aggregate_func - > addArena ( pool ) ;
} ) ;
2017-04-01 07:20:54 +00:00
}
}
}
2022-07-08 19:24:33 +00:00
return {
. key_columns = std : : move ( key_columns ) ,
. aggregate_columns = std : : move ( aggregate_columns ) ,
. final_aggregate_columns = std : : move ( final_aggregate_columns ) ,
. aggregate_columns_data = std : : move ( aggregate_columns_data ) ,
} ;
}
template < typename Filler >
Block Aggregator : : prepareBlockAndFill (
AggregatedDataVariants & data_variants ,
bool final ,
size_t rows ,
Filler & & filler ) const
{
auto & & out_cols = prepareOutputBlockColumns ( data_variants . aggregates_pools , final , rows ) ;
auto & & [ key_columns , aggregate_columns , final_aggregate_columns , aggregate_columns_data ] = out_cols ;
2020-12-22 11:30:29 +00:00
filler ( key_columns , aggregate_columns_data , final_aggregate_columns , final ) ;
2017-12-15 03:47:43 +00:00
2022-07-08 19:24:33 +00:00
return finalizeBlock ( std : : move ( out_cols ) , final , rows ) ;
}
Block Aggregator : : finalizeBlock ( OutputBlockColumns & & out_cols , bool final , size_t rows ) const
{
auto & & [ key_columns , aggregate_columns , final_aggregate_columns , aggregate_columns_data ] = out_cols ;
Block res_header = getHeader ( final ) ;
2022-06-27 11:16:52 +00:00
Block res = res_header . cloneEmpty ( ) ;
2017-12-15 03:47:43 +00:00
for ( size_t i = 0 ; i < params . keys_size ; + + i )
res . getByPosition ( i ) . column = std : : move ( key_columns [ i ] ) ;
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
2017-12-17 05:21:04 +00:00
{
2020-05-02 11:27:25 +00:00
const auto & aggregate_column_name = params . aggregates [ i ] . column_name ;
2017-12-17 05:21:04 +00:00
if ( final )
2020-05-02 11:27:25 +00:00
res . getByName ( aggregate_column_name ) . column = std : : move ( final_aggregate_columns [ i ] ) ;
2017-12-17 05:21:04 +00:00
else
2020-05-02 11:27:25 +00:00
res . getByName ( aggregate_column_name ) . column = std : : move ( aggregate_columns [ i ] ) ;
2017-12-17 05:21:04 +00:00
}
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// Change the size of the columns-constants in the block.
2022-06-27 11:16:52 +00:00
size_t columns = res_header . columns ( ) ;
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < columns ; + + i )
2019-06-27 19:28:52 +00:00
if ( isColumnConst ( * res . getByPosition ( i ) . column ) )
2017-12-15 03:47:43 +00:00
res . getByPosition ( i ) . column = res . getByPosition ( i ) . column - > cut ( 0 , rows ) ;
2017-04-01 07:20:54 +00:00
return res ;
2015-01-02 03:16:28 +00:00
}
2020-06-17 19:36:27 +00:00
2021-04-01 22:16:54 +00:00
void Aggregator : : addSingleKeyToAggregateColumns (
Fix memory leak in AggregatingInOrderTransform
Reproducer:
# NOTE: we need clickhouse from 33957 since right now LSan is broken due to getauxval().
$ url=https://s3.amazonaws.com/clickhouse-builds/33957/e04b862673644d313712607a0078f5d1c48b5377/package_asan/clickhouse
$ wget $url -o clickhouse-asan
$ chmod +x clickhouse-asan
$ ./clickhouse-asan server &
$ ./clickhouse-asan client
:) create table data (key Int, value String) engine=MergeTree() order by key
:) insert into data select number%5, toString(number) from numbers(10e6)
# usually it is enough one query, benchmark is just for stability of the results
# note, that if the exception was not happen from AggregatingInOrderTransform then add --continue_on_errors and wait
$ ./clickhouse-asan benchmark --query 'select key, uniqCombined64(value), groupArray(value) from data group by key' --optimize_aggregation_in_order=1 --memory_tracker_fault_probability=0.01, max_untracked_memory='2Mi'
LSan report:
==24595==ERROR: LeakSanitizer: detected memory leaks
Direct leak of 3932160 byte(s) in 6 object(s) allocated from:
0 0xcadba93 in realloc ()
1 0xcc108d9 in Allocator<false, false>::realloc() obj-x86_64-linux-gnu/../src/Common/Allocator.h:134:30
2 0xde19eae in void DB::PODArrayBase<>::realloc<DB::Arena*&>(unsigned long, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:161:25
3 0xde5f039 in void DB::PODArrayBase<>::reserveForNextSize<DB::Arena*&>(DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h
4 0xde5f039 in void DB::PODArray<>::push_back<>(DB::GroupArrayNodeString*&, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:432:19
5 0xde5f039 in DB::GroupArrayGeneralImpl<>::add() const obj-x86_64-linux-gnu/../src/AggregateFunctions/AggregateFunctionGroupArray.h:465:31
6 0xde5f039 in DB::IAggregateFunctionHelper<>::addBatchSinglePlaceFromInterval() const obj-x86_64-linux-gnu/../src/AggregateFunctions/IAggregateFunction.h:481:53
7 0x299df134 in DB::Aggregator::executeOnIntervalWithoutKeyImpl() obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:869:31
8 0x2ca75f7d in DB::AggregatingInOrderTransform::consume() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingInOrderTransform.cpp:124:13
...
SUMMARY: AddressSanitizer: 4523184 byte(s) leaked in 12 allocation(s).
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-02-01 15:40:48 +00:00
AggregatedDataVariants & data_variants ,
2021-04-02 12:10:49 +00:00
MutableColumns & aggregate_columns ) const
2020-04-18 09:51:21 +00:00
{
Fix memory leak in AggregatingInOrderTransform
Reproducer:
# NOTE: we need clickhouse from 33957 since right now LSan is broken due to getauxval().
$ url=https://s3.amazonaws.com/clickhouse-builds/33957/e04b862673644d313712607a0078f5d1c48b5377/package_asan/clickhouse
$ wget $url -o clickhouse-asan
$ chmod +x clickhouse-asan
$ ./clickhouse-asan server &
$ ./clickhouse-asan client
:) create table data (key Int, value String) engine=MergeTree() order by key
:) insert into data select number%5, toString(number) from numbers(10e6)
# usually it is enough one query, benchmark is just for stability of the results
# note, that if the exception was not happen from AggregatingInOrderTransform then add --continue_on_errors and wait
$ ./clickhouse-asan benchmark --query 'select key, uniqCombined64(value), groupArray(value) from data group by key' --optimize_aggregation_in_order=1 --memory_tracker_fault_probability=0.01, max_untracked_memory='2Mi'
LSan report:
==24595==ERROR: LeakSanitizer: detected memory leaks
Direct leak of 3932160 byte(s) in 6 object(s) allocated from:
0 0xcadba93 in realloc ()
1 0xcc108d9 in Allocator<false, false>::realloc() obj-x86_64-linux-gnu/../src/Common/Allocator.h:134:30
2 0xde19eae in void DB::PODArrayBase<>::realloc<DB::Arena*&>(unsigned long, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:161:25
3 0xde5f039 in void DB::PODArrayBase<>::reserveForNextSize<DB::Arena*&>(DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h
4 0xde5f039 in void DB::PODArray<>::push_back<>(DB::GroupArrayNodeString*&, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:432:19
5 0xde5f039 in DB::GroupArrayGeneralImpl<>::add() const obj-x86_64-linux-gnu/../src/AggregateFunctions/AggregateFunctionGroupArray.h:465:31
6 0xde5f039 in DB::IAggregateFunctionHelper<>::addBatchSinglePlaceFromInterval() const obj-x86_64-linux-gnu/../src/AggregateFunctions/IAggregateFunction.h:481:53
7 0x299df134 in DB::Aggregator::executeOnIntervalWithoutKeyImpl() obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:869:31
8 0x2ca75f7d in DB::AggregatingInOrderTransform::consume() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingInOrderTransform.cpp:124:13
...
SUMMARY: AddressSanitizer: 4523184 byte(s) leaked in 12 allocation(s).
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-02-01 15:40:48 +00:00
auto & data = data_variants . without_key ;
size_t i = 0 ;
try
2021-03-18 20:17:09 +00:00
{
Fix memory leak in AggregatingInOrderTransform
Reproducer:
# NOTE: we need clickhouse from 33957 since right now LSan is broken due to getauxval().
$ url=https://s3.amazonaws.com/clickhouse-builds/33957/e04b862673644d313712607a0078f5d1c48b5377/package_asan/clickhouse
$ wget $url -o clickhouse-asan
$ chmod +x clickhouse-asan
$ ./clickhouse-asan server &
$ ./clickhouse-asan client
:) create table data (key Int, value String) engine=MergeTree() order by key
:) insert into data select number%5, toString(number) from numbers(10e6)
# usually it is enough one query, benchmark is just for stability of the results
# note, that if the exception was not happen from AggregatingInOrderTransform then add --continue_on_errors and wait
$ ./clickhouse-asan benchmark --query 'select key, uniqCombined64(value), groupArray(value) from data group by key' --optimize_aggregation_in_order=1 --memory_tracker_fault_probability=0.01, max_untracked_memory='2Mi'
LSan report:
==24595==ERROR: LeakSanitizer: detected memory leaks
Direct leak of 3932160 byte(s) in 6 object(s) allocated from:
0 0xcadba93 in realloc ()
1 0xcc108d9 in Allocator<false, false>::realloc() obj-x86_64-linux-gnu/../src/Common/Allocator.h:134:30
2 0xde19eae in void DB::PODArrayBase<>::realloc<DB::Arena*&>(unsigned long, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:161:25
3 0xde5f039 in void DB::PODArrayBase<>::reserveForNextSize<DB::Arena*&>(DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h
4 0xde5f039 in void DB::PODArray<>::push_back<>(DB::GroupArrayNodeString*&, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:432:19
5 0xde5f039 in DB::GroupArrayGeneralImpl<>::add() const obj-x86_64-linux-gnu/../src/AggregateFunctions/AggregateFunctionGroupArray.h:465:31
6 0xde5f039 in DB::IAggregateFunctionHelper<>::addBatchSinglePlaceFromInterval() const obj-x86_64-linux-gnu/../src/AggregateFunctions/IAggregateFunction.h:481:53
7 0x299df134 in DB::Aggregator::executeOnIntervalWithoutKeyImpl() obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:869:31
8 0x2ca75f7d in DB::AggregatingInOrderTransform::consume() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingInOrderTransform.cpp:124:13
...
SUMMARY: AddressSanitizer: 4523184 byte(s) leaked in 12 allocation(s).
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-02-01 15:40:48 +00:00
for ( i = 0 ; i < params . aggregates_size ; + + i )
{
auto & column_aggregate_func = assert_cast < ColumnAggregateFunction & > ( * aggregate_columns [ i ] ) ;
column_aggregate_func . getData ( ) . push_back ( data + offsets_of_aggregate_states [ i ] ) ;
}
2021-03-18 20:17:09 +00:00
}
Fix memory leak in AggregatingInOrderTransform
Reproducer:
# NOTE: we need clickhouse from 33957 since right now LSan is broken due to getauxval().
$ url=https://s3.amazonaws.com/clickhouse-builds/33957/e04b862673644d313712607a0078f5d1c48b5377/package_asan/clickhouse
$ wget $url -o clickhouse-asan
$ chmod +x clickhouse-asan
$ ./clickhouse-asan server &
$ ./clickhouse-asan client
:) create table data (key Int, value String) engine=MergeTree() order by key
:) insert into data select number%5, toString(number) from numbers(10e6)
# usually it is enough one query, benchmark is just for stability of the results
# note, that if the exception was not happen from AggregatingInOrderTransform then add --continue_on_errors and wait
$ ./clickhouse-asan benchmark --query 'select key, uniqCombined64(value), groupArray(value) from data group by key' --optimize_aggregation_in_order=1 --memory_tracker_fault_probability=0.01, max_untracked_memory='2Mi'
LSan report:
==24595==ERROR: LeakSanitizer: detected memory leaks
Direct leak of 3932160 byte(s) in 6 object(s) allocated from:
0 0xcadba93 in realloc ()
1 0xcc108d9 in Allocator<false, false>::realloc() obj-x86_64-linux-gnu/../src/Common/Allocator.h:134:30
2 0xde19eae in void DB::PODArrayBase<>::realloc<DB::Arena*&>(unsigned long, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:161:25
3 0xde5f039 in void DB::PODArrayBase<>::reserveForNextSize<DB::Arena*&>(DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h
4 0xde5f039 in void DB::PODArray<>::push_back<>(DB::GroupArrayNodeString*&, DB::Arena*&) obj-x86_64-linux-gnu/../src/Common/PODArray.h:432:19
5 0xde5f039 in DB::GroupArrayGeneralImpl<>::add() const obj-x86_64-linux-gnu/../src/AggregateFunctions/AggregateFunctionGroupArray.h:465:31
6 0xde5f039 in DB::IAggregateFunctionHelper<>::addBatchSinglePlaceFromInterval() const obj-x86_64-linux-gnu/../src/AggregateFunctions/IAggregateFunction.h:481:53
7 0x299df134 in DB::Aggregator::executeOnIntervalWithoutKeyImpl() obj-x86_64-linux-gnu/../src/Interpreters/Aggregator.cpp:869:31
8 0x2ca75f7d in DB::AggregatingInOrderTransform::consume() obj-x86_64-linux-gnu/../src/Processors/Transforms/AggregatingInOrderTransform.cpp:124:13
...
SUMMARY: AddressSanitizer: 4523184 byte(s) leaked in 12 allocation(s).
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-02-01 15:40:48 +00:00
catch ( . . . )
{
/// Rollback
for ( size_t rollback_i = 0 ; rollback_i < i ; + + rollback_i )
{
auto & column_aggregate_func = assert_cast < ColumnAggregateFunction & > ( * aggregate_columns [ rollback_i ] ) ;
column_aggregate_func . getData ( ) . pop_back ( ) ;
}
throw ;
}
data = nullptr ;
2021-03-18 20:17:09 +00:00
}
2014-05-28 14:54:42 +00:00
2021-03-22 22:12:14 +00:00
void Aggregator : : addArenasToAggregateColumns (
const AggregatedDataVariants & data_variants ,
2021-04-02 12:10:49 +00:00
MutableColumns & aggregate_columns ) const
2021-03-18 20:17:09 +00:00
{
2020-04-18 09:51:21 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
{
2021-03-22 22:12:14 +00:00
auto & column_aggregate_func = assert_cast < ColumnAggregateFunction & > ( * aggregate_columns [ i ] ) ;
for ( const auto & pool : data_variants . aggregates_pools )
2020-05-12 14:50:13 +00:00
column_aggregate_func . addArena ( pool ) ;
2020-04-18 09:51:21 +00:00
}
}
2014-05-28 14:54:42 +00:00
2020-04-18 09:51:21 +00:00
void Aggregator : : createStatesAndFillKeyColumnsWithSingleKey (
AggregatedDataVariants & data_variants ,
2020-05-08 13:28:18 +00:00
Columns & key_columns ,
2020-04-18 09:51:21 +00:00
size_t key_row ,
2021-04-02 12:10:49 +00:00
MutableColumns & final_key_columns ) const
2020-04-18 09:51:21 +00:00
{
AggregateDataPtr place = data_variants . aggregates_pool - > alignedAlloc ( total_size_of_aggregate_states , align_aggregate_states ) ;
createAggregateStates ( place ) ;
data_variants . without_key = place ;
for ( size_t i = 0 ; i < params . keys_size ; + + i )
{
2020-05-07 20:13:51 +00:00
final_key_columns [ i ] - > insertFrom ( * key_columns [ i ] . get ( ) , key_row ) ;
2020-04-18 09:51:21 +00:00
}
}
2014-05-22 18:58:41 +00:00
2017-07-25 12:16:14 +00:00
Block Aggregator : : prepareBlockAndFillWithoutKey ( AggregatedDataVariants & data_variants , bool final , bool is_overflows ) const
2015-01-02 03:16:28 +00:00
{
2017-04-01 07:20:54 +00:00
size_t rows = 1 ;
auto filler = [ & data_variants , this ] (
2017-12-15 03:47:43 +00:00
MutableColumns & key_columns ,
2017-04-01 07:20:54 +00:00
AggregateColumnsData & aggregate_columns ,
2017-12-15 03:47:43 +00:00
MutableColumns & final_aggregate_columns ,
2018-08-27 18:16:32 +00:00
bool final_ )
2017-04-01 07:20:54 +00:00
{
if ( data_variants . type = = AggregatedDataVariants : : Type : : without_key | | params . overflow_row )
{
AggregatedDataWithoutKey & data = data_variants . without_key ;
2021-05-08 09:56:42 +00:00
if ( ! data )
throw Exception ( " Wrong data variant passed. " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-06-08 04:23:56 +00:00
if ( ! final_ )
2017-04-01 07:20:54 +00:00
{
2020-06-08 04:23:56 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
2017-04-01 07:20:54 +00:00
aggregate_columns [ i ] - > push_back ( data + offsets_of_aggregate_states [ i ] ) ;
data = nullptr ;
2020-06-08 04:23:56 +00:00
}
else
{
2020-12-22 11:30:29 +00:00
/// Always single-thread. It's safe to pass current arena from 'aggregates_pool'.
insertAggregatesIntoColumns ( data , final_aggregate_columns , data_variants . aggregates_pool ) ;
2020-06-08 04:23:56 +00:00
}
2017-04-01 07:20:54 +00:00
if ( params . overflow_row )
for ( size_t i = 0 ; i < params . keys_size ; + + i )
key_columns [ i ] - > insertDefault ( ) ;
}
} ;
Block block = prepareBlockAndFill ( data_variants , final , rows , filler ) ;
if ( is_overflows )
block . info . is_overflows = true ;
if ( final )
destroyWithoutKey ( data_variants ) ;
2017-07-25 12:16:14 +00:00
return block ;
2015-01-02 03:16:28 +00:00
}
2014-05-22 18:58:41 +00:00
2017-07-25 12:16:14 +00:00
Block Aggregator : : prepareBlockAndFillSingleLevel ( AggregatedDataVariants & data_variants , bool final ) const
2015-01-02 03:16:28 +00:00
{
2017-04-01 07:20:54 +00:00
size_t rows = data_variants . sizeWithoutOverflowRow ( ) ;
auto filler = [ & data_variants , this ] (
2017-12-15 03:47:43 +00:00
MutableColumns & key_columns ,
2017-04-01 07:20:54 +00:00
AggregateColumnsData & aggregate_columns ,
2017-12-15 03:47:43 +00:00
MutableColumns & final_aggregate_columns ,
2018-08-27 18:16:32 +00:00
bool final_ )
2017-04-01 07:20:54 +00:00
{
# define M(NAME) \
else if ( data_variants . type = = AggregatedDataVariants : : Type : : NAME ) \
convertToBlockImpl ( * data_variants . NAME , data_variants . NAME - > data , \
2020-12-22 11:30:29 +00:00
key_columns , aggregate_columns , final_aggregate_columns , data_variants . aggregates_pool , final_ ) ;
2017-04-01 07:20:54 +00:00
2020-03-08 23:48:08 +00:00
if ( false ) { } // NOLINT
2017-04-01 07:20:54 +00:00
APPLY_FOR_VARIANTS_SINGLE_LEVEL ( M )
# undef M
else
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
} ;
2017-07-25 12:16:14 +00:00
return prepareBlockAndFill ( data_variants , final , rows , filler ) ;
2015-01-02 03:16:28 +00:00
}
2016-08-02 01:46:05 +00:00
BlocksList Aggregator : : prepareBlocksAndFillTwoLevel ( AggregatedDataVariants & data_variants , bool final , ThreadPool * thread_pool ) const
2015-01-02 03:16:28 +00:00
{
# define M(NAME) \
2017-04-01 07:20:54 +00:00
else if ( data_variants . type = = AggregatedDataVariants : : Type : : NAME ) \
return prepareBlocksAndFillTwoLevelImpl ( data_variants , * data_variants . NAME , final , thread_pool ) ;
2015-01-02 03:16:28 +00:00
2020-03-08 23:48:08 +00:00
if ( false ) { } // NOLINT
2017-04-01 07:20:54 +00:00
APPLY_FOR_VARIANTS_TWO_LEVEL ( M )
2015-01-02 03:16:28 +00:00
# undef M
2017-04-01 07:20:54 +00:00
else
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
2015-01-02 03:16:28 +00:00
}
template < typename Method >
BlocksList Aggregator : : prepareBlocksAndFillTwoLevelImpl (
2017-04-01 07:20:54 +00:00
AggregatedDataVariants & data_variants ,
Method & method ,
bool final ,
ThreadPool * thread_pool ) const
2015-01-02 03:16:28 +00:00
{
2020-12-22 18:29:47 +00:00
size_t max_threads = thread_pool ? thread_pool - > getMaxThreads ( ) : 1 ;
2020-12-22 11:30:29 +00:00
if ( max_threads > data_variants . aggregates_pools . size ( ) )
for ( size_t i = data_variants . aggregates_pools . size ( ) ; i < max_threads ; + + i )
data_variants . aggregates_pools . push_back ( std : : make_shared < Arena > ( ) ) ;
2021-03-17 06:33:57 +00:00
std : : atomic < UInt32 > next_bucket_to_merge = 0 ;
auto converter = [ & ] ( size_t thread_id , ThreadGroupStatusPtr thread_group )
2017-04-01 07:20:54 +00:00
{
2018-11-20 17:08:34 +00:00
if ( thread_group )
CurrentThread : : attachToIfDetached ( thread_group ) ;
2020-12-22 11:30:29 +00:00
2021-03-17 06:33:57 +00:00
BlocksList blocks ;
while ( true )
{
UInt32 bucket = next_bucket_to_merge . fetch_add ( 1 ) ;
if ( bucket > = Method : : Data : : NUM_BUCKETS )
break ;
2020-12-22 11:30:29 +00:00
2021-03-17 06:33:57 +00:00
if ( method . data . impls [ bucket ] . empty ( ) )
continue ;
/// Select Arena to avoid race conditions
Arena * arena = data_variants . aggregates_pools . at ( thread_id ) . get ( ) ;
blocks . emplace_back ( convertOneBucketToBlock ( data_variants , method , arena , final , bucket ) ) ;
}
return blocks ;
2017-04-01 07:20:54 +00:00
} ;
2017-04-02 17:37:49 +00:00
/// packaged_task is used to ensure that exceptions are automatically thrown into the main stream.
2017-04-01 07:20:54 +00:00
2021-03-17 06:33:57 +00:00
std : : vector < std : : packaged_task < BlocksList ( ) > > tasks ( max_threads ) ;
2017-04-01 07:20:54 +00:00
try
{
2021-03-17 06:33:57 +00:00
for ( size_t thread_id = 0 ; thread_id < max_threads ; + + thread_id )
2017-04-01 07:20:54 +00:00
{
2021-03-17 06:33:57 +00:00
tasks [ thread_id ] = std : : packaged_task < BlocksList ( ) > (
[ group = CurrentThread : : getGroup ( ) , thread_id , & converter ] { return converter ( thread_id , group ) ; } ) ;
2017-04-01 07:20:54 +00:00
if ( thread_pool )
2021-03-17 06:33:57 +00:00
thread_pool - > scheduleOrThrowOnError ( [ thread_id , & tasks ] { tasks [ thread_id ] ( ) ; } ) ;
2017-04-01 07:20:54 +00:00
else
2021-03-17 06:33:57 +00:00
tasks [ thread_id ] ( ) ;
2017-04-01 07:20:54 +00:00
}
}
catch ( . . . )
{
2017-04-02 17:37:49 +00:00
/// If this is not done, then in case of an exception, tasks will be destroyed before the threads are completed, and it will be bad.
2017-04-01 07:20:54 +00:00
if ( thread_pool )
thread_pool - > wait ( ) ;
throw ;
}
if ( thread_pool )
thread_pool - > wait ( ) ;
BlocksList blocks ;
for ( auto & task : tasks )
{
if ( ! task . valid ( ) )
continue ;
2021-03-17 06:33:57 +00:00
blocks . splice ( blocks . end ( ) , task . get_future ( ) . get ( ) ) ;
2017-04-01 07:20:54 +00:00
}
return blocks ;
2015-01-02 03:16:28 +00:00
}
2015-12-08 02:01:46 +00:00
BlocksList Aggregator : : convertToBlocks ( AggregatedDataVariants & data_variants , bool final , size_t max_threads ) const
2015-01-02 03:16:28 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Converting aggregated data to blocks " ) ;
2015-01-02 03:16:28 +00:00
2017-04-01 07:20:54 +00:00
Stopwatch watch ;
2015-01-02 03:16:28 +00:00
2017-04-01 07:20:54 +00:00
BlocksList blocks ;
2015-01-02 03:16:28 +00:00
2017-04-02 17:37:49 +00:00
/// In what data structure is the data aggregated?
2017-04-01 07:20:54 +00:00
if ( data_variants . empty ( ) )
return blocks ;
2015-01-02 03:16:28 +00:00
2017-04-01 07:20:54 +00:00
std : : unique_ptr < ThreadPool > thread_pool ;
2017-04-02 17:37:49 +00:00
if ( max_threads > 1 & & data_variants . sizeWithoutOverflowRow ( ) > 100000 /// TODO Make a custom threshold.
& & data_variants . isTwoLevel ( ) ) /// TODO Use the shared thread pool with the `merge` function.
2017-04-01 07:20:54 +00:00
thread_pool = std : : make_unique < ThreadPool > ( max_threads ) ;
2015-01-02 03:16:28 +00:00
2017-07-25 12:16:14 +00:00
if ( data_variants . without_key )
blocks . emplace_back ( prepareBlockAndFillWithoutKey (
2017-04-01 07:20:54 +00:00
data_variants , final , data_variants . type ! = AggregatedDataVariants : : Type : : without_key ) ) ;
2015-01-02 03:16:28 +00:00
2017-04-01 07:20:54 +00:00
if ( data_variants . type ! = AggregatedDataVariants : : Type : : without_key )
{
if ( ! data_variants . isTwoLevel ( ) )
2017-07-25 12:16:14 +00:00
blocks . emplace_back ( prepareBlockAndFillSingleLevel ( data_variants , final ) ) ;
2017-04-01 07:20:54 +00:00
else
blocks . splice ( blocks . end ( ) , prepareBlocksAndFillTwoLevel ( data_variants , final , thread_pool . get ( ) ) ) ;
}
2014-02-26 11:44:54 +00:00
2017-04-01 07:20:54 +00:00
if ( ! final )
{
2017-04-02 17:37:49 +00:00
/// data_variants will not destroy the states of aggregate functions in the destructor.
/// Now ColumnAggregateFunction owns the states.
2017-04-01 07:20:54 +00:00
data_variants . aggregator = nullptr ;
}
2013-02-16 18:59:05 +00:00
2017-04-01 07:20:54 +00:00
size_t rows = 0 ;
size_t bytes = 0 ;
2015-01-02 03:16:28 +00:00
2017-04-01 07:20:54 +00:00
for ( const auto & block : blocks )
{
rows + = block . rows ( ) ;
bytes + = block . bytes ( ) ;
}
2012-02-27 06:28:20 +00:00
2017-04-01 07:20:54 +00:00
double elapsed_seconds = watch . elapsedSeconds ( ) ;
2021-04-15 18:00:16 +00:00
LOG_DEBUG ( log ,
2021-05-08 10:27:48 +00:00
" Converted aggregated data to blocks. {} rows, {} in {} sec. ({:.3f} rows/sec., {}/sec.) " ,
2020-05-30 21:35:52 +00:00
rows , ReadableSize ( bytes ) ,
2020-05-23 21:26:45 +00:00
elapsed_seconds , rows / elapsed_seconds ,
2020-05-30 21:35:52 +00:00
ReadableSize ( bytes / elapsed_seconds ) ) ;
2012-05-31 00:33:42 +00:00
2017-04-01 07:20:54 +00:00
return blocks ;
2012-02-27 06:28:20 +00:00
}
2018-12-04 10:31:15 +00:00
template < typename Method , typename Table >
2018-12-04 10:27:44 +00:00
void NO_INLINE Aggregator : : mergeDataNullKey (
Table & table_dst ,
Table & table_src ,
Arena * arena ) const
{
if constexpr ( Method : : low_cardinality_optimization )
{
if ( table_src . hasNullKeyData ( ) )
{
if ( ! table_dst . hasNullKeyData ( ) )
{
table_dst . hasNullKeyData ( ) = true ;
table_dst . getNullKeyData ( ) = table_src . getNullKeyData ( ) ;
}
else
{
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
aggregate_functions [ i ] - > merge (
table_dst . getNullKeyData ( ) + offsets_of_aggregate_states [ i ] ,
table_src . getNullKeyData ( ) + offsets_of_aggregate_states [ i ] ,
arena ) ;
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
aggregate_functions [ i ] - > destroy (
table_src . getNullKeyData ( ) + offsets_of_aggregate_states [ i ] ) ;
}
table_src . hasNullKeyData ( ) = false ;
table_src . getNullKeyData ( ) = nullptr ;
}
}
}
2021-06-13 12:38:57 +00:00
template < typename Method , bool use_compiled_functions , typename Table >
2015-01-03 05:39:21 +00:00
void NO_INLINE Aggregator : : mergeDataImpl (
2017-04-01 07:20:54 +00:00
Table & table_dst ,
Table & table_src ,
Arena * arena ) const
2015-01-03 05:39:21 +00:00
{
2018-12-04 10:27:44 +00:00
if constexpr ( Method : : low_cardinality_optimization )
2018-12-04 10:31:15 +00:00
mergeDataNullKey < Method , Table > ( table_dst , table_src , arena ) ;
2018-12-04 10:27:44 +00:00
2021-06-13 12:38:57 +00:00
table_src . mergeToViaEmplace ( table_dst , [ & ] ( AggregateDataPtr & __restrict dst , AggregateDataPtr & __restrict src , bool inserted )
2017-04-01 07:20:54 +00:00
{
if ( ! inserted )
{
2021-06-13 12:38:57 +00:00
# if USE_EMBEDDED_COMPILER
if constexpr ( use_compiled_functions )
2021-06-03 19:20:53 +00:00
{
2021-06-13 12:38:57 +00:00
const auto & compiled_functions = compiled_aggregate_functions_holder - > compiled_aggregate_functions ;
compiled_functions . merge_aggregate_states_function ( dst , src ) ;
2017-04-01 07:20:54 +00:00
2021-06-27 17:43:47 +00:00
if ( compiled_aggregate_functions_holder - > compiled_aggregate_functions . functions_count ! = params . aggregates_size )
{
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
{
if ( ! is_aggregate_function_compiled [ i ] )
aggregate_functions [ i ] - > merge ( dst + offsets_of_aggregate_states [ i ] , src + offsets_of_aggregate_states [ i ] , arena ) ;
}
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
{
if ( ! is_aggregate_function_compiled [ i ] )
aggregate_functions [ i ] - > destroy ( src + offsets_of_aggregate_states [ i ] ) ;
}
}
2021-06-03 19:20:53 +00:00
}
2021-06-27 17:43:47 +00:00
else
2021-06-04 10:43:11 +00:00
# endif
2021-06-27 17:43:47 +00:00
{
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
aggregate_functions [ i ] - > merge ( dst + offsets_of_aggregate_states [ i ] , src + offsets_of_aggregate_states [ i ] , arena ) ;
2021-06-03 19:20:53 +00:00
2021-06-27 17:43:47 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
aggregate_functions [ i ] - > destroy ( src + offsets_of_aggregate_states [ i ] ) ;
}
2017-04-01 07:20:54 +00:00
}
else
{
2019-08-08 16:16:19 +00:00
dst = src ;
2017-04-01 07:20:54 +00:00
}
2019-08-08 16:16:19 +00:00
src = nullptr ;
} ) ;
2017-04-01 07:20:54 +00:00
table_src . clearAndShrink ( ) ;
2015-01-03 05:39:21 +00:00
}
2015-10-22 01:44:33 +00:00
template < typename Method , typename Table >
void NO_INLINE Aggregator : : mergeDataNoMoreKeysImpl (
2017-04-01 07:20:54 +00:00
Table & table_dst ,
AggregatedDataWithoutKey & overflows ,
Table & table_src ,
Arena * arena ) const
2015-10-22 01:44:33 +00:00
{
2018-12-04 10:27:44 +00:00
/// Note : will create data for NULL key if not exist
if constexpr ( Method : : low_cardinality_optimization )
2018-12-04 10:31:15 +00:00
mergeDataNullKey < Method , Table > ( table_dst , table_src , arena ) ;
2018-12-04 10:27:44 +00:00
2019-08-08 16:16:19 +00:00
table_src . mergeToViaFind ( table_dst , [ & ] ( AggregateDataPtr dst , AggregateDataPtr & src , bool found )
2017-04-01 07:20:54 +00:00
{
2019-08-08 16:16:19 +00:00
AggregateDataPtr res_data = found ? dst : overflows ;
2015-10-22 01:44:33 +00:00
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
aggregate_functions [ i ] - > merge (
res_data + offsets_of_aggregate_states [ i ] ,
2019-08-08 16:16:19 +00:00
src + offsets_of_aggregate_states [ i ] ,
2017-04-01 07:20:54 +00:00
arena ) ;
2015-10-22 01:44:33 +00:00
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
2019-08-08 16:16:19 +00:00
aggregate_functions [ i ] - > destroy ( src + offsets_of_aggregate_states [ i ] ) ;
2015-12-23 07:06:34 +00:00
2019-08-08 16:16:19 +00:00
src = nullptr ;
} ) ;
2017-04-01 07:20:54 +00:00
table_src . clearAndShrink ( ) ;
2015-10-22 01:44:33 +00:00
}
template < typename Method , typename Table >
void NO_INLINE Aggregator : : mergeDataOnlyExistingKeysImpl (
2017-04-01 07:20:54 +00:00
Table & table_dst ,
Table & table_src ,
Arena * arena ) const
2015-10-22 01:44:33 +00:00
{
2018-12-04 10:27:44 +00:00
/// Note : will create data for NULL key if not exist
if constexpr ( Method : : low_cardinality_optimization )
2018-12-04 10:31:15 +00:00
mergeDataNullKey < Method , Table > ( table_dst , table_src , arena ) ;
2018-12-04 10:27:44 +00:00
2019-08-08 16:16:19 +00:00
table_src . mergeToViaFind ( table_dst ,
[ & ] ( AggregateDataPtr dst , AggregateDataPtr & src , bool found )
2017-04-01 07:20:54 +00:00
{
2019-08-08 16:16:19 +00:00
if ( ! found )
return ;
2015-10-22 01:44:33 +00:00
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
aggregate_functions [ i ] - > merge (
2019-08-08 16:16:19 +00:00
dst + offsets_of_aggregate_states [ i ] ,
src + offsets_of_aggregate_states [ i ] ,
2017-04-01 07:20:54 +00:00
arena ) ;
2015-10-22 01:44:33 +00:00
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
2019-08-08 16:16:19 +00:00
aggregate_functions [ i ] - > destroy ( src + offsets_of_aggregate_states [ i ] ) ;
2015-12-23 07:06:34 +00:00
2019-08-08 16:16:19 +00:00
src = nullptr ;
} ) ;
2017-04-01 07:20:54 +00:00
table_src . clearAndShrink ( ) ;
2015-10-22 01:44:33 +00:00
}
2015-01-03 05:39:21 +00:00
void NO_INLINE Aggregator : : mergeWithoutKeyDataImpl (
2017-04-01 07:20:54 +00:00
ManyAggregatedDataVariants & non_empty_data ) const
2015-01-03 05:39:21 +00:00
{
2017-04-01 07:20:54 +00:00
AggregatedDataVariantsPtr & res = non_empty_data [ 0 ] ;
2015-01-03 05:39:21 +00:00
2018-08-27 18:16:32 +00:00
/// We merge all aggregation results to the first.
for ( size_t result_num = 1 , size = non_empty_data . size ( ) ; result_num < size ; + + result_num )
2017-04-01 07:20:54 +00:00
{
AggregatedDataWithoutKey & res_data = res - > without_key ;
2018-08-27 18:16:32 +00:00
AggregatedDataWithoutKey & current_data = non_empty_data [ result_num ] - > without_key ;
2015-01-03 05:39:21 +00:00
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
aggregate_functions [ i ] - > merge ( res_data + offsets_of_aggregate_states [ i ] , current_data + offsets_of_aggregate_states [ i ] , res - > aggregates_pool ) ;
2015-01-03 05:39:21 +00:00
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
aggregate_functions [ i ] - > destroy ( current_data + offsets_of_aggregate_states [ i ] ) ;
2015-01-03 05:39:21 +00:00
2017-04-01 07:20:54 +00:00
current_data = nullptr ;
}
2015-01-03 05:39:21 +00:00
}
template < typename Method >
void NO_INLINE Aggregator : : mergeSingleLevelDataImpl (
2017-04-01 07:20:54 +00:00
ManyAggregatedDataVariants & non_empty_data ) const
2015-01-03 05:39:21 +00:00
{
2017-04-01 07:20:54 +00:00
AggregatedDataVariantsPtr & res = non_empty_data [ 0 ] ;
bool no_more_keys = false ;
2018-08-27 18:16:32 +00:00
/// We merge all aggregation results to the first.
for ( size_t result_num = 1 , size = non_empty_data . size ( ) ; result_num < size ; + + result_num )
2017-04-01 07:20:54 +00:00
{
if ( ! checkLimits ( res - > sizeWithoutOverflowRow ( ) , no_more_keys ) )
break ;
2018-08-27 18:16:32 +00:00
AggregatedDataVariants & current = * non_empty_data [ result_num ] ;
2017-04-01 07:20:54 +00:00
if ( ! no_more_keys )
2021-06-13 12:38:57 +00:00
{
# if USE_EMBEDDED_COMPILER
if ( compiled_aggregate_functions_holder )
{
mergeDataImpl < Method , true > (
getDataVariant < Method > ( * res ) . data ,
getDataVariant < Method > ( current ) . data ,
res - > aggregates_pool ) ;
}
else
# endif
{
mergeDataImpl < Method , false > (
getDataVariant < Method > ( * res ) . data ,
getDataVariant < Method > ( current ) . data ,
res - > aggregates_pool ) ;
}
}
2017-04-01 07:20:54 +00:00
else if ( res - > without_key )
2021-06-13 12:38:57 +00:00
{
2017-04-01 07:20:54 +00:00
mergeDataNoMoreKeysImpl < Method > (
getDataVariant < Method > ( * res ) . data ,
res - > without_key ,
getDataVariant < Method > ( current ) . data ,
res - > aggregates_pool ) ;
2021-06-13 12:38:57 +00:00
}
2017-04-01 07:20:54 +00:00
else
2021-06-13 12:38:57 +00:00
{
2017-04-01 07:20:54 +00:00
mergeDataOnlyExistingKeysImpl < Method > (
getDataVariant < Method > ( * res ) . data ,
getDataVariant < Method > ( current ) . data ,
res - > aggregates_pool ) ;
2021-06-13 12:38:57 +00:00
}
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// `current` will not destroy the states of aggregate functions in the destructor
2017-04-01 07:20:54 +00:00
current . aggregator = nullptr ;
}
2015-01-03 05:39:21 +00:00
}
2020-05-08 20:55:08 +00:00
# define M(NAME) \
template void NO_INLINE Aggregator : : mergeSingleLevelDataImpl < decltype ( AggregatedDataVariants : : NAME ) : : element_type > ( \
ManyAggregatedDataVariants & non_empty_data ) const ;
APPLY_FOR_VARIANTS_SINGLE_LEVEL ( M )
# undef M
2015-01-03 05:39:21 +00:00
2015-12-06 16:22:01 +00:00
template < typename Method >
void NO_INLINE Aggregator : : mergeBucketImpl (
2020-01-10 20:24:59 +00:00
ManyAggregatedDataVariants & data , Int32 bucket , Arena * arena , std : : atomic < bool > * is_cancelled ) const
2015-12-06 16:22:01 +00:00
{
2018-08-27 18:16:32 +00:00
/// We merge all aggregation results to the first.
2017-04-01 07:20:54 +00:00
AggregatedDataVariantsPtr & res = data [ 0 ] ;
2018-08-27 18:16:32 +00:00
for ( size_t result_num = 1 , size = data . size ( ) ; result_num < size ; + + result_num )
2017-04-01 07:20:54 +00:00
{
2020-01-10 20:24:59 +00:00
if ( is_cancelled & & is_cancelled - > load ( std : : memory_order_seq_cst ) )
return ;
2018-08-27 18:16:32 +00:00
AggregatedDataVariants & current = * data [ result_num ] ;
2021-06-13 12:38:57 +00:00
# if USE_EMBEDDED_COMPILER
if ( compiled_aggregate_functions_holder )
{
mergeDataImpl < Method , true > (
getDataVariant < Method > ( * res ) . data . impls [ bucket ] ,
getDataVariant < Method > ( current ) . data . impls [ bucket ] ,
arena ) ;
}
else
# endif
{
mergeDataImpl < Method , false > (
getDataVariant < Method > ( * res ) . data . impls [ bucket ] ,
getDataVariant < Method > ( current ) . data . impls [ bucket ] ,
arena ) ;
}
2017-04-01 07:20:54 +00:00
}
2015-12-06 16:22:01 +00:00
}
2019-09-06 12:19:59 +00:00
ManyAggregatedDataVariants Aggregator : : prepareVariantsToMerge ( ManyAggregatedDataVariants & data_variants ) const
2015-12-06 16:22:01 +00:00
{
2017-04-01 07:20:54 +00:00
if ( data_variants . empty ( ) )
throw Exception ( " Empty data passed to Aggregator::mergeAndConvertToBlocks. " , ErrorCodes : : EMPTY_DATA_PASSED ) ;
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Merging aggregated data " ) ;
2017-04-01 07:20:54 +00:00
2022-03-30 20:47:51 +00:00
if ( params . stats_collecting_params . isCollectionAndUseEnabled ( ) )
updateStatistics ( data_variants , params . stats_collecting_params ) ;
2017-04-01 07:20:54 +00:00
ManyAggregatedDataVariants non_empty_data ;
non_empty_data . reserve ( data_variants . size ( ) ) ;
for ( auto & data : data_variants )
if ( ! data - > empty ( ) )
non_empty_data . push_back ( data ) ;
if ( non_empty_data . empty ( ) )
2019-09-06 12:19:59 +00:00
return { } ;
2017-04-01 07:20:54 +00:00
if ( non_empty_data . size ( ) > 1 )
{
2017-04-02 17:37:49 +00:00
/// Sort the states in descending order so that the merge is more efficient (since all states are merged into the first).
2022-01-30 19:49:48 +00:00
: : sort ( non_empty_data . begin ( ) , non_empty_data . end ( ) ,
2017-04-01 07:20:54 +00:00
[ ] ( const AggregatedDataVariantsPtr & lhs , const AggregatedDataVariantsPtr & rhs )
{
return lhs - > sizeWithoutOverflowRow ( ) > rhs - > sizeWithoutOverflowRow ( ) ;
} ) ;
}
2017-04-02 17:37:49 +00:00
/// If at least one of the options is two-level, then convert all the options into two-level ones, if there are not such.
/// Note - perhaps it would be more optimal not to convert single-level versions before the merge, but merge them separately, at the end.
2017-04-01 07:20:54 +00:00
bool has_at_least_one_two_level = false ;
for ( const auto & variant : non_empty_data )
{
if ( variant - > isTwoLevel ( ) )
{
has_at_least_one_two_level = true ;
break ;
}
}
if ( has_at_least_one_two_level )
for ( auto & variant : non_empty_data )
if ( ! variant - > isTwoLevel ( ) )
2018-09-12 13:27:00 +00:00
variant - > convertToTwoLevel ( ) ;
2018-08-24 15:45:17 +00:00
2017-04-01 07:20:54 +00:00
AggregatedDataVariantsPtr & first = non_empty_data [ 0 ] ;
for ( size_t i = 1 , size = non_empty_data . size ( ) ; i < size ; + + i )
{
if ( first - > type ! = non_empty_data [ i ] - > type )
throw Exception ( " Cannot merge different aggregated data variants. " , ErrorCodes : : CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS ) ;
2017-04-02 17:37:49 +00:00
/** Elements from the remaining sets can be moved to the first data set.
* Therefore , it must own all the arenas of all other sets .
2017-04-01 07:20:54 +00:00
*/
first - > aggregates_pools . insert ( first - > aggregates_pools . end ( ) ,
non_empty_data [ i ] - > aggregates_pools . begin ( ) , non_empty_data [ i ] - > aggregates_pools . end ( ) ) ;
}
2019-09-06 12:19:59 +00:00
return non_empty_data ;
}
2015-10-23 01:43:42 +00:00
template < bool no_more_keys , typename Method , typename Table >
void NO_INLINE Aggregator : : mergeStreamsImplCase (
2017-04-01 07:20:54 +00:00
Arena * aggregates_pool ,
2019-01-21 10:39:24 +00:00
Method & method [[maybe_unused]] ,
2017-04-01 07:20:54 +00:00
Table & data ,
2022-05-19 07:34:41 +00:00
AggregateDataPtr overflow_row ,
size_t row_begin ,
size_t row_end ,
const AggregateColumnsConstData & aggregate_columns_data ,
2022-08-03 15:56:59 +00:00
const ColumnRawPtrs & key_columns ,
Arena * arena_for_keys ) const
2015-01-03 05:39:21 +00:00
{
2019-01-21 10:39:24 +00:00
typename Method : : State state ( key_columns , key_sizes , aggregation_state_cache ) ;
2017-04-01 07:20:54 +00:00
2022-05-19 07:34:41 +00:00
std : : unique_ptr < AggregateDataPtr [ ] > places ( new AggregateDataPtr [ row_end ] ) ;
2021-04-27 09:01:58 +00:00
2022-08-03 15:56:59 +00:00
if ( ! arena_for_keys )
arena_for_keys = aggregates_pool ;
2022-05-19 07:34:41 +00:00
for ( size_t i = row_begin ; i < row_end ; + + i )
2017-04-01 07:20:54 +00:00
{
2019-01-21 10:39:53 +00:00
AggregateDataPtr aggregate_data = nullptr ;
2017-04-01 07:20:54 +00:00
if ( ! no_more_keys )
2019-01-21 10:39:53 +00:00
{
2022-08-03 15:56:59 +00:00
auto emplace_result = state . emplaceKey ( data , i , * arena_for_keys ) ; // NOLINT
2019-01-21 10:39:53 +00:00
if ( emplace_result . isInserted ( ) )
{
emplace_result . setMapped ( nullptr ) ;
aggregate_data = aggregates_pool - > alignedAlloc ( total_size_of_aggregate_states , align_aggregate_states ) ;
createAggregateStates ( aggregate_data ) ;
emplace_result . setMapped ( aggregate_data ) ;
}
else
aggregate_data = emplace_result . getMapped ( ) ;
}
2017-04-01 07:20:54 +00:00
else
{
2022-08-03 15:56:59 +00:00
auto find_result = state . findKey ( data , i , * arena_for_keys ) ;
2019-01-21 10:39:53 +00:00
if ( find_result . isFound ( ) )
aggregate_data = find_result . getMapped ( ) ;
2017-04-01 07:20:54 +00:00
}
2018-08-23 13:22:03 +00:00
/// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys.
2019-01-21 10:39:53 +00:00
AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row ;
2021-04-27 09:01:58 +00:00
places [ i ] = value ;
}
2017-04-01 07:20:54 +00:00
2021-04-27 09:01:58 +00:00
for ( size_t j = 0 ; j < params . aggregates_size ; + + j )
{
2017-04-02 17:37:49 +00:00
/// Merge state of aggregate functions.
2021-04-27 09:01:58 +00:00
aggregate_functions [ j ] - > mergeBatch (
2022-05-19 07:34:41 +00:00
row_begin , row_end ,
2022-03-07 13:48:38 +00:00
places . get ( ) , offsets_of_aggregate_states [ j ] ,
2022-05-19 07:34:41 +00:00
aggregate_columns_data [ j ] - > data ( ) ,
2021-04-27 09:01:58 +00:00
aggregates_pool ) ;
2017-04-01 07:20:54 +00:00
}
2022-05-19 07:34:41 +00:00
}
template < typename Method , typename Table >
void NO_INLINE Aggregator : : mergeStreamsImpl (
2022-06-15 20:45:38 +00:00
Block block ,
2022-05-19 07:34:41 +00:00
Arena * aggregates_pool ,
Method & method ,
Table & data ,
AggregateDataPtr overflow_row ,
2022-08-03 15:56:59 +00:00
bool no_more_keys ,
Arena * arena_for_keys ) const
2022-05-19 07:34:41 +00:00
{
const AggregateColumnsConstData & aggregate_columns_data = params . makeAggregateColumnsData ( block ) ;
const ColumnRawPtrs & key_columns = params . makeRawKeyColumns ( block ) ;
mergeStreamsImpl < Method , Table > (
2022-08-03 15:56:59 +00:00
aggregates_pool , method , data , overflow_row , no_more_keys , 0 , block . rows ( ) , aggregate_columns_data , key_columns , arena_for_keys ) ;
2015-01-03 05:39:21 +00:00
}
2015-10-23 01:43:42 +00:00
template < typename Method , typename Table >
void NO_INLINE Aggregator : : mergeStreamsImpl (
2017-04-01 07:20:54 +00:00
Arena * aggregates_pool ,
Method & method ,
Table & data ,
AggregateDataPtr overflow_row ,
2022-05-19 07:34:41 +00:00
bool no_more_keys ,
size_t row_begin ,
size_t row_end ,
const AggregateColumnsConstData & aggregate_columns_data ,
2022-08-03 15:56:59 +00:00
const ColumnRawPtrs & key_columns ,
Arena * arena_for_keys ) const
2015-10-23 01:43:42 +00:00
{
2017-04-01 07:20:54 +00:00
if ( ! no_more_keys )
2022-08-03 15:56:59 +00:00
mergeStreamsImplCase < false > (
aggregates_pool , method , data , overflow_row , row_begin , row_end , aggregate_columns_data , key_columns , arena_for_keys ) ;
2017-04-01 07:20:54 +00:00
else
2022-08-03 15:56:59 +00:00
mergeStreamsImplCase < true > (
aggregates_pool , method , data , overflow_row , row_begin , row_end , aggregate_columns_data , key_columns , arena_for_keys ) ;
2015-10-23 01:43:42 +00:00
}
2022-05-19 07:34:41 +00:00
void NO_INLINE Aggregator : : mergeBlockWithoutKeyStreamsImpl (
2022-06-15 20:45:38 +00:00
Block block ,
2017-04-01 07:20:54 +00:00
AggregatedDataVariants & result ) const
2015-01-03 05:39:21 +00:00
{
2022-05-19 07:34:41 +00:00
AggregateColumnsConstData aggregate_columns = params . makeAggregateColumnsData ( block ) ;
mergeWithoutKeyStreamsImpl ( result , 0 , block . rows ( ) , aggregate_columns ) ;
}
void NO_INLINE Aggregator : : mergeWithoutKeyStreamsImpl (
AggregatedDataVariants & result ,
size_t row_begin ,
size_t row_end ,
const AggregateColumnsConstData & aggregate_columns_data ) const
{
2017-04-01 07:20:54 +00:00
AggregatedDataWithoutKey & res = result . without_key ;
if ( ! res )
{
2018-09-01 03:17:43 +00:00
AggregateDataPtr place = result . aggregates_pool - > alignedAlloc ( total_size_of_aggregate_states , align_aggregate_states ) ;
2017-04-01 07:20:54 +00:00
createAggregateStates ( place ) ;
res = place ;
}
2022-05-19 07:34:41 +00:00
for ( size_t row = row_begin ; row < row_end ; + + row )
2021-05-14 14:26:09 +00:00
{
/// Adding Values
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
2022-05-19 07:34:41 +00:00
aggregate_functions [ i ] - > merge ( res + offsets_of_aggregate_states [ i ] , ( * aggregate_columns_data [ i ] ) [ row ] , result . aggregates_pool ) ;
2021-05-14 14:26:09 +00:00
}
2015-01-03 05:39:21 +00:00
}
2022-05-19 07:34:41 +00:00
2021-09-08 15:30:48 +00:00
bool Aggregator : : mergeOnBlock ( Block block , AggregatedDataVariants & result , bool & no_more_keys ) const
2021-04-21 16:00:27 +00:00
{
/// `result` will destroy the states of aggregate functions in the destructor
result . aggregator = this ;
/// How to perform the aggregation?
if ( result . empty ( ) )
{
result . init ( method_chosen ) ;
result . keys_size = params . keys_size ;
result . key_sizes = key_sizes ;
LOG_TRACE ( log , " Aggregation method: {} " , result . getMethodName ( ) ) ;
}
if ( result . type = = AggregatedDataVariants : : Type : : without_key | | block . info . is_overflows )
2022-06-15 20:45:38 +00:00
mergeBlockWithoutKeyStreamsImpl ( std : : move ( block ) , result ) ;
2021-04-21 16:00:27 +00:00
# define M(NAME, IS_TWO_LEVEL) \
else if ( result . type = = AggregatedDataVariants : : Type : : NAME ) \
2022-06-15 20:45:38 +00:00
mergeStreamsImpl ( std : : move ( block ) , result . aggregates_pool , * result . NAME , result . NAME - > data , result . without_key , no_more_keys ) ;
2021-04-21 16:00:27 +00:00
APPLY_FOR_AGGREGATED_VARIANTS ( M )
# undef M
else if ( result . type ! = AggregatedDataVariants : : Type : : without_key )
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
size_t result_size = result . sizeWithoutOverflowRow ( ) ;
Int64 current_memory_usage = 0 ;
if ( auto * memory_tracker_child = CurrentThread : : getMemoryTracker ( ) )
if ( auto * memory_tracker = memory_tracker_child - > getParent ( ) )
current_memory_usage = memory_tracker - > get ( ) ;
/// Here all the results in the sum are taken into account, from different threads.
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation ;
2022-03-30 20:47:51 +00:00
bool worth_convert_to_two_level = worthConvertToTwoLevel (
params . group_by_two_level_threshold , result_size , params . group_by_two_level_threshold_bytes , result_size_bytes ) ;
2021-04-21 16:00:27 +00:00
/** Converting to a two-level data structure.
* It allows you to make , in the subsequent , an effective merge - either economical from memory or parallel .
*/
if ( result . isConvertibleToTwoLevel ( ) & & worth_convert_to_two_level )
result . convertToTwoLevel ( ) ;
/// Checking the constraints.
if ( ! checkLimits ( result_size , no_more_keys ) )
return false ;
/** Flush data to disk if too much RAM is consumed.
* Data can only be flushed to disk if a two - level aggregation structure is used .
*/
if ( params . max_bytes_before_external_group_by
& & result . isTwoLevel ( )
& & current_memory_usage > static_cast < Int64 > ( params . max_bytes_before_external_group_by )
& & worth_convert_to_two_level )
{
size_t size = current_memory_usage + params . min_free_disk_space ;
std : : string tmp_path = params . tmp_volume - > getDisk ( ) - > getPath ( ) ;
// enoughSpaceInDirectory() is not enough to make it right, since
// another process (or another thread of aggregator) can consume all
// space.
//
// But true reservation (IVolume::reserve()) cannot be used here since
// current_memory_usage does not takes compression into account and
// will reserve way more that actually will be used.
//
// Hence let's do a simple check.
if ( ! enoughSpaceInDirectory ( tmp_path , size ) )
throw Exception ( " Not enough space for external aggregation in " + tmp_path , ErrorCodes : : NOT_ENOUGH_SPACE ) ;
writeToTemporaryFile ( result , tmp_path ) ;
}
return true ;
}
2015-01-03 05:39:21 +00:00
2019-03-04 16:06:28 +00:00
void Aggregator : : mergeBlocks ( BucketToBlocks bucket_to_blocks , AggregatedDataVariants & result , size_t max_threads )
{
2017-04-01 07:20:54 +00:00
if ( bucket_to_blocks . empty ( ) )
return ;
2015-01-03 05:39:21 +00:00
2019-03-15 17:06:32 +00:00
UInt64 total_input_rows = 0 ;
2019-03-04 16:06:28 +00:00
for ( auto & bucket : bucket_to_blocks )
for ( auto & block : bucket . second )
2019-03-15 17:06:32 +00:00
total_input_rows + = block . rows ( ) ;
2019-03-04 16:06:28 +00:00
2017-04-02 17:37:49 +00:00
/** `minus one` means the absence of information about the bucket
* - in the case of single - level aggregation , as well as for blocks with " overflowing " values .
2018-09-28 20:17:38 +00:00
* If there is at least one block with a bucket number greater or equal than zero , then there was a two - level aggregation .
2017-04-01 07:20:54 +00:00
*/
auto max_bucket = bucket_to_blocks . rbegin ( ) - > first ;
2019-01-21 10:39:24 +00:00
bool has_two_level = max_bucket > = 0 ;
2015-01-03 05:39:21 +00:00
2017-04-01 07:20:54 +00:00
if ( has_two_level )
{
# define M(NAME) \
2018-08-27 17:42:13 +00:00
if ( method_chosen = = AggregatedDataVariants : : Type : : NAME ) \
method_chosen = AggregatedDataVariants : : Type : : NAME # # _two_level ;
2014-06-26 00:58:14 +00:00
2017-04-01 07:20:54 +00:00
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL ( M )
2014-06-26 00:58:14 +00:00
2017-04-01 07:20:54 +00:00
# undef M
}
2012-05-30 01:38:02 +00:00
2017-04-02 17:37:49 +00:00
/// result will destroy the states of aggregate functions in the destructor
2017-04-01 07:20:54 +00:00
result . aggregator = this ;
2015-03-23 01:10:06 +00:00
2018-08-27 17:42:13 +00:00
result . init ( method_chosen ) ;
2017-04-01 07:20:54 +00:00
result . keys_size = params . keys_size ;
result . key_sizes = key_sizes ;
2012-05-30 01:38:02 +00:00
2022-04-18 10:18:43 +00:00
bool has_blocks_with_unknown_bucket = bucket_to_blocks . contains ( - 1 ) ;
2015-01-17 04:49:13 +00:00
2017-04-02 17:37:49 +00:00
/// First, parallel the merge for the individual buckets. Then we continue merge the data not allocated to the buckets.
2017-04-01 07:20:54 +00:00
if ( has_two_level )
{
2017-04-02 17:37:49 +00:00
/** In this case, no_more_keys is not supported due to the fact that
* from different threads it is difficult to update the general state for " other " keys ( overflows ) .
* That is , the keys in the end can be significantly larger than max_rows_to_group_by .
2017-04-01 07:20:54 +00:00
*/
2015-10-23 01:43:42 +00:00
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Merging partially aggregated two-level data. " ) ;
2012-05-30 01:38:02 +00:00
2018-06-19 20:30:35 +00:00
auto merge_bucket = [ & bucket_to_blocks , & result , this ] ( Int32 bucket , Arena * aggregates_pool , ThreadGroupStatusPtr thread_group )
2017-04-01 07:20:54 +00:00
{
2018-11-20 17:08:34 +00:00
if ( thread_group )
CurrentThread : : attachToIfDetached ( thread_group ) ;
2012-05-30 01:38:02 +00:00
2017-04-01 07:20:54 +00:00
for ( Block & block : bucket_to_blocks [ bucket ] )
{
# define M(NAME) \
else if ( result . type = = AggregatedDataVariants : : Type : : NAME ) \
2022-06-15 20:45:38 +00:00
mergeStreamsImpl ( std : : move ( block ) , aggregates_pool , * result . NAME , result . NAME - > data . impls [ bucket ] , nullptr , false ) ;
2015-01-03 05:39:21 +00:00
2020-03-08 23:48:08 +00:00
if ( false ) { } // NOLINT
2017-04-01 07:20:54 +00:00
APPLY_FOR_VARIANTS_TWO_LEVEL ( M )
# undef M
else
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
}
} ;
std : : unique_ptr < ThreadPool > thread_pool ;
2018-06-04 14:17:24 +00:00
if ( max_threads > 1 & & total_input_rows > 100000 ) /// TODO Make a custom threshold.
2017-04-01 07:20:54 +00:00
thread_pool = std : : make_unique < ThreadPool > ( max_threads ) ;
for ( const auto & bucket_blocks : bucket_to_blocks )
{
const auto bucket = bucket_blocks . first ;
if ( bucket = = - 1 )
continue ;
result . aggregates_pools . push_back ( std : : make_shared < Arena > ( ) ) ;
Arena * aggregates_pool = result . aggregates_pools . back ( ) . get ( ) ;
2020-04-22 18:41:19 +00:00
auto task = [ group = CurrentThread : : getGroup ( ) , bucket , & merge_bucket , aggregates_pool ] { return merge_bucket ( bucket , aggregates_pool , group ) ; } ;
2017-04-01 07:20:54 +00:00
if ( thread_pool )
2019-10-17 14:41:27 +00:00
thread_pool - > scheduleOrThrowOnError ( task ) ;
2017-04-01 07:20:54 +00:00
else
task ( ) ;
}
2013-05-06 11:45:28 +00:00
2017-04-01 07:20:54 +00:00
if ( thread_pool )
thread_pool - > wait ( ) ;
2014-12-30 10:16:23 +00:00
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Merged partially aggregated two-level data. " ) ;
2017-04-01 07:20:54 +00:00
}
if ( has_blocks_with_unknown_bucket )
{
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Merging partially aggregated single-level data. " ) ;
2017-04-01 07:20:54 +00:00
bool no_more_keys = false ;
BlocksList & blocks = bucket_to_blocks [ - 1 ] ;
for ( Block & block : blocks )
{
if ( ! checkLimits ( result . sizeWithoutOverflowRow ( ) , no_more_keys ) )
break ;
if ( result . type = = AggregatedDataVariants : : Type : : without_key | | block . info . is_overflows )
2022-06-15 20:45:38 +00:00
mergeBlockWithoutKeyStreamsImpl ( std : : move ( block ) , result ) ;
2017-04-01 07:20:54 +00:00
# define M(NAME, IS_TWO_LEVEL) \
else if ( result . type = = AggregatedDataVariants : : Type : : NAME ) \
2022-06-15 20:45:38 +00:00
mergeStreamsImpl ( std : : move ( block ) , result . aggregates_pool , * result . NAME , result . NAME - > data , result . without_key , no_more_keys ) ;
2017-04-01 07:20:54 +00:00
APPLY_FOR_AGGREGATED_VARIANTS ( M )
# undef M
else if ( result . type ! = AggregatedDataVariants : : Type : : without_key )
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
}
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Merged partially aggregated single-level data. " ) ;
2017-04-01 07:20:54 +00:00
}
2015-01-03 05:39:21 +00:00
}
2015-07-30 23:41:02 +00:00
Block Aggregator : : mergeBlocks ( BlocksList & blocks , bool final )
{
2017-04-01 07:20:54 +00:00
if ( blocks . empty ( ) )
return { } ;
2015-07-30 23:41:02 +00:00
2017-07-25 12:16:14 +00:00
auto bucket_num = blocks . front ( ) . info . bucket_num ;
bool is_overflows = blocks . front ( ) . info . is_overflows ;
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Merging partially aggregated blocks (bucket = {}). " , bucket_num ) ;
2017-07-25 12:16:14 +00:00
Stopwatch watch ;
2017-04-01 07:20:54 +00:00
/** If possible, change 'method' to some_hash64. Otherwise, leave as is.
* Better hash function is needed because during external aggregation ,
* we may merge partitions of data with total number of keys far greater than 4 billion .
*/
2018-08-27 17:42:13 +00:00
auto merge_method = method_chosen ;
2016-09-23 05:49:55 +00:00
# define APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION(M) \
2018-01-09 00:19:58 +00:00
M ( key64 ) \
M ( key_string ) \
2017-04-01 07:20:54 +00:00
M ( key_fixed_string ) \
2018-01-09 00:19:58 +00:00
M ( keys128 ) \
M ( keys256 ) \
M ( serialized ) \
2016-09-23 05:49:55 +00:00
# define M(NAME) \
2018-02-24 04:46:14 +00:00
if ( merge_method = = AggregatedDataVariants : : Type : : NAME ) \
merge_method = AggregatedDataVariants : : Type : : NAME # # _hash64 ; \
2016-09-23 05:49:55 +00:00
2017-04-01 07:20:54 +00:00
APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION ( M )
2016-09-23 05:49:55 +00:00
# undef M
# undef APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION
2017-04-02 17:37:49 +00:00
/// Temporary data for aggregation.
2017-04-01 07:20:54 +00:00
AggregatedDataVariants result ;
2017-04-02 17:37:49 +00:00
/// result will destroy the states of aggregate functions in the destructor
2017-04-01 07:20:54 +00:00
result . aggregator = this ;
2018-02-24 04:46:14 +00:00
result . init ( merge_method ) ;
2017-04-01 07:20:54 +00:00
result . keys_size = params . keys_size ;
result . key_sizes = key_sizes ;
2022-08-03 15:56:59 +00:00
size_t source_rows = 0 ;
/// In some aggregation methods (e.g. serialized) aggregates pools are used also to store serialized aggregation keys.
/// Memory occupied by them will have the same lifetime as aggregate function states, while it is not actually necessary and leads to excessive memory consumption.
/// To avoid this we use a separate arena to allocate memory for aggregation keys. Its memory will be freed at this function return.
auto arena_for_keys = std : : make_shared < Arena > ( ) ;
2017-04-01 07:20:54 +00:00
for ( Block & block : blocks )
{
2022-08-03 15:56:59 +00:00
source_rows + = block . rows ( ) ;
2017-07-25 17:56:09 +00:00
if ( bucket_num > = 0 & & block . info . bucket_num ! = bucket_num )
bucket_num = - 1 ;
2017-07-25 12:16:14 +00:00
if ( result . type = = AggregatedDataVariants : : Type : : without_key | | is_overflows )
2022-06-15 20:45:38 +00:00
mergeBlockWithoutKeyStreamsImpl ( std : : move ( block ) , result ) ;
2017-04-01 07:20:54 +00:00
2022-08-03 15:56:59 +00:00
# define M(NAME, IS_TWO_LEVEL) \
else if ( result . type = = AggregatedDataVariants : : Type : : NAME ) \
mergeStreamsImpl ( std : : move ( block ) , result . aggregates_pool , * result . NAME , result . NAME - > data , nullptr , false , arena_for_keys . get ( ) ) ;
2017-04-01 07:20:54 +00:00
APPLY_FOR_AGGREGATED_VARIANTS ( M )
# undef M
else if ( result . type ! = AggregatedDataVariants : : Type : : without_key )
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
}
2017-07-25 12:16:14 +00:00
Block block ;
if ( result . type = = AggregatedDataVariants : : Type : : without_key | | is_overflows )
block = prepareBlockAndFillWithoutKey ( result , final , is_overflows ) ;
else
block = prepareBlockAndFillSingleLevel ( result , final ) ;
/// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods.
2017-04-01 07:20:54 +00:00
2017-07-25 12:16:14 +00:00
if ( ! final )
{
/// Pass ownership of aggregate function states from result to ColumnAggregateFunction objects in the resulting block.
result . aggregator = nullptr ;
2017-04-01 07:20:54 +00:00
}
2017-07-25 12:16:14 +00:00
size_t rows = block . rows ( ) ;
size_t bytes = block . bytes ( ) ;
double elapsed_seconds = watch . elapsedSeconds ( ) ;
2022-08-03 15:56:59 +00:00
LOG_DEBUG (
log ,
" Merged partially aggregated blocks for bucket #{}. Got {} rows, {} from {} source rows in {} sec. ({:.3f} rows/sec., {}/sec.) " ,
bucket_num ,
rows ,
ReadableSize ( bytes ) ,
source_rows ,
elapsed_seconds ,
rows / elapsed_seconds ,
2020-05-30 21:35:52 +00:00
ReadableSize ( bytes / elapsed_seconds ) ) ;
2017-04-01 07:20:54 +00:00
2017-07-25 12:16:14 +00:00
block . info . bucket_num = bucket_num ;
return block ;
2015-07-30 23:41:02 +00:00
}
2015-09-07 07:40:14 +00:00
template < typename Method >
void NO_INLINE Aggregator : : convertBlockToTwoLevelImpl (
2017-04-01 07:20:54 +00:00
Method & method ,
Arena * pool ,
2017-12-13 01:27:53 +00:00
ColumnRawPtrs & key_columns ,
2017-04-01 07:20:54 +00:00
const Block & source ,
std : : vector < Block > & destinations ) const
2015-09-07 07:40:14 +00:00
{
2019-01-21 10:39:24 +00:00
typename Method : : State state ( key_columns , key_sizes , aggregation_state_cache ) ;
2017-04-01 07:20:54 +00:00
size_t rows = source . rows ( ) ;
size_t columns = source . columns ( ) ;
/// Create a 'selector' that will contain bucket index for every row. It will be used to scatter rows to buckets.
IColumn : : Selector selector ( rows ) ;
/// For every row.
for ( size_t i = 0 ; i < rows ; + + i )
{
2018-12-03 13:00:01 +00:00
if constexpr ( Method : : low_cardinality_optimization )
{
if ( state . isNullAt ( i ) )
{
selector [ i ] = 0 ;
continue ;
}
}
2019-01-21 10:39:24 +00:00
/// Calculate bucket number from row hash.
auto hash = state . getHash ( method . data , i , * pool ) ;
2017-04-01 07:20:54 +00:00
auto bucket = method . data . getBucketFromHash ( hash ) ;
selector [ i ] = bucket ;
}
size_t num_buckets = destinations . size ( ) ;
for ( size_t column_idx = 0 ; column_idx < columns ; + + column_idx )
{
const ColumnWithTypeAndName & src_col = source . getByPosition ( column_idx ) ;
2017-12-15 03:47:43 +00:00
MutableColumns scattered_columns = src_col . column - > scatter ( num_buckets , selector ) ;
2017-04-01 07:20:54 +00:00
for ( size_t bucket = 0 , size = num_buckets ; bucket < size ; + + bucket )
{
if ( ! scattered_columns [ bucket ] - > empty ( ) )
{
Block & dst = destinations [ bucket ] ;
dst . info . bucket_num = bucket ;
2017-12-15 03:47:43 +00:00
dst . insert ( { std : : move ( scattered_columns [ bucket ] ) , src_col . type , src_col . name } ) ;
2017-04-01 07:20:54 +00:00
}
/** Inserted columns of type ColumnAggregateFunction will own states of aggregate functions
* by holding shared_ptr to source column . See ColumnAggregateFunction . h
*/
}
}
2015-09-07 07:40:14 +00:00
}
2021-09-08 15:30:48 +00:00
std : : vector < Block > Aggregator : : convertBlockToTwoLevel ( const Block & block ) const
2015-09-07 07:40:14 +00:00
{
2017-04-01 07:20:54 +00:00
if ( ! block )
return { } ;
2015-09-07 07:40:14 +00:00
2017-04-01 07:20:54 +00:00
AggregatedDataVariants data ;
2015-09-07 07:40:14 +00:00
2017-12-13 01:27:53 +00:00
ColumnRawPtrs key_columns ( params . keys_size ) ;
2015-09-07 07:40:14 +00:00
2017-04-02 17:37:49 +00:00
/// Remember the columns we will work with
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . keys_size ; + + i )
key_columns [ i ] = block . safeGetByPosition ( i ) . column . get ( ) ;
2015-09-07 07:40:14 +00:00
2018-08-27 17:42:13 +00:00
AggregatedDataVariants : : Type type = method_chosen ;
2017-04-01 07:20:54 +00:00
data . keys_size = params . keys_size ;
data . key_sizes = key_sizes ;
2015-09-07 07:40:14 +00:00
# define M(NAME) \
2017-04-01 07:20:54 +00:00
else if ( type = = AggregatedDataVariants : : Type : : NAME ) \
type = AggregatedDataVariants : : Type : : NAME # # _two_level ;
2015-09-07 07:40:14 +00:00
2020-03-08 23:48:08 +00:00
if ( false ) { } // NOLINT
2017-04-01 07:20:54 +00:00
APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL ( M )
2015-09-07 07:40:14 +00:00
# undef M
2017-04-01 07:20:54 +00:00
else
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
2015-09-07 07:40:14 +00:00
2017-04-01 07:20:54 +00:00
data . init ( type ) ;
2015-09-07 07:40:14 +00:00
2017-04-01 07:20:54 +00:00
size_t num_buckets = 0 ;
2015-09-07 07:40:14 +00:00
# define M(NAME) \
2017-04-01 07:20:54 +00:00
else if ( data . type = = AggregatedDataVariants : : Type : : NAME ) \
num_buckets = data . NAME - > data . NUM_BUCKETS ;
2015-09-07 07:40:14 +00:00
2020-03-08 23:48:08 +00:00
if ( false ) { } // NOLINT
2017-04-01 07:20:54 +00:00
APPLY_FOR_VARIANTS_TWO_LEVEL ( M )
2015-09-07 07:40:14 +00:00
# undef M
2017-04-01 07:20:54 +00:00
else
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
2015-09-07 07:40:14 +00:00
2017-04-01 07:20:54 +00:00
std : : vector < Block > splitted_blocks ( num_buckets ) ;
2015-09-07 07:40:14 +00:00
# define M(NAME) \
2017-04-01 07:20:54 +00:00
else if ( data . type = = AggregatedDataVariants : : Type : : NAME ) \
2018-09-12 13:27:00 +00:00
convertBlockToTwoLevelImpl ( * data . NAME , data . aggregates_pool , \
2019-08-11 21:45:18 +00:00
key_columns , block , splitted_blocks ) ;
2015-09-07 07:40:14 +00:00
2020-03-08 23:48:08 +00:00
if ( false ) { } // NOLINT
2017-04-01 07:20:54 +00:00
APPLY_FOR_VARIANTS_TWO_LEVEL ( M )
2015-09-07 07:40:14 +00:00
# undef M
2017-04-01 07:20:54 +00:00
else
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
2015-09-07 07:40:14 +00:00
2017-04-01 07:20:54 +00:00
return splitted_blocks ;
2015-09-07 07:40:14 +00:00
}
2015-12-06 19:42:28 +00:00
template < typename Method , typename Table >
2017-12-01 20:21:35 +00:00
void NO_INLINE Aggregator : : destroyImpl ( Table & table ) const
2015-01-03 05:39:21 +00:00
{
2019-08-08 16:16:19 +00:00
table . forEachMapped ( [ & ] ( AggregateDataPtr & data )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/** If an exception (usually a lack of memory, the MemoryTracker throws) arose
* after inserting the key into a hash table , but before creating all states of aggregate functions ,
* then data will be equal nullptr .
2017-04-01 07:20:54 +00:00
*/
if ( nullptr = = data )
2019-08-08 16:16:19 +00:00
return ;
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
2020-06-07 22:46:58 +00:00
aggregate_functions [ i ] - > destroy ( data + offsets_of_aggregate_states [ i ] ) ;
2017-04-01 07:20:54 +00:00
data = nullptr ;
2019-08-08 16:16:19 +00:00
} ) ;
2015-12-06 19:42:28 +00:00
}
void Aggregator : : destroyWithoutKey ( AggregatedDataVariants & result ) const
{
2017-04-01 07:20:54 +00:00
AggregatedDataWithoutKey & res_data = result . without_key ;
2015-12-06 19:42:28 +00:00
2017-04-01 07:20:54 +00:00
if ( nullptr ! = res_data )
{
for ( size_t i = 0 ; i < params . aggregates_size ; + + i )
2020-06-07 22:46:58 +00:00
aggregate_functions [ i ] - > destroy ( res_data + offsets_of_aggregate_states [ i ] ) ;
2015-12-06 19:42:28 +00:00
2017-04-01 07:20:54 +00:00
res_data = nullptr ;
}
2012-05-30 01:38:02 +00:00
}
2021-09-08 15:30:48 +00:00
void Aggregator : : destroyAllAggregateStates ( AggregatedDataVariants & result ) const
2013-02-16 18:59:05 +00:00
{
2020-03-08 21:40:00 +00:00
if ( result . empty ( ) )
2017-04-01 07:20:54 +00:00
return ;
2013-02-16 18:59:05 +00:00
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Destroying aggregate states " ) ;
2013-02-16 18:59:05 +00:00
2017-04-02 17:37:49 +00:00
/// In what data structure is the data aggregated?
2017-04-01 07:20:54 +00:00
if ( result . type = = AggregatedDataVariants : : Type : : without_key | | params . overflow_row )
destroyWithoutKey ( result ) ;
2013-02-16 18:59:05 +00:00
2014-12-30 10:16:23 +00:00
# define M(NAME, IS_TWO_LEVEL) \
2017-04-01 07:20:54 +00:00
else if ( result . type = = AggregatedDataVariants : : Type : : NAME ) \
2017-12-01 20:21:35 +00:00
destroyImpl < decltype ( result . NAME ) : : element_type > ( result . NAME - > data ) ;
2014-12-30 10:16:23 +00:00
2020-03-08 23:48:08 +00:00
if ( false ) { } // NOLINT
2017-04-01 07:20:54 +00:00
APPLY_FOR_AGGREGATED_VARIANTS ( M )
2014-12-30 10:16:23 +00:00
# undef M
2017-04-01 07:20:54 +00:00
else if ( result . type ! = AggregatedDataVariants : : Type : : without_key )
throw Exception ( " Unknown aggregated data variant. " , ErrorCodes : : UNKNOWN_AGGREGATED_DATA_VARIANT ) ;
2013-02-16 18:59:05 +00:00
}
2013-05-03 10:20:53 +00:00
2011-09-19 01:42:16 +00:00
}