You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2023/04/19 13:34:47 UTC

[doris] 21/36: [chore](hashtable) Use doris' Allocator to replace std::allocator in phmap (#18735)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-2.0-alpha
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 353f5fc24692729dd258442c2624806abd7a1fd7
Author: Jerry Hu <mr...@gmail.com>
AuthorDate: Tue Apr 18 09:58:28 2023 +0800

    [chore](hashtable) Use doris' Allocator to replace std::allocator in phmap (#18735)
---
 be/src/exprs/hybrid_set.h                          |  7 ++-
 be/src/olap/hll.cpp                                |  2 +-
 be/src/olap/hll.h                                  |  4 +-
 be/src/olap/in_list_predicate.h                    |  2 -
 be/src/olap/reader.cpp                             |  2 -
 be/src/olap/rowset/segment_v2/binary_dict_page.h   |  2 -
 .../vec/aggregate_functions/aggregate_function.h   |  5 +-
 .../aggregate_functions/aggregate_function_topn.h  |  4 +-
 .../aggregate_functions/aggregate_function_uniq.h  |  5 +-
 be/src/vec/common/hash_table/ph_hash_map.h         |  3 +-
 be/src/vec/common/hash_table/phmap_fwd_decl.h      | 54 ++++++++++++++++++++++
 11 files changed, 68 insertions(+), 22 deletions(-)

diff --git a/be/src/exprs/hybrid_set.h b/be/src/exprs/hybrid_set.h
index 67c8f43099..54a30cd343 100644
--- a/be/src/exprs/hybrid_set.h
+++ b/be/src/exprs/hybrid_set.h
@@ -17,14 +17,13 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
-
 #include "common/object_pool.h"
 #include "runtime/decimalv2_value.h"
 #include "runtime/define_primitive_type.h"
 #include "runtime/primitive_type.h"
 #include "vec/columns/column_nullable.h"
 #include "vec/columns/column_string.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 #include "vec/common/string_ref.h"
 
 namespace doris {
@@ -149,7 +148,7 @@ template <typename T>
 class DynamicContainer {
 public:
     using Self = DynamicContainer;
-    using Iterator = typename phmap::flat_hash_set<T>::iterator;
+    using Iterator = typename vectorized::flat_hash_set<T>::iterator;
     using ElementType = T;
 
     DynamicContainer() = default;
@@ -168,7 +167,7 @@ public:
     size_t size() const { return _set.size(); }
 
 private:
-    phmap::flat_hash_set<T> _set;
+    vectorized::flat_hash_set<T> _set;
 };
 
 // TODO Maybe change void* parameter to template parameter better.
diff --git a/be/src/olap/hll.cpp b/be/src/olap/hll.cpp
index 7c234d1703..8f7a61e712 100644
--- a/be/src/olap/hll.cpp
+++ b/be/src/olap/hll.cpp
@@ -49,7 +49,7 @@ void HyperLogLog::_convert_explicit_to_register() {
         _update_registers(value);
     }
     // clear _hash_set
-    phmap::flat_hash_set<uint64_t>().swap(_hash_set);
+    vectorized::flat_hash_set<uint64_t>().swap(_hash_set);
 }
 
 // Change HLL_DATA_EXPLICIT to HLL_DATA_FULL directly, because HLL_DATA_SPARSE
diff --git a/be/src/olap/hll.h b/be/src/olap/hll.h
index a3347f4626..f1ad81100b 100644
--- a/be/src/olap/hll.h
+++ b/be/src/olap/hll.h
@@ -18,7 +18,6 @@
 #pragma once
 
 #include <math.h>
-#include <parallel_hashmap/phmap.h>
 #include <stdio.h>
 
 #include <map>
