Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 8629E200B6B for ; Fri, 9 Sep 2016 17:53:24 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 84B7C160AC2; Fri, 9 Sep 2016 15:53:24 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 18CDA160AA3 for ; Fri, 9 Sep 2016 17:53:21 +0200 (CEST) Received: (qmail 44531 invoked by uid 500); 9 Sep 2016 15:53:21 -0000 Mailing-List: contact commits-help@quickstep.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@quickstep.incubator.apache.org Delivered-To: mailing list commits@quickstep.incubator.apache.org Received: (qmail 44522 invoked by uid 99); 9 Sep 2016 15:53:21 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 09 Sep 2016 15:53:21 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 97155C0361 for ; Fri, 9 Sep 2016 15:53:20 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -5.446 X-Spam-Level: X-Spam-Status: No, score=-5.446 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-1.426] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id wQjI9xzjLGXg for ; Fri, 9 Sep 2016 15:53:10 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id D176260D7C for ; Fri, 9 Sep 2016 15:53:07 +0000 (UTC) Received: (qmail 34353 invoked by uid 99); 9 Sep 2016 15:51:37 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 09 Sep 2016 15:51:37 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id CCB54EF9A1; Fri, 9 Sep 2016 15:51:15 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: hbdeshmukh@apache.org To: commits@quickstep.incubator.apache.org Date: Fri, 09 Sep 2016 15:51:29 -0000 Message-Id: <7618b259961c4a50abe90b8de8f1909c@git.apache.org> In-Reply-To: <4adeb684e11e4b549b68cdd7e3387ec1@git.apache.org> References: <4adeb684e11e4b549b68cdd7e3387ec1@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [15/19] incubator-quickstep git commit: Modified Aggregation unit test. Ran clang-format. archived-at: Fri, 09 Sep 2016 15:53:24 -0000 http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2a9efc4e/storage/FastHashTable.hpp ---------------------------------------------------------------------- diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp index 909fcc0..f1e8d1a 100644 --- a/storage/FastHashTable.hpp +++ b/storage/FastHashTable.hpp @@ -35,8 +35,8 @@ #include "storage/TupleReference.hpp" #include "storage/ValueAccessor.hpp" #include "storage/ValueAccessorUtil.hpp" -#include "threading/SpinSharedMutex.hpp" #include "threading/SpinMutex.hpp" +#include "threading/SpinSharedMutex.hpp" #include "types/Type.hpp" #include "types/TypedValue.hpp" #include "utility/BloomFilter.hpp" @@ -50,23 +50,13 @@ namespace quickstep { */ /** - * @brief Base class for hash table. + * @brief Base class for the hash table implementation in which the payload can + * be just a bunch of bytes. This implementation is suitable for + * aggregation hash table with multiple aggregation handles (e.g. SUM, + * MAX, MIN etc). * - * This class is templated so that the core hash-table logic can be reused in - * different contexts requiring different value types and semantics (e.g. - * hash-joins vs. hash-based grouping for aggregates vs. hash-based indices). - * The base template defines the interface that HashTables provide to clients - * and implements some common functionality for all HashTables. There a few - * different (also templated) implementation classes that inherit from this - * base class and have different physical layouts with different performance - * characteristics. As of this writing, they are: - * 1. LinearOpenAddressingHashTable - All keys/values are stored directly - * in a single array of buckets. Collisions are handled by simply - * advancing to the "next" adjacent bucket until an empty bucket is - * found. This implementation is vulnerable to performance degradation - * due to the formation of bucket chains when there are many duplicate - * and/or consecutive keys. - * 2. SeparateChainingHashTable - Keys/values are stored in a separate + * At present there is one implementation for this base class. + * 1. SeparateChainingHashTable - Keys/values are stored in a separate * region of memory from the base hash table slot array. Every bucket * has a "next" pointer so that entries that collide (i.e. map to the * same base slot) form chains of pointers with each other. Although @@ -74,22 +64,12 @@ namespace quickstep { * LinearOpenAddressingHashTable, it does not have the same * vulnerabilities to key skew, and it additionally supports a very * efficient bucket-preallocation mechanism that minimizes cache - * coherency overhead when multiple threads are building a HashTable - * as part of a hash-join. - * 3. SimpleScalarSeparateChainingHashTable - A simplified version of - * SeparateChainingHashTable that is only usable for single, scalar - * keys with a reversible hash function. This implementation exploits - * the reversible hash to avoid storing separate copies of keys at all, - * and to skip an extra key comparison when hash codes collide. + * coherency overhead when multiple threads are building a HashTable. * * @note If you need to create a HashTable and not just use it as a client, see * HashTableFactory, which simplifies the process of creating a * HashTable. * - * @param ValueT The mapped value in this hash table. Must be - * copy-constructible. For a serializable hash table, ValueT must also - * be trivially copyable and trivially destructible (and beware of - * pointers to external memory). * @param resizable Whether this hash table is resizable (using memory from a * StorageManager) or not (using a private, fixed memory allocation). * @param serializable If true, this hash table can safely be saved to and @@ -115,21 +95,15 @@ template class FastHashTable : public HashTableBase { + serializable, + force_key_copy, + allow_duplicate_keys> { static_assert(!(serializable && resizable && !force_key_copy), "A HashTable must have force_key_copy=true when serializable " "and resizable are both true."); - // TODO(chasseur): GCC 4.8.3 doesn't yet implement - // std::is_trivially_copyable. In the future, we should include a - // static_assert that prevents a serializable HashTable from being used with - // a ValueT which is not trivially copyable. - public: // Shadow template parameters. This is useful for shared test harnesses. -// typedef ValueT value_type; static constexpr bool template_resizable = resizable; static constexpr bool template_serializable = serializable; static constexpr bool template_force_key_copy = force_key_copy; @@ -162,8 +136,9 @@ class FastHashTable : public HashTableBasegetID(); blob_.release(); @@ -212,8 +187,7 @@ class FastHashTable : public HashTableBase &key, - const uint8_t *value_ptr); + HashTablePutResult putCompositeKey(const std::vector &key, + const std::uint8_t *value_ptr); /** * @brief Add (multiple) new entries into the hash table from a @@ -378,7 +352,7 @@ class FastHashTable : public HashTableBase bool upsert(const TypedValue &key, - const uint8_t *initial_value_ptr, + const std::uint8_t *initial_value_ptr, FunctorT *functor); /** @@ -421,18 +395,18 @@ class FastHashTable : public HashTableBase bool upsertCompositeKeyFast(const std::vector &key, - const uint8_t *init_value_ptr, - FunctorT *functor); + const std::uint8_t *init_value_ptr, + FunctorT *functor); template bool upsertCompositeKeyFast(const std::vector &key, - const uint8_t *init_value_ptr, - FunctorT *functor, - int index); + const std::uint8_t *init_value_ptr, + FunctorT *functor, + int index); bool upsertCompositeKeyFast(const std::vector &key, - const uint8_t *init_value_ptr, - const uint8_t *source_state); + const std::uint8_t *init_value_ptr, + const std::uint8_t *source_state); /** * @brief Apply a functor to (multiple) entries in this hash table, with keys @@ -481,10 +455,11 @@ class FastHashTable : public HashTableBase> &argument_ids, - ValueAccessor *accessor, - const attribute_id key_attr_id, - const bool check_for_null_keys); + bool upsertValueAccessorFast( + const std::vector> &argument_ids, + ValueAccessor *accessor, + const attribute_id key_attr_id, + const bool check_for_null_keys); /** * @brief Apply a functor to (multiple) entries in this hash table, with keys @@ -582,7 +557,7 @@ class FastHashTable : public HashTableBase &key) const = 0; - virtual const uint8_t* getSingleCompositeKey(const std::vector &key, - int index) const = 0; + virtual const std::uint8_t* getSingleCompositeKey( + const std::vector &key) const = 0; + virtual const std::uint8_t *getSingleCompositeKey( + const std::vector &key, int index) const = 0; /** * @brief Lookup a key against this hash table to find matching entries. @@ -634,7 +610,8 @@ class FastHashTable : public HashTableBase *values) const = 0; + virtual void getAll(const TypedValue &key, + std::vector *values) const = 0; /** * @brief Lookup a composite key against this hash table to find matching @@ -659,8 +636,9 @@ class FastHashTable : public HashTableBase &key, - std::vector *values) const = 0; + virtual void getAllCompositeKey( + const std::vector &key, + std::vector *values) const = 0; /** * @brief Lookup (multiple) keys from a ValueAccessor and apply a functor to @@ -726,7 +704,8 @@ class FastHashTable : public HashTableBase - void getAllFromValueAccessorCompositeKey(ValueAccessor *accessor, - const std::vector &key_attr_ids, - const bool check_for_null_keys, - FunctorT *functor) const; + void getAllFromValueAccessorCompositeKey( + ValueAccessor *accessor, + const std::vector &key_attr_ids, + const bool check_for_null_keys, + FunctorT *functor) const; /** * @brief Apply the functor to each key with a match in the hash table. @@ -842,10 +823,8 @@ class FastHashTable : public HashTableBase(accessor, - key_attr_id, - check_for_null_keys, - functor); + return runOverKeysFromValueAccessor( + accessor, key_attr_id, check_for_null_keys, functor); } /** @@ -869,10 +848,8 @@ class FastHashTable : public HashTableBase &key_attr_ids, const bool check_for_null_keys, FunctorT *functor) const { - return runOverKeysFromValueAccessorCompositeKey(accessor, - key_attr_ids, - check_for_null_keys, - functor); + return runOverKeysFromValueAccessorCompositeKey( + accessor, key_attr_ids, check_for_null_keys, functor); } /** @@ -896,10 +873,8 @@ class FastHashTable : public HashTableBase(accessor, - key_attr_id, - check_for_null_keys, - functor); + return runOverKeysFromValueAccessor( + accessor, key_attr_id, check_for_null_keys, functor); } /** @@ -923,10 +898,8 @@ class FastHashTable : public HashTableBase &key_attr_ids, const bool check_for_null_keys, FunctorT *functor) const { - return runOverKeysFromValueAccessorCompositeKey(accessor, - key_attr_ids, - check_for_null_keys, - functor); + return runOverKeysFromValueAccessorCompositeKey( + accessor, key_attr_ids, check_for_null_keys, functor); } /** @@ -983,8 +956,7 @@ class FastHashTable : public HashTableBase - std::size_t forEachCompositeKeyFast(FunctorT *functor, - int index) const; + std::size_t forEachCompositeKeyFast(FunctorT *functor, int index) const; /** * @brief A call to this function will cause a bloom filter to be built @@ -1037,7 +1009,8 @@ class FastHashTable : public HashTableBase &&probe_attribute_ids) { + inline void addProbeSideAttributeIds( + std::vector &&probe_attribute_ids) { probe_attribute_ids_.push_back(probe_attribute_ids); } @@ -1065,30 +1038,32 @@ class FastHashTable : public HashTableBase &key_types, - const std::size_t num_entries, - const std::vector &handles, - const std::vector &payload_sizes, - StorageManager *storage_manager, - const bool adjust_hashes, - const bool use_scalar_literal_hash, - const bool preallocate_supported) - : key_types_(key_types), - scalar_key_inline_(true), - key_inline_(nullptr), - adjust_hashes_(adjust_hashes), - use_scalar_literal_hash_(use_scalar_literal_hash), - preallocate_supported_(preallocate_supported), - handles_(handles), - total_payload_size_(std::accumulate(payload_sizes.begin(), payload_sizes.end(), sizeof(SpinMutex))), - storage_manager_(storage_manager), - hash_table_memory_(nullptr), - hash_table_memory_size_(0) { + FastHashTable(const std::vector &key_types, + const std::size_t num_entries, + const std::vector &handles, + const std::vector &payload_sizes, + StorageManager *storage_manager, + const bool adjust_hashes, + const bool use_scalar_literal_hash, + const bool preallocate_supported) + : key_types_(key_types), + scalar_key_inline_(true), + key_inline_(nullptr), + adjust_hashes_(adjust_hashes), + use_scalar_literal_hash_(use_scalar_literal_hash), + preallocate_supported_(preallocate_supported), + handles_(handles), + num_handles_(handles.size()), + total_payload_size_(std::accumulate( + payload_sizes.begin(), payload_sizes.end(), sizeof(SpinMutex))), + storage_manager_(storage_manager), + hash_table_memory_(nullptr), + hash_table_memory_size_(0) { DEBUG_ASSERT(resizable); std::size_t running_sum = sizeof(SpinMutex); for (auto size : payload_sizes) { - payload_offsets_.emplace_back(running_sum); - running_sum+=size; + payload_offsets_.emplace_back(running_sum); + running_sum += size; } } @@ -1122,14 +1097,14 @@ class FastHashTable : public HashTableBase &key_types, - void *hash_table_memory, - const std::size_t hash_table_memory_size, - const bool new_hash_table, - const bool hash_table_memory_zeroed, - const bool adjust_hashes, - const bool use_scalar_literal_hash, - const bool preallocate_supported) + FastHashTable(const std::vector &key_types, + void *hash_table_memory, + const std::size_t hash_table_memory_size, + const bool new_hash_table, + const bool hash_table_memory_zeroed, + const bool adjust_hashes, + const bool use_scalar_literal_hash, + const bool preallocate_supported) : key_types_(key_types), scalar_key_inline_(true), key_inline_(nullptr), @@ -1169,16 +1144,17 @@ class FastHashTable : public HashTableBase &key, - const std::size_t variable_key_size, - const std::uint8_t *init_value_ptr, - HashTablePreallocationState *prealloc_state) = 0; - + virtual HashTablePutResult putInternal( + const TypedValue &key, + const std::size_t variable_key_size, + const std::uint8_t &value, + HashTablePreallocationState *prealloc_state) = 0; + + virtual HashTablePutResult putCompositeKeyInternalFast( + const std::vector &key, + const std::size_t variable_key_size, + const std::uint8_t *init_value_ptr, + HashTablePreallocationState *prealloc_state) = 0; // Helpers for upsert. Both return a pointer to the value corresponding to // 'key'. If this HashTable is resizable, 'resize_shared_mutex_' should be @@ -1186,13 +1162,15 @@ class FastHashTable : public HashTableBase &key, - const std::uint8_t *init_value_ptr, - const std::size_t variable_key_size) = 0; + virtual std::uint8_t *upsertCompositeKeyInternalFast( + const std::vector &key, + const std::uint8_t *init_value_ptr, + const std::size_t variable_key_size) = 0; // Helpers for forEach. Each return true on success, false if no more entries // exist to iterate over. After a successful call, '*key' is overwritten with @@ -1200,10 +1178,10 @@ class FastHashTable : public HashTableBase *key, - const uint8_t **value, + const std::uint8_t **value, std::size_t *entry_num) const = 0; // Helpers for getAllFromValueAccessor. Each return true on success, false if @@ -1213,11 +1191,11 @@ class FastHashTable : public HashTableBase &key, const std::size_t hash_code, - const uint8_t **value, + const std::uint8_t **value, std::size_t *entry_num) const = 0; // Return true if key exists in the hash table. @@ -1250,15 +1228,17 @@ class FastHashTable : public HashTableBase key_types_; + const std::vector key_types_; // Information about whether key components are stored inline or in a // separate variable-length storage region. This is usually determined by a @@ -1275,6 +1255,7 @@ class FastHashTable : public HashTableBase handles_; + const unsigned int num_handles_; const std::size_t total_payload_size_; std::vector payload_offsets_; @@ -1341,12 +1322,11 @@ class FastHashTable : public HashTableBase probe_bloom_filters_; + std::vector probe_bloom_filters_; std::vector> probe_attribute_ids_; DISALLOW_COPY_AND_ASSIGN(FastHashTable); }; - /** * @brief An instantiation of the HashTable template for use in aggregations. * @note This has force_key_copy = true, so that we don't have dangling pointers @@ -1363,11 +1343,11 @@ template -HashTablePutResult FastHashTable - ::put(const TypedValue &key, - const uint8_t &value) { - const std::size_t variable_size = (force_key_copy && !scalar_key_inline_) ? key.getDataSize() - : 0; +HashTablePutResult +FastHashTable:: + put(const TypedValue &key, const std::uint8_t &value) { + const std::size_t variable_size = + (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0; if (resizable) { HashTablePutResult result = HashTablePutResult::kOutOfSpace; while (result == HashTablePutResult::kOutOfSpace) { @@ -1389,16 +1369,19 @@ template -HashTablePutResult FastHashTable - ::putCompositeKeyFast(const std::vector &key, - const std::uint8_t* init_value_ptr) { - const std::size_t variable_size = calculateVariableLengthCompositeKeyCopySize(key); +HashTablePutResult +FastHashTable:: + putCompositeKey(const std::vector &key, + const std::uint8_t *init_value_ptr) { + const std::size_t variable_size = + calculateVariableLengthCompositeKeyCopySize(key); if (resizable) { HashTablePutResult result = HashTablePutResult::kOutOfSpace; while (result == HashTablePutResult::kOutOfSpace) { { SpinSharedMutexSharedLock lock(resize_shared_mutex_); - result = putCompositeKeyInternalFast(key, variable_size, init_value_ptr, nullptr); + result = putCompositeKeyInternalFast( + key, variable_size, init_value_ptr, nullptr); } if (result == HashTablePutResult::kOutOfSpace) { resize(0, variable_size); @@ -1406,21 +1389,22 @@ HashTablePutResult FastHashTable template -HashTablePutResult FastHashTable - ::putValueAccessor(ValueAccessor *accessor, - const attribute_id key_attr_id, - const bool check_for_null_keys, - FunctorT *functor) { +HashTablePutResult +FastHashTable:: + putValueAccessor(ValueAccessor *accessor, + const attribute_id key_attr_id, + const bool check_for_null_keys, + FunctorT *functor) { HashTablePutResult result = HashTablePutResult::kOutOfSpace; std::size_t variable_size; HashTablePreallocationState prealloc_state; @@ -1428,111 +1412,120 @@ HashTablePutResult FastHashTable HashTablePutResult { // NOLINT(build/c++11) - if (using_prealloc) { - std::size_t total_entries = 0; - std::size_t total_variable_key_size = 0; - if (check_for_null_keys || (force_key_copy && !scalar_key_inline_)) { - // If we need to filter out nulls OR make variable copies, make a - // prepass over the ValueAccessor. - while (accessor->next()) { - TypedValue key = accessor->getTypedValue(key_attr_id); - if (check_for_null_keys && key.isNull()) { - continue; - } - ++total_entries; - total_variable_key_size += (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0; - } - accessor->beginIteration(); - } else { - total_entries = accessor->getNumTuples(); - } - if (resizable) { - bool prealloc_succeeded = false; - while (!prealloc_succeeded) { - { - SpinSharedMutexSharedLock lock(resize_shared_mutex_); - prealloc_succeeded = this->preallocateForBulkInsert(total_entries, - total_variable_key_size, - &prealloc_state); + if (using_prealloc) { + std::size_t total_entries = 0; + std::size_t total_variable_key_size = 0; + if (check_for_null_keys || (force_key_copy && !scalar_key_inline_)) { + // If we need to filter out nulls OR make variable copies, make a + // prepass over the ValueAccessor. + while (accessor->next()) { + TypedValue key = accessor->getTypedValue(key_attr_id); + if (check_for_null_keys && key.isNull()) { + continue; + } + ++total_entries; + total_variable_key_size += (force_key_copy && !scalar_key_inline_) + ? key.getDataSize() + : 0; + } + accessor->beginIteration(); + } else { + total_entries = accessor->getNumTuples(); } - if (!prealloc_succeeded) { - this->resize(total_entries, total_variable_key_size); + if (resizable) { + bool prealloc_succeeded = false; + while (!prealloc_succeeded) { + { + SpinSharedMutexSharedLock lock(resize_shared_mutex_); + prealloc_succeeded = this->preallocateForBulkInsert( + total_entries, total_variable_key_size, &prealloc_state); + } + if (!prealloc_succeeded) { + this->resize(total_entries, total_variable_key_size); + } + } + } else { + using_prealloc = this->preallocateForBulkInsert( + total_entries, total_variable_key_size, &prealloc_state); } } - } else { - using_prealloc = this->preallocateForBulkInsert(total_entries, - total_variable_key_size, - &prealloc_state); - } - } - std::unique_ptr thread_local_bloom_filter; - if (has_build_side_bloom_filter_) { - thread_local_bloom_filter.reset(new BloomFilter(build_bloom_filter_->getRandomSeed(), - build_bloom_filter_->getNumberOfHashes(), - build_bloom_filter_->getBitArraySize())); - } - if (resizable) { - while (result == HashTablePutResult::kOutOfSpace) { - { - result = HashTablePutResult::kOK; - SpinSharedMutexSharedLock lock(resize_shared_mutex_); + std::unique_ptr thread_local_bloom_filter; + if (has_build_side_bloom_filter_) { + thread_local_bloom_filter.reset( + new BloomFilter(build_bloom_filter_->getRandomSeed(), + build_bloom_filter_->getNumberOfHashes(), + build_bloom_filter_->getBitArraySize())); + } + if (resizable) { + while (result == HashTablePutResult::kOutOfSpace) { + { + result = HashTablePutResult::kOK; + SpinSharedMutexSharedLock lock(resize_shared_mutex_); + while (accessor->next()) { + TypedValue key = accessor->getTypedValue(key_attr_id); + if (check_for_null_keys && key.isNull()) { + continue; + } + variable_size = (force_key_copy && !scalar_key_inline_) + ? key.getDataSize() + : 0; + result = this->putInternal( + key, + variable_size, + (*functor)(*accessor), + using_prealloc ? &prealloc_state : nullptr); + // Insert into bloom filter, if enabled. + if (has_build_side_bloom_filter_) { + thread_local_bloom_filter->insertUnSafe( + static_cast(key.getDataPtr()), + key.getDataSize()); + } + if (result == HashTablePutResult::kDuplicateKey) { + DEBUG_ASSERT(!using_prealloc); + return result; + } else if (result == HashTablePutResult::kOutOfSpace) { + DEBUG_ASSERT(!using_prealloc); + break; + } + } + } + if (result == HashTablePutResult::kOutOfSpace) { + this->resize(0, variable_size); + accessor->previous(); + } + } + } else { while (accessor->next()) { TypedValue key = accessor->getTypedValue(key_attr_id); if (check_for_null_keys && key.isNull()) { continue; } - variable_size = (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0; - result = this->putInternal(key, - variable_size, - (*functor)(*accessor), - using_prealloc ? &prealloc_state : nullptr); + variable_size = + (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0; + result = + this->putInternal(key, + variable_size, + (*functor)(*accessor), + using_prealloc ? &prealloc_state : nullptr); // Insert into bloom filter, if enabled. if (has_build_side_bloom_filter_) { - thread_local_bloom_filter->insertUnSafe(static_cast(key.getDataPtr()), - key.getDataSize()); + thread_local_bloom_filter->insertUnSafe( + static_cast(key.getDataPtr()), + key.getDataSize()); } - if (result == HashTablePutResult::kDuplicateKey) { - DEBUG_ASSERT(!using_prealloc); + if (result != HashTablePutResult::kOK) { return result; - } else if (result == HashTablePutResult::kOutOfSpace) { - DEBUG_ASSERT(!using_prealloc); - break; } } } - if (result == HashTablePutResult::kOutOfSpace) { - this->resize(0, variable_size); - accessor->previous(); - } - } - } else { - while (accessor->next()) { - TypedValue key = accessor->getTypedValue(key_attr_id); - if (check_for_null_keys && key.isNull()) { - continue; - } - variable_size = (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0; - result = this->putInternal(key, - variable_size, - (*functor)(*accessor), - using_prealloc ? &prealloc_state : nullptr); - // Insert into bloom filter, if enabled. + // Update the build side bloom filter with thread local copy, if + // available. if (has_build_side_bloom_filter_) { - thread_local_bloom_filter->insertUnSafe(static_cast(key.getDataPtr()), - key.getDataSize()); - } - if (result != HashTablePutResult::kOK) { - return result; + build_bloom_filter_->bitwiseOr(thread_local_bloom_filter.get()); } - } - } - // Update the build side bloom filter with thread local copy, if available. - if (has_build_side_bloom_filter_) { - build_bloom_filter_->bitwiseOr(thread_local_bloom_filter.get()); - } - return HashTablePutResult::kOK; - }); + return HashTablePutResult::kOK; + }); } template template -HashTablePutResult FastHashTable - ::putValueAccessorCompositeKey(ValueAccessor *accessor, - const std::vector &key_attr_ids, - const bool check_for_null_keys, - FunctorT *functor) { +HashTablePutResult +FastHashTable:: + putValueAccessorCompositeKey(ValueAccessor *accessor, + const std::vector &key_attr_ids, + const bool check_for_null_keys, + FunctorT *functor) { DEBUG_ASSERT(key_types_.size() == key_attr_ids.size()); HashTablePutResult result = HashTablePutResult::kOutOfSpace; std::size_t variable_size; @@ -1555,50 +1549,79 @@ HashTablePutResult FastHashTable HashTablePutResult { // NOLINT(build/c++11) - if (using_prealloc) { - std::size_t total_entries = 0; - std::size_t total_variable_key_size = 0; - if (check_for_null_keys || force_key_copy) { - // If we need to filter out nulls OR make variable copies, make a - // prepass over the ValueAccessor. - while (accessor->next()) { - if (this->GetCompositeKeyFromValueAccessor(*accessor, - key_attr_ids, - check_for_null_keys, - &key_vector)) { - continue; - } - ++total_entries; - total_variable_key_size += this->calculateVariableLengthCompositeKeyCopySize(key_vector); - } - accessor->beginIteration(); - } else { - total_entries = accessor->getNumTuples(); - } - if (resizable) { - bool prealloc_succeeded = false; - while (!prealloc_succeeded) { - { - SpinSharedMutexSharedLock lock(resize_shared_mutex_); - prealloc_succeeded = this->preallocateForBulkInsert(total_entries, - total_variable_key_size, - &prealloc_state); + if (using_prealloc) { + std::size_t total_entries = 0; + std::size_t total_variable_key_size = 0; + if (check_for_null_keys || force_key_copy) { + // If we need to filter out nulls OR make variable copies, make a + // prepass over the ValueAccessor. + while (accessor->next()) { + if (this->GetCompositeKeyFromValueAccessor(*accessor, + key_attr_ids, + check_for_null_keys, + &key_vector)) { + continue; + } + ++total_entries; + total_variable_key_size += + this->calculateVariableLengthCompositeKeyCopySize(key_vector); + } + accessor->beginIteration(); + } else { + total_entries = accessor->getNumTuples(); } - if (!prealloc_succeeded) { - this->resize(total_entries, total_variable_key_size); + if (resizable) { + bool prealloc_succeeded = false; + while (!prealloc_succeeded) { + { + SpinSharedMutexSharedLock lock(resize_shared_mutex_); + prealloc_succeeded = this->preallocateForBulkInsert( + total_entries, total_variable_key_size, &prealloc_state); + } + if (!prealloc_succeeded) { + this->resize(total_entries, total_variable_key_size); + } + } + } else { + using_prealloc = this->preallocateForBulkInsert( + total_entries, total_variable_key_size, &prealloc_state); } } - } else { - using_prealloc = this->preallocateForBulkInsert(total_entries, - total_variable_key_size, - &prealloc_state); - } - } - if (resizable) { - while (result == HashTablePutResult::kOutOfSpace) { - { - result = HashTablePutResult::kOK; - SpinSharedMutexSharedLock lock(resize_shared_mutex_); + if (resizable) { + while (result == HashTablePutResult::kOutOfSpace) { + { + result = HashTablePutResult::kOK; + SpinSharedMutexSharedLock lock(resize_shared_mutex_); + while (accessor->next()) { + if (this->GetCompositeKeyFromValueAccessor(*accessor, + key_attr_ids, + check_for_null_keys, + &key_vector)) { + continue; + } + variable_size = + this->calculateVariableLengthCompositeKeyCopySize( + key_vector); + result = this->putCompositeKeyInternal( + key_vector, + variable_size, + (*functor)(*accessor), + using_prealloc ? &prealloc_state : nullptr); + if (result == HashTablePutResult::kDuplicateKey) { + DEBUG_ASSERT(!using_prealloc); + return result; + } else if (result == HashTablePutResult::kOutOfSpace) { + DEBUG_ASSERT(!using_prealloc); + break; + } + } + } + if (result == HashTablePutResult::kOutOfSpace) { + this->resize(0, variable_size); + accessor->previous(); + } + } + } else { while (accessor->next()) { if (this->GetCompositeKeyFromValueAccessor(*accessor, key_attr_ids, @@ -1606,46 +1629,21 @@ HashTablePutResult FastHashTablecalculateVariableLengthCompositeKeyCopySize(key_vector); - result = this->putCompositeKeyInternal(key_vector, - variable_size, - (*functor)(*accessor), - using_prealloc ? &prealloc_state : nullptr); - if (result == HashTablePutResult::kDuplicateKey) { - DEBUG_ASSERT(!using_prealloc); + variable_size = + this->calculateVariableLengthCompositeKeyCopySize(key_vector); + result = this->putCompositeKeyInternal( + key_vector, + variable_size, + (*functor)(*accessor), + using_prealloc ? &prealloc_state : nullptr); + if (result != HashTablePutResult::kOK) { return result; - } else if (result == HashTablePutResult::kOutOfSpace) { - DEBUG_ASSERT(!using_prealloc); - break; } } } - if (result == HashTablePutResult::kOutOfSpace) { - this->resize(0, variable_size); - accessor->previous(); - } - } - } else { - while (accessor->next()) { - if (this->GetCompositeKeyFromValueAccessor(*accessor, - key_attr_ids, - check_for_null_keys, - &key_vector)) { - continue; - } - variable_size = this->calculateVariableLengthCompositeKeyCopySize(key_vector); - result = this->putCompositeKeyInternal(key_vector, - variable_size, - (*functor)(*accessor), - using_prealloc ? &prealloc_state : nullptr); - if (result != HashTablePutResult::kOK) { - return result; - } - } - } - return HashTablePutResult::kOK; - }); + return HashTablePutResult::kOK; + }); } template template -bool FastHashTable - ::upsert(const TypedValue &key, - const uint8_t *initial_value_ptr, - FunctorT *functor) { +bool FastHashTable::upsert(const TypedValue &key, + const std::uint8_t + *initial_value_ptr, + FunctorT *functor) { DEBUG_ASSERT(!allow_duplicate_keys); - const std::size_t variable_size = (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0; + const std::size_t variable_size = + (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0; if (resizable) { for (;;) { { SpinSharedMutexSharedLock resize_lock(resize_shared_mutex_); - uint8_t *value = upsertInternalFast(key, variable_size, initial_value_ptr); + std::uint8_t *value = + upsertInternalFast(key, variable_size, initial_value_ptr); if (value != nullptr) { (*functor)(value); return true; @@ -1672,7 +1675,8 @@ bool FastHashTable *>(destination_hash_table)) {} + explicit HashTableMergerFast( + AggregationStateHashTableBase *destination_hash_table) + : destination_hash_table_( + static_cast *>( + destination_hash_table)) {} /** * @brief The operator for the functor. @@ -1702,8 +1709,8 @@ class HashTableMergerFast { * aggregation hash table. **/ inline void operator()(const std::vector &group_by_key, - const uint8_t *source_state) { - const uint8_t *original_state = + const std::uint8_t *source_state) { + const std::uint8_t *original_state = destination_hash_table_->getSingleCompositeKey(group_by_key); if (original_state != nullptr) { // The CHECK is required as upsertCompositeKey can return false if the @@ -1712,7 +1719,7 @@ class HashTableMergerFast { CHECK(destination_hash_table_->upsertCompositeKeyFast( group_by_key, original_state, source_state)); } else { - destination_hash_table_->putCompositeKeyFast(group_by_key, source_state); + destination_hash_table_->putCompositeKey(group_by_key, source_state); } } @@ -1722,23 +1729,27 @@ class HashTableMergerFast { DISALLOW_COPY_AND_ASSIGN(HashTableMergerFast); }; - template template -bool FastHashTable - ::upsertCompositeKeyFast(const std::vector &key, - const std::uint8_t *init_value_ptr, - FunctorT *functor) { +bool FastHashTable:: + upsertCompositeKeyFast(const std::vector &key, + const std::uint8_t *init_value_ptr, + FunctorT *functor) { DEBUG_ASSERT(!allow_duplicate_keys); - const std::size_t variable_size = calculateVariableLengthCompositeKeyCopySize(key); + const std::size_t variable_size = + calculateVariableLengthCompositeKeyCopySize(key); if (resizable) { for (;;) { { SpinSharedMutexSharedLock resize_lock(resize_shared_mutex_); - uint8_t *value = upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size); + std::uint8_t *value = + upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size); if (value != nullptr) { (*functor)(value); return true; @@ -1747,7 +1758,8 @@ bool FastHashTable template -bool FastHashTable - ::upsertCompositeKeyFast(const std::vector &key, - const std::uint8_t *init_value_ptr, - FunctorT *functor, - int index) { +bool FastHashTable:: + upsertCompositeKeyFast(const std::vector &key, + const std::uint8_t *init_value_ptr, + FunctorT *functor, + int index) { DEBUG_ASSERT(!allow_duplicate_keys); - const std::size_t variable_size = calculateVariableLengthCompositeKeyCopySize(key); + const std::size_t variable_size = + calculateVariableLengthCompositeKeyCopySize(key); if (resizable) { for (;;) { { SpinSharedMutexSharedLock resize_lock(resize_shared_mutex_); - uint8_t *value = upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size); + std::uint8_t *value = + upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size); if (value != nullptr) { - (*functor)(value+payload_offsets_[index]); + (*functor)(value + payload_offsets_[index]); return true; } } resize(0, variable_size); } } else { - uint8_t *value = upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size); + std::uint8_t *value = + upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size); if (value == nullptr) { return false; } else { - (*functor)(value+payload_offsets_[index]); + (*functor)(value + payload_offsets_[index]); return true; } } } - template -bool FastHashTable - ::upsertCompositeKeyFast(const std::vector &key, - const std::uint8_t *init_value_ptr, - const std::uint8_t *source_state) { +bool FastHashTable:: + upsertCompositeKeyFast(const std::vector &key, + const std::uint8_t *init_value_ptr, + const std::uint8_t *source_state) { DEBUG_ASSERT(!allow_duplicate_keys); - const std::size_t variable_size = calculateVariableLengthCompositeKeyCopySize(key); + const std::size_t variable_size = + calculateVariableLengthCompositeKeyCopySize(key); if (resizable) { for (;;) { { SpinSharedMutexSharedLock resize_lock(resize_shared_mutex_); - uint8_t *value = upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size); + std::uint8_t *value = + upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size); if (value != nullptr) { - SpinMutexLock lock(*(reinterpret_cast(value))); - for (unsigned int k = 0; k < handles_.size(); ++k) { - handles_[k]->mergeStatesFast(source_state + payload_offsets_[k], value + payload_offsets_[k]); - } + SpinMutexLock lock(*(reinterpret_cast(value))); + for (unsigned int k = 0; k < num_handles_; ++k) { + handles_[k]->mergeStatesFast(source_state + payload_offsets_[k], + value + payload_offsets_[k]); + } return true; } } resize(0, variable_size); } } else { - uint8_t *value = upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size); + std::uint8_t *value = + upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size); if (value == nullptr) { return false; } else { SpinMutexLock lock(*(reinterpret_cast(value))); - for (unsigned int k = 0; k < handles_.size(); ++k) { - handles_[k]->mergeStatesFast(source_state + payload_offsets_[k], value + payload_offsets_[k]); + for (unsigned int k = 0; k < num_handles_; ++k) { + handles_[k]->mergeStatesFast(source_state + payload_offsets_[k], + value + payload_offsets_[k]); } return true; } @@ -1836,86 +1861,102 @@ template -bool FastHashTable - ::upsertValueAccessorFast(const std::vector> &argument_ids, - ValueAccessor *accessor, - const attribute_id key_attr_id, - const bool check_for_null_keys) { +bool FastHashTable:: + upsertValueAccessorFast( + const std::vector> &argument_ids, + ValueAccessor *accessor, + const attribute_id key_attr_id, + const bool check_for_null_keys) { DEBUG_ASSERT(!allow_duplicate_keys); std::size_t variable_size; std::vector local; return InvokeOnAnyValueAccessor( accessor, [&](auto *accessor) -> bool { // NOLINT(build/c++11) - if (resizable) { - bool continuing = true; - while (continuing) { - { - continuing = false; - SpinSharedMutexSharedLock lock(resize_shared_mutex_); + if (resizable) { + bool continuing = true; + while (continuing) { + { + continuing = false; + SpinSharedMutexSharedLock lock(resize_shared_mutex_); + while (accessor->next()) { + TypedValue key = accessor->getTypedValue(key_attr_id); + if (check_for_null_keys && key.isNull()) { + continue; + } + variable_size = (force_key_copy && !scalar_key_inline_) + ? key.getDataSize() + : 0; + std::uint8_t *value = + this->upsertInternalFast(key, variable_size, nullptr); + if (value == nullptr) { + continuing = true; + break; + } else { + SpinMutexLock lock(*(reinterpret_cast(value))); + for (unsigned int k = 0; k < num_handles_; ++k) { + local.clear(); + if (argument_ids[k].size()) { + local.emplace_back( + accessor->getTypedValue(argument_ids[k].front())); + } + handles_[k]->updateState(local, + value + payload_offsets_[k]); + } + } + } + } + if (continuing) { + this->resize(0, variable_size); + accessor->previous(); + } + } + } else { while (accessor->next()) { TypedValue key = accessor->getTypedValue(key_attr_id); if (check_for_null_keys && key.isNull()) { continue; } - variable_size = (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0; - uint8_t *value = this->upsertInternalFast(key, variable_size, nullptr); + variable_size = + (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0; + std::uint8_t *value = + this->upsertInternalFast(key, variable_size, nullptr); if (value == nullptr) { - continuing = true; - break; + return false; } else { SpinMutexLock lock(*(reinterpret_cast(value))); - for (unsigned int k = 0; k < handles_.size(); ++k) { - local.clear(); - if (argument_ids[k].size()) { - local.emplace_back(accessor->getTypedValue(argument_ids[k].front())); - } - handles_[k]->iterateInlFast(local, value + payload_offsets_[k]); + for (unsigned int k = 0; k < num_handles_; ++k) { + local.clear(); + if (argument_ids[k].size()) { + local.emplace_back( + accessor->getTypedValue(argument_ids[k].front())); + } + handles_[k]->updateState(local, value + payload_offsets_[k]); } } } } - if (continuing) { - this->resize(0, variable_size); - accessor->previous(); - } - } - } else { - while (accessor->next()) { - TypedValue key = accessor->getTypedValue(key_attr_id); - if (check_for_null_keys && key.isNull()) { - continue; - } - variable_size = (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0; - uint8_t *value = this->upsertInternalFast(key, variable_size, nullptr); - if (value == nullptr) { - return false; - } else { - SpinMutexLock lock(*(reinterpret_cast(value))); - for (unsigned int k = 0; k < handles_.size(); ++k) { - local.clear(); - if (argument_ids[k].size()) { - local.emplace_back(accessor->getTypedValue(argument_ids[k].front())); - } - handles_[k]->iterateInlFast(local, value + payload_offsets_[k]); - } - } - } - } - return true; - }); + return true; + }); } template -bool FastHashTable - ::upsertValueAccessorCompositeKeyFast(const std::vector> &argument_ids, - ValueAccessor *accessor, - const std::vector &key_attr_ids, - const bool check_for_null_keys) { +bool FastHashTable:: + upsertValueAccessorCompositeKeyFast( + const std::vector> &argument_ids, + ValueAccessor *accessor, + const std::vector &key_attr_ids, + const bool check_for_null_keys) { DEBUG_ASSERT(!allow_duplicate_keys); std::size_t variable_size; std::vector key_vector; @@ -1924,12 +1965,47 @@ bool FastHashTable bool { // NOLINT(build/c++11) - if (resizable) { - bool continuing = true; - while (continuing) { - { - continuing = false; - SpinSharedMutexSharedLock lock(resize_shared_mutex_); + if (resizable) { + bool continuing = true; + while (continuing) { + { + continuing = false; + SpinSharedMutexSharedLock lock(resize_shared_mutex_); + while (accessor->next()) { + if (this->GetCompositeKeyFromValueAccessor(*accessor, + key_attr_ids, + check_for_null_keys, + &key_vector)) { + continue; + } + variable_size = + this->calculateVariableLengthCompositeKeyCopySize( + key_vector); + std::uint8_t *value = this->upsertCompositeKeyInternalFast( + key_vector, nullptr, variable_size); + if (value == nullptr) { + continuing = true; + break; + } else { + SpinMutexLock lock(*(reinterpret_cast(value))); + for (unsigned int k = 0; k < num_handles_; ++k) { + local.clear(); + if (argument_ids[k].size()) { + local.emplace_back( + accessor->getTypedValue(argument_ids[k].front())); + } + handles_[k]->updateState(local, + value + payload_offsets_[k]); + } + } + } + } + if (continuing) { + this->resize(0, variable_size); + accessor->previous(); + } + } + } else { while (accessor->next()) { if (this->GetCompositeKeyFromValueAccessor(*accessor, key_attr_ids, @@ -1937,59 +2013,28 @@ bool FastHashTablecalculateVariableLengthCompositeKeyCopySize(key_vector); - uint8_t *value = this->upsertCompositeKeyInternalFast(key_vector, - nullptr, - variable_size); + variable_size = + this->calculateVariableLengthCompositeKeyCopySize(key_vector); + std::uint8_t *value = this->upsertCompositeKeyInternalFast( + key_vector, nullptr, variable_size); if (value == nullptr) { - continuing = true; - break; + return false; } else { SpinMutexLock lock(*(reinterpret_cast(value))); - for (unsigned int k = 0; k < handles_.size(); ++k) { - local.clear(); - if (argument_ids[k].size()) { - local.emplace_back(accessor->getTypedValue(argument_ids[k].front())); - } - handles_[k]->iterateInlFast(local, value + payload_offsets_[k]); + for (unsigned int k = 0; k < num_handles_; ++k) { + local.clear(); + if (argument_ids[k].size()) { + local.emplace_back( + accessor->getTypedValue(argument_ids[k].front())); + } + handles_[k]->updateState(local, value + payload_offsets_[k]); } } } } - if (continuing) { - this->resize(0, variable_size); - accessor->previous(); - } - } - } else { - while (accessor->next()) { - if (this->GetCompositeKeyFromValueAccessor(*accessor, - key_attr_ids, - check_for_null_keys, - &key_vector)) { - continue; - } - variable_size = this->calculateVariableLengthCompositeKeyCopySize(key_vector); - uint8_t *value = this->upsertCompositeKeyInternalFast(key_vector, - nullptr, - variable_size); - if (value == nullptr) { - return false; - } else { - SpinMutexLock lock(*(reinterpret_cast(value))); - for (unsigned int k = 0; k < handles_.size(); ++k) { - local.clear(); - if (argument_ids[k].size()) { - local.emplace_back(accessor->getTypedValue(argument_ids[k].front())); - } - handles_[k]->iterateInlFast(local, value + payload_offsets_[k]); - } - } - } - } - return true; - }); + return true; + }); } template template -void FastHashTable - ::getAllFromValueAccessor(ValueAccessor *accessor, - const attribute_id key_attr_id, - const bool check_for_null_keys, - FunctorT *functor) const { +void FastHashTable:: + getAllFromValueAccessor(ValueAccessor *accessor, + const attribute_id key_attr_id, + const bool check_for_null_keys, + FunctorT *functor) const { // Pass through to method with additional template parameters for less // branching in inner loop. if (check_for_null_keys) { @@ -2048,45 +2096,52 @@ template template -void FastHashTable - ::getAllFromValueAccessorCompositeKey(ValueAccessor *accessor, - const std::vector &key_attr_ids, - const bool check_for_null_keys, - FunctorT *functor) const { +void FastHashTable:: + getAllFromValueAccessorCompositeKey( + ValueAccessor *accessor, + const std::vector &key_attr_ids, + const bool check_for_null_keys, + FunctorT *functor) const { DEBUG_ASSERT(key_types_.size() == key_attr_ids.size()); std::vector key_vector; key_vector.resize(key_attr_ids.size()); InvokeOnAnyValueAccessor( accessor, [&](auto *accessor) -> void { // NOLINT(build/c++11) - while (accessor->next()) { - bool null_key = false; - for (std::vector::size_type key_idx = 0; - key_idx < key_types_.size(); - ++key_idx) { - key_vector[key_idx] = accessor->getTypedValue(key_attr_ids[key_idx]); - if (check_for_null_keys && key_vector[key_idx].isNull()) { - null_key = true; - break; - } - } - if (null_key) { - continue; - } + while (accessor->next()) { + bool null_key = false; + for (std::vector::size_type key_idx = 0; + key_idx < key_types_.size(); + ++key_idx) { + key_vector[key_idx] = + accessor->getTypedValue(key_attr_ids[key_idx]); + if (check_for_null_keys && key_vector[key_idx].isNull()) { + null_key = true; + break; + } + } + if (null_key) { + continue; + } - const std::size_t hash_code - = adjust_hashes_ ? this->AdjustHash(this->hashCompositeKey(key_vector)) - : this->hashCompositeKey(key_vector); - std::size_t entry_num = 0; - const uint8_t *value; - while (this->getNextEntryForCompositeKey(key_vector, hash_code, &value, &entry_num)) { - (*functor)(*accessor, *value); - if (!allow_duplicate_keys) { - break; + const std::size_t hash_code = + adjust_hashes_ + ? this->AdjustHash(this->hashCompositeKey(key_vector)) + : this->hashCompositeKey(key_vector); + std::size_t entry_num = 0; + const std::uint8_t *value; + while (this->getNextEntryForCompositeKey( + key_vector, hash_code, &value, &entry_num)) { + (*functor)(*accessor, *value); + if (!allow_duplicate_keys) { + break; + } + } } - } - } - }); + }); } template template void FastHashTable:: + serializable, + force_key_copy, + allow_duplicate_keys>:: getAllFromValueAccessorWithExtraWorkForFirstMatch( ValueAccessor *accessor, const attribute_id key_attr_id, @@ -2106,29 +2161,34 @@ void FastHashTable void { // NOLINT(build/c++11) - while (accessor->next()) { - TypedValue key = accessor->getTypedValue(key_attr_id); - if (check_for_null_keys && key.isNull()) { - continue; - } - const std::size_t hash_code = - adjust_hashes_ ? FastHashTable - ::AdjustHash(key.getHash()) - : key.getHash(); - std::size_t entry_num = 0; - const uint8_t *value; - if (this->getNextEntryForKey(key, hash_code, &value, &entry_num)) { - functor->recordMatch(*accessor); - (*functor)(*accessor, *value); - if (!allow_duplicate_keys) { - continue; - } - while (this->getNextEntryForKey(key, hash_code, &value, &entry_num)) { - (*functor)(*accessor, *value); + while (accessor->next()) { + TypedValue key = accessor->getTypedValue(key_attr_id); + if (check_for_null_keys && key.isNull()) { + continue; + } + const std::size_t hash_code = + adjust_hashes_ + ? FastHashTable< + resizable, + serializable, + force_key_copy, + allow_duplicate_keys>::AdjustHash(key.getHash()) + : key.getHash(); + std::size_t entry_num = 0; + const std::uint8_t *value; + if (this->getNextEntryForKey(key, hash_code, &value, &entry_num)) { + functor->recordMatch(*accessor); + (*functor)(*accessor, *value); + if (!allow_duplicate_keys) { + continue; + } + while ( + this->getNextEntryForKey(key, hash_code, &value, &entry_num)) { + (*functor)(*accessor, *value); + } + } } - } - } - }); // NOLINT(whitespace/parens) + }); // NOLINT(whitespace/parens) } template template -void FastHashTable - ::getAllFromValueAccessorCompositeKeyWithExtraWorkForFirstMatch( +void FastHashTable:: + getAllFromValueAccessorCompositeKeyWithExtraWorkForFirstMatch( ValueAccessor *accessor, const std::vector &key_attr_ids, const bool check_for_null_keys, @@ -2148,39 +2211,46 @@ void FastHashTable void { // NOLINT(build/c++11) - while (accessor->next()) { - bool null_key = false; - for (std::vector::size_type key_idx = 0; - key_idx < key_types_.size(); - ++key_idx) { - key_vector[key_idx] = accessor->getTypedValue(key_attr_ids[key_idx]); - if (check_for_null_keys && key_vector[key_idx].isNull()) { - null_key = true; - break; - } - } - if (null_key) { - continue; - } + while (accessor->next()) { + bool null_key = false; + for (std::vector::size_type key_idx = 0; + key_idx < key_types_.size(); + ++key_idx) { + key_vector[key_idx] = + accessor->getTypedValue(key_attr_ids[key_idx]); + if (check_for_null_keys && key_vector[key_idx].isNull()) { + null_key = true; + break; + } + } + if (null_key) { + continue; + } - const std::size_t hash_code = - adjust_hashes_ ? FastHashTable - ::AdjustHash(this->hashCompositeKey(key_vector)) - : this->hashCompositeKey(key_vector); - std::size_t entry_num = 0; - const uint8_t *value; - if (this->getNextEntryForCompositeKey(key_vector, hash_code, &value, &entry_num)) { - functor->recordMatch(*accessor); - (*functor)(*accessor, *value); - if (!allow_duplicate_keys) { - continue; - } - while (this->getNextEntryForCompositeKey(key_vector, hash_code, &value, &entry_num)) { - (*functor)(*accessor, *value); + const std::size_t hash_code = + adjust_hashes_ + ? FastHashTable:: + AdjustHash(this->hashCompositeKey(key_vector)) + : this->hashCompositeKey(key_vector); + std::size_t entry_num = 0; + const std::uint8_t *value; + if (this->getNextEntryForCompositeKey( + key_vector, hash_code, &value, &entry_num)) { + functor->recordMatch(*accessor); + (*functor)(*accessor, *value); + if (!allow_duplicate_keys) { + continue; + } + while (this->getNextEntryForCompositeKey( + key_vector, hash_code, &value, &entry_num)) { + (*functor)(*accessor, *value); + } + } } - } - } - }); // NOLINT(whitespace/parens) + }); // NOLINT(whitespace/parens) } template template void FastHashTable:: + serializable, + force_key_copy, + allow_duplicate_keys>:: runOverKeysFromValueAccessor(ValueAccessor *accessor, const attribute_id key_attr_id, const bool check_for_null_keys, FunctorT *functor) const { - InvokeOnAnyValueAccessor( - accessor, - [&](auto *accessor) -> void { // NOLINT(build/c++11) - while (accessor->next()) { - TypedValue key = accessor->getTypedValue(key_attr_id); - if (check_for_null_keys && key.isNull()) { - if (!run_if_match_found) { - (*functor)(*accessor); - continue; - } - } - if (run_if_match_found) { - if (this->hasKey(key)) { - (*functor)(*accessor); - } - } else { - if (!this->hasKey(key)) { - (*functor)(*accessor); - } - } - } - }); // NOLINT(whitespace/parens) + InvokeOnAnyValueAccessor(accessor, + [&](auto *accessor) -> void { // NOLINT(build/c++11) + while (accessor->next()) { + TypedValue key = + accessor->getTypedValue(key_attr_id); + if (check_for_null_keys && key.isNull()) { + if (!run_if_match_found) { + (*functor)(*accessor); + continue; + } + } + if (run_if_match_found) { + if (this->hasKey(key)) { + (*functor)(*accessor); + } + } else { + if (!this->hasKey(key)) { + (*functor)(*accessor); + } + } + } + }); // NOLINT(whitespace/parens) } template template -void FastHashTable - ::runOverKeysFromValueAccessorCompositeKey(ValueAccessor *accessor, - const std::vector &key_attr_ids, - const bool check_for_null_keys, - FunctorT *functor) const { +void FastHashTable:: + runOverKeysFromValueAccessorCompositeKey( + ValueAccessor *accessor, + const std::vector &key_attr_ids, + const bool check_for_null_keys, + FunctorT *functor) const { DEBUG_ASSERT(key_types_.size() == key_attr_ids.size()); std::vector key_vector; key_vector.resize(key_attr_ids.size()); InvokeOnAnyValueAccessor( accessor, [&](auto *accessor) -> void { // NOLINT(build/c++11) - while (accessor->next()) { - bool null_key = false; - for (std::vector::size_type key_idx = 0; - key_idx < key_types_.size(); - ++key_idx) { - key_vector[key_idx] = accessor->getTypedValue(key_attr_ids[key_idx]); - if (check_for_null_keys && key_vector[key_idx].isNull()) { - null_key = true; - break; - } - } - if (null_key) { - if (!run_if_match_found) { - (*functor)(*accessor); - continue; - } - } + while (accessor->next()) { + bool null_key = false; + for (std::vector::size_type key_idx = 0; + key_idx < key_types_.size(); + ++key_idx) { + key_vector[key_idx] = + accessor->getTypedValue(key_attr_ids[key_idx]); + if (check_for_null_keys && key_vector[key_idx].isNull()) { + null_key = true; + break; + } + } + if (null_key) { + if (!run_if_match_found) { + (*functor)(*accessor); + continue; + } + } - if (run_if_match_found) { - if (this->hasCompositeKey(key_vector)) { - (*functor)(*accessor); + if (run_if_match_found) { + if (this->hasCompositeKey(key_vector)) { + (*functor)(*accessor); + } + } else if (!this->hasCompositeKey(key_vector)) { + (*functor)(*accessor); + } } - } else if (!this->hasCompositeKey(key_vector)) { - (*functor)(*accessor); - } - } - }); // NOLINT(whitespace/parens) + }); // NOLINT(whitespace/parens) } template template -std::size_t FastHashTable - ::forEach(FunctorT *functor) const { +std::size_t +FastHashTable:: + forEach(FunctorT *functor) const { std::size_t entries_visited = 0; std::size_t entry_num = 0; TypedValue key; - const uint8_t *value_ptr; + const std::uint8_t *value_ptr; while (getNextEntry(&key, &value_ptr, &entry_num)) { ++entries_visited; (*functor)(key, *value_ptr); @@ -2288,12 +2364,13 @@ template template -std::size_t FastHashTable - ::forEachCompositeKeyFast(FunctorT *functor) const { +std::size_t +FastHashTable:: + forEachCompositeKeyFast(FunctorT *functor) const { std::size_t entries_visited = 0; std::size_t entry_num = 0; std::vector key; - const uint8_t *value_ptr; + const std::uint8_t *value_ptr; while (getNextEntryCompositeKey(&key, &value_ptr, &entry_num)) { ++entries_visited; (*functor)(key, value_ptr); @@ -2302,22 +2379,21 @@ std::size_t FastHashTable template -std::size_t FastHashTable - ::forEachCompositeKeyFast(FunctorT *functor, - int index) const { +std::size_t +FastHashTable:: + forEachCompositeKeyFast(FunctorT *functor, int index) const { std::size_t entries_visited = 0; std::size_t entry_num = 0; std::vector key; - const uint8_t *value_ptr; + const std::uint8_t *value_ptr; while (getNextEntryCompositeKey(&key, &value_ptr, &entry_num)) { ++entries_visited; - (*functor)(key, value_ptr+payload_offsets_[index]); + (*functor)(key, value_ptr + payload_offsets_[index]); key.clear(); } return entries_visited; @@ -2327,8 +2403,9 @@ template -inline std::size_t FastHashTable - ::hashCompositeKey(const std::vector &key) const { +inline std::size_t +FastHashTable