diff --git a/src/Common/HashMapsTemplate.h b/src/Common/HashMapsTemplate.h index 8eb33d1d1d4..09e1a031935 100644 --- a/src/Common/HashMapsTemplate.h +++ b/src/Common/HashMapsTemplate.h @@ -4,7 +4,8 @@ #include #include #include -#include +#include +#include namespace DB { @@ -24,9 +25,14 @@ void serializeHashMap(const Map & map, MappedSerializer && mapped_serializer, Wr }); } -template +template void deserializeHashMap(Map & map, MappedDeserializer && mapped_deserializer, Arena & pool, ReadBuffer & rb) { + using Mapped = std::decay_t::mapped_type; + + constexpr bool is_string_hash_map + = std::is_same_v, StringHashMap> || std::is_same_v, TwoLevelStringHashMap>; + /// For StringHashMap or TwoLevelStringHashMap, it requires StringRef key padded 8 keys(left and right). /// So far, the Arena's MemoryChunk is always padding right 15, so we just pad left 8 here if constexpr (is_string_hash_map) @@ -60,6 +66,20 @@ void deserializeHashMap(Map & map, MappedDeserializer && mapped_deserializer, Ar pool.setPaddingLeft(0); } +template +void serializeTwoLevelHashMap(const Map & map, MappedSerializer && mapped_serializer, WriteBuffer & wb) +{ + serializeHashMap(map, std::move(mapped_serializer), wb); + map.writeUpdatedBuckets(wb); +} + +template +void deserializeTwoLevelHashMap(Map & map, MappedDeserializer && mapped_deserializer, Arena & pool, ReadBuffer & rb) +{ + deserializeHashMap(map, std::move(mapped_deserializer), pool, rb); + map.readUpdatedBuckets(rb); /// recover buckets updated status +} + /// HashMapsTemplate is a taken from HashJoin class and make it standalone /// and could be shared among different components @@ -187,7 +207,7 @@ struct HashMapsTemplate #define M(NAME) \ case HashType::NAME: { \ assert(NAME); \ - deserializeHashMap(*NAME, mapped_deserializer, pool, rb); \ + deserializeHashMap(*NAME, mapped_deserializer, pool, rb); \ return; \ } APPLY_FOR_HASH_KEY_VARIANTS(M) diff --git a/src/Common/HashTable/TimeBucketHashMap.h b/src/Common/HashTable/TimeBucketHashMap.h index 172d1d1c192..827c396f8ef 100644 --- a/src/Common/HashTable/TimeBucketHashMap.h +++ b/src/Common/HashTable/TimeBucketHashMap.h @@ -33,6 +33,20 @@ class TimeBucketHashMapTable p.second.forEachValue(func); } + template + void ALWAYS_INLINE forEachValueOfUpdatedBuckets(Func && func, bool reset_updated = false) + { + for (auto & p : this->impls) + { + if (this->isBucketUpdated(p.first)) + { + p.second.forEachValue(func); + if (reset_updated) + this->resetUpdatedBucket(p.first); + } + } + } + typename Cell::Mapped & ALWAYS_INLINE operator[](const Key & x) { LookupResult it; diff --git a/src/Common/HashTable/TimeBucketHashTable.h b/src/Common/HashTable/TimeBucketHashTable.h index 77c1cdbe8aa..023a10ba9de 100644 --- a/src/Common/HashTable/TimeBucketHashTable.h +++ b/src/Common/HashTable/TimeBucketHashTable.h @@ -108,7 +108,9 @@ class TimeBucketHashTable : private boost::noncopyable, protected Hash /// empty using ConstLookupResult = typename Impl::ConstLookupResult; /// FIXME, choose a better perf data structure + /// Usually we don't have too many time buckets std::map impls; + std::unordered_map updated_buckets; Impl sentinel; TimeBucketHashTable() { } @@ -263,6 +265,7 @@ class TimeBucketHashTable : private boost::noncopyable, protected Hash /// empty { auto window = windowKey(key_holder); impls[window].emplace(key_holder, it, inserted, hash_value); + updated_buckets[window] = true; /// updated } LookupResult ALWAYS_INLINE find(Key x, size_t hash_value) @@ -289,6 +292,7 @@ class TimeBucketHashTable : private boost::noncopyable, protected Hash /// empty { DB::writeIntBinary(p.first); p.second.write(wb); + DB::writeBinary(updated_buckets[p.first], wb); } } @@ -309,7 +313,12 @@ class TimeBucketHashTable : private boost::noncopyable, protected Hash /// empty /// Write key and key-value separator DB::writeIntText(p.first, wb); DB::writeChar(KEY_VALUE_SEPARATOR, wb); + /// + DB::writeChar('<', wb); p.second.writeText(wb); + DB::writeChar(',', wb); + DB::writeBoolText(updated_buckets[p.first], wb); + DB::writeChar('>', wb); } DB::writeChar(END_BUCKET_MARKER, wb); } @@ -327,6 +336,7 @@ class TimeBucketHashTable : private boost::noncopyable, protected Hash /// empty assert(key != 0); assert(!impls.contains(key)); impls[key].read(rb); + DB::readBinary(updated_buckets[key], rb); } } @@ -349,7 +359,12 @@ class TimeBucketHashTable : private boost::noncopyable, protected Hash /// empty assert(key != 0); assert(!impls.contains(key)); + /// + DB::assertChar('<', rb); impls[key].readText(rb); + DB::assertChar(',', rb); + DB::readBoolText(updated_buckets[key], rb); + DB::assertChar('>', rb); } DB::assertChar(END_BUCKET_MARKER, rb); } @@ -402,6 +417,7 @@ class TimeBucketHashTable : private boost::noncopyable, protected Hash /// empty last_removed_watermark = it->first; ++removed; + updated_buckets.erase(it->first); it = impls.erase(it); } else @@ -438,4 +454,45 @@ class TimeBucketHashTable : private boost::noncopyable, protected Hash /// empty return buckets; } + + bool isBucketUpdated(Int64 bucket_) const + { + auto it = updated_buckets.find(bucket_); + if (it != updated_buckets.end()) + return it->second; + + return false; + } + + void resetUpdatedBucket(Int64 bucket_) + { + auto it = updated_buckets.find(bucket_); + if (it != updated_buckets.end()) + it->second = false; + } + + void writeUpdatedBuckets(DB::WriteBuffer & wb) const + { + DB::writeVarUInt(updated_buckets.size(), wb); + for (const auto & [bucket, updated] : updated_buckets) + { + DB::writeIntBinary(bucket, wb); + DB::writeBinary(updated, wb); + } + } + + void readUpdatedBuckets(DB::ReadBuffer & rb) + { + size_t size = 0; + DB::readVarUInt(size, rb); + updated_buckets.clear(); + Int64 bucket = 0; + bool updated = false; + for (size_t i = 0; i < size; ++i) + { + DB::readIntBinary(bucket, rb); + DB::readBinary(updated, rb); + updated_buckets.emplace(bucket, updated); + } + } }; diff --git a/src/Common/HashTable/TwoLevelHashMap.h b/src/Common/HashTable/TwoLevelHashMap.h index 3e618ca0a50..26008468974 100644 --- a/src/Common/HashTable/TwoLevelHashMap.h +++ b/src/Common/HashTable/TwoLevelHashMap.h @@ -38,6 +38,20 @@ class TwoLevelHashMapTable : public TwoLevelHashTableimpls[i].forEachValue(func); } + template + void ALWAYS_INLINE forEachValueOfUpdatedBuckets(Func && func, bool reset_updated = false) + { + for (auto i = 0u; i < this->NUM_BUCKETS; ++i) + { + if (this->isBucketUpdated(i)) + { + this->impls[i].forEachValue(func); + if (reset_updated) + this->resetUpdatedBucket(i); + } + } + } + template void ALWAYS_INLINE mergeToViaEmplace(Self & that, Func && func) { diff --git a/src/Common/HashTable/TwoLevelHashTable.h b/src/Common/HashTable/TwoLevelHashTable.h index 7e865cb48da..46d9e3ad637 100644 --- a/src/Common/HashTable/TwoLevelHashTable.h +++ b/src/Common/HashTable/TwoLevelHashTable.h @@ -90,6 +90,7 @@ class TwoLevelHashTable : using ConstLookupResult = typename Impl::ConstLookupResult; Impl impls[NUM_BUCKETS]; + bool updated_buckets[NUM_BUCKETS] = {false}; TwoLevelHashTable() = default; @@ -119,6 +120,7 @@ class TwoLevelHashTable : size_t hash_value = cell->getHash(src); size_t buck = getBucketFromHash(hash_value); impls[buck].insertUniqueNonZero(cell, hash_value); + updated_buckets[buck] = true; } } @@ -271,6 +273,7 @@ class TwoLevelHashTable : { size_t buck = getBucketFromHash(hash_value); impls[buck].emplace(key_holder, it, inserted, hash_value); + updated_buckets[buck] = true; } LookupResult ALWAYS_INLINE find(Key x, size_t hash_value) @@ -292,7 +295,10 @@ class TwoLevelHashTable : void write(DB::WriteBuffer & wb) const { for (UInt32 i = 0; i < NUM_BUCKETS; ++i) + { impls[i].write(wb); + DB::writeBinary(updated_buckets[i], wb); + } } void writeText(DB::WriteBuffer & wb) const @@ -301,14 +307,23 @@ class TwoLevelHashTable : { if (i != 0) DB::writeChar(',', wb); + + /// + DB::writeChar('<', wb); impls[i].writeText(wb); + DB::writeChar(',', wb); + DB::writeBoolText(updated_buckets[i], wb); + DB::writeChar('>', wb); } } void read(DB::ReadBuffer & rb) { for (UInt32 i = 0; i < NUM_BUCKETS; ++i) + { impls[i].read(rb); + DB::readBinary(updated_buckets[i], rb); + } } void readText(DB::ReadBuffer & rb) @@ -317,7 +332,13 @@ class TwoLevelHashTable : { if (i != 0) DB::assertChar(',', rb); + + /// + DB::assertChar('<', rb); impls[i].readText(rb); + DB::assertChar(',', rb); + DB::readBoolText(updated_buckets[i], rb); + DB::assertChar('>', rb); } } @@ -365,5 +386,31 @@ class TwoLevelHashTable : std::iota(bucket_ids.begin(), bucket_ids.end(), 0); return bucket_ids; } + + bool isBucketUpdated(Int64 bucket_) const + { + return updated_buckets[bucket_]; + } + + void resetUpdatedBucket(Int64 bucket_) + { + updated_buckets[bucket_] = false; + } + + void writeUpdatedBuckets(DB::WriteBuffer & wb) const + { + DB::writeVarUInt(NUM_BUCKETS, wb); + for (const auto & elem : updated_buckets) + DB::writeBinary(elem, wb); + } + + void readUpdatedBuckets(DB::ReadBuffer & rb) + { + size_t size = 0; + DB::readVarUInt(size, rb); + assert(size == NUM_BUCKETS); + for (auto & elem : updated_buckets) + DB::readBinary(elem, rb); + } /// proton : ends }; diff --git a/src/Common/HashTable/TwoLevelStringHashMap.h b/src/Common/HashTable/TwoLevelStringHashMap.h index a351543edb0..3501861a3ee 100644 --- a/src/Common/HashTable/TwoLevelStringHashMap.h +++ b/src/Common/HashTable/TwoLevelStringHashMap.h @@ -29,6 +29,20 @@ class TwoLevelStringHashMap : public TwoLevelStringHashTableimpls[i].forEachValue(func); } + template + void ALWAYS_INLINE forEachValueOfUpdatedBuckets(Func && func, bool reset_updated = false) + { + for (auto i = 0u; i < this->NUM_BUCKETS; ++i) + { + if (this->isBucketUpdated(i)) + { + this->impls[i].forEachValue(func); + if (reset_updated) + this->resetUpdatedBucket(i); + } + } + } + template void ALWAYS_INLINE mergeToViaEmplace(Self & that, Func && func) { diff --git a/src/Common/HashTable/TwoLevelStringHashTable.h b/src/Common/HashTable/TwoLevelStringHashTable.h index e0485f5aaa6..e1a3910ecf4 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -39,6 +39,7 @@ class TwoLevelStringHashTable : private boost::noncopyable using ConstLookupResult = typename Impl::ConstLookupResult; Impl impls[NUM_BUCKETS]; + bool updated_buckets[NUM_BUCKETS] = {false}; TwoLevelStringHashTable() {} @@ -53,24 +54,28 @@ class TwoLevelStringHashTable : private boost::noncopyable size_t hash_value = v.getHash(src.m1); size_t buck = getBucketFromHash(hash_value); impls[buck].m1.insertUniqueNonZero(&v, hash_value); + updated_buckets[buck] = true; } for (auto & v : src.m2) { size_t hash_value = v.getHash(src.m2); size_t buck = getBucketFromHash(hash_value); impls[buck].m2.insertUniqueNonZero(&v, hash_value); + updated_buckets[buck] = true; } for (auto & v : src.m3) { size_t hash_value = v.getHash(src.m3); size_t buck = getBucketFromHash(hash_value); impls[buck].m3.insertUniqueNonZero(&v, hash_value); + updated_buckets[buck] = true; } for (auto & v : src.ms) { size_t hash_value = v.getHash(src.ms); size_t buck = getBucketFromHash(hash_value); impls[buck].ms.insertUniqueNonZero(&v, hash_value); + updated_buckets[buck] = true; } } @@ -84,6 +89,9 @@ class TwoLevelStringHashTable : private boost::noncopyable const size_t sz = x.size; if (sz == 0) { + if constexpr (std::is_same_v) + self.updated_buckets[0] = true; + keyHolderDiscardKey(key_holder); return func(self.impls[0].m0, VoidKey{}, 0); } @@ -94,6 +102,9 @@ class TwoLevelStringHashTable : private boost::noncopyable // string keys. Put them to the generic table. auto res = hash(x); auto buck = getBucketFromHash(res); + if constexpr (std::is_same_v) + self.updated_buckets[buck] = true; + return func(self.impls[buck].ms, std::forward(key_holder), res); } @@ -126,6 +137,9 @@ class TwoLevelStringHashTable : private boost::noncopyable } auto res = hash(k8); auto buck = getBucketFromHash(res); + if constexpr (std::is_same_v) + self.updated_buckets[buck] = true; + keyHolderDiscardKey(key_holder); return func(self.impls[buck].m1, k8, res); } @@ -137,6 +151,9 @@ class TwoLevelStringHashTable : private boost::noncopyable n[1] >>= s; auto res = hash(k16); auto buck = getBucketFromHash(res); + if constexpr (std::is_same_v) + self.updated_buckets[buck] = true; + keyHolderDiscardKey(key_holder); return func(self.impls[buck].m2, k16, res); } @@ -148,6 +165,9 @@ class TwoLevelStringHashTable : private boost::noncopyable n[2] >>= s; auto res = hash(k24); auto buck = getBucketFromHash(res); + if constexpr (std::is_same_v) + self.updated_buckets[buck] = true; + keyHolderDiscardKey(key_holder); return func(self.impls[buck].m3, k24, res); } @@ -155,6 +175,9 @@ class TwoLevelStringHashTable : private boost::noncopyable { auto res = hash(x); auto buck = getBucketFromHash(res); + if constexpr (std::is_same_v) + self.updated_buckets[buck] = true; + return func(self.impls[buck].ms, std::forward(key_holder), res); } } @@ -179,7 +202,10 @@ class TwoLevelStringHashTable : private boost::noncopyable void write(DB::WriteBuffer & wb) const { for (UInt32 i = 0; i < NUM_BUCKETS; ++i) + { impls[i].write(wb); + DB::writeBinary(updated_buckets[i], wb); + } } void writeText(DB::WriteBuffer & wb) const @@ -188,14 +214,22 @@ class TwoLevelStringHashTable : private boost::noncopyable { if (i != 0) DB::writeChar(',', wb); + /// + DB::writeChar('<', wb); impls[i].writeText(wb); + DB::writeChar(',', wb); + DB::writeBoolText(updated_buckets[i], wb); + DB::writeChar('>', wb); } } void read(DB::ReadBuffer & rb) { for (UInt32 i = 0; i < NUM_BUCKETS; ++i) + { impls[i].read(rb); + DB::readBinary(updated_buckets[i], rb); + } } void readText(DB::ReadBuffer & rb) @@ -205,6 +239,12 @@ class TwoLevelStringHashTable : private boost::noncopyable if (i != 0) DB::assertChar(',', rb); impls[i].readText(rb); + /// + DB::assertChar('<', rb); + impls[i].readText(rb); + DB::assertChar(',', rb); + DB::readBoolText(updated_buckets[i], rb); + DB::assertChar('>', rb); } } @@ -252,4 +292,30 @@ class TwoLevelStringHashTable : private boost::noncopyable std::iota(bucket_ids.begin(), bucket_ids.end(), 0); return bucket_ids; } + + bool isBucketUpdated(Int64 bucket_) const + { + return updated_buckets[bucket_]; + } + + void resetUpdatedBucket(Int64 bucket_) + { + updated_buckets[bucket_] = false; + } + + void writeUpdatedBuckets(DB::WriteBuffer & wb) const + { + DB::writeVarUInt(NUM_BUCKETS, wb); + for (const auto & elem : updated_buckets) + DB::writeBinary(elem, wb); + } + + void readUpdatedBuckets(DB::ReadBuffer & rb) + { + size_t size = 0; + DB::readVarUInt(size, rb); + assert(size == NUM_BUCKETS); + for (auto & elem : updated_buckets) + DB::readBinary(elem, rb); + } }; diff --git a/src/Common/serde.h b/src/Common/serde.h index ce44f491337..b6bfa951081 100644 --- a/src/Common/serde.h +++ b/src/Common/serde.h @@ -1,32 +1,9 @@ #pragma once -#include -#include #include namespace DB { -/// REQUIRES: The object must support versioned serialization/deserialization -template -concept VersionedSerializable - = requires(const S & s, WB & wb, VersionType version, Args &&... args) { s.serialize(wb, version, std::forward(args)...); }; - -template -concept VersionedDeserializable - = requires(S & s, RB & rb, VersionType version, Args &&... args) { s.deserialize(rb, version, std::forward(args)...); }; - -template S> -void ALWAYS_INLINE serialize(const S & s, WB & wb, VersionType version, Args &&... args) -{ - s.serialize(wb, version, std::forward(args)...); -} - -template S> -void ALWAYS_INLINE deserialize(S & s, RB & rb, VersionType version, Args &&... args) -{ - s.deserialize(rb, version, std::forward(args)...); -} - /// macro tag to indicate the data members or struct or class will /// be serialized / deserialized via network or file system IO. /// Hence, data structure versioning / backward / forward compatibility diff --git a/src/Interpreters/Streaming/AggregationUtils.cpp b/src/Interpreters/Streaming/AggregationUtils.cpp new file mode 100644 index 00000000000..6f79b641e01 --- /dev/null +++ b/src/Interpreters/Streaming/AggregationUtils.cpp @@ -0,0 +1,119 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + +namespace Streaming +{ +OutputBlockColumns prepareOutputBlockColumns( + const Aggregator::Params & params, + const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions, + const Block & res_header, + Arenas & aggregates_pools, + bool final, + size_t rows) +{ + MutableColumns key_columns(params.keys_size); + MutableColumns aggregate_columns(params.aggregates_size); + MutableColumns final_aggregate_columns(params.aggregates_size); + Aggregator::AggregateColumnsData aggregate_columns_data(params.aggregates_size); + + for (size_t i = 0; i < params.keys_size; ++i) + { + key_columns[i] = res_header.safeGetByPosition(i).type->createColumn(); + key_columns[i]->reserve(rows); + } + + for (size_t i = 0; i < params.aggregates_size; ++i) + { + if (!final) + { + const auto & aggregate_column_name = params.aggregates[i].column_name; + aggregate_columns[i] = res_header.getByName(aggregate_column_name).type->createColumn(); + + /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. + ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); + + /// proton: starts + column_aggregate_func.setKeepState(params.keep_state); + /// proton: ends + + /// Add arenas to ColumnAggregateFunction, which can result in moving ownership to it if reference count + /// get dropped in other places + for (auto & pool : aggregates_pools) + column_aggregate_func.addArena(pool); + + aggregate_columns_data[i] = &column_aggregate_func.getData(); + aggregate_columns_data[i]->reserve(rows); + } + else + { + final_aggregate_columns[i] = aggregate_functions[i]->getReturnType()->createColumn(); + final_aggregate_columns[i]->reserve(rows); + + if (aggregate_functions[i]->isState()) + { + auto callback = [&](IColumn & column) + { + /// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states. + if (auto * column_aggregate_func = typeid_cast(&column)) + for (auto & pool : aggregates_pools) + column_aggregate_func->addArena(pool); + }; + + callback(*final_aggregate_columns[i]); + final_aggregate_columns[i]->forEachSubcolumnRecursively(callback); + } + } + } + + if (key_columns.size() != params.keys_size) + throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + + std::vector raw_key_columns; + raw_key_columns.reserve(key_columns.size()); + for (auto & column : key_columns) + raw_key_columns.push_back(column.get()); + + return { + .key_columns = std::move(key_columns), + .raw_key_columns = std::move(raw_key_columns), + .aggregate_columns = std::move(aggregate_columns), + .final_aggregate_columns = std::move(final_aggregate_columns), + .aggregate_columns_data = std::move(aggregate_columns_data), + }; +} + +Block finalizeBlock(const Aggregator::Params & params, const Block & res_header, OutputBlockColumns && out_cols, bool final, size_t rows) +{ + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + + Block res = res_header.cloneEmpty(); + + 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) + { + const auto & aggregate_column_name = params.aggregates[i].column_name; + if (final) + res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]); + else + res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]); + } + + /// Change the size of the columns-constants in the block. + size_t columns = res_header.columns(); + for (size_t i = 0; i < columns; ++i) + if (isColumnConst(*res.getByPosition(i).column)) + res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows); + + return res; +} +} +} diff --git a/src/Interpreters/Streaming/AggregationUtils.h b/src/Interpreters/Streaming/AggregationUtils.h new file mode 100644 index 00000000000..6f6875e72fc --- /dev/null +++ b/src/Interpreters/Streaming/AggregationUtils.h @@ -0,0 +1,27 @@ +#pragma once + +#include + +namespace DB::Streaming +{ + +struct OutputBlockColumns +{ + MutableColumns key_columns; + std::vector raw_key_columns; + MutableColumns aggregate_columns; + MutableColumns final_aggregate_columns; + Aggregator::AggregateColumnsData aggregate_columns_data; +}; + + +OutputBlockColumns prepareOutputBlockColumns( + const Aggregator::Params & params, + const Aggregator::AggregateFunctionsPlainPtrs & aggregate_functions, + const Block & res_header, + Arenas & aggregates_pools, + bool final, + size_t rows); + +Block finalizeBlock(const Aggregator::Params & params, const Block & res_header, OutputBlockColumns && out_cols, bool final, size_t rows); +} diff --git a/src/Interpreters/Streaming/Aggregator.cpp b/src/Interpreters/Streaming/Aggregator.cpp index 7273c6ab81d..8de5a063791 100644 --- a/src/Interpreters/Streaming/Aggregator.cpp +++ b/src/Interpreters/Streaming/Aggregator.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -103,6 +104,61 @@ Columns materializeKeyColumns(Columns & columns, ColumnRawPtrs & key_columns, co return materialized_columns; } + +template +BlocksList convertBucketsInParallel(ThreadPool * thread_pool, const std::vector & buckets, BucketConverter && bucket_converter) +{ + std::atomic next_bucket_idx_to_merge = 0; + auto converter = [&](const std::atomic_flag * cancelled) { + BlocksList blocks; + Arena arena; + while (true) + { + if (cancelled && cancelled->test()) + break; + + UInt32 bucket_idx = next_bucket_idx_to_merge.fetch_add(1); + if (bucket_idx >= buckets.size()) + break; + + auto bucket = buckets[bucket_idx]; + blocks.splice(blocks.end(), bucket_converter(bucket, &arena)); + } + return blocks; + }; + + size_t num_threads = thread_pool ? std::min(thread_pool->getMaxThreads(), buckets.size()) : 1; + if (num_threads <= 1) + { + return converter(nullptr); + } + + /// Process in parallel + auto results = std::make_shared>(); + results->resize(num_threads); + thread_pool->setMaxThreads(num_threads); + { + std::atomic_flag cancelled; + SCOPE_EXIT_SAFE(cancelled.test_and_set();); + + for (size_t thread_id = 0; thread_id < num_threads; ++thread_id) + { + thread_pool->scheduleOrThrowOnError([thread_id, group = CurrentThread::getGroup(), results, &converter, &cancelled] { + CurrentThread::attachToIfDetached(group); + SCOPE_EXIT_SAFE( CurrentThread::detachQueryIfNotDetached() ); + (*results)[thread_id] = converter(&cancelled); + }); + } + + thread_pool->wait(); + } + + BlocksList blocks; + for (auto & result : *results) + blocks.splice(blocks.end(), std::move(result)); + + return blocks; +} } AggregatedDataVariants::~AggregatedDataVariants() @@ -120,6 +176,30 @@ AggregatedDataVariants::~AggregatedDataVariants() } } +void AggregatedDataVariants::reset() +{ + assert(aggregator); + /// Clear states + if (!aggregator->all_aggregates_has_trivial_destructor) + aggregator->destroyAllAggregateStates(*this); + + /// Clear hash map + switch (type) + { + case AggregatedDataVariants::Type::EMPTY: break; + case AggregatedDataVariants::Type::without_key: break; + + #define M(NAME, IS_TWO_LEVEL) \ + case AggregatedDataVariants::Type::NAME: NAME.reset(); break; + APPLY_FOR_AGGREGATED_VARIANTS_STREAMING(M) + #undef M + } + invalidate(); + + /// Reset pool + resetAndCreateAggregatesPools(); +} + void AggregatedDataVariants::convertToTwoLevel() { if (aggregator) @@ -143,6 +223,17 @@ void AggregatedDataVariants::convertToTwoLevel() } } +void AggregatedDataVariants::serialize(WriteBuffer & wb, const Aggregator & aggregator_) const +{ + /// We cannot use itself `aggregator` since if there is no data, it is nullptr. + aggregator_.checkpoint(*this, wb); +} + +void AggregatedDataVariants::deserialize(ReadBuffer & rb, const Aggregator & aggregator_) +{ + aggregator_.recover(*this, rb); +} + Block Aggregator::getHeader(bool final) const { return params.getHeader(final); @@ -268,6 +359,9 @@ void Aggregator::Params::explain(JSONBuilder::JSONMap & map) const Aggregator::Aggregator(const Params & params_) : params(params_), log(&Poco::Logger::get("StreamingAggregator")) { + if (params.overflow_row) [[unlikely]] + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Overflow row processing is not implemented in streaming aggregation"); + /// Use query-level memory tracker if (auto * memory_tracker_child = CurrentThread::getMemoryTracker()) if (auto * memory_tracker = memory_tracker_child->getParent()) @@ -282,8 +376,14 @@ Aggregator::Aggregator(const Params & params_) : params(params_), log(&Poco::Lo total_size_of_aggregate_states = 0; all_aggregates_has_trivial_destructor = true; + if (trackingUpdatesType() == TrackingUpdatesType::Updates) + { + total_size_of_aggregate_states = sizeof(TrackingUpdates); + align_aggregate_states = alignof(TrackingUpdates); + } + // aggregate_states will be aligned as below: - // |<-- state_1 -->|<-- pad_1 -->|<-- state_2 -->|<-- pad_2 -->| ..... + // |<-- [UpdatesTrackingData] -->||<-- 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 @@ -318,12 +418,13 @@ Aggregator::Aggregator(const Params & params_) : params(params_), log(&Poco::Lo aggregation_state_cache = AggregatedDataVariants::createCache(method_chosen, cache_settings); #if USE_EMBEDDED_COMPILER - compileAggregateFunctionsIfNeeded(); + /// TODO: Support compile aggregate functions + // compileAggregateFunctionsIfNeeded(); #endif } #if USE_EMBEDDED_COMPILER - +/* void Aggregator::compileAggregateFunctionsIfNeeded() { static std::unordered_map aggregate_functions_description_to_count; @@ -398,7 +499,7 @@ void Aggregator::compileAggregateFunctionsIfNeeded() } } } - +*/ #endif AggregatedDataVariants::Type Aggregator::chooseAggregationMethod() @@ -650,15 +751,18 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethodTimeBucketTwoLev } /// proton: ends -template -void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const +void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data, bool prefix_with_updates_tracking_state) const { - for (size_t j = 0; j < params.aggregates_size; ++j) + /// Initialize reserved TrackingUpdates + assert(aggregate_data); + if (prefix_with_updates_tracking_state) { - if constexpr (skip_compiled_aggregate_functions) - if (is_aggregate_function_compiled[j]) - continue; + if (trackingUpdatesType() == TrackingUpdatesType::Updates) + new (aggregate_data) TrackingUpdates(); + } + for (size_t j = 0; j < params.aggregates_size; ++j) + { try { /** An exception may occur if there is a shortage of memory. @@ -670,13 +774,7 @@ void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const catch (...) { for (size_t rollback_j = 0; rollback_j < j; ++rollback_j) - { - if constexpr (skip_compiled_aggregate_functions) - if (is_aggregate_function_compiled[j]) - continue; - aggregate_functions[rollback_j]->destroy(aggregate_data + offsets_of_aggregate_states[rollback_j]); - } throw; } @@ -688,13 +786,11 @@ void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const size_t row_begin, size_t row_end, ColumnRawPtrs & key_columns, - AggregateFunctionInstruction * aggregate_instructions, - bool no_more_keys, - AggregateDataPtr overflow_row) const + AggregateFunctionInstruction * aggregate_instructions) const { #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - return executeImpl(*result.NAME, result.aggregates_pool, row_begin, row_end, key_columns, aggregate_instructions, no_more_keys, overflow_row); + return executeImplBatch(*result.NAME, result.aggregates_pool, row_begin, row_end, key_columns, aggregate_instructions); if (false) {} // NOLINT APPLY_FOR_AGGREGATED_VARIANTS_STREAMING(M) @@ -708,53 +804,19 @@ void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const * Inline does not make sense, since the inner loop is entirely inside this function. */ template -[[nodiscard]] bool NO_INLINE Aggregator::executeImpl( +[[nodiscard]] bool NO_INLINE Aggregator::executeImplBatch( Method & method, Arena * aggregates_pool, size_t row_begin, size_t row_end, ColumnRawPtrs & key_columns, - AggregateFunctionInstruction * aggregate_instructions, - bool no_more_keys, - AggregateDataPtr overflow_row) const + AggregateFunctionInstruction * aggregate_instructions) const { typename Method::State state(key_columns, key_sizes, aggregation_state_cache); - if (!no_more_keys) - { -#if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder) - { - return executeImplBatch(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row); - } - else -#endif - { - return executeImplBatch(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row); - } - } - else - { - return executeImplBatch(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row); - } -} - -template -[[nodiscard]] bool NO_INLINE Aggregator::executeImplBatch( - Method & method, - typename Method::State & state, - Arena * aggregates_pool, - size_t row_begin, - size_t row_end, - AggregateFunctionInstruction * aggregate_instructions, - AggregateDataPtr overflow_row) const -{ /// Optimization for special case when there are no aggregate functions. - if (params.aggregates_size == 0) + if (params.aggregates_size == 0 && !needTrackUpdates()) { - if constexpr (no_more_keys) - return false; - /// For all rows. AggregateDataPtr place = aggregates_pool->alloc(0); for (size_t i = row_begin; i < row_end; ++i) @@ -765,7 +827,7 @@ template bool need_finalization = false; /// Optimization for special case when aggregating by 8bit key. - if constexpr (!no_more_keys && std::is_same_v) + if constexpr (std::is_same_v) { /// We use another method if there are aggregate functions with -Array combinator. bool has_arrays = false; @@ -778,7 +840,7 @@ template } } - if (!has_arrays) + if (!has_arrays && !needTrackUpdates()) { for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) { @@ -789,8 +851,9 @@ template inst->state_offset, [&](AggregateDataPtr & aggregate_data) { - aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); - createAggregateStates(aggregate_data); + auto data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(data, /*prefix_with_updates_tracking_state=*/ false); + aggregate_data = data; }, state.getKeyData(), inst->batch_arguments, @@ -821,100 +884,29 @@ template { AggregateDataPtr aggregate_data = nullptr; - if constexpr (!no_more_keys) - { - auto emplace_result = state.emplaceKey(method.data, i, *aggregates_pool); - - /// 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); - - aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); - -#if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - { - 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(aggregate_data); - } - -#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 emplace_result = state.emplaceKey(method.data, i, *aggregates_pool); - 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); - } -#endif - } - else -#endif - { - createAggregateStates(aggregate_data); - } + /// 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); - emplace_result.setMapped(aggregate_data); - } - else - aggregate_data = emplace_result.getMapped(); + aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(aggregate_data); - assert(aggregate_data != nullptr); + emplace_result.setMapped(aggregate_data); } else - { - /// 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; - } + aggregate_data = emplace_result.getMapped(); + assert(aggregate_data != nullptr); places[i] = aggregate_data; } -#if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - { - std::vector 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 = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function; - add_into_aggregate_states_function(row_begin, row_end, columns_data.data(), places.get()); - } -#endif - /// Add values to the aggregate functions. for (size_t i = 0; i < aggregate_functions.size(); ++i) { -#if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - if (is_aggregate_function_compiled[i]) - continue; -#endif - AggregateFunctionInstruction * inst = aggregate_instructions + i; if (inst->offsets) @@ -938,10 +930,12 @@ template } } + if (needTrackUpdates()) + TrackingUpdates::addBatch(row_begin, row_end, places.get(), aggregate_instructions ? aggregate_instructions->delta_column : nullptr); + return need_finalization; } -template [[nodiscard]] bool NO_INLINE Aggregator::executeWithoutKeyImpl( AggregatedDataWithoutKey & res, size_t row_begin, @@ -949,55 +943,12 @@ template AggregateFunctionInstruction * aggregate_instructions, Arena * arena) const { -#if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - { - std::vector 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; - add_into_aggregate_states_function_single_place(row_begin, row_end, columns_data.data(), res); - -#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 - } -#endif - /// Adding values bool should_finalize = false; for (size_t i = 0; i < aggregate_functions.size(); ++i) { AggregateFunctionInstruction * inst = aggregate_instructions + i; -#if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - if (is_aggregate_function_compiled[i]) - continue; -#endif if (inst->offsets) inst->batch_that->addBatchSinglePlace( inst->offsets[static_cast(row_begin) - 1], @@ -1026,28 +977,12 @@ template } } - return should_finalize; -} + if (needTrackUpdates()) + TrackingUpdates::addBatchSinglePlace(row_begin, row_end, res, aggregate_instructions ? aggregate_instructions->delta_column : nullptr); -void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl( - AggregatedDataWithoutKey & res, - size_t row_begin, - size_t row_end, - AggregateFunctionInstruction * aggregate_instructions, - Arena * arena, - const IColumn * delta_col) -{ - /// Adding values - for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) - { - if (inst->offsets) - inst->batch_that->addBatchSinglePlaceFromInterval(inst->offsets[row_begin], inst->offsets[row_end - 1], res + inst->state_offset, inst->batch_arguments, arena, -1, delta_col); - else - inst->batch_that->addBatchSinglePlaceFromInterval(row_begin, row_end, res + inst->state_offset, inst->batch_arguments, arena, -1, delta_col); - } + return should_finalize; } - void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns & aggregate_columns, Columns & materialized_columns, AggregateFunctionInstructions & aggregate_functions_instructions, NestedColumnsHolder & nested_columns_holder) const { @@ -1109,8 +1044,7 @@ std::pair Aggregator::executeOnBlock( const Block & block, AggregatedDataVariants & result, ColumnRawPtrs & key_columns, - AggregateColumns & aggregate_columns, - bool & no_more_keys) const + AggregateColumns & aggregate_columns) const { return executeOnBlock( block.getColumns(), @@ -1118,8 +1052,7 @@ std::pair Aggregator::executeOnBlock( block.rows(), result, key_columns, - aggregate_columns, - no_more_keys); + aggregate_columns); } /// return {should_abort, need_finalization} @@ -1129,8 +1062,7 @@ std::pair Aggregator::executeOnBlock( size_t row_end, AggregatedDataVariants & result, ColumnRawPtrs & key_columns, - AggregateColumns & aggregate_columns, - bool & no_more_keys) const + AggregateColumns & aggregate_columns) const { std::pair return_result = {false, false}; auto & need_abort = return_result.first; @@ -1163,33 +1095,17 @@ std::pair Aggregator::executeOnBlock( AggregateFunctionInstructions aggregate_functions_instructions; prepareAggregateInstructions(columns, aggregate_columns, materialized_columns, aggregate_functions_instructions, nested_columns_holder); - initStatesForWithoutKeyOrOverflow(result); + initStatesForWithoutKey(result); /// We select one of the aggregation methods and call it. /// For the case when there are no keys (all aggregate into one row). if (result.type == AggregatedDataVariants::Type::without_key) - { - /// TODO: Enable compilation after investigation -// #if USE_EMBEDDED_COMPILER -// if (compiled_aggregate_functions_holder) -// { -// executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool); -// } -// else -// #endif - { - need_finalization = executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool); - } - } + need_finalization = executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool); else - { - /// This is where data is written that does not fit in `max_rows_to_group_by` with `group_by_overflow_mode = any`. - AggregateDataPtr overflow_row_ptr = params.overflow_row ? result.without_key : nullptr; - need_finalization = executeImpl(result, row_begin, row_end, key_columns, aggregate_functions_instructions.data(), no_more_keys, overflow_row_ptr); - } + need_finalization = executeImpl(result, row_begin, row_end, key_columns, aggregate_functions_instructions.data()); - need_abort = checkAndProcessResult(result, no_more_keys); + need_abort = checkAndProcessResult(result); return return_result; } @@ -1223,7 +1139,7 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, co data_variants.init(data_variants.type); data_variants.aggregates_pools = Arenas(1, std::make_shared()); data_variants.aggregates_pool = data_variants.aggregates_pools.back().get(); - initStatesForWithoutKeyOrOverflow(data_variants); + initStatesForWithoutKey(data_variants); block_out.flush(); compressed_buf.next(); @@ -1274,273 +1190,53 @@ Block Aggregator::convertOneBucketToBlockImpl( Arena * arena, bool final, bool clear_states, - size_t bucket) const + Int64 bucket, + ConvertType type) const { - Block block = prepareBlockAndFill(data_variants, final, clear_states, method.data.impls[bucket].size(), - [bucket, &method, arena, this] ( - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - bool final_, - bool clear_states_) - { - convertToBlockImpl(method, method.data.impls[bucket], - key_columns, aggregate_columns, final_aggregate_columns, arena, final_, clear_states_); - }); - - block.info.bucket_num = static_cast(bucket); + Block block = convertToBlockImpl(method, method.data.impls[bucket], arena, data_variants.aggregates_pools, final, method.data.impls[bucket].size(), clear_states, type); + block.info.bucket_num = static_cast(bucket); + method.data.resetUpdatedBucket(bucket); /// finalized return block; } -Block Aggregator::convertOneBucketToBlock(AggregatedDataVariants & variants, bool final, ConvertAction action, size_t bucket) const +template +void Aggregator::writeToTemporaryFileImpl( + AggregatedDataVariants & data_variants, + Method & method, + NativeWriter & out) const { - auto method = variants.type; - Block block; - bool clear_states = shouldClearStates(action, final); - if (false) {} // NOLINT -#define M(NAME) \ - else if (method == AggregatedDataVariants::Type::NAME) \ - block = convertOneBucketToBlockImpl(variants, *variants.NAME, variants.aggregates_pool, final, clear_states, bucket); - - APPLY_FOR_VARIANTS_TIME_BUCKET_TWO_LEVEL(M) -#undef M - else - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - - return block; -} + size_t max_temporary_block_size_rows = 0; + size_t max_temporary_block_size_bytes = 0; -Block Aggregator::mergeAndConvertOneBucketToBlock( - ManyAggregatedDataVariants & variants, bool final, ConvertAction action, size_t bucket) const -{ - auto prepared_data_ptr = prepareVariantsToMerge(variants); - if (prepared_data_ptr->empty()) - return {}; + auto update_max_sizes = [&](const Block & block) + { + size_t block_size_rows = block.rows(); + size_t block_size_bytes = block.bytes(); - auto & merged_data = *prepared_data_ptr->at(0); - auto method = merged_data.type; - Arena * arena = merged_data.aggregates_pool; - bool clear_states = shouldClearStates(action, final); - Block block; + if (block_size_rows > max_temporary_block_size_rows) + max_temporary_block_size_rows = block_size_rows; + if (block_size_bytes > max_temporary_block_size_bytes) + max_temporary_block_size_bytes = block_size_bytes; + }; - if (false) {} // NOLINT -#define M(NAME) \ - else if (method == AggregatedDataVariants::Type::NAME) \ - { \ - mergeBucketImpl(*prepared_data_ptr, final, clear_states, bucket, arena); \ - block = convertOneBucketToBlockImpl(merged_data, *merged_data.NAME, arena, final, clear_states, bucket); \ + for (auto bucket : method.data.buckets()) + { + Block block = convertOneBucketToBlockImpl(data_variants, method, data_variants.aggregates_pool, false, false, bucket); + out.write(block); + update_max_sizes(block); } - APPLY_FOR_VARIANTS_ALL_TWO_LEVEL(M) -#undef M + /// Pass ownership of the aggregate functions states: + /// `data_variants` will not destroy them in the destructor, they are now owned by ColumnAggregateFunction objects. + data_variants.aggregator = nullptr; - return block; + LOG_DEBUG(log, "Max size of temporary block: {} rows, {}.", max_temporary_block_size_rows, ReadableSize(max_temporary_block_size_bytes)); } -BlocksList -Aggregator::mergeAndConvertToBlocks(ManyAggregatedDataVariants & data_variants, bool final, ConvertAction action, size_t max_threads) const + +bool Aggregator::checkLimits(size_t result_size) const { - auto prepared_data_ptr = prepareVariantsToMerge(data_variants); - if (prepared_data_ptr->empty()) - return {}; - - if (unlikely(params.overflow_row)) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Overflow row processing is not implemented in streaming aggregation"); - - bool clear_states = shouldClearStates(action, final); - BlocksList blocks; - auto & first = *prepared_data_ptr->at(0); - if (first.type == AggregatedDataVariants::Type::without_key) - blocks.emplace_back(mergeAndConvertWithoutKeyToBlock(*prepared_data_ptr, final, clear_states)); - else if (first.isTwoLevel()) - blocks.splice(blocks.end(), mergeAndConvertTwoLevelToBlocks(*prepared_data_ptr, final, max_threads, clear_states)); - else - blocks.emplace_back(mergeAndConvertSingleLevelToBlock(*prepared_data_ptr, final, clear_states)); - - if (clear_states) - clearDataVariants(first); - - return blocks; -} - -Block Aggregator::mergeAndConvertWithoutKeyToBlock(ManyAggregatedDataVariants & non_empty_data, bool final, bool clear_states) const -{ - auto & first = *non_empty_data.at(0); - assert(first.type == AggregatedDataVariants::Type::without_key); - mergeWithoutKeyDataImpl(non_empty_data, clear_states); - return prepareBlockAndFillWithoutKey(first, final, false, clear_states); -} - -Block Aggregator::mergeAndConvertSingleLevelToBlock(ManyAggregatedDataVariants & non_empty_data, bool final, bool clear_states) const -{ - auto & first = *non_empty_data.at(0); - if (false) - { - } // NOLINT -#define M(NAME) \ - else if (first.type == AggregatedDataVariants::Type::NAME) \ - mergeSingleLevelDataImpl(non_empty_data, clear_states); - - APPLY_FOR_VARIANTS_SINGLE_LEVEL_STREAMING(M) -#undef M - else throw Exception("Unknown single level aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - - return prepareBlockAndFillSingleLevel(first, final, clear_states); -} - -BlocksList Aggregator::mergeAndConvertTwoLevelToBlocks( - ManyAggregatedDataVariants & non_empty_data, bool final, size_t max_threads, bool clear_states) const -{ - auto & first = *non_empty_data.at(0); - assert(first.isTwoLevel()); -#define M(NAME) \ - else if (first.type == AggregatedDataVariants::Type::NAME) return mergeAndConvertTwoLevelToBlocksImpl< \ - decltype(first.NAME)::element_type>(non_empty_data, final, max_threads, clear_states); - - if (false) - { - } // NOLINT - APPLY_FOR_VARIANTS_ALL_TWO_LEVEL(M) -#undef M - else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); -} - -template -BlocksList Aggregator::mergeAndConvertTwoLevelToBlocksImpl( - ManyAggregatedDataVariants & non_empty_data, bool final, size_t max_threads, bool clear_states) const -{ - auto & first = *non_empty_data.at(0); - - std::vector buckets; - if (first.isStaticBucketTwoLevel()) - buckets = getDataVariant(first).data.buckets(); - else - { - assert(first.isTimeBucketTwoLevel()); - std::set buckets_set; - for (auto & data_variants : non_empty_data) - { - auto tmp_buckets = getDataVariant(*data_variants).data.buckets(); - buckets_set.insert(tmp_buckets.begin(), tmp_buckets.end()); - } - buckets.assign(buckets_set.begin(), buckets_set.end()); - } - - std::atomic next_bucket_idx_to_merge = 0; - auto converter = [&](size_t thread_id, ThreadGroupStatusPtr thread_group, const std::atomic_flag * cancelled) { - SCOPE_EXIT_SAFE(if (thread_group) CurrentThread::detachQueryIfNotDetached();); - if (thread_group) - CurrentThread::attachToIfDetached(thread_group); - - BlocksList blocks; - while (true) - { - if (cancelled && cancelled->test()) - break; - - UInt32 bucket_idx = next_bucket_idx_to_merge.fetch_add(1); - if (bucket_idx >= buckets.size()) - break; - - auto bucket = buckets[bucket_idx]; - - /// Merge one bucket into first one - Arena * arena = first.aggregates_pools.at(thread_id).get(); - mergeBucketImpl(non_empty_data, final, clear_states, bucket, arena); - auto & method = getDataVariant(first); - if (method.data.impls[bucket].empty()) - continue; - - /// Convert one bucket of first one - blocks.emplace_back(convertOneBucketToBlockImpl(first, method, arena, final, clear_states, bucket)); - } - - if (clear_states) - { - for (size_t i = 1; i < non_empty_data.size(); ++i) - clearDataVariants(*non_empty_data[i]); - } - - return blocks; - }; - - auto num_threads = std::min(max_threads, buckets.size()); - if (num_threads <= 1) - return converter(0, nullptr, nullptr); - - /// Process in parallel - /// proton FIXME : separate final vs non-final converting. For non-final converting, we don't need - /// each arena for each thread. - for (size_t i = first.aggregates_pools.size(); i < num_threads; ++i) - first.aggregates_pools.push_back(std::make_shared()); - - auto results = std::make_shared>(); - results->resize(num_threads); - ThreadPool thread_pool(num_threads); - { - std::atomic_flag cancelled; - SCOPE_EXIT_SAFE(cancelled.test_and_set();); - - for (size_t thread_id = 0; thread_id < num_threads; ++thread_id) - thread_pool.scheduleOrThrowOnError([thread_id, group = CurrentThread::getGroup(), results, &converter, &cancelled] { - (*results)[thread_id] = converter(thread_id, group, &cancelled); - }); - - thread_pool.wait(); - } - - BlocksList blocks; - for (auto & result : *results) - blocks.splice(blocks.end(), std::move(result)); - - return blocks; -} - -template -void Aggregator::writeToTemporaryFileImpl( - AggregatedDataVariants & data_variants, - Method & method, - NativeWriter & out) const -{ - size_t max_temporary_block_size_rows = 0; - size_t max_temporary_block_size_bytes = 0; - - auto update_max_sizes = [&](const Block & block) - { - size_t block_size_rows = block.rows(); - size_t block_size_bytes = block.bytes(); - - if (block_size_rows > max_temporary_block_size_rows) - max_temporary_block_size_rows = block_size_rows; - if (block_size_bytes > max_temporary_block_size_bytes) - max_temporary_block_size_bytes = block_size_bytes; - }; - - for (auto bucket : method.data.buckets()) - { - Block block = convertOneBucketToBlockImpl(data_variants, method, data_variants.aggregates_pool, false, false, bucket); - out.write(block); - update_max_sizes(block); - } - - if (params.overflow_row) - { - Block block = prepareBlockAndFillWithoutKey(data_variants, false, true, false); - out.write(block); - update_max_sizes(block); - } - - /// Pass ownership of the aggregate functions states: - /// `data_variants` will not destroy them in the destructor, they are now owned by ColumnAggregateFunction objects. - data_variants.aggregator = nullptr; - - LOG_DEBUG(log, "Max size of temporary block: {} rows, {}.", max_temporary_block_size_rows, ReadableSize(max_temporary_block_size_bytes)); -} - - -bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const -{ - if (!no_more_keys && params.max_rows_to_group_by && result_size > params.max_rows_to_group_by) + if (params.max_rows_to_group_by && result_size > params.max_rows_to_group_by) { switch (params.group_by_overflow_mode) { @@ -1553,8 +1249,7 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const return false; case OverflowMode::ANY: - no_more_keys = true; - break; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Streaming aggregation doesn't support 'OverflowMode::ANY'"); } } @@ -1566,44 +1261,25 @@ bool Aggregator::checkLimits(size_t result_size, bool & no_more_keys) const template -void Aggregator::convertToBlockImpl( - Method & method, - Table & data, - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - Arena * arena, - bool final, - bool clear_states) const +Block Aggregator::convertToBlockImpl( + Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows, bool clear_states, ConvertType type) const { if (data.empty()) - return; - - if (key_columns.size() != params.keys_size) - throw Exception{"Aggregate. Unexpected key columns size.", ErrorCodes::LOGICAL_ERROR}; + { + auto && out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, rows); + return {finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows)}; + } - std::vector raw_key_columns; - raw_key_columns.reserve(key_columns.size()); - for (auto & column : key_columns) - raw_key_columns.push_back(column.get()); + Block res; if (final) { -#if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder) - { - static constexpr bool use_compiled_functions = !Method::low_cardinality_optimization; - convertToBlockImplFinal(method, data, std::move(raw_key_columns), final_aggregate_columns, arena, clear_states); - } - else -#endif - { - convertToBlockImplFinal(method, data, std::move(raw_key_columns), final_aggregate_columns, arena, clear_states); - } + res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows, clear_states, type); } else { - convertToBlockImplNotFinal(method, data, std::move(raw_key_columns), aggregate_columns); + assert(type == ConvertType::Normal); + res = convertToBlockImplNotFinal(method, data, aggregates_pools, rows); } /// In order to release memory early. @@ -1611,6 +1287,8 @@ void Aggregator::convertToBlockImpl( if (clear_states) data.clearAndShrink(); /// proton: ends + + return res; } @@ -1618,7 +1296,8 @@ template inline void Aggregator::insertAggregatesIntoColumns( Mapped & mapped, MutableColumns & final_aggregate_columns, - Arena * arena) const + Arena * arena, + bool clear_states) const { /** Final values of aggregate functions are inserted to columns. * Then states of aggregate functions, that are not longer needed, are destroyed. @@ -1657,7 +1336,7 @@ inline void Aggregator::insertAggregatesIntoColumns( /// proton: starts /// For streaming aggregation, we hold up to the states - if (params.keep_state) + if (!clear_states) { if (exception) std::rethrow_exception(exception); @@ -1690,112 +1369,16 @@ inline void Aggregator::insertAggregatesIntoColumns( std::rethrow_exception(exception); } - -template -void NO_INLINE Aggregator::convertToBlockImplFinal( - Method & method, - Table & data, - std::vector key_columns, - MutableColumns & final_aggregate_columns, - Arena * arena, - bool clear_states) const +Block Aggregator::insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena, bool clear_states) const { - if constexpr (Method::low_cardinality_optimization) - { - if (data.hasNullKeyData()) - { - key_columns[0]->insertDefault(); - insertAggregatesIntoColumns(data.getNullKeyData(), final_aggregate_columns, arena); - } - } - - auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes); - const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - - PaddedPODArray places; - places.reserve(data.size()); - - data.forEachValue([&](const auto & key, auto & mapped) - { - /// Ingore invalid mapped, there are two cases: - /// 1) mapped was destroyed (it's a bug) - /// 2) no mapped states for retracted data (means it's an new group key, but no retracted data) - if (!mapped) - return; - - /// For UDA with own emit strategy, there are two special cases to be handled: - /// 1. not all groups need to be emitted. therefore proton needs to pick groups - /// that should emits, and only emit those groups while keep other groups unchanged. - /// 2. a single block trigger multiple emits. In this case, proton need insert the - /// same key multiple times for each emit result of this group. - - /// for non-UDA or UDA without emit strategy, 'should_emit' is always true. - /// For UDA with emit strategy, it is true only if the group should emit. - size_t emit_times = 1; - if (params.group_by == Params::GroupBy::USER_DEFINED) - { - assert(aggregate_functions.size() == 1); - emit_times = aggregate_functions[0]->getEmitTimes(mapped + offsets_of_aggregate_states[0]); - } - - if (emit_times > 0) - { - /// duplicate key for each emit - for (size_t i = 0; i < emit_times; i++) - method.insertKeyIntoColumns(key, key_columns, key_sizes_ref); - - places.emplace_back(mapped); - - /// Mark the cell as destroyed so it will not be destroyed in destructor. - /// proton: starts. Here we push the `mapped` to `places`, for streaming - /// case, we don't want aggregate function to destroy the places - if (clear_states) - mapped = nullptr; - } - }); - std::exception_ptr exception; size_t aggregate_functions_destroy_index = 0; try { -#if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - { - /** 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 columns_data; - - auto compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; - - for (size_t i = 0; i < params.aggregates_size; ++i) - { - if (!is_aggregate_function_compiled[i]) - continue; - - auto & final_aggregate_column = final_aggregate_columns[i]; - final_aggregate_column = final_aggregate_column->cloneResized(places.size()); - columns_data.emplace_back(getColumnData(final_aggregate_column.get())); - } - - auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function; - insert_aggregates_into_columns_function(0, places.size(), columns_data.data(), places.data()); - } -#endif - for (; aggregate_functions_destroy_index < params.aggregates_size;) { - if constexpr (use_compiled_functions) - { - if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) - { - ++aggregate_functions_destroy_index; - continue; - } - } - - auto & final_aggregate_column = final_aggregate_columns[aggregate_functions_destroy_index]; + auto & final_aggregate_column = out_cols.final_aggregate_columns[aggregate_functions_destroy_index]; size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; /** We increase aggregate_functions_destroy_index because by function contract if insertResultIntoBatch @@ -1842,15 +1425,6 @@ void NO_INLINE Aggregator::convertToBlockImplFinal( for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index) { - if constexpr (use_compiled_functions) - { - if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) - { - ++aggregate_functions_destroy_index; - continue; - } - } - bool is_state = aggregate_functions[aggregate_functions_destroy_index]->isState(); bool destroy_place_after_insert = !is_state && clear_states; if (destroy_place_after_insert) @@ -1877,137 +1451,124 @@ void NO_INLINE Aggregator::convertToBlockImplFinal( if (exception) std::rethrow_exception(exception); + + return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size()); } template -void NO_INLINE Aggregator::convertToBlockImplNotFinal( - Method & method, - Table & data, - std::vector key_columns, - AggregateColumnsData & aggregate_columns) const +Block NO_INLINE Aggregator::convertToBlockImplFinal( + Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows, bool clear_states, ConvertType type) const { + constexpr bool final = true; + auto out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, rows); + if constexpr (Method::low_cardinality_optimization) { if (data.hasNullKeyData()) { - key_columns[0]->insertDefault(); - - for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); - - data.getNullKeyData() = nullptr; + assert(type == ConvertType::Normal); + out_cols.key_columns[0]->insertDefault(); + insertAggregatesIntoColumns(data.getNullKeyData(), out_cols.final_aggregate_columns, arena, clear_states); } } - auto shuffled_key_sizes = method.shuffleKeyColumns(key_columns, key_sizes); - const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; + auto shuffled_key_sizes = method.shuffleKeyColumns(out_cols.raw_key_columns, key_sizes); + const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - data.forEachValue([&](const auto & key, auto & mapped) - { - method.insertKeyIntoColumns(key, key_columns, key_sizes_ref); + PaddedPODArray places; + places.reserve(rows); - /// reserved, so push_back does not throw exceptions - for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i]->push_back(mapped + offsets_of_aggregate_states[i]); + bool only_updates = (type == ConvertType::Updates); - /// proton: starts. For streaming aggr, we hold on to the states - /// Since it is not final, we shall never clear the state - /// if (!params.keep_state) - /// mapped = nullptr; - /// proton: ends. - }); -} + data.forEachValue([&](const auto & key, auto & mapped) + { + /// Ingore invalid mapped, there are two cases: + /// 1) mapped was destroyed (it's a bug) + /// 2) no mapped states for retracted data (means it's an new group key, but no retracted data) + if (!mapped) + return; + if (only_updates) + { + if (!TrackingUpdates::updated(mapped)) + return; -template -Block Aggregator::prepareBlockAndFill( - AggregatedDataVariants & data_variants, - bool final, - bool clear_states, - size_t rows, - Filler && filler) const -{ - 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); + /// Finalized it for current coverting + TrackingUpdates::resetUpdated(mapped); + } - Block header = getHeader(final); + /// For UDA with own emit strategy, there are two special cases to be handled: + /// 1. not all groups need to be emitted. therefore proton needs to pick groups + /// that should emits, and only emit those groups while keep other groups unchanged. + /// 2. a single block trigger multiple emits. In this case, proton need insert the + /// same key multiple times for each emit result of this group. - for (size_t i = 0; i < params.keys_size; ++i) - { - key_columns[i] = header.safeGetByPosition(i).type->createColumn(); - key_columns[i]->reserve(rows); - } + /// for non-UDA or UDA without emit strategy, 'should_emit' is always true. + /// For UDA with emit strategy, it is true only if the group should emit. + size_t emit_times = 1; + if (params.group_by == Params::GroupBy::USER_DEFINED) + { + assert(aggregate_functions.size() == 1); + emit_times = aggregate_functions[0]->getEmitTimes(mapped + offsets_of_aggregate_states[0]); + } - for (size_t i = 0; i < params.aggregates_size; ++i) - { - if (!final) + if (emit_times > 0) { - const auto & aggregate_column_name = params.aggregates[i].column_name; - aggregate_columns[i] = header.getByName(aggregate_column_name).type->createColumn(); + /// duplicate key for each emit + for (size_t i = 0; i < emit_times; i++) + method.insertKeyIntoColumns(key, out_cols.raw_key_columns, key_sizes_ref); - /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. - ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); + places.emplace_back(mapped); - /// proton: starts - column_aggregate_func.setKeepState(params.keep_state); - /// proton: ends + /// Mark the cell as destroyed so it will not be destroyed in destructor. + /// proton: starts. Here we push the `place` to `places`, for streaming + /// case, we don't want aggregate function to destroy the places + if (clear_states) + mapped = nullptr; + } + }); - /// Add arenas to ColumnAggregateFunction, which can result in moving ownership to it if reference count - /// get dropped in other places - for (auto & pool : data_variants.aggregates_pools) - column_aggregate_func.addArena(pool); + return insertResultsIntoColumns(places, std::move(out_cols), arena, clear_states); +} - aggregate_columns_data[i] = &column_aggregate_func.getData(); - aggregate_columns_data[i]->reserve(rows); - } - else +template +Block NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const +{ + constexpr bool final = false; + auto out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, rows); + + if constexpr (Method::low_cardinality_optimization) + { + if (data.hasNullKeyData()) { - final_aggregate_columns[i] = aggregate_functions[i]->getReturnType()->createColumn(); - final_aggregate_columns[i]->reserve(rows); + out_cols.raw_key_columns[0]->insertDefault(); - if (aggregate_functions[i]->isState()) - { - /// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states. - if (auto * column_aggregate_func = typeid_cast(final_aggregate_columns[i].get())) - for (auto & pool : data_variants.aggregates_pools) - column_aggregate_func->addArena(pool); + for (size_t i = 0; i < params.aggregates_size; ++i) + out_cols.aggregate_columns_data[i]->push_back(data.getNullKeyData() + offsets_of_aggregate_states[i]); - /// Aggregate state can be wrapped into array if aggregate function ends with -Resample combinator. - final_aggregate_columns[i]->forEachSubcolumn([&data_variants](IColumn::WrappedPtr & subcolumn) - { - if (auto * column_aggregate_func = typeid_cast(subcolumn.get())) - for (auto & pool : data_variants.aggregates_pools) - column_aggregate_func->addArena(pool); - }); - } + data.getNullKeyData() = nullptr; } } - filler(key_columns, aggregate_columns_data, final_aggregate_columns, final, clear_states); - - Block res = header.cloneEmpty(); - - for (size_t i = 0; i < params.keys_size; ++i) - res.getByPosition(i).column = std::move(key_columns[i]); + auto shuffled_key_sizes = method.shuffleKeyColumns(out_cols.raw_key_columns, key_sizes); + const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - for (size_t i = 0; i < params.aggregates_size; ++i) + data.forEachValue([&](const auto & key, auto & mapped) { - const auto & aggregate_column_name = params.aggregates[i].column_name; - if (final) - res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]); - else - res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]); - } + method.insertKeyIntoColumns(key, out_cols.raw_key_columns, key_sizes_ref); - /// Change the size of the columns-constants in the block. - size_t columns = header.columns(); - for (size_t i = 0; i < columns; ++i) - if (isColumnConst(*res.getByPosition(i).column)) - res.getByPosition(i).column = res.getByPosition(i).column->cut(0, rows); + /// reserved, so push_back does not throw exceptions + for (size_t i = 0; i < params.aggregates_size; ++i) + out_cols.aggregate_columns_data[i]->push_back(mapped + offsets_of_aggregate_states[i]); - return res; + /// proton: starts. For streaming aggr, we hold on to the states + /// Since it is not final, we shall never clear the state + /// if (!params.keep_state) + /// mapped = nullptr; + /// proton: ends. + }); + + return finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows); } void Aggregator::addSingleKeyToAggregateColumns( @@ -2034,23 +1595,7 @@ void Aggregator::addArenasToAggregateColumns( } } -void Aggregator::createStatesAndFillKeyColumnsWithSingleKey( - AggregatedDataVariants & data_variants, - Columns & key_columns, - size_t key_row, - MutableColumns & final_key_columns) const -{ - 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) - { - final_key_columns[i]->insertFrom(*key_columns[i].get(), key_row); - } -} - -Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows, bool clear_states) const +Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool clear_states, ConvertType type) const { /// proton: starts. if (!data_variants.without_key) @@ -2060,189 +1605,98 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va } /// proton: ends. + auto res_header = getHeader(final); size_t rows = 1; + auto && out_cols = prepareOutputBlockColumns(params, aggregate_functions, res_header, data_variants.aggregates_pools, final, rows); + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; - auto filler = [&data_variants, this]( - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - bool final_, - bool clear_states_) - { - if (data_variants.type == AggregatedDataVariants::Type::without_key || params.overflow_row) - { - AggregatedDataWithoutKey & data = data_variants.without_key; - - if (!data) - throw Exception("Wrong data variant passed.", ErrorCodes::LOGICAL_ERROR); - - if (!final_) - { - for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i]->push_back(data + offsets_of_aggregate_states[i]); + assert(data_variants.type == AggregatedDataVariants::Type::without_key); - /// proton: starts - if (clear_states_) - data = nullptr; - /// proton: ends - } - else - { - /// Always single-thread. It's safe to pass current arena from 'aggregates_pool'. - insertAggregatesIntoColumns(data, final_aggregate_columns, data_variants.aggregates_pool); - } + if (type == ConvertType::Updates && !TrackingUpdates::updated(data_variants.without_key)) + return res_header.cloneEmpty(); - if (params.overflow_row) - for (size_t i = 0; i < params.keys_size; ++i) - key_columns[i]->insertDefault(); + AggregatedDataWithoutKey & data = [&]() -> AggregateDataPtr & { + if (type == ConvertType::Updates) + { + TrackingUpdates::resetUpdated(data_variants.without_key); + return data_variants.without_key; } - }; + else + return data_variants.without_key; + }(); - Block block = prepareBlockAndFill(data_variants, final, clear_states, rows, filler); + if (!data) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong data variant passed."); - if (is_overflows) - block.info.is_overflows = true; + if (!final) + { + for (size_t i = 0; i < params.aggregates_size; ++i) + aggregate_columns_data[i]->push_back(data + offsets_of_aggregate_states[i]); + data = nullptr; + } + else + { + /// Always single-thread. It's safe to pass current arena from 'aggregates_pool'. + insertAggregatesIntoColumns(data, final_aggregate_columns, data_variants.aggregates_pool, clear_states); + } - return block; + return finalizeBlock(params, res_header, std::move(out_cols), final, rows); } -Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final, bool clear_states) const +Block Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final, bool clear_states, ConvertType type) const { - size_t rows = data_variants.sizeWithoutOverflowRow(); - - auto filler = [&data_variants, this]( - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - bool final_, - bool clear_states_) - { - #define M(NAME) \ - else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ - convertToBlockImpl(*data_variants.NAME, data_variants.NAME->data, \ - key_columns, aggregate_columns, final_aggregate_columns, data_variants.aggregates_pool, final_, clear_states_); - - if (false) {} // NOLINT - APPLY_FOR_VARIANTS_SINGLE_LEVEL_STREAMING(M) - #undef M - else - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - }; + const size_t rows = data_variants.sizeWithoutOverflowRow(); +#define M(NAME) \ + else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ + return convertToBlockImpl(*data_variants.NAME, data_variants.NAME->data, data_variants.aggregates_pool, data_variants.aggregates_pools, final, rows, clear_states, type); - return prepareBlockAndFill(data_variants, final, clear_states, rows, filler); + if (false) {} // NOLINT + APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) +#undef M + else throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant."); } - -BlocksList Aggregator::prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, size_t max_threads, bool clear_states) const +BlocksList Aggregator::prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, bool clear_states, size_t max_threads, ConvertType type) const { + /// TODO Make a custom threshold. + /// TODO Use the shared thread pool with the `merge` function. std::unique_ptr thread_pool; - if (max_threads > 1 && data_variants.sizeWithoutOverflowRow() > 100000 /// TODO Make a custom threshold. - && data_variants.isStaticBucketTwoLevel()) /// TODO Use the shared thread pool with the `merge` function. + if (max_threads > 1 && data_variants.sizeWithoutOverflowRow() > 100000 + && final && type == ConvertType::Normal) /// use single thread for non-final or retracted data or updated data thread_pool = std::make_unique(max_threads); + if (false) {} // NOLINT #define M(NAME) \ else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ - return prepareBlocksAndFillTwoLevelImpl(data_variants, *data_variants.NAME, final, clear_states, thread_pool.get()); + return prepareBlocksAndFillTwoLevelImpl(data_variants, *data_variants.NAME, final, clear_states, thread_pool.get(), type); - if (false) {} // NOLINT APPLY_FOR_VARIANTS_ALL_TWO_LEVEL(M) #undef M else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } - template BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( AggregatedDataVariants & data_variants, Method & method, bool final, bool clear_states, - ThreadPool * thread_pool) const + ThreadPool * thread_pool, + ConvertType type) const { - size_t max_threads = thread_pool ? thread_pool->getMaxThreads() : 1; - /// proton FIXME : separate final vs non-final converting. For non-final converting, we don't need - /// each arena for each thread. - for (size_t i = data_variants.aggregates_pools.size(); i < max_threads; ++i) - data_variants.aggregates_pools.push_back(std::make_shared()); - - auto buckets = method.data.buckets(); - std::atomic next_bucket_idx_to_merge = 0; - - auto converter = [&](size_t thread_id, ThreadGroupStatusPtr thread_group) - { - SCOPE_EXIT_SAFE( - if (thread_group) - CurrentThread::detachQueryIfNotDetached(); - ); - if (thread_group) - CurrentThread::attachToIfDetached(thread_group); - - BlocksList blocks; - while (true) - { - UInt32 bucket_idx = next_bucket_idx_to_merge.fetch_add(1); - - if (bucket_idx >= buckets.size()) - break; - - auto bucket = buckets[bucket_idx]; - 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(convertOneBucketToBlockImpl(data_variants, method, arena, final, clear_states, bucket)); - } - return blocks; - }; - - /// packaged_task is used to ensure that exceptions are automatically thrown into the main stream. - - std::vector> tasks(max_threads); - - try - { - for (size_t thread_id = 0; thread_id < max_threads; ++thread_id) - { - tasks[thread_id] = std::packaged_task( - [group = CurrentThread::getGroup(), thread_id, &converter] { return converter(thread_id, group); }); - - if (thread_pool) - thread_pool->scheduleOrThrowOnError([thread_id, &tasks] { tasks[thread_id](); }); - else - tasks[thread_id](); - } - } - catch (...) - { - /// 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. - if (thread_pool) - thread_pool->wait(); - - throw; - } - - if (thread_pool) - thread_pool->wait(); - - BlocksList blocks; - - for (auto & task : tasks) - { - if (!task.valid()) - continue; - - blocks.splice(blocks.end(), task.get_future().get()); - } + return convertBucketsInParallel(thread_pool, method.data.buckets(), [&](Int64 bucket, Arena * arena) -> BlocksList { + /// Skip no changed bucket if only updated is requested + if (type == ConvertType::Updates && !method.data.isBucketUpdated(bucket)) + return {}; - return blocks; + return {convertOneBucketToBlockImpl(data_variants, method, arena, final, clear_states, bucket, type)}; + }); } - -BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, bool final, ConvertAction action, size_t max_threads) const +BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, bool final, size_t max_threads) const { - LOG_TRACE(log, "Converting aggregated data to blocks"); + LOG_DEBUG(log, "Converting aggregated data to blocks"); Stopwatch watch; @@ -2252,29 +1706,18 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b if (data_variants.empty()) return blocks; - bool clear_states = shouldClearStates(action, final); + bool clear_states = final && !params.keep_state; - if (data_variants.without_key) - /// When without_key is setup, it doesn't necessary mean no GROUP BY keys, it may be overflow - blocks.emplace_back(prepareBlockAndFillWithoutKey( - data_variants, final, data_variants.type != AggregatedDataVariants::Type::without_key, clear_states)); - - if (data_variants.type != AggregatedDataVariants::Type::without_key) - { - if (data_variants.isTwoLevel()) - blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, max_threads, clear_states)); - else - blocks.emplace_back(prepareBlockAndFillSingleLevel(data_variants, final, clear_states)); - } + if (data_variants.type == AggregatedDataVariants::Type::without_key) + blocks.emplace_back(prepareBlockAndFillWithoutKey(data_variants, final, clear_states)); + else if (!data_variants.isTwoLevel()) + blocks.emplace_back(prepareBlockAndFillSingleLevel(data_variants, final, clear_states)); + else + blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, clear_states, max_threads)); /// proton: starts. if (clear_states) - { - /// `data_variants` will not destroy the states of aggregate functions in the destructor, - /// since already cleared up in `prepareBlocksAndFill...()` - data_variants.aggregator = nullptr; - clearDataVariants(data_variants); - } + data_variants.reset(); /// proton: ends size_t rows = 0; @@ -2287,7 +1730,7 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b } double elapsed_seconds = watch.elapsedSeconds(); - LOG_INFO(log, + LOG_DEBUG(log, "Converted aggregated data to blocks. {} rows, {} in {} sec. ({:.3f} rows/sec., {}/sec.)", rows, ReadableSize(bytes), elapsed_seconds, rows / elapsed_seconds, @@ -2296,7 +1739,6 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b return blocks; } - template void NO_INLINE Aggregator::mergeDataNullKey( Table & table_dst, @@ -2339,7 +1781,7 @@ void NO_INLINE Aggregator::mergeDataNullKey( } -template +template void NO_INLINE Aggregator::mergeDataImpl( Table & table_dst, Table & table_src, @@ -2352,7 +1794,6 @@ void NO_INLINE Aggregator::mergeDataImpl( auto func = [&](AggregateDataPtr & __restrict dst, AggregateDataPtr & __restrict src, bool inserted) { - /// proton: starts if (inserted) { /// If there are multiple sources, there are more than one AggregatedDataVariant. Aggregator always creates a new AggregatedDataVariant and merge all other @@ -2360,72 +1801,12 @@ void NO_INLINE Aggregator::mergeDataImpl( /// If it does not alloc new memory for the 'dst' (i.e. aggregate state of the new AggregatedDataVariant which get destroyed after finalize()) but reuse /// that from the 'src' to store the final aggregated result, it will cause the data from other AggregatedDataVariant will be merged multiple times and /// generate incorrect aggregated result. - dst = arena->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); -#if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - { - const auto & compiled_aggregate_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; - compiled_aggregate_functions.create_aggregate_states_function(dst); - if (compiled_aggregate_functions.functions_count != aggregate_functions.size()) - { - static constexpr bool skip_compiled_aggregate_functions = true; - createAggregateStates(dst); - } - -#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 = dst + 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 - } - else -#endif - { - createAggregateStates(dst); - } - } - /// proton: ends - -#if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - { - const auto & compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; - compiled_functions.merge_aggregate_states_function(dst, src); - - 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) -// { -// /// proton: starts -// if (!is_aggregate_function_compiled[i] && !params.streaming) -// aggregate_functions[i]->destroy(src + offsets_of_aggregate_states[i]); -// /// proton: ends -// } - } - } - else -#endif - { - 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); + auto aggregate_data = arena->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(aggregate_data); + dst = aggregate_data; } - if (clear_states) - destroyAggregateStates(src); + mergeAggregateStates(dst, src, arena, clear_states); }; if constexpr (std::is_same_v) @@ -2433,152 +1814,50 @@ void NO_INLINE Aggregator::mergeDataImpl( else table_src.mergeToViaEmplace(table_dst, func, std::move(key_handler)); + /// In order to release memory early. if (clear_states) table_src.clearAndShrink(); /// proton: ends } - -template -void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl( - Table & table_dst, - AggregatedDataWithoutKey & overflows, - Table & table_src, - Arena * arena, - bool clear_states) const +void NO_INLINE Aggregator::mergeWithoutKeyDataImpl(ManyAggregatedDataVariants & non_empty_data, bool clear_states) const { - /// Note : will create data for NULL key if not exist - if constexpr (Method::low_cardinality_optimization) - mergeDataNullKey(table_dst, table_src, arena, clear_states); + AggregatedDataVariantsPtr & res = non_empty_data[0]; - table_src.mergeToViaFind(table_dst, [&](AggregateDataPtr dst, AggregateDataPtr & src, bool found) + /// We merge all aggregation results to the first. + for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num) { - AggregateDataPtr res_data = found ? dst : overflows; - - for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_functions[i]->merge( - res_data + offsets_of_aggregate_states[i], - src + offsets_of_aggregate_states[i], - arena); + AggregatedDataVariants & current = *non_empty_data[result_num]; + mergeAggregateStates(res->without_key, current.without_key, res->aggregates_pool, clear_states); - /// proton : starts + /// In order to release memory early. if (clear_states) - destroyAggregateStates(src); - /// proton : ends - }); - - if (clear_states) - table_src.clearAndShrink(); + current.reset(); + } } -template -void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl( - Table & table_dst, - Table & table_src, - Arena * arena, - bool clear_states) const +template +void NO_INLINE Aggregator::mergeSingleLevelDataImpl(ManyAggregatedDataVariants & non_empty_data, bool clear_states) const { - /// Note : will create data for NULL key if not exist - if constexpr (Method::low_cardinality_optimization) - mergeDataNullKey(table_dst, table_src, arena, clear_states); - - table_src.mergeToViaFind(table_dst, - [&](AggregateDataPtr dst, AggregateDataPtr & src, bool found) - { - if (!found) - return; - - 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); - - /// proton : starts - if (clear_states) - destroyAggregateStates(src); - }); - - if (clear_states) - table_src.clearAndShrink(); - /// proton : ends -} - - -void NO_INLINE Aggregator::mergeWithoutKeyDataImpl(ManyAggregatedDataVariants & non_empty_data, bool clear_states) const -{ - AggregatedDataVariantsPtr & res = non_empty_data[0]; + AggregatedDataVariantsPtr & res = non_empty_data[0]; /// We merge all aggregation results to the first. for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num) { - /// proton: starts. - mergeAggregateStates(res->without_key, non_empty_data[result_num]->without_key, res->aggregates_pool, clear_states); - /// proton: ends. - } -} - - -template -void NO_INLINE Aggregator::mergeSingleLevelDataImpl(ManyAggregatedDataVariants & non_empty_data, bool clear_states) const -{ - AggregatedDataVariantsPtr & res = non_empty_data[0]; - bool no_more_keys = false; - - /// We merge all aggregation results to the first. - for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num) - { - if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys)) + if (!checkLimits(res->sizeWithoutOverflowRow())) break; AggregatedDataVariants & current = *non_empty_data[result_num]; - if (!no_more_keys) - { -#if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder) - { - mergeDataImpl( - getDataVariant(*res).data, - getDataVariant(current).data, - res->aggregates_pool, - clear_states); - } - else -#endif - { - mergeDataImpl( - getDataVariant(*res).data, - getDataVariant(current).data, - res->aggregates_pool, - clear_states); - } - } - else if (res->without_key) - { - mergeDataNoMoreKeysImpl( - getDataVariant(*res).data, - res->without_key, - getDataVariant(current).data, - res->aggregates_pool, - clear_states); - } - else - { - mergeDataOnlyExistingKeysImpl( - getDataVariant(*res).data, - getDataVariant(current).data, - res->aggregates_pool, - clear_states); - } + mergeDataImpl( + getDataVariant(*res).data, + getDataVariant(current).data, + res->aggregates_pool, + clear_states); + /// In order to release memory early. if (clear_states) - { - /// `current` will not destroy the states of aggregate functions in the destructor, - /// since already cleared up in `mergeData...Impl()` - current.aggregator = nullptr; - clearDataVariants(current); - } + current.reset(); } } @@ -2588,472 +1867,189 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl(ManyAggregatedDataVariants & APPLY_FOR_VARIANTS_SINGLE_LEVEL_STREAMING(M) #undef M -template -void NO_INLINE Aggregator::mergeBucketImpl( - ManyAggregatedDataVariants & data, bool final, bool clear_states, Int64 bucket, Arena * arena, std::atomic * is_cancelled) const -{ - /// We merge all aggregation results to the first. - AggregatedDataVariantsPtr & res = data[0]; - for (size_t result_num = 1, size = data.size(); result_num < size; ++result_num) - { - if (is_cancelled && is_cancelled->load(std::memory_order_seq_cst)) - return; - - AggregatedDataVariants & current = *data[result_num]; -#if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder) - { - mergeDataImpl( - getDataVariant(*res).data.impls[bucket], - getDataVariant(current).data.impls[bucket], - arena, - clear_states); - } - else -#endif - { - mergeDataImpl( - getDataVariant(*res).data.impls[bucket], - getDataVariant(current).data.impls[bucket], - arena, - clear_states); - } - } -} - -ManyAggregatedDataVariantsPtr Aggregator::prepareVariantsToMerge(ManyAggregatedDataVariants & data_variants, bool always_merge_into_empty) const -{ - if (data_variants.empty()) - throw Exception("Empty data passed to Aggregator::prepareVariantsToMerge.", ErrorCodes::EMPTY_DATA_PASSED); - - LOG_TRACE(log, "Merging aggregated data"); - - auto non_empty_data = std::make_shared(); - - /// proton: starts: - for (auto & data : data_variants) - if (!data->empty()) - non_empty_data->push_back(data); - - if (non_empty_data->empty()) - return non_empty_data; - - if (non_empty_data->size() > 1 || always_merge_into_empty) - { - /// When do streaming merging, we shall not touch existing memory arenas and - /// all memory arenas merge to the first empty one, so we need create a new resulting arena - /// at position 0. - auto result_variants = std::make_shared(false); - result_variants->aggregator = this; - initDataVariants(*result_variants, method_chosen, key_sizes, params); - initStatesForWithoutKeyOrOverflow(*result_variants); - non_empty_data->insert(non_empty_data->begin(), result_variants); - } - - /// for streaming query, we don't need sort the arenas -// if (non_empty_data.size() > 1) -// { -// /// Sort the states in descending order so that the merge is more efficient (since all states are merged into the first). -// std::sort(non_empty_data.begin(), non_empty_data.end(), -// [](const AggregatedDataVariantsPtr & lhs, const AggregatedDataVariantsPtr & rhs) -// { -// return lhs->sizeWithoutOverflowRow() > rhs->sizeWithoutOverflowRow(); -// }); -// } - /// proton: ends - - /// 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. - - /// proton: starts. The first variant is for result aggregating - bool has_two_level - = std::any_of(non_empty_data->begin(), non_empty_data->end(), [](const auto & variant) { return variant->isTwoLevel(); }); - - if (has_two_level) - { - for (auto & variant : *non_empty_data) - if (!variant->isTwoLevel()) - variant->convertToTwoLevel(); - } - - AggregatedDataVariantsPtr & first = non_empty_data->at(0); - - for (size_t i = 1, size = non_empty_data->size(); i < size; ++i) - { - if (first->type != non_empty_data->at(i)->type) - throw Exception("Cannot merge different aggregated data variants.", ErrorCodes::CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS); - /** Elements from the remaining sets can be moved to the first data set. - * Therefore, it must own all the arenas of all other sets. - */ - first->aggregates_pools.insert(first->aggregates_pools.end(), - non_empty_data->at(i)->aggregates_pools.begin(), non_empty_data->at(i)->aggregates_pools.end()); - } - - assert(first->aggregates_pools.size() == non_empty_data->size()); - - return non_empty_data; -} - -template -void NO_INLINE Aggregator::mergeStreamsImplCase( - Block & block, - Arena * aggregates_pool, - Method & method [[maybe_unused]], - Table & data, - AggregateDataPtr overflow_row) const +BlocksList +Aggregator::mergeAndConvertToBlocks(ManyAggregatedDataVariants & data_variants, bool final, size_t max_threads) const { - ColumnRawPtrs key_columns(params.keys_size); - AggregateColumnsConstData aggregate_columns(params.aggregates_size); - - /// Remember the columns we will work with - for (size_t i = 0; i < params.keys_size; ++i) - key_columns[i] = block.safeGetByPosition(i).column.get(); + auto prepared_data_ptr = prepareVariantsToMerge(data_variants); + if (prepared_data_ptr->empty()) + return {}; - for (size_t i = 0; i < params.aggregates_size; ++i) + bool clear_states = final && !params.keep_state; + BlocksList blocks; + auto & first = *prepared_data_ptr->at(0); + if (first.type == AggregatedDataVariants::Type::without_key) { - const auto & aggregate_column_name = params.aggregates[i].column_name; - aggregate_columns[i] = &typeid_cast(*block.getByName(aggregate_column_name).column).getData(); + mergeWithoutKeyDataImpl(*prepared_data_ptr, clear_states); + blocks.emplace_back(prepareBlockAndFillWithoutKey(first, final, clear_states)); } - - typename Method::State state(key_columns, key_sizes, aggregation_state_cache); - - /// For all rows. - size_t rows = block.rows(); - std::unique_ptr places(new AggregateDataPtr[rows]); - - for (size_t i = 0; i < rows; ++i) + else if (!first.isTwoLevel()) { - AggregateDataPtr aggregate_data = nullptr; - - if (!no_more_keys) - { - auto emplace_result = state.emplaceKey(data, i, *aggregates_pool); - 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(); - } - else - { - auto find_result = state.findKey(data, i, *aggregates_pool); - if (find_result.isFound()) - aggregate_data = find_result.getMapped(); - } - - /// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys. + if (false) { } // NOLINT +#define M(NAME) \ + else if (first.type == AggregatedDataVariants::Type::NAME) \ + mergeSingleLevelDataImpl(*prepared_data_ptr, clear_states); - AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row; - places[i] = value; - } + APPLY_FOR_VARIANTS_SINGLE_LEVEL_STREAMING(M) +#undef M + else throw Exception("Unknown single level aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - for (size_t j = 0; j < params.aggregates_size; ++j) - { - /// Merge state of aggregate functions. - aggregate_functions[j]->mergeBatch( - 0, rows, - places.get(), offsets_of_aggregate_states[j], - aggregate_columns[j]->data(), - aggregates_pool); + blocks.emplace_back(prepareBlockAndFillSingleLevel(first, final, clear_states)); } - - /// Early release memory. - block.clear(); -} - -template -void NO_INLINE Aggregator::mergeStreamsImpl( - Block & block, - Arena * aggregates_pool, - Method & method, - Table & data, - AggregateDataPtr overflow_row, - bool no_more_keys) const -{ - if (!no_more_keys) - mergeStreamsImplCase(block, aggregates_pool, method, data, overflow_row); else - mergeStreamsImplCase(block, aggregates_pool, method, data, overflow_row); -} - - -void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl( - Block & block, - AggregatedDataVariants & result) const -{ - AggregateColumnsConstData aggregate_columns(params.aggregates_size); - - /// Remember the columns we will work with - for (size_t i = 0; i < params.aggregates_size; ++i) { - const auto & aggregate_column_name = params.aggregates[i].column_name; - aggregate_columns[i] = &typeid_cast(*block.getByName(aggregate_column_name).column).getData(); - } + auto total_size = std::accumulate(prepared_data_ptr->begin(), prepared_data_ptr->end(), 0ull, [](size_t size, const auto & variants) { + return size + variants->sizeWithoutOverflowRow(); + }); + /// TODO Make a custom threshold. + /// TODO Use the shared thread pool with the `merge` function. + std::unique_ptr thread_pool; + if (max_threads > 1 && total_size > 100000 && final) + thread_pool = std::make_unique(max_threads); - AggregatedDataWithoutKey & res = result.without_key; - if (!res) - { - AggregateDataPtr place = result.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); - createAggregateStates(place); - res = place; - } + if (false) { } // NOLINT +#define M(NAME) \ + else if (first.type == AggregatedDataVariants::Type::NAME) \ + blocks = mergeAndConvertTwoLevelToBlocksImpl(*prepared_data_ptr, final, clear_states, thread_pool.get()); - for (size_t row = 0, rows = block.rows(); row < rows; ++row) - { - /// Adding Values - for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_functions[i]->merge(res + offsets_of_aggregate_states[i], (*aggregate_columns[i])[row], result.aggregates_pool); + APPLY_FOR_VARIANTS_ALL_TWO_LEVEL(M) +#undef M + else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } - /// Early release memory. - block.clear(); -} - -bool Aggregator::mergeOnBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys) const -{ - /// `result` will destroy the states of aggregate functions in the destructor - result.aggregator = this; - - /// How to perform the aggregation? - if (result.empty()) + if (clear_states) { - result.init(method_chosen); - result.keys_size = params.keys_size; - result.key_sizes = key_sizes; - LOG_TRACE(log, "Aggregation method: {}", result.getMethodName()); + for (auto & variants : *prepared_data_ptr) + variants->reset(); } - if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows) - mergeWithoutKeyStreamsImpl(block, result); - -#define M(NAME, IS_TWO_LEVEL) \ - else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(block, result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys); - - APPLY_FOR_AGGREGATED_VARIANTS_STREAMING(M) -#undef M - else if (result.type != AggregatedDataVariants::Type::without_key) - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - - return checkAndProcessResult(result, no_more_keys); + return blocks; } - -void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVariants & result, size_t max_threads) +template +BlocksList Aggregator::mergeAndConvertTwoLevelToBlocksImpl( + ManyAggregatedDataVariants & non_empty_data, bool final, bool clear_states, ThreadPool * thread_pool) const { - if (bucket_to_blocks.empty()) - return; - - UInt64 total_input_rows = 0; - for (auto & bucket : bucket_to_blocks) - for (auto & block : bucket.second) - total_input_rows += block.rows(); - - /** `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. - * If there is at least one block with a bucket number greater or equal than zero, then there was a two-level aggregation. - */ - auto max_bucket = bucket_to_blocks.rbegin()->first; - bool has_two_level = max_bucket >= 0; + auto & first = *non_empty_data.at(0); - if (has_two_level) + std::vector buckets; + if (first.isStaticBucketTwoLevel()) { - #define M(NAME) \ - if (method_chosen == AggregatedDataVariants::Type::NAME) \ - method_chosen = AggregatedDataVariants::Type::NAME ## _two_level; - - APPLY_FOR_VARIANTS_CONVERTIBLE_TO_STATIC_BUCKET_TWO_LEVEL(M) - - #undef M + buckets = getDataVariant(first).data.buckets(); } - - /// result will destroy the states of aggregate functions in the destructor - result.aggregator = this; - - result.init(method_chosen); - result.keys_size = params.keys_size; - result.key_sizes = key_sizes; - - bool has_blocks_with_unknown_bucket = bucket_to_blocks.contains(-1); - - /// First, parallel the merge for the individual buckets. Then we continue merge the data not allocated to the buckets. - if (has_two_level) + else { - /** 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. - */ - - LOG_TRACE(log, "Merging partially aggregated two-level data."); - - auto merge_bucket = [&bucket_to_blocks, &result, this](size_t bucket, Arena * aggregates_pool, ThreadGroupStatusPtr thread_group) - { - SCOPE_EXIT_SAFE( - if (thread_group) - CurrentThread::detachQueryIfNotDetached(); - ); - if (thread_group) - CurrentThread::attachToIfDetached(thread_group); - - for (Block & block : bucket_to_blocks[static_cast(bucket)]) - { - #define M(NAME) \ - else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(block, aggregates_pool, *result.NAME, result.NAME->data.impls[bucket], nullptr, false); - - if (false) {} // NOLINT - APPLY_FOR_VARIANTS_ALL_TWO_LEVEL(M) - #undef M - else - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - } - }; - - std::unique_ptr thread_pool; - if (max_threads > 1 && total_input_rows > 100000) /// TODO Make a custom threshold. - thread_pool = std::make_unique(max_threads); - - for (const auto & bucket_blocks : bucket_to_blocks) + assert(first.isTimeBucketTwoLevel()); + std::unordered_set buckets_set; + for (auto & data_variants : non_empty_data) { - const auto bucket = bucket_blocks.first; - - if (bucket == -1) - continue; - - result.aggregates_pools.push_back(std::make_shared()); - Arena * aggregates_pool = result.aggregates_pools.back().get(); - - auto task = [group = CurrentThread::getGroup(), bucket, &merge_bucket, aggregates_pool]{ return merge_bucket(bucket, aggregates_pool, group); }; - - if (thread_pool) - thread_pool->scheduleOrThrowOnError(task); - else - task(); + auto tmp_buckets = getDataVariant(*data_variants).data.buckets(); + buckets_set.insert(tmp_buckets.begin(), tmp_buckets.end()); } - - if (thread_pool) - thread_pool->wait(); - - LOG_TRACE(log, "Merged partially aggregated two-level data."); + buckets.assign(buckets_set.begin(), buckets_set.end()); } - if (has_blocks_with_unknown_bucket) - { - LOG_TRACE(log, "Merging partially aggregated single-level data."); - - 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) - mergeWithoutKeyStreamsImpl(block, result); + return convertBucketsInParallel(thread_pool, buckets, [&](Int64 bucket, Arena * arena) -> BlocksList { + mergeBucketImpl(non_empty_data, bucket, arena, clear_states); + return {convertOneBucketToBlockImpl(first, getDataVariant(first), arena, final, clear_states, bucket)}; + }); +} - #define M(NAME, IS_TWO_LEVEL) \ - else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(block, result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys); +template +void NO_INLINE Aggregator::mergeBucketImpl( + ManyAggregatedDataVariants & data, Int64 bucket, Arena * arena, bool clear_states, std::atomic * is_cancelled) const +{ + /// We merge all aggregation results to the first. + AggregatedDataVariantsPtr & res = data[0]; + for (size_t result_num = 1, size = data.size(); result_num < size; ++result_num) + { + if (is_cancelled && is_cancelled->load(std::memory_order_seq_cst)) + return; - APPLY_FOR_AGGREGATED_VARIANTS_STREAMING(M) - #undef M - else if (result.type != AggregatedDataVariants::Type::without_key) - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - } + AggregatedDataVariants & current = *data[result_num]; + mergeDataImpl( + getDataVariant(*res).data.impls[bucket], + getDataVariant(current).data.impls[bucket], + arena, + clear_states); - LOG_TRACE(log, "Merged partially aggregated single-level data."); + /// Assume the current bucket has been finalized. + getDataVariant(current).data.resetUpdatedBucket(bucket); } } - -Block Aggregator::mergeBlocks(BlocksList & blocks, bool final, ConvertAction action) +ManyAggregatedDataVariantsPtr Aggregator::prepareVariantsToMerge(ManyAggregatedDataVariants & data_variants, bool always_merge_into_empty) const { - if (blocks.empty()) - return {}; - - auto bucket_num = blocks.front().info.bucket_num; - bool is_overflows = blocks.front().info.is_overflows; - - LOG_TRACE(log, "Merging partially aggregated blocks (bucket = {}).", bucket_num); - Stopwatch watch; + if (data_variants.empty()) + throw Exception("Empty data passed to Aggregator::prepareVariantsToMerge.", ErrorCodes::EMPTY_DATA_PASSED); - /** 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. - */ - auto merge_method = method_chosen; + LOG_TRACE(log, "Merging aggregated data"); -#define APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION(M) \ - M(key64) \ - M(key_string) \ - M(key_fixed_string) \ - M(keys128) \ - M(keys256) \ - M(serialized) \ + auto non_empty_data = std::make_shared(); -#define M(NAME) \ - if (merge_method == AggregatedDataVariants::Type::NAME) \ - merge_method = AggregatedDataVariants::Type::NAME ## _hash64; \ + /// proton: starts: + for (auto & data : data_variants) + if (!data->empty()) + non_empty_data->push_back(data); - APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION(M) -#undef M + if (non_empty_data->empty()) + return non_empty_data; -#undef APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION + if (non_empty_data->size() > 1 || always_merge_into_empty) + { + /// When do streaming merging, we shall not touch existing memory arenas and + /// all memory arenas merge to the first empty one, so we need create a new resulting arena + /// at position 0. + auto result_variants = std::make_shared(false); + result_variants->aggregator = this; + initDataVariants(*result_variants, method_chosen, key_sizes, params); + initStatesForWithoutKey(*result_variants); + non_empty_data->insert(non_empty_data->begin(), result_variants); + } - /// Temporary data for aggregation. - AggregatedDataVariants result; + /// for streaming query, we don't need sort the arenas +// if (non_empty_data.size() > 1) +// { +// /// Sort the states in descending order so that the merge is more efficient (since all states are merged into the first). +// std::sort(non_empty_data.begin(), non_empty_data.end(), +// [](const AggregatedDataVariantsPtr & lhs, const AggregatedDataVariantsPtr & rhs) +// { +// return lhs->sizeWithoutOverflowRow() > rhs->sizeWithoutOverflowRow(); +// }); +// } + /// proton: ends - /// result will destroy the states of aggregate functions in the destructor - result.aggregator = this; + /// 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. - /// proton: starts - initDataVariants(result, method_chosen, key_sizes, params); - /// proton: ends + /// proton: starts. The first variant is for result aggregating + bool has_two_level + = std::any_of(non_empty_data->begin(), non_empty_data->end(), [](const auto & variant) { return variant->isTwoLevel(); }); - for (Block & block : blocks) + if (has_two_level) { - if (bucket_num >= 0 && block.info.bucket_num != bucket_num) - bucket_num = -1; + for (auto & variant : *non_empty_data) + if (!variant->isTwoLevel()) + variant->convertToTwoLevel(); + } - if (result.type == AggregatedDataVariants::Type::without_key || is_overflows) - mergeWithoutKeyStreamsImpl(block, result); + AggregatedDataVariantsPtr & first = non_empty_data->at(0); - #define M(NAME, IS_TWO_LEVEL) \ - else if (result.type == AggregatedDataVariants::Type::NAME) \ - mergeStreamsImpl(block, result.aggregates_pool, *result.NAME, result.NAME->data, nullptr, false); + for (size_t i = 1, size = non_empty_data->size(); i < size; ++i) + { + if (first->type != non_empty_data->at(i)->type) + throw Exception("Cannot merge different aggregated data variants.", ErrorCodes::CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS); - APPLY_FOR_AGGREGATED_VARIANTS_STREAMING(M) - #undef M - else if (result.type != AggregatedDataVariants::Type::without_key) - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); + /** Elements from the remaining sets can be moved to the first data set. + * Therefore, it must own all the arenas of all other sets. + */ + first->aggregates_pools.insert(first->aggregates_pools.end(), + non_empty_data->at(i)->aggregates_pools.begin(), non_empty_data->at(i)->aggregates_pools.end()); } - bool clear_states = shouldClearStates(action, final); - Block block; - if (result.type == AggregatedDataVariants::Type::without_key || is_overflows) - block = prepareBlockAndFillWithoutKey(result, final, is_overflows, clear_states); - else - block = prepareBlockAndFillSingleLevel(result, final, clear_states); - /// NOTE: two-level data is not possible here - chooseAggregationMethod chooses only among single-level methods. - - size_t rows = block.rows(); - size_t bytes = block.bytes(); - double elapsed_seconds = watch.elapsedSeconds(); - LOG_DEBUG(log, "Merged partially aggregated blocks. {} rows, {}. in {} sec. ({:.3f} rows/sec., {}/sec.)", - rows, ReadableSize(bytes), - elapsed_seconds, rows / elapsed_seconds, - ReadableSize(bytes / elapsed_seconds)); + assert(first->aggregates_pools.size() == non_empty_data->size()); - block.info.bucket_num = bucket_num; - return block; + return non_empty_data; } template @@ -3202,7 +2198,7 @@ void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result) cons LOG_TRACE(log, "Destroying aggregate states"); /// In what data structure is the data aggregated? - if (result.type == AggregatedDataVariants::Type::without_key || params.overflow_row) + if (result.type == AggregatedDataVariants::Type::without_key) destroyWithoutKey(result); #define M(NAME, IS_TWO_LEVEL) \ @@ -3217,9 +2213,9 @@ void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result) cons } /// proton: starts. for streaming processing -void Aggregator::initStatesForWithoutKeyOrOverflow(AggregatedDataVariants & data_variants) const +void Aggregator::initStatesForWithoutKey(AggregatedDataVariants & data_variants) const { - if (!data_variants.without_key && (params.overflow_row || data_variants.type == AggregatedDataVariants::Type::without_key)) + if (!data_variants.without_key && data_variants.type == AggregatedDataVariants::Type::without_key) { AggregateDataPtr place = data_variants.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); createAggregateStates(place); @@ -3321,33 +2317,38 @@ std::vector Aggregator::bucketsBefore(const AggregatedDataVariants & resu /// 1) The keys can reside in hashmap or in arena /// 2) The state can reside in arena or in the aggregation function /// And there is a special one which is group without key -void Aggregator::checkpoint(const AggregatedDataVariants & data_variants, WriteBuffer & wb) +void Aggregator::checkpoint(const AggregatedDataVariants & data_variants, WriteBuffer & wb) const { + auto version = getVersion(); /// Serialization layout /// [version] + [states layout] - VersionType version = getVersion(); writeIntBinary(version, wb); if (version <= 1) - return doCheckpointLegacy(data_variants, wb); - - return doCheckpoint(data_variants, wb); + return const_cast(this)->doCheckpointLegacy(data_variants, wb); + else if (version <= 2) + return doCheckpointV2(data_variants, wb); + else + return doCheckpointV3(data_variants, wb); } -void Aggregator::recover(AggregatedDataVariants & data_variants, ReadBuffer & rb) +void Aggregator::recover(AggregatedDataVariants & data_variants, ReadBuffer & rb) const { /// Serialization layout /// [version] + [states layout] VersionType recovered_version = 0; readIntBinary(recovered_version, rb); + assert(recovered_version <= getVersion()); + /// So far, no broken changes from `recovered_version` to `version`. /// FIXME: Legacy layout needs to be cleaned after no use if (recovered_version <= 1) - return doRecoverLegacy(data_variants, rb); - - /// Recover STATE V2 - return doRecover(data_variants, rb); + return const_cast(this)->doRecoverLegacy(data_variants, rb); + else if (recovered_version <= 2) + return doRecoverV2(data_variants, rb); + else + return doRecoverV3(data_variants, rb); } void Aggregator::doCheckpointLegacy(const AggregatedDataVariants & data_variants, WriteBuffer & wb) @@ -3370,7 +2371,7 @@ void Aggregator::doCheckpointLegacy(const AggregatedDataVariants & data_variants /// FIXME, set a good max_threads /// For ConvertAction::Checkpoint, don't clear state `data_variants` - auto blocks = convertToBlocks(const_cast(data_variants), false, ConvertAction::Checkpoint, 8); + auto blocks = convertToBlocks(const_cast(data_variants), false, 8); /// assert(!blocks.empty()); @@ -3473,7 +2474,7 @@ void Aggregator::recoverStatesWithoutKey(AggregatedDataVariants & data_variants, /// may have internal states as well assert(!data_variants.without_key); - initStatesForWithoutKeyOrOverflow(data_variants); + initStatesForWithoutKey(data_variants); AggregatedDataWithoutKey & data = data_variants.without_key; AggregateColumnsData aggregate_columns(params.aggregates_size); @@ -3614,7 +2615,7 @@ void Aggregator::recoverStatesTwoLevel(AggregatedDataVariants & data_variants, B /// The complexity of checkpoint the state of Aggregator is a combination of the following 2 cases /// 1) without key states (without_key or overflow rows) /// 2) hash table states -void Aggregator::doCheckpoint(const AggregatedDataVariants & data_variants, WriteBuffer & wb) +void Aggregator::doCheckpointV2(const AggregatedDataVariants & data_variants, WriteBuffer & wb) const { /// Serialization layout, there are 2 cases: /// 1) Without key: [uint8][uint16][aggr-func-state-without-key] @@ -3652,7 +2653,7 @@ void Aggregator::doCheckpoint(const AggregatedDataVariants & data_variants, Writ else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } -void Aggregator::doRecover(AggregatedDataVariants & data_variants, ReadBuffer & rb) +void Aggregator::doRecoverV2(AggregatedDataVariants & data_variants, ReadBuffer & rb) const { UInt8 inited = 0; readIntBinary(inited, rb); @@ -3687,58 +2688,19 @@ void Aggregator::doRecover(AggregatedDataVariants & data_variants, ReadBuffer & if (is_two_level && !data_variants.isTwoLevel()) data_variants.convertToTwoLevel(); - bool use_string_hash_map = data_variants.type == AggregatedDataVariants::Type::key_string - || data_variants.type == AggregatedDataVariants::Type::key_string_two_level - || data_variants.type == AggregatedDataVariants::Type::key_fixed_string - || data_variants.type == AggregatedDataVariants::Type::key_fixed_string_two_level; - /// [aggr-func-state-in-hash-map] if (false) { } // NOLINT #define M(NAME, IS_TWO_LEVEL) \ - else if (data_variants.type == AggregatedDataVariants::Type::NAME) { \ - if (use_string_hash_map) \ - DB::deserializeHashMap(data_variants.NAME->data, [this](auto & mapped, Arena & pool, ReadBuffer & rb_) { deserializeAggregateStates(mapped, rb_, &pool); }, *data_variants.aggregates_pool, rb); \ - else \ - DB::deserializeHashMap(data_variants.NAME->data, [this](auto & mapped, Arena & pool, ReadBuffer & rb_) { deserializeAggregateStates(mapped, rb_, &pool); }, *data_variants.aggregates_pool, rb); \ - } + else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ + DB::deserializeHashMap(data_variants.NAME->data, [this](auto & mapped, Arena & pool, ReadBuffer & rb_) { deserializeAggregateStates(mapped, rb_, &pool); }, *data_variants.aggregates_pool, rb); APPLY_FOR_AGGREGATED_VARIANTS_STREAMING(M) #undef M else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } -bool Aggregator::shouldClearStates(ConvertAction action, bool final_) const -{ - /// For streaming processing, data_variants.aggregator will never be nullptr once set - /// and we will never move the ownership of the states to `ColumnAggregateFunction` - /// unless we don't need keep the states - - switch (action) - { - case ConvertAction::DistributedMerge: - /// Distributed processing case. Only clear states on initiator - return final_; - case ConvertAction::WriteToTmpFS: - /// We are dumping all states to file system in case of memory is not efficient - /// In this case, we should not keep the states - return true; - case ConvertAction::Checkpoint: - /// Checkpoint is snapshot of in-memory states, we shall not clear the states - return false; - case ConvertAction::InternalMerge: - return false; - case ConvertAction::RetractedEmit: - return true; - case ConvertAction::StreamingEmit: - [[fallthrough]]; - default: - /// By default, streaming processing needs hold on to the states - return !params.keep_state; - } -} - VersionType Aggregator::getVersionFromRevision(UInt64 revision) const { if (revision >= STATE_V2_MIN_REVISION) @@ -3757,10 +2719,9 @@ template void NO_INLINE Aggregator::spliceBucketsImpl( AggregatedDataVariants & data_dest, AggregatedDataVariants & data_src, - bool final, - bool clear_states, const std::vector & gcd_buckets, - Arena * arena) const + Arena * arena, + bool clear_states) const { /// In order to merge state with same other keys of different gcd buckets, reset the window group keys to zero /// create a new key, where the window key part is 0, and the other key parts are the same as the original value. @@ -3794,37 +2755,29 @@ void NO_INLINE Aggregator::spliceBucketsImpl( auto & table_dest = getDataVariant(data_dest).data.impls; auto & table_src = getDataVariant(data_src).data.impls; -#if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder) - { - for (auto bucket : gcd_buckets) - mergeDataImpl(table_dest[0], table_src[bucket], arena, clear_states, zero_out_window_keys_func); - } - else -#endif - { - for (auto bucket : gcd_buckets) - mergeDataImpl(table_dest[0], table_src[bucket], arena, clear_states, zero_out_window_keys_func); - } + for (auto bucket : gcd_buckets) + mergeDataImpl(table_dest[0], table_src[bucket], arena, clear_states, zero_out_window_keys_func); } -Block Aggregator::spliceAndConvertBucketsToBlock( - AggregatedDataVariants & variants, bool final, ConvertAction action, const std::vector & gcd_buckets) const +Block Aggregator::spliceAndConvertBucketsToBlock(AggregatedDataVariants & variants, bool final, const std::vector & gcd_buckets) const { - AggregatedDataVariants result_variants; - result_variants.aggregator = this; - initDataVariants(result_variants, method_chosen, key_sizes, params); - initStatesForWithoutKeyOrOverflow(result_variants); - - auto method = result_variants.type; - Arena * arena = result_variants.aggregates_pool; - bool clear_states = shouldClearStates(action, final); + assert(variants.isTimeBucketTwoLevel()); + if (false) {} // NOLINT #define M(NAME) \ - else if (method == AggregatedDataVariants::Type::NAME) \ + else if (variants.type == AggregatedDataVariants::Type::NAME) \ { \ - spliceBucketsImpl(result_variants, variants, final, clear_states, gcd_buckets, arena); \ - return convertOneBucketToBlockImpl(result_variants, *result_variants.NAME, arena, final, clear_states, 0); \ + if (gcd_buckets.size() > 1) \ + { \ + AggregatedDataVariants result_variants; \ + result_variants.aggregator = this; \ + initDataVariants(result_variants, method_chosen, key_sizes, params); \ + initStatesForWithoutKey(result_variants); \ + spliceBucketsImpl(result_variants, variants, gcd_buckets, result_variants.aggregates_pool, /*clear_states*/ false); \ + return convertOneBucketToBlockImpl(result_variants, *result_variants.NAME, result_variants.aggregates_pool, final, /*clear_states*/ true, 0); \ + } \ + else \ + return convertOneBucketToBlockImpl(variants, *variants.NAME, variants.aggregates_pool, final, /*clear_states*/ false, gcd_buckets[0]); \ } APPLY_FOR_VARIANTS_TIME_BUCKET_TWO_LEVEL(M) @@ -3835,31 +2788,31 @@ Block Aggregator::spliceAndConvertBucketsToBlock( UNREACHABLE(); } -Block Aggregator::mergeAndSpliceAndConvertBucketsToBlock( - ManyAggregatedDataVariants & variants, bool final, ConvertAction action, const std::vector & gcd_buckets) const +Block Aggregator::mergeAndSpliceAndConvertBucketsToBlock(ManyAggregatedDataVariants & variants, bool final, const std::vector & gcd_buckets) const { - auto prepared_data = prepareVariantsToMerge(variants); + bool need_splice = gcd_buckets.size() > 1; + auto prepared_data = prepareVariantsToMerge(variants, /*always_merge_into_empty*/ need_splice); if (prepared_data->empty()) return {}; - AggregatedDataVariants result_variants; - result_variants.aggregator = this; - initDataVariants(result_variants, method_chosen, key_sizes, params); - initStatesForWithoutKeyOrOverflow(result_variants); - - auto method = result_variants.type; - Arena * arena = result_variants.aggregates_pool; - bool clear_states = shouldClearStates(action, final); + auto & first = *prepared_data->at(0); + assert(first.isTimeBucketTwoLevel()); + Arena * arena = first.aggregates_pool; if (false) {} // NOLINT #define M(NAME) \ - else if (method == AggregatedDataVariants::Type::NAME) \ + else if (first.type == AggregatedDataVariants::Type::NAME) \ { \ - using Method = decltype(result_variants.NAME)::element_type; \ + using Method = decltype(first.NAME)::element_type; \ for (auto bucket : gcd_buckets) \ - mergeBucketImpl(*prepared_data, final, clear_states, bucket, arena); \ - spliceBucketsImpl(result_variants, *prepared_data->at(0), final, clear_states, gcd_buckets, arena); \ - return convertOneBucketToBlockImpl(result_variants, *result_variants.NAME, arena, final, clear_states, 0); \ + mergeBucketImpl(*prepared_data, bucket, arena, /*clear_states*/ false); \ + if (need_splice) \ + { \ + spliceBucketsImpl(first, first, gcd_buckets, arena, /*clear_states*/ true); \ + return convertOneBucketToBlockImpl(first, *first.NAME, arena, final, /*clear_states*/ true, 0); \ + } \ + else \ + return convertOneBucketToBlockImpl(first, *first.NAME, arena, final, /*clear_states*/ false, gcd_buckets[0]); \ } APPLY_FOR_VARIANTS_TIME_BUCKET_TWO_LEVEL(M) @@ -3994,8 +2947,7 @@ std::pair Aggregator::executeAndRetractOnBlock( AggregatedDataVariants & result, AggregatedDataVariants & retracted_result, ColumnRawPtrs & key_columns, - AggregateColumns & aggregate_columns, - bool & no_more_keys) const + AggregateColumns & aggregate_columns) const { std::pair return_result = {false, false}; auto & need_abort = return_result.first; @@ -4019,8 +2971,6 @@ std::pair Aggregator::executeAndRetractOnBlock( AggregateFunctionInstructions aggregate_functions_instructions; prepareAggregateInstructions(columns, aggregate_columns, materialized_columns, aggregate_functions_instructions, nested_columns_holder); - assert(!params.overflow_row && !no_more_keys); - retracted_result.aggregator = this; if (result.type == AggregatedDataVariants::Type::without_key) { @@ -4032,13 +2982,13 @@ std::pair Aggregator::executeAndRetractOnBlock( if (result.without_key) { - initStatesForWithoutKeyOrOverflow(retracted_result); + initStatesForWithoutKey(retracted_result); mergeAggregateStates(retracted_result.without_key, result.without_key, retracted_result.aggregates_pool, false); } } - initStatesForWithoutKeyOrOverflow(result); - need_finalization = executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool); + initStatesForWithoutKey(result); + need_finalization = executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool); } else { @@ -4057,7 +3007,7 @@ std::pair Aggregator::executeAndRetractOnBlock( #undef M } - need_abort = checkAndProcessResult(result, no_more_keys); + need_abort = checkAndProcessResult(result); /// it's possible for gloabl single level hash table was converted to two level table after `checkAndProcessResult`, /// so we also convert retarcted data to two level. if (result.isTwoLevel() && !retracted_result.isTwoLevel()) @@ -4105,8 +3055,6 @@ void Aggregator::mergeRetractedGroupsImpl( AggregatedDataVariantsPtr & res = aggregated_data[0]; AggregatedDataVariantsPtr & retracted_res = retracted_data[0]; - bool no_more_keys = false; - using Table = typename Method::Data; Table & dst_table = getDataVariant(*res).data; Table & dst_retracted_table = getDataVariant(*retracted_res).data; @@ -4125,11 +3073,9 @@ void Aggregator::mergeRetractedGroupsImpl( /// `dst_retracted` <= (thread-1: group-1) + (thread-2: group-2) for (size_t result_num = 1, size = retracted_data.size(); result_num < size; ++result_num) { - if (!checkLimits(retracted_res->sizeWithoutOverflowRow(), no_more_keys)) + if (!checkLimits(retracted_res->sizeWithoutOverflowRow())) break; - assert(!no_more_keys); - auto & src_retracted_table = getDataVariant(*retracted_data[result_num]).data; src_retracted_table.mergeToViaEmplace(dst_retracted_table, [&](AggregateDataPtr & __restrict dst, AggregateDataPtr & __restrict src, bool inserted) { if (inserted) @@ -4143,11 +3089,9 @@ void Aggregator::mergeRetractedGroupsImpl( /// `dst_retracted` <= (thread-1: group-2) + (thread-2: group-1) for (size_t result_num = 1, size = retracted_data.size(); result_num < size; ++result_num) { - if (!checkLimits(retracted_res->sizeWithoutOverflowRow(), no_more_keys)) + if (!checkLimits(retracted_res->sizeWithoutOverflowRow())) break; - assert(!no_more_keys); - auto & current_retracted = *retracted_data[result_num]; Table & src_retracted_table = getDataVariant(current_retracted).data; Table & src_aggregated_table = getDataVariant(*aggregated_data[result_num]).data; @@ -4165,17 +3109,16 @@ void Aggregator::mergeRetractedGroupsImpl( }}); /// Reset retracted data after finalization - clearDataVariants(current_retracted); + current_retracted.reset(); } /// 3) Merge new/updated groups (based on all changed groups) /// `dst` <= (thread-1: group-1 group-2) + (thread-2: group-1 group-2) for (size_t result_num = 1, size = aggregated_data.size(); result_num < size; ++result_num) { - if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys)) + if (!checkLimits(res->sizeWithoutOverflowRow())) break; - assert(!no_more_keys); Table & src_aggregated_table = getDataVariant(*aggregated_data[result_num]).data; dst_retracted_table.forEachValue([&](const auto & key, auto & mapped) { /// Merge new/updated groups @@ -4192,7 +3135,7 @@ void Aggregator::mergeRetractedGroupsImpl( if (inserted) dst_it->getMapped() = nullptr; - + auto find_it = src_aggregated_table.find(key); if (find_it) mergeAggregateStates( @@ -4211,8 +3154,9 @@ void Aggregator::mergeAggregateStates(AggregateDataPtr & dst, AggregateDataPtr & if (!dst) { - dst = arena->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); - createAggregateStates(dst); + auto aggregate_data = arena->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(aggregate_data); + dst = aggregate_data; } for (size_t i = 0; i < params.aggregates_size; ++i) @@ -4263,30 +3207,117 @@ void Aggregator::deserializeAggregateStates(AggregateDataPtr & place, ReadBuffer } } -void Aggregator::clearDataVariants(AggregatedDataVariants & data_variants) const +void Aggregator::doCheckpointV3(const AggregatedDataVariants & data_variants, WriteBuffer & wb) const { - /// Clear states - destroyAllAggregateStates(data_variants); + /// Serialization layout, there are 2 cases: + /// 1) Without key: [uint8][uint16][aggr-func-state-without-key] + /// 2) Otherwise: [uint8][uint16][aggr-func-state-for-overflow-row][is_two_level][aggr-func-state-in-hash-map] + bool inited = !data_variants.empty(); + writeBinary(inited, wb); + if (!inited) + return; /// No aggregated data yet - /// Clear hash map - switch (data_variants.type) - { - case AggregatedDataVariants::Type::EMPTY: break; - case AggregatedDataVariants::Type::without_key: break; + writeIntBinary(static_cast(data_variants.type), wb); - #define M(NAME, IS_TWO_LEVEL) \ - case AggregatedDataVariants::Type::NAME: data_variants.NAME.reset(); break; - APPLY_FOR_AGGREGATED_VARIANTS_STREAMING(M) - #undef M + writeIntBinary(static_cast(trackingUpdatesType()), wb); + + auto state_serializer = [this](auto place, auto & wb_) { + assert(place); + if (trackingUpdatesType() == TrackingUpdatesType::Updates) + TrackingUpdates::serialize(place, wb_); + + for (size_t i = 0; i < params.aggregates_size; ++i) + aggregate_functions[i]->serialize(place + offsets_of_aggregate_states[i], wb_); + }; + + /// [aggr-func-state-without-key] + if (data_variants.type == AggregatedDataVariants::Type::without_key) + state_serializer(data_variants.without_key, wb); + + /// [aggr-func-state-in-hash-map] +#define M(NAME, IS_TWO_LEVEL) \ + else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ + { \ + if constexpr (IS_TWO_LEVEL) \ + DB::serializeTwoLevelHashMap(data_variants.NAME->data, [&](const auto & mapped, WriteBuffer & wb_) { state_serializer(mapped, wb_); }, wb); \ + else \ + DB::serializeHashMap(data_variants.NAME->data, [&](const auto & mapped, WriteBuffer & wb_) { state_serializer(mapped, wb_); }, wb); \ } - data_variants.invalidate(); - /// Reset pool - data_variants.aggregates_pools = Arenas(1, std::make_shared()); - data_variants.aggregates_pool = data_variants.aggregates_pools.back().get(); + APPLY_FOR_AGGREGATED_VARIANTS_STREAMING(M) +#undef M + else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); +} + +void Aggregator::doRecoverV3(AggregatedDataVariants & data_variants, ReadBuffer & rb) const +{ + bool inited = !data_variants.empty(); + readBinary(inited, rb); + if (!inited) + return; + + UInt8 recovered_data_variants_type_uint8; + readIntBinary(recovered_data_variants_type_uint8, rb); + AggregatedDataVariants::Type recovered_data_variants_type = static_cast(recovered_data_variants_type_uint8); + + data_variants.aggregator = this; + initDataVariants(data_variants, method_chosen, key_sizes, params); + /// Data variants is inited with single level hashmap, however the checkpoint states are 2 levels + /// which means data variants was converted to two level + if (data_variants.type != recovered_data_variants_type) + if (data_variants.isConvertibleToTwoLevel()) + data_variants.convertToTwoLevel(); + + if (data_variants.type != recovered_data_variants_type) + throw Exception( + ErrorCodes::RECOVER_CHECKPOINT_FAILED, + "Failed to recover aggregation checkpoint. Aggregated data variant type is not compatible, checkpointed={}, current={}", + magic_enum::enum_name(recovered_data_variants_type), + magic_enum::enum_name(method_chosen)); + + UInt8 recovered_expanded_data_type_uint8; + readIntBinary(recovered_expanded_data_type_uint8, rb); + TrackingUpdatesType recovered_expanded_data_type = static_cast(recovered_expanded_data_type_uint8); + if (recovered_expanded_data_type != trackingUpdatesType()) + throw Exception( + ErrorCodes::RECOVER_CHECKPOINT_FAILED, + "Failed to recover aggregation checkpoint. Expanded data type is not the same, checkpointed={}, current={}", + magic_enum::enum_name(recovered_expanded_data_type), + magic_enum::enum_name(trackingUpdatesType())); + + auto state_deserializer = [this](auto & place, auto & rb_, Arena * arena) { + place = nullptr; /// exception-safety - if you can not allocate memory or create states, then destructors will not be called. + auto aggregate_data = arena->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(aggregate_data); + place = aggregate_data; + + if (trackingUpdatesType() == TrackingUpdatesType::Updates) + TrackingUpdates::deserialize(place, rb_); + + for (size_t i = 0; i < params.aggregates_size; ++i) + aggregate_functions[i]->deserialize(place + offsets_of_aggregate_states[i], rb_, std::nullopt, arena); + }; + + /// [aggr-func-state-without-key] + if (data_variants.type == AggregatedDataVariants::Type::without_key) + state_deserializer(data_variants.without_key, rb, data_variants.aggregates_pool); + + /// [aggr-func-state-in-hash-map] +#define M(NAME, IS_TWO_LEVEL) \ + else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ + { \ + if constexpr (IS_TWO_LEVEL) \ + DB::deserializeTwoLevelHashMap(data_variants.NAME->data, [&](auto & mapped, Arena & pool, ReadBuffer & rb_) { state_deserializer(mapped, rb_, &pool); }, *data_variants.aggregates_pool, rb); \ + else \ + DB::deserializeHashMap(data_variants.NAME->data, [&](auto & mapped, Arena & pool, ReadBuffer & rb_) { state_deserializer(mapped, rb_, &pool); }, *data_variants.aggregates_pool, rb); \ + } + + APPLY_FOR_AGGREGATED_VARIANTS_STREAMING(M) +#undef M + else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } -bool Aggregator::checkAndProcessResult(AggregatedDataVariants & result, bool & no_more_keys) const +bool Aggregator::checkAndProcessResult(AggregatedDataVariants & result) const { size_t result_size = result.sizeWithoutOverflowRow(); Int64 current_memory_usage = 0; @@ -4307,7 +3338,7 @@ bool Aggregator::checkAndProcessResult(AggregatedDataVariants & result, bool & n result.convertToTwoLevel(); /// Checking the constraints. - if (!checkLimits(result_size, no_more_keys)) + if (!checkLimits(result_size)) return true; /** Flush data to disk if too much RAM is consumed. @@ -4340,6 +3371,141 @@ bool Aggregator::checkAndProcessResult(AggregatedDataVariants & result, bool & n return false; } +BlocksList Aggregator::convertUpdatesToBlocks(AggregatedDataVariants & data_variants) const +{ + LOG_DEBUG(log, "Converting updated aggregated data to blocks"); + + Stopwatch watch; + + BlocksList blocks; + + /// In what data structure is the data aggregated? + if (data_variants.empty()) + return blocks; + + constexpr bool final = true; + constexpr bool clear_states = false; + if (data_variants.type == AggregatedDataVariants::Type::without_key) + blocks.emplace_back(prepareBlockAndFillWithoutKey(data_variants, final, clear_states, ConvertType::Updates)); + else if (!data_variants.isTwoLevel()) + blocks.emplace_back(prepareBlockAndFillSingleLevel(data_variants, final, clear_states, ConvertType::Updates)); + else + blocks.splice(blocks.end(), prepareBlocksAndFillTwoLevel(data_variants, final, clear_states, /*max_threads*/ 1, ConvertType::Updates)); + + size_t rows = 0; + size_t bytes = 0; + + for (const auto & block : blocks) + { + rows += block.rows(); + bytes += block.bytes(); + } + + double elapsed_seconds = watch.elapsedSeconds(); + LOG_DEBUG(log, + "Converted updated aggregated data to blocks. {} rows, {} in {} sec. ({:.3f} rows/sec., {}/sec.)", + rows, ReadableSize(bytes), + elapsed_seconds, rows / elapsed_seconds, + ReadableSize(bytes / elapsed_seconds)); + + return blocks; +} + + +template +void NO_INLINE Aggregator::mergeUpdateGroupsImpl(ManyAggregatedDataVariants & non_empty_data, Arena * arena) const +{ + AggregatedDataVariantsPtr & res = non_empty_data[0]; + auto & dst_table = getDataVariant(*res).data; + /// Always merge updated data into empty first. + assert(dst_table.empty()); + + /// For example: + /// thread-1 thread-2 + /// group-1 updated non-updated + /// group-2 non-updated updated + /// group-3 non-updated non-updated + /// + /// 1) Collect all updated groups + /// `dst` <= (group-1, group-2) + using Table = typename Method::Data; + for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num) + { + if (!checkLimits(res->sizeWithoutOverflowRow())) + break; + + auto & src_table = getDataVariant(*non_empty_data[result_num]).data; + auto merge_updated_func = [&](const auto & key, auto & mapped) { + /// Skip no updated group + if (!TrackingUpdates::updated(mapped)) + return; + + typename Table::LookupResult dst_it; + bool inserted; + /// For StringRef `key`, it is safe to store to `dst_table` + /// since the `dst_table` is temporary and the `src_table` will not be cleaned in the meantime + dst_table.emplace(key, dst_it, inserted); + if (inserted) + { + auto & dst = dst_it->getMapped(); + dst = nullptr; /// exception-safety - if you can not allocate memory or create states, then destructors will not be called. + auto aggregate_data = arena->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(aggregate_data, /*prefix_with_updates_tracking_state=*/ false); + dst = aggregate_data; + } + }; + + if constexpr (is_two_level) + src_table.forEachValueOfUpdatedBuckets(std::move(merge_updated_func), /*reset_updated*/ true); + else + src_table.forEachValue(std::move(merge_updated_func)); + } + + /// 2) Merge all updated groups parts for each thread (based on `1)` ) + /// `dst` <= (thread-1: group-1 group-2) + (thread-2: group-1 group-2) + for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num) + { + auto & src_table = getDataVariant(*non_empty_data[result_num]).data; + dst_table.forEachValue([&](const auto & key, auto & mapped) { + if (auto find_it = src_table.find(key)) + { + mergeAggregateStates(mapped, find_it->getMapped(), arena, /*clear_states*/ false); + /// NOTE: We always reset the updated flag after merged + TrackingUpdates::resetUpdated(find_it->getMapped()); + } + }); + } +} + +AggregatedDataVariantsPtr Aggregator::mergeUpdateGroups(ManyAggregatedDataVariants & data_variants) const +{ + auto prepared_data_ptr = prepareVariantsToMerge(data_variants, /*always_merge_into_empty*/ true); + if (prepared_data_ptr->empty()) + return {}; + + BlocksList blocks; + auto & first = *prepared_data_ptr->at(0); + if (first.type == AggregatedDataVariants::Type::without_key) + { + if (std::ranges::none_of(*prepared_data_ptr, [](auto & variants) { + return variants->without_key && TrackingUpdates::updated(variants->without_key); + })) + return {}; + + mergeWithoutKeyDataImpl(*prepared_data_ptr, /*clear_states*/ false); + } + +#define M(NAME, IS_TWO_LEVEL) \ + else if (first.type == AggregatedDataVariants::Type::NAME) \ + mergeUpdateGroupsImpl(*prepared_data_ptr, first.aggregates_pool); + + APPLY_FOR_AGGREGATED_VARIANTS_STREAMING(M) +#undef M + else throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); + + return prepared_data_ptr->at(0); +} + void Aggregator::updateMetrics(const AggregatedDataVariants & variants, AggregatedDataMetrics & metrics) const { switch (variants.type) diff --git a/src/Interpreters/Streaming/Aggregator.h b/src/Interpreters/Streaming/Aggregator.h index e90535dea75..37aeac85bfe 100644 --- a/src/Interpreters/Streaming/Aggregator.h +++ b/src/Interpreters/Streaming/Aggregator.h @@ -37,11 +37,13 @@ #include #include #include +#include #include #include #include #include #include +#include #include /// proton: ends @@ -74,15 +76,10 @@ namespace Streaming * best suited for different cases, and this approach is just one of them, chosen for a combination of reasons. */ -enum class ConvertAction : uint8_t +enum class ConvertType : uint8_t { - Unkonwn = 0, - DistributedMerge, - WriteToTmpFS, - Checkpoint, - StreamingEmit, - InternalMerge, - RetractedEmit + Normal = 0, + Updates = 1, }; /// using TimeBucketAggregatedDataWithUInt16Key = TimeBucketHashMap>; @@ -103,7 +100,8 @@ using TimeBucketAggregatedDataWithKeys256TwoLevel = TimeBucketHashMap()); + aggregates_pool = aggregates_pools.back().get(); + /// Enable GC for arena by default. For cases like global aggregation, we will disable it further in \init + aggregates_pool->enableRecycle(true); + } + /// Number of rows (different keys). size_t size() const { @@ -558,12 +567,17 @@ struct AggregatedDataVariants : private boost::noncopyable throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } } + + void serialize(WriteBuffer & wb, const Aggregator & aggregator_) const; + void deserialize(ReadBuffer & rb, const Aggregator & aggregator_); }; using AggregatedDataVariantsPtr = std::shared_ptr; using ManyAggregatedDataVariants = std::vector; using ManyAggregatedDataVariantsPtr = std::shared_ptr; +struct OutputBlockColumns; + /** How are "total" values calculated with WITH TOTALS? * (For more details, see TotalsHavingTransform.) * @@ -650,6 +664,8 @@ class Aggregator final size_t window_keys_num; WindowParamsPtr window_params; + + TrackingUpdatesType tracking_updates_type; /// proton: ends /// proton: starts @@ -670,7 +686,8 @@ class Aggregator final GroupBy streaming_group_by_ = GroupBy::OTHER, ssize_t delta_col_pos_ = -1, size_t window_keys_num_ = 0, - WindowParamsPtr window_params_ = nullptr) + WindowParamsPtr window_params_ = nullptr, + TrackingUpdatesType tracking_updates_type_ = TrackingUpdatesType::None) : src_header(src_header_), intermediate_header(intermediate_header_), keys(keys_), aggregates(aggregates_), keys_size(keys.size()), aggregates_size(aggregates.size()), @@ -687,7 +704,8 @@ class Aggregator final group_by(streaming_group_by_), delta_col_pos(delta_col_pos_), window_keys_num(window_keys_num_), - window_params(window_params_) + window_params(window_params_), + tracking_updates_type(tracking_updates_type_) { } /// proton: ends @@ -727,17 +745,21 @@ class Aggregator final /// Process one block. Return {should_abort, need_finalization} pair /// should_abort: if the processing should be aborted (with group_by_overflow_mode = 'break') return true, otherwise false. /// need_finalization : only for UDA aggregation. If there is no UDA, always false - std::pair executeOnBlock(const Block & block, + std::pair executeOnBlock( + const Block & block, AggregatedDataVariants & result, ColumnRawPtrs & key_columns, - AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block - bool & no_more_keys) const; + AggregateColumns & aggregate_columns /// Passed to not create them anew for each block + ) const; - std::pair executeOnBlock(Columns columns, - size_t row_begin, size_t row_end, + std::pair executeOnBlock( + Columns columns, + size_t row_begin, + size_t row_end, AggregatedDataVariants & result, - ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block - bool & no_more_keys) const; + ColumnRawPtrs & key_columns, + AggregateColumns & aggregate_columns /// Passed to not create them anew for each block + ) const; /// Execute and retract state for changed groups: /// 1) For new group: @@ -759,14 +781,10 @@ class Aggregator final AggregatedDataVariants & result, AggregatedDataVariants & retracted_result, ColumnRawPtrs & key_columns, - AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block - bool & no_more_keys) const; - - bool mergeOnBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys) const; + AggregateColumns & aggregate_columns /// Passed to not create them anew for each block + ) const; /** Convert the aggregation data structure into a block. - * If overflow_row = true, then aggregates for rows that are not included in max_rows_to_group_by are put in the first block. - * * If final = false, then ColumnAggregateFunction is created as the aggregation columns with the state of the calculations, * which can then be combined with other states (for distributed query processing or checkpoint). * If final = true, then columns with ready values are created as aggregate columns. @@ -786,22 +804,30 @@ class Aggregator final * a. SELECT count(), avg(i), sum(k) FROM ( <-- second level global aggr, need prune its state at this level * SELECT avg(i) AS i, sum(k) AS k FROM my_stream GROUP BY device_id <-- first level global aggr, don't prune states * ); + * + * \param max_threads - limits max threads for converting two level aggregate state in parallel */ - BlocksList convertToBlocks(AggregatedDataVariants & data_variants, bool final, ConvertAction action, size_t max_threads) const; - BlocksList mergeAndConvertToBlocks(ManyAggregatedDataVariants & data_variants, bool final, ConvertAction action, size_t max_threads) const; - - Block convertOneBucketToBlock(AggregatedDataVariants & data_variants, bool final, ConvertAction action, size_t bucket) const; - Block mergeAndConvertOneBucketToBlock(ManyAggregatedDataVariants & variants, bool final, ConvertAction action, size_t bucket) const; + BlocksList convertToBlocks(AggregatedDataVariants & data_variants, bool final, size_t max_threads) const; + BlocksList mergeAndConvertToBlocks(ManyAggregatedDataVariants & data_variants, bool final, size_t max_threads) const; - /// Used by hop window function, merge multiple gcd windows (buckets) to a hop window + /// For Tumble/Session window function, there is only one bucket + /// For Hop window function, merge multiple gcd windows (buckets) to a hop window /// For examples: /// gcd_bucket1 - [00:00, 00:02) /// => result block - [00:00, 00:04) /// gcd_bucket2 - [00:02, 00:04) - Block spliceAndConvertBucketsToBlock( - AggregatedDataVariants & variants, bool final, ConvertAction action, const std::vector & gcd_buckets) const; - Block mergeAndSpliceAndConvertBucketsToBlock( - ManyAggregatedDataVariants & variants, bool final, ConvertAction action, const std::vector & gcd_buckets) const; + Block spliceAndConvertBucketsToBlock(AggregatedDataVariants & variants, bool final, const std::vector & gcd_buckets) const; + Block mergeAndSpliceAndConvertBucketsToBlock(ManyAggregatedDataVariants & variants, bool final, const std::vector & gcd_buckets) const; + + /// Only convert the states of update groups tracked + BlocksList convertUpdatesToBlocks(AggregatedDataVariants & data_variants) const; + + /// \return: merged updated data if exists, when there is no update data, return nullptr + AggregatedDataVariantsPtr mergeUpdateGroups(ManyAggregatedDataVariants & data_variants) const; + + /// For some streaming queries with `emit on update` or `emit changelog`, need tracking updates (with retract) + bool needTrackUpdates() const { return params.tracking_updates_type != TrackingUpdatesType::None; } + TrackingUpdatesType trackingUpdatesType() const { return params.tracking_updates_type; } /// Used for merge changed groups and return the of changed groups std::pair @@ -813,22 +839,12 @@ class Aggregator final /// If @p always_merge_into_empty is true, always add an empty variants at front even if there is only one ManyAggregatedDataVariantsPtr prepareVariantsToMerge(ManyAggregatedDataVariants & data_variants, bool always_merge_into_empty = false) const; - using BucketToBlocks = std::map; - /// Merge partially aggregated blocks separated to buckets into one data structure. - void mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVariants & result, size_t max_threads); - - /// Merge several partially aggregated blocks into one. - /// Precondition: for all blocks block.info.is_overflows flag must be the same. - /// (either all blocks are from overflow data or none blocks are). - /// The resulting block has the same value of is_overflows flag. - Block mergeBlocks(BlocksList & blocks, bool final, ConvertAction action); - /** Split block with partially-aggregated data to many blocks, as if two-level method of aggregation was used. * This is needed to simplify merging of that data with other results, that are already two-level. */ std::vector convertBlockToTwoLevel(const Block & block) const; - void initStatesForWithoutKeyOrOverflow(AggregatedDataVariants & data_variants) const; + void initStatesForWithoutKey(AggregatedDataVariants & data_variants) const; /// For external aggregation. void writeToTemporaryFile(AggregatedDataVariants & data_variants, const String & tmp_path) const; @@ -933,8 +949,7 @@ class Aggregator final /** Create states of aggregate functions for one key. */ - template - void createAggregateStates(AggregateDataPtr & aggregate_data) const; + void createAggregateStates(AggregateDataPtr & aggregate_data, bool prefix_with_updates_tracking_state = true) const; /** Call `destroy` methods for states of aggregate functions. * Used in the exception handler for aggregation, since RAII in this case is not applicable. @@ -946,35 +961,19 @@ class Aggregator final size_t row_begin, size_t row_end, ColumnRawPtrs & key_columns, - AggregateFunctionInstruction * aggregate_instructions, - bool no_more_keys, - AggregateDataPtr overflow_row = nullptr) const; + AggregateFunctionInstruction * aggregate_instructions) const; /// Process one data block, aggregate the data into a hash table. template - bool executeImpl( - Method & method, - Arena * aggregates_pool, - size_t row_begin, - size_t row_end, - ColumnRawPtrs & key_columns, - AggregateFunctionInstruction * aggregate_instructions, - bool no_more_keys, - AggregateDataPtr overflow_row) const; - - /// Specialization for a particular value no_more_keys. - template bool executeImplBatch( Method & method, - typename Method::State & state, Arena * aggregates_pool, size_t row_begin, size_t row_end, - AggregateFunctionInstruction * aggregate_instructions, - AggregateDataPtr overflow_row) const; + ColumnRawPtrs & key_columns, + AggregateFunctionInstruction * aggregate_instructions) const; /// For case when there are no keys (all aggregate into one row). For UDA with own strategy, return 'true' means the UDA should emit after execution - template bool executeWithoutKeyImpl( AggregatedDataWithoutKey & res, size_t row_begin, @@ -982,14 +981,6 @@ class Aggregator final AggregateFunctionInstruction * aggregate_instructions, Arena * arena) const; - static void executeOnIntervalWithoutKeyImpl( - AggregatedDataWithoutKey & res, - size_t row_begin, - size_t row_end, - AggregateFunctionInstruction * aggregate_instructions, - Arena * arena, - const IColumn * delta_col); - template void writeToTemporaryFileImpl( AggregatedDataVariants & data_variants, @@ -1006,7 +997,7 @@ class Aggregator final /// Merge data from hash table `src` into `dst`. using EmptyKeyHandler = void *; - template + template void mergeDataImpl( Table & table_dst, Table & table_src, @@ -1014,68 +1005,31 @@ class Aggregator final bool clear_states, KeyHandler && key_handler = nullptr) const; - /// Merge data from hash table `src` into `dst`, but only for keys that already exist in dst. In other cases, merge the data into `overflows`. - template - void mergeDataNoMoreKeysImpl( - Table & table_dst, - AggregatedDataWithoutKey & overflows, - Table & table_src, - Arena * arena, - bool clear_states) const; - - /// Same, but ignores the rest of the keys. - template - void mergeDataOnlyExistingKeysImpl( - Table & table_dst, - Table & table_src, - Arena * arena, - bool clear_states) const; - void mergeWithoutKeyDataImpl(ManyAggregatedDataVariants & non_empty_data, bool clear_states) const; template void mergeSingleLevelDataImpl(ManyAggregatedDataVariants & non_empty_data, bool clear_states) const; template - void convertToBlockImpl( - Method & method, - Table & data, - MutableColumns & key_columns, - AggregateColumnsData & aggregate_columns, - MutableColumns & final_aggregate_columns, - Arena * arena, - bool final, - bool clear_states) const; + Block convertToBlockImpl( + Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool final, size_t rows, bool clear_states, ConvertType type) const; template void insertAggregatesIntoColumns( Mapped & mapped, MutableColumns & final_aggregate_columns, - Arena * arena) const; - - template - void convertToBlockImplFinal( - Method & method, - Table & data, - std::vector key_columns, - MutableColumns & final_aggregate_columns, Arena * arena, bool clear_states) const; + Block insertResultsIntoColumns( + PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena, bool clear_states) const; + template - void convertToBlockImplNotFinal( - Method & method, - Table & data, - std::vector key_columns, - AggregateColumnsData & aggregate_columns) const; + Block convertToBlockImplFinal( + Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows, bool clear_states, ConvertType type) const; - template - Block prepareBlockAndFill( - AggregatedDataVariants & data_variants, - bool final, - bool clear_states, - size_t rows, - Filler && filler) const; + template + Block convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t rows) const; template Block convertOneBucketToBlockImpl( @@ -1084,26 +1038,31 @@ class Aggregator final Arena * arena, bool final, bool clear_states, - size_t bucket) const; + Int64 bucket, + ConvertType type = ConvertType::Normal) const; /// proton: starts. template void spliceBucketsImpl( AggregatedDataVariants & data_dest, AggregatedDataVariants & data_src, - bool final, - bool clear_states, const std::vector & gcd_buckets, - Arena * arena) const; + Arena * arena, + bool clear_states) const; template BlocksList mergeAndConvertTwoLevelToBlocksImpl( - ManyAggregatedDataVariants & non_empty_data, bool final, size_t max_threads, bool clear_states) const; + ManyAggregatedDataVariants & non_empty_data, bool final, bool clear_states, ThreadPool * thread_pool) const; - Block mergeAndConvertWithoutKeyToBlock(ManyAggregatedDataVariants & non_empty_data, bool final, bool clear_states) const; - Block mergeAndConvertSingleLevelToBlock(ManyAggregatedDataVariants & non_empty_data, bool final, bool clear_states) const; - BlocksList - mergeAndConvertTwoLevelToBlocks(ManyAggregatedDataVariants & non_empty_data, bool final, size_t max_threads, bool clear_states) const; + void mergeAggregateStates(AggregateDataPtr & dst, AggregateDataPtr & src, Arena * arena, bool clear_states) const; + + void destroyAggregateStates(AggregateDataPtr & place) const; + + void serializeAggregateStates(const AggregateDataPtr & place, WriteBuffer & wb) const; + void deserializeAggregateStates(AggregateDataPtr & place, ReadBuffer & rb, Arena * arena) const; + + /// \return true means execution must be aborted, false means normal + bool checkAndProcessResult(AggregatedDataVariants & result) const; template bool executeAndRetractImpl( @@ -1119,55 +1078,21 @@ class Aggregator final template void mergeRetractedGroupsImpl(ManyAggregatedDataVariants & aggregated_data, ManyAggregatedDataVariants & retracted_data) const; - void mergeAggregateStates(AggregateDataPtr & dst, AggregateDataPtr & src, Arena * arena, bool clear_states) const; - - void destroyAggregateStates(AggregateDataPtr & place) const; - - void serializeAggregateStates(const AggregateDataPtr & place, WriteBuffer & wb) const; - void deserializeAggregateStates(AggregateDataPtr & place, ReadBuffer & rb, Arena * arena) const; - - void clearDataVariants(AggregatedDataVariants & data_variants) const; - - /// @return does need abort ? - bool checkAndProcessResult(AggregatedDataVariants & result, bool & no_more_keys) const; + template + void mergeUpdateGroupsImpl(ManyAggregatedDataVariants & non_empty_data, Arena * arena) const; /// proton: ends. - Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows, bool clear_states) const; - Block prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final, bool clear_states) const; - BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, size_t max_threads, bool clear_states) const; + Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool clear_states, ConvertType type = ConvertType::Normal) const; + Block prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final, bool clear_states, ConvertType type = ConvertType::Normal) const; + BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, bool clear_states, size_t max_threads, ConvertType type = ConvertType::Normal) const; template BlocksList prepareBlocksAndFillTwoLevelImpl( - AggregatedDataVariants & data_variants, - Method & method, - bool final, - bool clear_states, - ThreadPool * thread_pool) const; - - template - void mergeStreamsImplCase( - Block & block, - Arena * aggregates_pool, - Method & method, - Table & data, - AggregateDataPtr overflow_row) const; - - template - void mergeStreamsImpl( - Block & block, - Arena * aggregates_pool, - Method & method, - Table & data, - AggregateDataPtr overflow_row, - bool no_more_keys) const; - - void mergeWithoutKeyStreamsImpl( - Block & block, - AggregatedDataVariants & result) const; + AggregatedDataVariants & data_variants, Method & method, bool final, bool clear_states, ThreadPool * thread_pool, ConvertType type) const; template void mergeBucketImpl( - ManyAggregatedDataVariants & data, bool final, bool clear_states, Int64 bucket, Arena * arena, std::atomic * is_cancelled = nullptr) const; + ManyAggregatedDataVariants & data, Int64 bucket, Arena * arena, bool clear_states, std::atomic * is_cancelled = nullptr) const; template void convertBlockToTwoLevelImpl( @@ -1188,9 +1113,8 @@ class Aggregator final * If it is exceeded, then, depending on the group_by_overflow_mode, either * - throws an exception; * - returns false, which means that execution must be aborted; - * - sets the variable no_more_keys to true. */ - bool checkLimits(size_t result_size, bool & no_more_keys) const; + bool checkLimits(size_t result_size) const; void prepareAggregateInstructions( Columns columns, @@ -1207,30 +1131,29 @@ class Aggregator final const AggregatedDataVariants & data_variants, MutableColumns & aggregate_columns) const; - void createStatesAndFillKeyColumnsWithSingleKey( - AggregatedDataVariants & data_variants, - Columns & key_columns, size_t key_row, - MutableColumns & final_key_columns) const; - /// proton: starts void setupAggregatesPoolTimestamps(size_t row_begin, size_t row_end, const ColumnRawPtrs & key_columns, Arena * aggregates_pool) const; - inline bool shouldClearStates(ConvertAction action, bool final_) const; +public: + /// Existed versions: + /// STATE V1 - Legacy version (REVISION 1) + /// STATE V2 - REVISION 1 (Enable revision) + /// STATE V3 - REVISION 3 (Add updates tracking state) + static constexpr UInt64 STATE_V2_MIN_REVISION = 1; + // static constexpr UInt64 STATE_V3_MIN_REVISION = 3; /// will enable it later VersionType getVersionFromRevision(UInt64 revision) const; VersionType getVersion() const; -public: - /// Existed versions: - /// STATE VERSION 1 - Legacy version - /// STATE VERSION 2 - REVISION 1 (Enable revision) - static constexpr UInt64 STATE_V2_MIN_REVISION = 1; + void checkpoint(const AggregatedDataVariants & data_variants, WriteBuffer & wb) const; + void recover(AggregatedDataVariants & data_variants, ReadBuffer & rb) const; - void checkpoint(const AggregatedDataVariants & data_variants, WriteBuffer & wb); - void recover(AggregatedDataVariants & data_variants, ReadBuffer & rb); +private: + void doCheckpointV3(const AggregatedDataVariants & data_variants, WriteBuffer & wb) const; + void doRecoverV3(AggregatedDataVariants & data_variants, ReadBuffer & rb) const; - void doCheckpoint(const AggregatedDataVariants & data_variants, WriteBuffer & wb); - void doRecover(AggregatedDataVariants & data_variants, ReadBuffer & rb); + void doCheckpointV2(const AggregatedDataVariants & data_variants, WriteBuffer & wb) const; + void doRecoverV2(AggregatedDataVariants & data_variants, ReadBuffer & rb) const; /// [Legacy] void doCheckpointLegacy(const AggregatedDataVariants & data_variants, WriteBuffer & wb); diff --git a/src/Interpreters/Streaming/HashJoin.cpp b/src/Interpreters/Streaming/HashJoin.cpp index 108cc609cc1..91da977f3eb 100644 --- a/src/Interpreters/Streaming/HashJoin.cpp +++ b/src/Interpreters/Streaming/HashJoin.cpp @@ -2546,9 +2546,9 @@ void HashJoin::serialize(WriteBuffer & wb, VersionType version) const /// Part-4: Buffered data of left/right join stream if (bidirectional_hash_join) - DB::serialize(left_data, wb, version); + left_data.serialize(wb, version); - DB::serialize(right_data, wb, version); + right_data.serialize(wb, version); /// Part-5: Asof type (Optional) bool need_asof = streaming_strictness == Strictness::Range || streaming_strictness == Strictness::Asof; @@ -2564,12 +2564,12 @@ void HashJoin::serialize(WriteBuffer & wb, VersionType version) const if (join_results.has_value()) { assert(retract_push_down && emit_changelog); - DB::serialize(*join_results, wb, version, *this); + join_results->serialize(wb, version, *this); } /// Part-7: Others DB::writeIntBinary(combined_watermark.load(), wb); - DB::serialize(join_metrics, wb, version); + join_metrics.serialize(wb, version); } void HashJoin::deserialize(ReadBuffer & rb, VersionType version) @@ -2664,9 +2664,9 @@ void HashJoin::deserialize(ReadBuffer & rb, VersionType version) /// Part-4: Buffered data of left/right join stream if (bidirectional_hash_join) - DB::deserialize(left_data, rb, version); + left_data.deserialize(rb, version); - DB::deserialize(right_data, rb, version); + right_data.deserialize(rb, version); /// Part-5: Asof type (Optional) bool need_asof = streaming_strictness == Strictness::Range || streaming_strictness == Strictness::Asof; @@ -2705,7 +2705,7 @@ void HashJoin::deserialize(ReadBuffer & rb, VersionType version) join_results.has_value()); assert(retract_push_down && emit_changelog); - DB::deserialize(*join_results, rb, version, *this); + join_results->deserialize(rb, version, *this); } /// Part-7: Others @@ -2713,7 +2713,7 @@ void HashJoin::deserialize(ReadBuffer & rb, VersionType version) DB::readIntBinary(recovered_combined_watermark, rb); combined_watermark = recovered_combined_watermark; - DB::deserialize(join_metrics, rb, version); + join_metrics.deserialize(rb, version); } void HashJoin::JoinResults::serialize(WriteBuffer & wb, VersionType version, const HashJoin & join) const @@ -2723,7 +2723,7 @@ void HashJoin::JoinResults::serialize(WriteBuffer & wb, VersionType version, con serializeHashJoinMapsVariants(blocks, *maps, wb, version, sample_block, join); if (version <= CachedBlockMetrics::SERDE_REQUIRED_MAX_VERSION) - DB::serialize(metrics, wb, version); + metrics.serialize(wb, version); } void HashJoin::JoinResults::deserialize(ReadBuffer & rb, VersionType version, const HashJoin & join) @@ -2733,7 +2733,7 @@ void HashJoin::JoinResults::deserialize(ReadBuffer & rb, VersionType version, co deserializeHashJoinMapsVariants(blocks, *maps, rb, version, pool, sample_block, join); if (version <= CachedBlockMetrics::SERDE_REQUIRED_MAX_VERSION) - DB::deserialize(metrics, rb, version); + metrics.deserialize(rb, version); } void HashJoin::JoinData::serialize(WriteBuffer & wb, VersionType version) const @@ -2748,7 +2748,7 @@ void HashJoin::JoinData::serialize(WriteBuffer & wb, VersionType version) const if (has_primary_key_hash_table) { SerializedRowRefListMultipleToIndices serialized_row_ref_list_multiple_to_indices; - DB::serialize(*buffered_data, wb, version, &serialized_row_ref_list_multiple_to_indices); + buffered_data->serialize(wb, version, &serialized_row_ref_list_multiple_to_indices); primary_key_hash_table->map.serialize( /*MappedSerializer*/ @@ -2758,7 +2758,7 @@ void HashJoin::JoinData::serialize(WriteBuffer & wb, VersionType version) const wb); } else - DB::serialize(*buffered_data, wb, version, nullptr); + buffered_data->serialize(wb, version, nullptr); } void HashJoin::JoinData::deserialize(ReadBuffer & rb, VersionType version) @@ -2789,7 +2789,7 @@ void HashJoin::JoinData::deserialize(ReadBuffer & rb, VersionType version) if (has_primary_key_hash_table) { DeserializedIndicesToRowRefListMultiple deserialized_indices_to_multiple_ref; - DB::deserialize(*buffered_data, rb, version, &deserialized_indices_to_multiple_ref); + buffered_data->deserialize(rb, version, &deserialized_indices_to_multiple_ref); primary_key_hash_table->map.deserialize( /*MappedDeserializer*/ @@ -2801,7 +2801,7 @@ void HashJoin::JoinData::deserialize(ReadBuffer & rb, VersionType version) rb); } else - DB::deserialize(*buffered_data, rb, version, nullptr); + buffered_data->deserialize(rb, version, nullptr); } void HashJoin::JoinGlobalMetrics::serialize(WriteBuffer & wb, VersionType) const @@ -2828,7 +2828,7 @@ void serializeHashJoinMapsVariants( SerializedRowRefListMultipleToIndices * serialized_row_ref_list_multiple_to_indices) { SerializedBlocksToIndices serialized_blocks_to_indices; - DB::serialize(blocks, wb, version, header, &serialized_blocks_to_indices); + blocks.serialize(wb, version, header, &serialized_blocks_to_indices); assert(maps.map_variants.size() >= 1); DB::writeIntBinary(static_cast(maps.map_variants.size()), wb); @@ -2883,7 +2883,7 @@ void deserializeHashJoinMapsVariants( DeserializedIndicesToRowRefListMultiple * deserialized_indices_to_multiple_ref) { DeserializedIndicesToBlocks deserialized_indices_to_blocks; - DB::deserialize(blocks, rb, version, header, &deserialized_indices_to_blocks); + blocks.deserialize(rb, version, header, &deserialized_indices_to_blocks); UInt16 maps_size; DB::readIntBinary(maps_size, rb); diff --git a/src/Interpreters/Streaming/TrackingUpdatesData.h b/src/Interpreters/Streaming/TrackingUpdatesData.h new file mode 100644 index 00000000000..bd44c79e3d7 --- /dev/null +++ b/src/Interpreters/Streaming/TrackingUpdatesData.h @@ -0,0 +1,105 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ +using AggregateDataPtr = char *; +using ConstAggregateDataPtr = const char *; + +namespace Streaming +{ +SERDE struct TrackingUpdates +{ + static ALWAYS_INLINE TrackingUpdates & data(AggregateDataPtr __restrict place) { return *reinterpret_cast(place); } + static ALWAYS_INLINE const TrackingUpdates & data(ConstAggregateDataPtr __restrict place) { return *reinterpret_cast(place); } + + static ALWAYS_INLINE bool empty(ConstAggregateDataPtr __restrict place) { return data(place).updates == 0; } + static ALWAYS_INLINE bool updated(ConstAggregateDataPtr __restrict place) { return data(place).updated_since_last_finalization; } + static ALWAYS_INLINE void setUpdated(AggregateDataPtr __restrict place) { data(place).updated_since_last_finalization = true; } + static ALWAYS_INLINE void resetUpdated(AggregateDataPtr __restrict place) { data(place).updated_since_last_finalization = false; } + + static void addBatch(size_t row_begin, size_t row_end, AggregateDataPtr * places, const IColumn * delta_col) + { + if (delta_col == nullptr) + { + for (size_t i = row_begin; i < row_end; ++i) + if (places[i]) + data(places[i]).add(); + } + else + { + const auto & delta_flags = assert_cast(*delta_col).getData(); + for (size_t i = row_begin; i < row_end; ++i) + { + if (places[i]) + { + if (delta_flags[i] >= 0) + data(places[i]).add(); + else + data(places[i]).negate(); + } + } + } + } + + static void addBatchSinglePlace(size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, const IColumn * delta_col) + { + if (!place) + return; + + auto & data_ex = data(place); + if (delta_col == nullptr) + data_ex.updates += row_end - row_begin; + else + { + const auto & delta_flags = assert_cast(*delta_col).getData(); + data_ex.updates = std::accumulate(delta_flags.begin(), delta_flags.end(), data_ex.updates); + } + + data_ex.updated_since_last_finalization = true; + } + + static void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & wb) + { + const auto & data_ex = data(place); + writeIntBinary(data_ex.updates, wb); + writeBinary(data_ex.updated_since_last_finalization, wb); + } + + static void deserialize(AggregateDataPtr __restrict place, ReadBuffer & rb) + { + auto & data_ex = data(place); + readIntBinary(data_ex.updates, rb); + readBinary(data_ex.updated_since_last_finalization, rb); + } + + ALWAYS_INLINE void add() + { + ++updates; + updated_since_last_finalization = true; + } + + ALWAYS_INLINE void negate() + { + --updates; + updated_since_last_finalization = true; + } + + /// Used to track if the target to be tracked has zero sum changes + UInt64 updates = 0; + + /// Used to track if the target group tracked has updates since last finalization + bool updated_since_last_finalization = true; +}; + +enum class TrackingUpdatesType : uint8_t +{ + None = 0, + Updates = 1, +}; + +} +} diff --git a/src/Interpreters/Streaming/joinData.cpp b/src/Interpreters/Streaming/joinData.cpp index 7896803d732..9c2853a3d31 100644 --- a/src/Interpreters/Streaming/joinData.cpp +++ b/src/Interpreters/Streaming/joinData.cpp @@ -326,18 +326,18 @@ void BufferedStreamData::serialize( DB::writeIntBinary(block_id, wb); assert(current_hash_blocks); - DB::serialize(*current_hash_blocks, wb, version, sample_block, *join, serialized_row_ref_list_multiple_to_indices); + current_hash_blocks->serialize(wb, version, sample_block, *join, serialized_row_ref_list_multiple_to_indices); DB::writeIntBinary(static_cast(range_bucket_hash_blocks.size()), wb); for (const auto & [bucket, hash_blocks] : range_bucket_hash_blocks) { DB::writeIntBinary(bucket, wb); assert(hash_blocks); - DB::serialize(*hash_blocks, wb, version, sample_block, *join, serialized_row_ref_list_multiple_to_indices); + hash_blocks->serialize(wb, version, sample_block, *join, serialized_row_ref_list_multiple_to_indices); } if (version <= CachedBlockMetrics::SERDE_REQUIRED_MAX_VERSION) - DB::serialize(metrics, wb, version); + metrics.serialize(wb, version); } void BufferedStreamData::deserialize( @@ -358,7 +358,7 @@ void BufferedStreamData::deserialize( DB::readIntBinary(block_id, rb); assert(current_hash_blocks); - DB::deserialize(*current_hash_blocks, rb, version, sample_block, *join, deserialized_indices_to_row_ref_list_multiple); + current_hash_blocks->deserialize(rb, version, sample_block, *join, deserialized_indices_to_row_ref_list_multiple); UInt32 size; Int64 bucket; @@ -370,11 +370,11 @@ void BufferedStreamData::deserialize( assert(inserted); /// Init hash table join->initHashMaps(iter->second->maps->map_variants); - DB::deserialize(*iter->second, rb, version, sample_block, *join, deserialized_indices_to_row_ref_list_multiple); + iter->second->deserialize(rb, version, sample_block, *join, deserialized_indices_to_row_ref_list_multiple); } if (version <= CachedBlockMetrics::SERDE_REQUIRED_MAX_VERSION) - DB::deserialize(metrics, rb, version); + metrics.deserialize(rb, version); } HashBlocksPtr BufferedStreamData::newHashBlocks() diff --git a/src/Interpreters/Streaming/tests/gtest_streaming_hash_join.cpp b/src/Interpreters/Streaming/tests/gtest_streaming_hash_join.cpp index 244eb64a6a6..7b0fdefedfe 100644 --- a/src/Interpreters/Streaming/tests/gtest_streaming_hash_join.cpp +++ b/src/Interpreters/Streaming/tests/gtest_streaming_hash_join.cpp @@ -258,14 +258,14 @@ std::shared_ptr initHashJoin( void serdeAndCheck(const Streaming::HashJoin & join, Streaming::HashJoin & recovered_join, std::string_view msg) { WriteBufferFromOwnString wb; - DB::serialize(join, wb, ProtonRevision::getVersionRevision()); + join.serialize(wb, ProtonRevision::getVersionRevision()); auto original_string = wb.str(); ReadBufferFromOwnString rb(original_string); - DB::deserialize(recovered_join, rb, ProtonRevision::getVersionRevision()); + recovered_join.deserialize(rb, ProtonRevision::getVersionRevision()); WriteBufferFromOwnString wb2; - DB::serialize(recovered_join, wb2, ProtonRevision::getVersionRevision()); + recovered_join.serialize(wb2, ProtonRevision::getVersionRevision()); auto recovered_string = wb2.str(); ASSERT_EQ(original_string, recovered_string) << msg << ": (FAILED)\n"; diff --git a/src/Processors/Transforms/Streaming/AggregatingHelper.cpp b/src/Processors/Transforms/Streaming/AggregatingHelper.cpp index 6fa6139d38a..c6bca59183f 100644 --- a/src/Processors/Transforms/Streaming/AggregatingHelper.cpp +++ b/src/Processors/Transforms/Streaming/AggregatingHelper.cpp @@ -30,12 +30,13 @@ Chunk mergeBlocksToChunk(BlocksList && blocks) return merged_chunk; } -Chunk convertToChunkImpl(AggregatedDataVariants & data, const AggregatingTransformParams & params, ConvertAction action) +Chunk convertToChunkImpl(AggregatedDataVariants & data, const AggregatingTransformParams & params) { if (data.empty()) return {}; - auto blocks = params.aggregator.convertToBlocks(data, params.final, action, params.params.max_threads); + BlocksList blocks = params.aggregator.convertToBlocks(data, params.final, params.params.max_threads); + /// FIXME: When global aggr states was converted two level hash table, the merged chunk may be too large return mergeBlocksToChunk(std::move(blocks)); } @@ -45,12 +46,12 @@ namespace AggregatingHelper { Chunk convertToChunk(AggregatedDataVariants & data, const AggregatingTransformParams & params) { - return convertToChunkImpl(data, params, ConvertAction::StreamingEmit); + return convertToChunkImpl(data, params); } Chunk mergeAndConvertToChunk(ManyAggregatedDataVariants & data, const AggregatingTransformParams & params) { - auto blocks = params.aggregator.mergeAndConvertToBlocks(data, params.final, ConvertAction::StreamingEmit, params.params.max_threads); + auto blocks = params.aggregator.mergeAndConvertToBlocks(data, params.final, params.params.max_threads); /// FIXME: When global aggr states was converted two level hash table, the merged chunk may be too large return mergeBlocksToChunk(std::move(blocks)); } @@ -58,58 +59,51 @@ Chunk mergeAndConvertToChunk(ManyAggregatedDataVariants & data, const Aggregatin Chunk spliceAndConvertBucketsToChunk( AggregatedDataVariants & data, const AggregatingTransformParams & params, const std::vector & buckets) { - if (buckets.size() == 1) - return convertToChunk(params.aggregator.convertOneBucketToBlock(data, params.final, ConvertAction::StreamingEmit, buckets[0])); - else - return convertToChunk(params.aggregator.spliceAndConvertBucketsToBlock(data, params.final, ConvertAction::InternalMerge, buckets)); + return convertToChunk(params.aggregator.spliceAndConvertBucketsToBlock(data, params.final, buckets)); } Chunk mergeAndSpliceAndConvertBucketsToChunk( ManyAggregatedDataVariants & data, const AggregatingTransformParams & params, const std::vector & buckets) { - if (buckets.size() == 1) - return convertToChunk( - params.aggregator.mergeAndConvertOneBucketToBlock(data, params.final, ConvertAction::StreamingEmit, buckets[0])); - else - return convertToChunk( - params.aggregator.mergeAndSpliceAndConvertBucketsToBlock(data, params.final, ConvertAction::InternalMerge, buckets)); + return convertToChunk(params.aggregator.mergeAndSpliceAndConvertBucketsToBlock(data, params.final, buckets)); } ChunkPair -convertToChangelogChunk(AggregatedDataVariants & data, RetractedDataVariants & retracted_data, const AggregatingTransformParams & params) -{ - if (data.empty()) - return {}; - - assert(!retracted_data.empty()); - - auto retracted_chunk = convertToChunkImpl(retracted_data, params, ConvertAction::RetractedEmit); - if (retracted_chunk) - { - auto retracted_delta_col = ColumnInt8::create(retracted_chunk.rows(), Int8(-1)); - retracted_chunk.addColumn(std::move(retracted_delta_col)); - retracted_chunk.setConsecutiveDataFlag(); - } - - auto chunk = convertToChunkImpl(data, params, ConvertAction::StreamingEmit); - if (chunk) - { - auto delta_col = ColumnInt8::create(chunk.rows(), Int8(1)); - chunk.addColumn(std::move(delta_col)); - } - - return {std::move(retracted_chunk), std::move(chunk)}; -} - -ChunkPair mergeAndConvertToChangelogChunk( - ManyAggregatedDataVariants & data, ManyRetractedDataVariants & retracted_data, const AggregatingTransformParams & params) -{ - auto [merged_data, merged_retracted_data] = params.aggregator.mergeRetractedGroups(data, retracted_data); - if (!merged_data) - return {}; - - assert(merged_retracted_data); - return convertToChangelogChunk(*merged_data, *merged_retracted_data, params); + convertToChangelogChunk(AggregatedDataVariants & data, RetractedDataVariants & retracted_data, const AggregatingTransformParams & params) + { + if (data.empty()) + return {}; + + assert(!retracted_data.empty()); + + auto retracted_chunk = convertToChunk(retracted_data, params); + if (retracted_chunk) + { + auto retracted_delta_col = ColumnInt8::create(retracted_chunk.rows(), Int8(-1)); + retracted_chunk.addColumn(std::move(retracted_delta_col)); + retracted_chunk.setConsecutiveDataFlag(); + } + retracted_data.reset(); /// Clean up retract data after finalized + + auto chunk = convertToChunk(data, params); + if (chunk) + { + auto delta_col = ColumnInt8::create(chunk.rows(), Int8(1)); + chunk.addColumn(std::move(delta_col)); + } + + return {std::move(retracted_chunk), std::move(chunk)}; + } + + ChunkPair mergeAndConvertToChangelogChunk( + ManyAggregatedDataVariants & data, ManyRetractedDataVariants & retracted_data, const AggregatingTransformParams & params) + { + auto [merged_data, merged_retracted_data] = params.aggregator.mergeRetractedGroups(data, retracted_data); + if (!merged_data) + return {}; + + assert(merged_retracted_data); + return convertToChangelogChunk(*merged_data, *merged_retracted_data, params); } } } diff --git a/src/Processors/Transforms/Streaming/AggregatingTransform.cpp b/src/Processors/Transforms/Streaming/AggregatingTransform.cpp index 629a047c1f3..1bd1c496237 100644 --- a/src/Processors/Transforms/Streaming/AggregatingTransform.cpp +++ b/src/Processors/Transforms/Streaming/AggregatingTransform.cpp @@ -189,22 +189,11 @@ void AggregatingTransform::consume(Chunk chunk) std::pair AggregatingTransform::executeOrMergeColumns(Chunk & chunk, size_t num_rows) { auto columns = chunk.detachColumns(); - if (params->only_merge) - { - auto block = getInputs().front().getHeader().cloneWithColumns(columns); - materializeBlockInplace(block); - /// FIXME - /// Blocking finalization during execution on current variant - std::lock_guard lock(variants_mutex); - auto success = params->aggregator.mergeOnBlock(block, variants, no_more_keys); - return {!success, false}; - } - else - { - /// Blocking finalization during execution on current variant - std::lock_guard lock(variants_mutex); - return params->aggregator.executeOnBlock(std::move(columns), 0, num_rows, variants, key_columns, aggregate_columns, no_more_keys); - } + assert(!params->only_merge && !no_more_keys); + + /// Blocking finalization during execution on current variant + std::lock_guard lock(variants_mutex); + return params->aggregator.executeOnBlock(std::move(columns), 0, num_rows, variants, key_columns, aggregate_columns); } void AggregatingTransform::emitVersion(Chunk & chunk) @@ -498,7 +487,7 @@ void AggregatingTransform::checkpoint(CheckpointContextPtr ckpt_ctx) } /// Serializing no shared data - params->aggregator.checkpoint(variants, wb); + variants.serialize(wb, params->aggregator); DB::writeIntBinary(watermark, wb); @@ -554,7 +543,7 @@ void AggregatingTransform::recover(CheckpointContextPtr ckpt_ctx) } /// Serializing local or stable data during checkpointing - params->aggregator.recover(variants, rb); + variants.deserialize(rb, params->aggregator); DB::readIntBinary(watermark, rb); diff --git a/src/Processors/Transforms/Streaming/AggregatingTransformWithSubstream.cpp b/src/Processors/Transforms/Streaming/AggregatingTransformWithSubstream.cpp index 918d1337658..208549bb6f4 100644 --- a/src/Processors/Transforms/Streaming/AggregatingTransformWithSubstream.cpp +++ b/src/Processors/Transforms/Streaming/AggregatingTransformWithSubstream.cpp @@ -226,10 +226,10 @@ std::pair AggregatingTransformWithSubstream::executeOrMergeColumns(C /// according to partition keys auto num_rows = chunk.getNumRows(); - assert(!params->only_merge); + assert(!params->only_merge && !no_more_keys); return params->aggregator.executeOnBlock( - chunk.detachColumns(), 0, num_rows, substream_ctx->variants, key_columns, aggregate_columns, no_more_keys); + chunk.detachColumns(), 0, num_rows, substream_ctx->variants, key_columns, aggregate_columns); } SubstreamContextPtr AggregatingTransformWithSubstream::getOrCreateSubstreamContext(const SubstreamID & id) @@ -294,7 +294,7 @@ void SubstreamContext::serialize(WriteBuffer & wb, VersionType version) const { DB::Streaming::serialize(id, wb); - aggregating_transform->params->aggregator.checkpoint(variants, wb); + variants.serialize(wb, aggregating_transform->params->aggregator); DB::writeIntBinary(finalized_watermark, wb); @@ -312,7 +312,7 @@ void SubstreamContext::deserialize(ReadBuffer & rb, VersionType version) { DB::Streaming::deserialize(id, rb); - aggregating_transform->params->aggregator.recover(variants, rb); + variants.deserialize(rb, aggregating_transform->params->aggregator); DB::readIntBinary(finalized_watermark, rb); diff --git a/src/Processors/Transforms/Streaming/ChangelogConvertTransform.cpp b/src/Processors/Transforms/Streaming/ChangelogConvertTransform.cpp index 2b9f8ceb183..7521957516b 100644 --- a/src/Processors/Transforms/Streaming/ChangelogConvertTransform.cpp +++ b/src/Processors/Transforms/Streaming/ChangelogConvertTransform.cpp @@ -407,7 +407,7 @@ void ChangelogConvertTransform::checkpoint(CheckpointContextPtr ckpt_ctx) { ckpt_ctx->coordinator->checkpoint(getVersion(), getLogicID(), ckpt_ctx, [this](WriteBuffer & wb) { SerializedBlocksToIndices serialized_blocks_to_indices; - DB::serialize(source_chunks, wb, getVersion(), getInputs().front().getHeader(), &serialized_blocks_to_indices); + source_chunks.serialize(wb, getVersion(), getInputs().front().getHeader(), &serialized_blocks_to_indices); index.serialize( /*MappedSerializer*/ @@ -420,7 +420,7 @@ void ChangelogConvertTransform::checkpoint(CheckpointContextPtr ckpt_ctx) DB::writeIntBinary(late_rows, wb); if (version <= CachedBlockMetrics::SERDE_REQUIRED_MAX_VERSION) - DB::serialize(cached_block_metrics, wb, getVersion()); + cached_block_metrics.serialize(wb, getVersion()); }); } @@ -428,7 +428,7 @@ void ChangelogConvertTransform::recover(CheckpointContextPtr ckpt_ctx) { ckpt_ctx->coordinator->recover(getLogicID(), ckpt_ctx, [this](VersionType version_, ReadBuffer & rb) { DeserializedIndicesToBlocks deserialized_indices_to_blocks; - DB::deserialize(source_chunks, rb, version_, getInputs().front().getHeader(), &deserialized_indices_to_blocks); + source_chunks.deserialize(rb, version_, getInputs().front().getHeader(), &deserialized_indices_to_blocks); index.deserialize( /*MappedDeserializer*/ @@ -442,7 +442,7 @@ void ChangelogConvertTransform::recover(CheckpointContextPtr ckpt_ctx) DB::readIntBinary(late_rows, rb); if (version_ <= CachedBlockMetrics::SERDE_REQUIRED_MAX_VERSION) - DB::deserialize(cached_block_metrics, rb, version_); + cached_block_metrics.deserialize(rb, version_); }); } } diff --git a/src/Processors/Transforms/Streaming/GlobalAggregatingTransform.cpp b/src/Processors/Transforms/Streaming/GlobalAggregatingTransform.cpp index 6d19e51fcc0..365c5621a37 100644 --- a/src/Processors/Transforms/Streaming/GlobalAggregatingTransform.cpp +++ b/src/Processors/Transforms/Streaming/GlobalAggregatingTransform.cpp @@ -57,7 +57,7 @@ GlobalAggregatingTransform::GlobalAggregatingTransform( const auto & data = std::any_cast(field); DB::writeIntBinary(data.size(), wb); for (const auto & elem : data) - params->aggregator.checkpoint(*elem, wb); + elem->serialize(wb, params->aggregator); }, /// Field deserializer [this](std::any & field, ReadBuffer & rb, VersionType) { @@ -68,7 +68,7 @@ GlobalAggregatingTransform::GlobalAggregatingTransform( for (auto & elem : data) { elem = std::make_shared(); - params->aggregator.recover(*elem, rb); + elem->deserialize(rb, params->aggregator); } }}); } @@ -102,7 +102,7 @@ std::pair GlobalAggregatingTransform::executeOrMergeColumns(Chunk & { if (params->emit_changelog) { - assert(!params->only_merge); + assert(!params->only_merge && !no_more_keys); auto & retracted_variants = many_data->getField()[current_variant]; auto & aggregated_variants = many_data->variants[current_variant]; @@ -110,7 +110,7 @@ std::pair GlobalAggregatingTransform::executeOrMergeColumns(Chunk & /// Blocking finalization during execution on current variant std::lock_guard lock(variants_mutex); return params->aggregator.executeAndRetractOnBlock( - chunk.detachColumns(), 0, num_rows, *aggregated_variants, *retracted_variants, key_columns, aggregate_columns, no_more_keys); + chunk.detachColumns(), 0, num_rows, *aggregated_variants, *retracted_variants, key_columns, aggregate_columns); } else return AggregatingTransform::executeOrMergeColumns(chunk, num_rows); diff --git a/src/Processors/Transforms/Streaming/GlobalAggregatingTransformWithSubstream.cpp b/src/Processors/Transforms/Streaming/GlobalAggregatingTransformWithSubstream.cpp index e223ee5b623..b682e02ea85 100644 --- a/src/Processors/Transforms/Streaming/GlobalAggregatingTransformWithSubstream.cpp +++ b/src/Processors/Transforms/Streaming/GlobalAggregatingTransformWithSubstream.cpp @@ -32,16 +32,16 @@ SubstreamContextPtr GlobalAggregatingTransformWithSubstream::getOrCreateSubstrea { substream_ctx->setField( {std::make_shared(), - /// Field serializer - [this](const std::any & field, WriteBuffer & wb, VersionType) { - const auto & data = std::any_cast(field); - params->aggregator.checkpoint(*data, wb); - }, - /// Field deserializer - [this](std::any & field, ReadBuffer & rb, VersionType) { - auto & data = std::any_cast(field); - params->aggregator.recover(*data, rb); - }}); + /// Field serializer + [this](const std::any & field, WriteBuffer & wb, VersionType) { + const auto & data = std::any_cast(field); + data->serialize(wb, params->aggregator); + }, + /// Field deserializer + [this](std::any & field, ReadBuffer & rb, VersionType) { + auto & data = std::any_cast(field); + data->deserialize(rb, params->aggregator); + }}); } return substream_ctx; } @@ -51,14 +51,14 @@ GlobalAggregatingTransformWithSubstream::executeOrMergeColumns(Chunk & chunk, co { if (params->emit_changelog) { - assert(!params->only_merge); + assert(!params->only_merge && !no_more_keys); auto num_rows = chunk.getNumRows(); auto & retracted_variants = substream_ctx->getField(); auto & aggregated_variants = substream_ctx->variants; return params->aggregator.executeAndRetractOnBlock( - chunk.detachColumns(), 0, num_rows, aggregated_variants, *retracted_variants, key_columns, aggregate_columns, no_more_keys); + chunk.detachColumns(), 0, num_rows, aggregated_variants, *retracted_variants, key_columns, aggregate_columns); } else return AggregatingTransformWithSubstream::executeOrMergeColumns(chunk, substream_ctx); diff --git a/tests/stream/test_stream_smoke/0001_view_case.json b/tests/stream/test_stream_smoke/0001_view_case.json index cc73303442b..b5eae39feb0 100644 --- a/tests/stream/test_stream_smoke/0001_view_case.json +++ b/tests/stream/test_stream_smoke/0001_view_case.json @@ -482,8 +482,8 @@ "steps":[ {"statements": [ {"client":"python", "query_type": "table", "query":"drop view if exists test1_mv_union"}, - {"client":"python","query_id":"300", "wait":3, "query_type": "table", "query":"create materialized view test1_mv_union as (select id, sum(value) as sum_value from test1_mv group by id limit 4 union select id, sum(value) as sum_value from test1_mv group by id limit 4)"}, - {"client":"python", "query_id":"301", "query_type": "stream","depends_on_stream":"test1_mv_union","wait":2,"query_end_timer":5,"drop_view":"test1_mv_union", "drop_view_wait":1, "query":"select id, sum_value from test1_mv_union settings seek_to='earliest'"} + {"client":"python", "wait":3, "query_type": "table", "query":"create materialized view test1_mv_union as (select id, sum(value) as sum_value from test1_mv group by id union select id, sum(value) as sum_value from test1_mv group by id)"}, + {"client":"python", "query_id":"301", "query_type": "stream","depends_on_stream":"test1_mv_union","wait":2, "query":"select id, sum_value from test1_mv_union settings seek_to='earliest'"} ]}, {"inputs": [ @@ -498,7 +498,8 @@ ["dev2", "ca", 76, "\"create_time\":\"2021-11-02 20:00:10\"", "2020-02-02 20:01:05"], ["dev2", "ca", 80, "\"create_time\":\"2021-11-02 20:00:01\"", "2020-02-02 20:01:03"], ["dev8", "ca", 67, "\"create_time\":\"2021-11-02 20:00:01\"", "2020-02-02 20:01:02"], - ["dev8", "ca", 77, "\"create_time\":\"2021-11-02 20:00:10\"", "2020-02-02 20:01:08"]]} + ["dev8", "ca", 77, "\"create_time\":\"2021-11-02 20:00:10\"", "2020-02-02 20:01:08"]], + "kill":301, "kill_wait":3, "drop_view":"test1_mv_union", "drop_view_wait":2} ]} ], diff --git a/tests/stream/test_stream_smoke/0013_changelog_stream13.yaml b/tests/stream/test_stream_smoke/0013_changelog_stream13.yaml index 12c2345964a..73af7a1dce0 100644 --- a/tests/stream/test_stream_smoke/0013_changelog_stream13.yaml +++ b/tests/stream/test_stream_smoke/0013_changelog_stream13.yaml @@ -368,12 +368,12 @@ tests: wait: 2 depends_on_stream: changelog_kv_13 query_id: '13108' - query: select count_distinct(val), sum_distinct(val) from changelog_kv_13; + query: select count_distinct(val), sum_distinct(val) from changelog_kv_13 emit periodic 1s; - client: python query_type: table depends_on: '13108' - wait: 3 + wait: 2 kill: '13108' kill_wait: 2 query: insert into changelog_kv_13(id, val, _tp_delta) values(2, 1, 1)(2, 1, -1)(3, 2, 1)(3, 2, -1); diff --git a/tests/stream/test_stream_smoke/0013_changelog_stream14.yaml b/tests/stream/test_stream_smoke/0013_changelog_stream14.yaml index 09912690e77..056513b5aea 100644 --- a/tests/stream/test_stream_smoke/0013_changelog_stream14.yaml +++ b/tests/stream/test_stream_smoke/0013_changelog_stream14.yaml @@ -207,34 +207,34 @@ tests: wait: 2 depends_on_stream: test_changelog_14 query_id: '15112' - query: select group_uniq_array(val), _tp_delta from test_changelog_14 emit changelog; + query: select group_uniq_array(val), _tp_delta from test_changelog_14 emit changelog periodic 1s; - client: python query_type: table depends_on: '15112' - wait: 3 + wait: 2 query: insert into test_changelog_14(id, val) values(1, 1); - client: python query_type: table - wait: 2 + wait: 1 query: insert into test_changelog_14(id, val) values(1, 2); - client: python query_type: table - wait: 2 + wait: 1 query: insert into test_changelog_14(id, val) values(2, 3); - client: python query_type: table - wait: 2 + wait: 1 query: insert into test_changelog_14(id, val) values(3, 3); - client: python query_type: table kill : '15112' kill_wait: 2 - wait: 2 + wait: 1 query: insert into test_changelog_14(id, val) values(3, 4); expected_results: - query_id: '15112' @@ -278,27 +278,27 @@ tests: - client: python query_type: table depends_on: '15113' - wait: 3 + wait: 2 query: insert into test_changelog_14(id, val) values(1, 1), (2,2); - client: python query_type: table kill: '15113' - kill_wait: 2 - wait: 3 + kill_wait: 3 + wait: 2 query: insert into test_changelog_14(id, val, _tp_delta) values(3, 3, +1), (2, 2, -1); - statements: - client: python query_type: stream query_id: 15113-1 - wait: 2 + wait: 1 terminate: manual query: recover from '15113' - client: python query_type: table - depends_on_stream: test_changelog_14 + depends_on: '15113' wait: 2 query: insert into test_changelog_14(id, val) values(4, 4), (5,5); @@ -314,7 +314,7 @@ tests: - client: python query_type: table - wait: 2 + wait: 3 query: kill query where query_id='15113-1' sync - client: python diff --git a/tests/stream/test_stream_smoke/0013_changelog_stream2.json b/tests/stream/test_stream_smoke/0013_changelog_stream2.json index 42b8e4d7a38..42c6b1ff542 100644 --- a/tests/stream/test_stream_smoke/0013_changelog_stream2.json +++ b/tests/stream/test_stream_smoke/0013_changelog_stream2.json @@ -191,7 +191,7 @@ {"client":"python", "query_type": "table", "exist":"test14_append_stream1_2", "exist_wait":2, "wait":1, "query":"create stream if not exists test14_append_stream1_2 (i int, k1 int, k2 string)"}, {"client":"python", "query_type": "table", "exist":"test14_append_stream2_2", "exist_wait":2, "wait":1, "query":"create stream if not exists test14_append_stream2_2 (j int, kk1 int, kk2 string) primary key(kk1, kk2) settings mode='versioned_kv'"}, {"client":"python", "query_type": "table", "exist":"test14_append_stream3_2", "exist_wait":2, "wait":1, "query":"create stream if not exists test14_append_stream3_2 (k int, kkk1 int, kkk2 string) primary key (kkk1, kkk2) settings mode='versioned_kv'"}, - {"client":"python", "query_type": "stream", "query_id":"1444", "wait":1, "terminate":"manual", "query":"select a.i, a.k1, a.k2, b.j, b.kk1, b.kk2, c.k, c.kkk1, c.kkk2, _tp_delta from test14_append_stream1_2 as a inner all join test14_append_stream2_2 as b on a.i = b.j inner all join test14_append_stream3_2 as c on b.kk2 = c.kkk2"}, + {"client":"python", "query_type": "stream", "depends_on_stream":"test14_append_stream3_2", "query_id":"1444", "wait":1, "terminate":"manual", "query":"select a.i, a.k1, a.k2, b.j, b.kk1, b.kk2, c.k, c.kkk1, c.kkk2, _tp_delta from test14_append_stream1_2 as a inner all join test14_append_stream2_2 as b on a.i = b.j inner all join test14_append_stream3_2 as c on b.kk2 = c.kkk2"}, {"client":"python", "query_type": "table", "depends_on":"1444", "wait":1, "query": "insert into test14_append_stream3_2(k, kkk1, kkk2) values (3, 2, 'k2')"}, {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream2_2(j, kk1, kk2) values (1, 1, 'k2')"}, {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream1_2(i, k1, k2) values (1, 1, 'k')"}, @@ -221,13 +221,10 @@ "statements": [ {"client":"python", "query_type":"table", "query":"drop stream if exists test14_append_stream_2"}, {"client":"python", "query_type": "table", "exist":"test14_append_stream_2", "exist_wait":2, "wait":1, "query":"create stream if not exists test14_append_stream_2 (id int, name string, val float, ts datetime)"}, - {"client":"python", "query_type": "stream", "query_id":"1445", "wait":1, "terminate":"manual", "query":"select count(), min(val), max(val), sum(val), avg(val) from changelog(test14_append_stream_2, id, name)"}, - {"client":"python", "query_type": "table", "depends_on":"1445", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 11.1 ,'2020-02-02 20:00:00')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'a', 22.2 ,'2020-02-02 20:00:01')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 33.3 ,'2020-02-02 20:00:02')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'a', 44.4 ,'2020-02-02 20:00:03')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 22.2 ,'2020-02-02 20:00:04')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'b', 44.4 ,'2020-02-02 20:00:05')"}, + {"client":"python", "query_type": "stream", "depends_on_stream":"test14_append_stream_2", "query_id":"1445", "wait":1, "terminate":"manual", "query":"select count(), min(val), max(val), sum(val), avg(val) from changelog(test14_append_stream_2, id, name) emit periodic 1s"}, + {"client":"python", "query_type": "table", "depends_on":"1445", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 11.1 ,'2020-02-02 20:00:00') (2, 'a', 22.2 ,'2020-02-02 20:00:01')"}, + {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 33.3 ,'2020-02-02 20:00:02') (2, 'a', 44.4 ,'2020-02-02 20:00:03')"}, + {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 22.2 ,'2020-02-02 20:00:04') (2, 'b', 44.4 ,'2020-02-02 20:00:05')"}, {"client":"python", "query_type": "table", "kill":"1445", "kill_wait":3, "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 33.3 ,'2020-02-02 20:00:05')"} ] } @@ -236,10 +233,10 @@ { "query_id":"1445", "expected_results":[ - [2, 11.100000381469727, 22.200000762939453, 33.30000114440918, 16.65000057220459], - [2, 33.29999923706055, 44.400001525878906, 77.70000076293945, 38.85000038146973], - [4, 22.200000762939453, 44.400001525878906, 144.3000030517578, 36.07500076293945], - [4, 33.29999923706055, 44.400001525878906, 155.4000015258789, 38.85000038146973] + [2, 11.1, 22.2, 33.3, 16.65], + [2, 33.3, 44.4, 77.7, 38.85], + [4, 22.2, 44.4, 144.3, 36.075], + [4, 33.3, 44.4, 155.4, 38.85] ] } ] @@ -254,15 +251,11 @@ "statements": [ {"client":"python", "query_type":"table", "query":"drop stream if exists test14_append_stream_2"}, {"client":"python", "query_type": "table", "exist":"test14_append_stream_2", "exist_wait":2, "wait":1, "query":"create stream if not exists test14_append_stream_2 (id int, name string, val float, ts datetime)"}, - {"client":"python", "query_type": "stream", "query_id":"1446", "wait":1, "terminate":"manual", "query":"select count(), min(val), max(val), sum(val), avg(val) from changelog(test14_append_stream_2, id, name) group by id"}, - {"client":"python", "query_type": "table", "depends_on":"1446", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 11.1 ,'2020-02-02 20:00:00')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'a', 33.3 ,'2020-02-02 20:00:01')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 22.2 ,'2020-02-02 20:00:02')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'b', 22.2 ,'2020-02-02 20:00:03')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'c', 33.3 ,'2020-02-02 20:00:04')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'c', 11.1 ,'2020-02-02 20:00:05')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 44.4 ,'2020-02-02 20:00:06')"}, - {"client":"python", "query_type": "table", "kill":"1446", "kill_wait":3, "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'b', 44.4 ,'2020-02-02 20:00:07')"} + {"client":"python", "query_type": "stream", "depends_on_stream":"test14_append_stream_2", "query_id":"1446", "wait":1, "terminate":"manual", "query":"select count(), min(val), max(val), sum(val), avg(val) from changelog(test14_append_stream_2, id, name) group by id emit periodic 1s"}, + {"client":"python", "query_type": "table", "depends_on":"1446", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 11.1 ,'2020-02-02 20:00:00') (2, 'a', 33.3 ,'2020-02-02 20:00:01')"}, + {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 22.2 ,'2020-02-02 20:00:02') (2, 'b', 22.2 ,'2020-02-02 20:00:03')"}, + {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'c', 33.3 ,'2020-02-02 20:00:04') (2, 'c', 11.1 ,'2020-02-02 20:00:05')"}, + {"client":"python", "query_type": "table", "kill":"1446", "kill_wait":3, "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 44.4 ,'2020-02-02 20:00:06') (2, 'b', 44.4 ,'2020-02-02 20:00:07')"} ] } ], @@ -270,14 +263,14 @@ { "query_id":"1446", "expected_results":[ - [1, 33.29999923706055, 33.29999923706055, 33.29999923706055, 33.29999923706055], - [1, 11.100000381469727, 11.100000381469727, 11.100000381469727, 11.100000381469727], - [2, 22.200000762939453, 33.29999923706055, 55.5, 27.75], - [2, 11.100000381469727, 22.200000762939453, 33.30000114440918, 16.65000057220459], - [3, 11.100000381469727, 33.29999923706055, 66.60000038146973, 22.200000127156574], - [3, 11.100000381469727, 33.29999923706055, 66.60000038146973, 22.200000127156574], - [3, 11.100000381469727, 44.400001525878906, 88.80000114440918, 29.600000381469727], - [3, 11.100000381469727, 44.400001525878906, 88.80000114440918, 29.600000381469727] + [1, 33.3, 33.3, 33.3, 33.3], + [1, 11.1, 11.1, 11.1, 11.1], + [2, 22.2, 33.3, 55.5, 27.75], + [2, 11.1, 22.2, 33.3, 16.65], + [3, 11.1, 33.3, 66.6, 22.2], + [3, 11.1, 33.3, 66.6, 22.2], + [3, 11.1, 44.4, 88.8, 29.6], + [3, 11.1, 44.4, 88.8, 29.6] ] } ] @@ -292,15 +285,11 @@ "statements": [ {"client":"python", "query_type":"table", "query":"drop stream if exists test14_append_stream_2"}, {"client":"python", "query_type": "table", "exist":"test14_append_stream_2", "exist_wait":2, "wait":1, "query":"create stream if not exists test14_append_stream_2 (id int, name string, val float, ts datetime)"}, - {"client":"python", "query_type": "stream", "query_id":"1447", "wait":1, "terminate":"manual", "query":"select count(), min(val), max(val), sum(val), avg(val) from changelog(test14_append_stream_2, id, name) group by id"}, - {"client":"python", "query_type": "table", "depends_on":"1447", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 11.1 ,'2020-02-02 20:00:00')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'a', 33.3 ,'2020-02-02 20:00:01')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 22.2 ,'2020-02-02 20:00:02')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'b', 22.2 ,'2020-02-02 20:00:03')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'c', 33.3 ,'2020-02-02 20:00:04')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'c', 11.1 ,'2020-02-02 20:00:05')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 44.4 ,'2020-02-02 20:00:06')"}, - {"client":"python", "query_type": "table", "kill":"1447", "kill_wait":3, "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'b', 44.4 ,'2020-02-02 20:00:07')"} + {"client":"python", "query_type": "stream", "query_id":"1447", "wait":1, "terminate":"manual", "query":"select count(), min(val), max(val), sum(val), avg(val) from changelog(test14_append_stream_2, id, name) group by id emit periodic 1s"}, + {"client":"python", "query_type": "table", "depends_on":"1447", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 11.1 ,'2020-02-02 20:00:00') (2, 'a', 33.3 ,'2020-02-02 20:00:01')"}, + {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 22.2 ,'2020-02-02 20:00:02') (2, 'b', 22.2 ,'2020-02-02 20:00:03')"}, + {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'c', 33.3 ,'2020-02-02 20:00:04') (2, 'c', 11.1 ,'2020-02-02 20:00:05')"}, + {"client":"python", "query_type": "table", "kill":"1447", "kill_wait":3, "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 44.4 ,'2020-02-02 20:00:06') (2, 'b', 44.4 ,'2020-02-02 20:00:07')"} ] } ], @@ -308,14 +297,14 @@ { "query_id":"1447", "expected_results":[ - [1, 33.29999923706055, 33.29999923706055, 33.29999923706055, 33.29999923706055], - [1, 11.100000381469727, 11.100000381469727, 11.100000381469727, 11.100000381469727], - [2, 22.200000762939453, 33.29999923706055, 55.5, 27.75], - [2, 11.100000381469727, 22.200000762939453, 33.30000114440918, 16.65000057220459], - [3, 11.100000381469727, 33.29999923706055, 66.60000038146973, 22.200000127156574], - [3, 11.100000381469727, 33.29999923706055, 66.60000038146973, 22.200000127156574], - [3, 11.100000381469727, 44.400001525878906, 88.80000114440918, 29.600000381469727], - [3, 11.100000381469727, 44.400001525878906, 88.80000114440918, 29.600000381469727] + [1, 33.3, 33.3, 33.3, 33.3], + [1, 11.1, 11.1, 11.1, 11.1], + [2, 22.2, 33.3, 55.5, 27.75], + [2, 11.1, 22.2, 33.3, 16.65], + [3, 11.1, 33.3, 66.6, 22.2], + [3, 11.1, 33.3, 66.6, 22.2], + [3, 11.1, 44.4, 88.8, 29.6], + [3, 11.1, 44.4, 88.8, 29.6] ] } ] @@ -344,8 +333,8 @@ { "query_id":"1448", "expected_results":[ - [1, 3, 11.100000381469727, 44.400001525878906, 88.80000114440918, 29.600000381469727, "2020-02-02 20:00:00", "2020-02-02 20:00:05"], - [2, 2, 22.200000762939453, 44.400001525878906, 66.60000228881836, 33.30000114440918, "2020-02-02 20:00:00", "2020-02-02 20:00:05"] + [1, 3, 11.1, 44.4, 88.8, 29.6, "2020-02-02 20:00:00", "2020-02-02 20:00:05"], + [2, 2, 22.2, 44.4, 66.6, 33.3, "2020-02-02 20:00:00", "2020-02-02 20:00:05"] ] } ] @@ -360,7 +349,7 @@ "statements": [ {"client":"python", "query_type":"table", "query":"drop stream if exists test14_append_stream_2"}, {"client":"python", "query_type": "table", "exist":"test14_append_stream_2", "exist_wait":2, "wait":1, "query":"create stream if not exists test14_append_stream_2 (id int, name string, val float, ts datetime)"}, - {"client":"python", "query_type": "stream", "query_id":"1449", "wait":1, "terminate":"manual", "query":"with subquery as (select * from changelog(test14_append_stream_2, id, name))select id, count(*), min(val), max(val) from subquery group by id"}, + {"client":"python", "query_type": "stream", "query_id":"1449", "wait":1, "terminate":"manual", "query":"with subquery as (select * from changelog(test14_append_stream_2, id, name))select id, count(*), min(val), max(val) from subquery group by id emit periodic 1s"}, {"client":"python", "query_type": "table", "depends_on":"1449", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 11.1 ,'2020-02-02 20:00:00')"}, {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 22.2 ,'2020-02-02 20:00:02')"}, {"client":"python", "query_type": "table", "kill":"1449", "kill_wait":3, "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 33.3 ,'2020-02-02 20:00:04')"} @@ -371,8 +360,9 @@ { "query_id":"1449", "expected_results":[ - [1, 2, 11.100000381469727, 22.200000762939453], - [1, 2, 22.200000762939453, 33.29999923706055] + [1, 1, 11.1, 11.1], + [1, 2, 11.1, 22.2], + [1, 2, 22.2, 33.3] ] } ] @@ -387,15 +377,11 @@ "statements": [ {"client":"python", "query_type":"table", "query":"drop stream if exists test14_append_stream_2"}, {"client":"python", "query_type": "table", "exist":"test14_append_stream_2", "exist_wait":2, "wait":1, "query":"create stream if not exists test14_append_stream_2 (id int, name string, val float, ts datetime)"}, - {"client":"python", "query_type": "stream", "query_id":"1450", "wait":1, "terminate":"manual", "query":"with subquery as (select id, count() as cnt, min(val) as min_val, max(val) as max_val from changelog(test14_append_stream_2, id, name) group by id)select count(*), sum(cnt), min(min_val), max(max_val) from subquery"}, - {"client":"python", "query_type": "table", "depends_on":"1450", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 11.1 ,'2020-02-02 20:00:00')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'a', 33.3 ,'2020-02-02 20:00:01')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 22.2 ,'2020-02-02 20:00:02')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'b', 22.2 ,'2020-02-02 20:00:03')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'c', 33.3 ,'2020-02-02 20:00:04')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'c', 11.1 ,'2020-02-02 20:00:05')"}, - {"client":"python", "query_type": "table", "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 44.4 ,'2020-02-02 20:00:06')"}, - {"client":"python", "query_type": "table", "kill":"1450", "kill_wait":3, "wait":1, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (2, 'b', 44.4 ,'2020-02-02 20:00:07')"} + {"client":"python", "query_type": "stream", "depends_on_stream":"test14_append_stream_2", "query_id":"1450", "wait":1, "terminate":"manual", "query":"with subquery as (select id, count() as cnt, min(val) as min_val, max(val) as max_val from changelog(test14_append_stream_2, id, name) group by id)select count(*), sum(cnt), min(min_val), max(max_val) from subquery"}, + {"client":"python", "query_type": "table", "depends_on":"1450", "wait":2, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'a', 11.1 ,'2020-02-02 20:00:00') (2, 'a', 33.3 ,'2020-02-02 20:00:01')"}, + {"client":"python", "query_type": "table", "wait":2, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 22.2 ,'2020-02-02 20:00:02') (2, 'b', 22.2 ,'2020-02-02 20:00:03')"}, + {"client":"python", "query_type": "table", "wait":2, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'c', 33.3 ,'2020-02-02 20:00:04') (2, 'c', 11.1 ,'2020-02-02 20:00:05')"}, + {"client":"python", "query_type": "table", "kill":"1450", "kill_wait":3, "wait":2, "query": "insert into test14_append_stream_2 (id, name, val, ts) values (1, 'b', 44.4 ,'2020-02-02 20:00:06') (2, 'b', 44.4 ,'2020-02-02 20:00:07')"} ] } ], @@ -403,10 +389,10 @@ { "query_id":"1450", "expected_results":[ - [2, 2, 11.100000381469727, 33.29999923706055], - [2, 4, 11.100000381469727, 33.29999923706055], - [2, 6, 11.100000381469727, 33.29999923706055], - [2, 6, 11.100000381469727, 44.400001525878906] + [2, 2, 11.1, 33.3], + [2, 4, 11.1, 33.3], + [2, 6, 11.1, 33.3], + [2, 6, 11.1, 44.4] ] } ] diff --git a/tests/stream/test_stream_smoke/0018_query_state7_view.json b/tests/stream/test_stream_smoke/0018_query_state7_view.json index 2803f5c8d77..79a53ddd044 100644 --- a/tests/stream/test_stream_smoke/0018_query_state7_view.json +++ b/tests/stream/test_stream_smoke/0018_query_state7_view.json @@ -548,7 +548,7 @@ {"client":"python", "query_type": "table", "wait":1, "query":"drop view if exists test19_state_mv7"}, {"client":"python", "query_type": "table", "wait":1, "query":"drop stream if exists test19_state_stream7"}, {"client":"python", "query_type": "table", "exist":"test19_state_stream7", "exist_wait":2, "wait":1, "query":"create stream test19_state_stream7 (id string, location string, value float, timestamp datetime64(3) default now64(3))"}, - {"client":"python", "query_type": "table", "exist":"test19_state_mv7", "exist_wait":2, "wait":1, "query":"create materialized view test19_state_mv7 as (select id, sum(value) as sum_value from test19_state_stream7 group by id limit 4 union select id, sum(value) as sum_value from test19_state_stream7 group by id limit 4)"}, + {"client":"python", "query_type": "table", "exist":"test19_state_mv7", "exist_wait":2, "wait":1, "query":"create materialized view test19_state_mv7 as (select id, sum(value) as sum_value from test19_state_stream7 group by id union select id, sum(value) as sum_value from test19_state_stream7 group by id)"}, {"client":"python", "query_type": "stream", "query_id":"19177", "wait":1, "terminate":"manual", "query":"subscribe to select id, sum_value from test19_state_mv7 settings checkpoint_interval=1"}, {"client":"python", "query_type": "table", "depends_on":"19177", "kill":"19177", "kill_wait":3, "wait":1, "query": "insert into test19_state_stream7(id, location, value, timestamp) values ('dev1', 'ca', 57.3, '2020-02-02 20:00:00')('dev2', 'ca', 58.3, '2020-02-02 20:00:03')"} ] @@ -579,8 +579,10 @@ "expected_results":[ ["dev1", "57.3"], ["dev2", "127.3"], + ["dev4", "67"], ["dev1", "57.3"], - ["dev2", "127.3"] + ["dev2", "127.3"], + ["dev4", "67"] ] } ] diff --git a/tests/stream/test_stream_smoke/0030_two_level_global_aggr.yaml b/tests/stream/test_stream_smoke/0030_two_level_global_aggr.yaml index 09a1c4d9240..9c5657d1111 100644 --- a/tests/stream/test_stream_smoke/0030_two_level_global_aggr.yaml +++ b/tests/stream/test_stream_smoke/0030_two_level_global_aggr.yaml @@ -131,7 +131,7 @@ tests: query_id: 3100 depends_on_stream: test_31_multishards_stream query: | - subscribe to with cte as (select i as key, count() from test_31_multishards_stream where _tp_time > earliest_ts() group by key settings group_by_two_level_threshold=50) select count() from cte settings checkpoint_interval=2, emit_during_backfill=false; + subscribe to with cte as (select i as key, count() from test_31_multishards_stream where _tp_time > earliest_ts() group by key settings group_by_two_level_threshold=50) select count() from cte settings checkpoint_interval=2; - client: python query_type: table @@ -207,7 +207,7 @@ tests: depends_on_stream: test_31_multishards_stream wait: 1 query: | - subscribe to with cte as (select i as key, count() from changelog(test_31_multishards_stream, i) where _tp_time > earliest_ts() group by key emit changelog settings group_by_two_level_threshold=50) select count() from cte settings checkpoint_interval=2, emit_during_backfill=false; + subscribe to with cte as (select i as key, count() from changelog(test_31_multishards_stream, i) where _tp_time > earliest_ts() group by key emit changelog settings group_by_two_level_threshold=50) select count() from cte settings checkpoint_interval=2; - client: python query_type: table diff --git a/tests/stream/test_stream_smoke/0099_fixed_issues.json b/tests/stream/test_stream_smoke/0099_fixed_issues.json index 1422c6d81a7..3f298ec5413 100644 --- a/tests/stream/test_stream_smoke/0099_fixed_issues.json +++ b/tests/stream/test_stream_smoke/0099_fixed_issues.json @@ -640,7 +640,7 @@ {"client":"python", "query_type": "table", "query": "drop stream if exists v_12183487"}, {"client":"python", "query_type": "table", "wait":2, "query": "create stream v_12183487(id int, val int) primary key id settings shards=3;"}, {"client":"python", "query_type": "stream", "wait":2, "depends_on_stream":"v_12183487", "query_id":"12183487213", "query":"subscribe to select sum_distinct_streaming(val), sum(val), count_distinct(val), count(val) from v_12183487 settings checkpoint_interval=1;"}, - {"client":"python", "query_type": "table", "depends_on_stream": "v_12183487", "kill":"12183487213", "kill_wait":2, "wait": 3, "query": "insert into v_12183487(id, val) values(3, 30);"} + {"client":"python", "query_type": "table", "depends_on": "12183487213", "kill":"12183487213", "kill_wait":3, "wait": 2, "query": "insert into v_12183487(id, val) values(3, 30);"} ] }, {