You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@doris.apache.org by GitBox <gi...@apache.org> on 2019/07/23 09:46:41 UTC

[GitHub] [incubator-doris] chaoyli commented on a change in pull request #1529: Refactor storage aggregate framework

chaoyli commented on a change in pull request #1529: Refactor storage aggregate framework
URL: https://github.com/apache/incubator-doris/pull/1529#discussion_r306225251
 
 

 ##########
 File path: be/src/olap/aggregate_func.h
 ##########
 @@ -15,221 +15,283 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef DORIS_BE_SRC_OLAP_AGGREGATE_FUNC_H
-#define DORIS_BE_SRC_OLAP_AGGREGATE_FUNC_H
+#pragma once
 
 #include "olap/hll.h"
 #include "olap/types.h"
 #include "util/arena.h"
 
 namespace doris {
 
-using AggregateFunc = void (*)(char* left, const char* right, Arena* arena);
-using FinalizeFunc = void (*)(char* data);
+using AggeInitFunc = void (*)(char* dst, Arena* arena);
+using AggUpdateFunc = void (*)(char* dst, const char* src, Arena* arena);
+using AggFinalizeFunc = void (*)(char* data, Arena* arena);
 
-template<FieldAggregationMethod agg_method,
-        FieldType field_type> struct AggregateFuncTraits {};
+// This class contains information about aggregate operation.
+class AggregateInfo {
+public:
+    // Init function will initialize aggregation execute environment in dst.
+    // For example: for sum, we just initial dst to 0. For HLL column, it will
+    // allocate and init context used to compute HLL.
+    //
+    // Memory Note: For plain memory can be allocated from arena, whose lifetime
+    // will last util finalize function is called. Memory allocated from heap should
+    // be freed in finalize functioin to avoid memory leak.
+    inline void init(void* dst, Arena* arena) const {
+        _init_fn((char*)dst, arena);
+    }
+
+    // Actually do the aggregate operation. dst is the context which is initialized
+    // by init function, src is the current value which is to be aggregated.
+    // For example: For sum, dst is the current sum, and src is the next value which
+    // will be added to sum.
+    // This function usually is used when load function.
+    //
+    // Memory Note: Same with init function.
+    inline void update(void* dst, const void* src, Arena* arena) const {
+        _update_fn((char*)dst, (const char*)src, arena);
+    }
+
+    // Merge aggregated intermediate data. Data stored in engine is aggregated,
+    // because storage has done some aggregate when loading or compaction.
+    // So this function is often used in read operation.
+    // 
+    // Memory Note: Same with init function.
+    inline void merge(void* dst, const void* src, Arena* arena) const {
+        _merge_fn((char*)dst, (const char*)src, arena);
+    }
+
+    // Finalize function convert intermediate context into final format. For example:
+    // For HLL type, finalize function will serialize the aggregate context into a slice.
+    // For input src points to the context, and when function is finished, result will be
+    // saved in src.
+    //
+    // Memory Note: All heap memory allocated in init and update function should be freed
+    // before this function return. Memory allocated from arena will be still available
+    // and will be freed by client.
+    inline void finalize(void* src, Arena* arena) const {
+        _finalize_fn((char*)src, arena);
+    }
 
-template<FieldType field_type>
-struct AggregateFuncTraits<OLAP_FIELD_AGGREGATION_NONE, field_type> {
-    static void aggregate(char* left, const char* right, Arena* arena) {}
+private:
+    void (*_init_fn)(char* dst, Arena* arena);
+    void (*_update_fn)(char* dst, const char* src, Arena* arena);
+    void (*_merge_fn)(char* dst, const char* src, Arena* arena);
+    void (*_finalize_fn)(char* dst, Arena* arena);
+
+    friend class AggregateFuncResolver;
+
+    template<typename Traits>
+    AggregateInfo(const Traits& traits);
+};
+
+struct BaseAggregateFuncs {
+    // Default init function will set to null
+    static void init(char* dst, Arena* arena) {
+        *reinterpret_cast<bool*>(dst) = true;
+    }
+
+    // Default update do nothing.
+    static void update(char* dst, const char* src, Arena* arena) {
+    }
+
+    // For most aggregate method, its merge and update are same. If merge
+    // is same with update, keep merge nullptr to avoid duplicate code.
+    // AggregateInfo constructor will set merge function to update function
+    // when merge is nullptr.
+    AggUpdateFunc merge = nullptr;
+
+    // Default finalize do nothing.
+    static void finalize(char* src, Arena* arena) {
+    }
+};
+
+template<FieldAggregationMethod agg_method, FieldType field_type>
+struct AggregateFuncTraits : public BaseAggregateFuncs {
 };
 
 template <FieldType field_type>
-struct AggregateFuncTraits<OLAP_FIELD_AGGREGATION_MIN, field_type> {
-    static void aggregate(char* left, const char* right, Arena* arena) {
-        typedef typename FieldTypeTraits<field_type>::CppType CppType;
-        bool l_null = *reinterpret_cast<bool*>(left);
-        bool r_null = *reinterpret_cast<const bool*>(right);
-        if (l_null) {
-            return;
-        } else if (r_null) {
-            *reinterpret_cast<bool*>(left) = true;
-        } else {
-            CppType* l_val = reinterpret_cast<CppType*>(left + 1);
-            const CppType* r_val = reinterpret_cast<const CppType*>(right + 1);
-            if (*r_val < *l_val) { *l_val = *r_val; }
+struct AggregateFuncTraits<OLAP_FIELD_AGGREGATION_MIN, field_type> : public BaseAggregateFuncs {
+    typedef typename FieldTypeTraits<field_type>::CppType CppType;
+
+    static void update(char* dst, const char* src, Arena* arena) {
+        bool src_null = *reinterpret_cast<const bool*>(src);
+        // ignore null value
+        if (src_null) return;
+
+        bool dst_null = *reinterpret_cast<bool*>(dst);
+        CppType* dst_val = reinterpret_cast<CppType*>(dst + 1);
 
 Review comment:
   if dst_null == true, dst + 1 has non-sense, it will be undefined behavior.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@doris.apache.org
For additional commands, e-mail: dev-help@doris.apache.org