@@ -30,6 +29,7 @@
 #endif
 
 #include "gutil/macros.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 
 namespace doris {
 
@@ -268,7 +268,7 @@ public:
 
 private:
     HllDataType _type = HLL_DATA_EMPTY;
-    phmap::flat_hash_set<uint64_t> _hash_set;
+    vectorized::flat_hash_set<uint64_t> _hash_set;
 
     // This field is much space consuming(HLL_REGISTERS_COUNT), we create
     // it only when it is really needed.
diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h
index 7d0001cf1c..e207bf28d5 100644
--- a/be/src/olap/in_list_predicate.h
+++ b/be/src/olap/in_list_predicate.h
@@ -17,8 +17,6 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
-
 #include <cstdint>
 #include <roaring/roaring.hh>
 
diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp
index f3ffd44d32..b21691af2d 100644
--- a/be/src/olap/reader.cpp
+++ b/be/src/olap/reader.cpp
@@ -17,8 +17,6 @@
 
 #include "olap/reader.h"
 
-#include <parallel_hashmap/phmap.h>
-
 #include "common/status.h"
 #include "exprs/create_predicate_function.h"
 #include "exprs/hybrid_set.h"
diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.h b/be/src/olap/rowset/segment_v2/binary_dict_page.h
index 26bf127a1f..b570b042dc 100644
--- a/be/src/olap/rowset/segment_v2/binary_dict_page.h
+++ b/be/src/olap/rowset/segment_v2/binary_dict_page.h
@@ -17,8 +17,6 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
-
 #include <functional>
 #include <memory>
 #include <string>
diff --git a/be/src/vec/aggregate_functions/aggregate_function.h b/be/src/vec/aggregate_functions/aggregate_function.h
index c55612ec15..2d6b07e403 100644
--- a/be/src/vec/aggregate_functions/aggregate_function.h
+++ b/be/src/vec/aggregate_functions/aggregate_function.h
@@ -20,9 +20,8 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
-
 #include "vec/columns/column_complex.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 #include "vec/core/block.h"
 #include "vec/core/column_numbers.h"
 #include "vec/core/field.h"
@@ -221,7 +220,7 @@ public:
                       std::is_same_v<Derived,
                                      AggregateFunctionBitmapOp<AggregateFunctionBitmapUnionOp>>) {
             if (agg_many) {
-                phmap::flat_hash_map<AggregateDataPtr, std::vector<int>> place_rows;
+                flat_hash_map<AggregateDataPtr, std::vector<int>> place_rows;
                 for (int i = 0; i < batch_size; ++i) {
                     auto iter = place_rows.find(places[i] + place_offset);
                     if (iter == place_rows.end()) {
diff --git a/be/src/vec/aggregate_functions/aggregate_function_topn.h b/be/src/vec/aggregate_functions/aggregate_function_topn.h
index 99ec3dadd9..7bfacf7977 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_topn.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_topn.h
@@ -17,7 +17,6 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
 #include <rapidjson/stringbuffer.h>
 #include <rapidjson/writer.h>
 
@@ -29,6 +28,7 @@
 #include "vec/columns/column_array.h"
 #include "vec/columns/column_string.h"
 #include "vec/columns/column_vector.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 #include "vec/core/types.h"
 #include "vec/data_types/data_type_array.h"
 #include "vec/data_types/data_type_string.h"
@@ -183,7 +183,7 @@ struct AggregateFunctionTopNData {
 
     int top_num = 0;
     uint64_t capacity = 0;
-    phmap::flat_hash_map<T, uint64_t> counter_map;
+    flat_hash_map<T, uint64_t> counter_map;
 };
 
 struct AggregateFunctionTopNImplInt {
diff --git a/be/src/vec/aggregate_functions/aggregate_function_uniq.h b/be/src/vec/aggregate_functions/aggregate_function_uniq.h
index ff17082752..592cb5a925 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_uniq.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_uniq.h
@@ -20,8 +20,6 @@
 
 #pragma once
 
-#include <parallel_hashmap/phmap.h>
-
 #include <type_traits>
 
 #include "gutil/hash/city.h"
@@ -31,6 +29,7 @@
 #include "vec/common/assert_cast.h"
 #include "vec/common/bit_cast.h"
 #include "vec/common/hash_table/hash_set.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 #include "vec/common/typeid_cast.h"
 #include "vec/data_types/data_type_number.h"
 
@@ -47,7 +46,7 @@ struct AggregateFunctionUniqExactData {
     using Key = std::conditional_t<is_string_key, UInt128, T>;
     using Hash = std::conditional_t<is_string_key, UInt128TrivialHash, HashCRC32<Key>>;
 
-    using Set = phmap::flat_hash_set<Key, Hash>;
+    using Set = flat_hash_set<Key, Hash>;
 
     static UInt128 ALWAYS_INLINE get_key(const StringRef& value) {
         UInt128 key;
diff --git a/be/src/vec/common/hash_table/ph_hash_map.h b/be/src/vec/common/hash_table/ph_hash_map.h
index 2a847eda75..da51f31cf9 100644
--- a/be/src/vec/common/hash_table/ph_hash_map.h
+++ b/be/src/vec/common/hash_table/ph_hash_map.h
@@ -23,6 +23,7 @@
 
 #include "vec/common/hash_table/hash.h"
 #include "vec/common/hash_table/hash_table_utils.h"
+#include "vec/common/hash_table/phmap_fwd_decl.h"
 
 template <typename Key, typename Mapped>
 ALWAYS_INLINE inline auto lookup_result_get_mapped(std::pair<const Key, Mapped>* it) {
@@ -36,7 +37,7 @@ public:
     using Self = PHHashMap;
     using Hash = HashMethod;
     using cell_type = std::pair<const Key, Mapped>;
-    using HashMapImpl = phmap::flat_hash_map<Key, Mapped, Hash>;
+    using HashMapImpl = doris::vectorized::flat_hash_map<Key, Mapped, Hash>;
 
     using key_type = Key;
     using mapped_type = Mapped;
diff --git a/be/src/vec/common/hash_table/phmap_fwd_decl.h b/be/src/vec/common/hash_table/phmap_fwd_decl.h
new file mode 100644
index 0000000000..d37eb898e1
--- /dev/null
+++ b/be/src/vec/common/hash_table/phmap_fwd_decl.h
@@ -0,0 +1,54 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <parallel_hashmap/phmap.h>
+
+#include "vec/common/allocator.h"
+
+namespace doris::vectorized {
+
+/// `Allocator_` implements several interfaces of `std::allocator`
+/// which `phmap::flat_hash_map` will use.
+template <typename T>
+class Allocator_ : private Allocator<false, true> {
+public:
+    using value_type = T;
+    using pointer = T*;
+
+    Allocator_() = default;
+
+    template <typename T_>
+    Allocator_(const Allocator_<T_>&) {};
+
+    constexpr T* allocate(size_t n) { return static_cast<T*>(Allocator::alloc(n * sizeof(T))); }
+
+    void deallocate(pointer p, size_t n) { Allocator::free(p, n * sizeof(T)); }
+
+    friend bool operator==(const Allocator_&, const Allocator_&) { return true; }
+};
+
+template <typename K, typename V, typename Hash = phmap::Hash<K>, typename Eq = phmap::EqualTo<K>,
+          typename Alloc = Allocator_<phmap::Pair<const K, V>>>
+using flat_hash_map = phmap::flat_hash_map<K, V, Hash, Eq, Alloc>;
+
+template <typename K, typename Hash = phmap::Hash<K>, typename Eq = phmap::EqualTo<K>,
+          typename Alloc = Allocator_<K>>
+using flat_hash_set = phmap::flat_hash_set<K, Hash, Eq, Alloc>;
+
+} // namespace doris::vectorized


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