You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by ra...@apache.org on 2016/08/21 10:34:38 UTC

[3/7] incubator-quickstep git commit: Modified Aggregation unit test. Ran clang-format.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dad7d6f3/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
index 909fcc0..9b67734 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"
@@ -115,9 +115,9 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 class FastHashTable : public HashTableBase<resizable,
-                                       serializable,
-                                       force_key_copy,
-                                       allow_duplicate_keys> {
+                                           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.");
@@ -129,7 +129,7 @@ class FastHashTable : public HashTableBase<resizable,
 
  public:
   // Shadow template parameters. This is useful for shared test harnesses.
-//  typedef ValueT value_type;
+  //  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 +162,9 @@ class FastHashTable : public HashTableBase<resizable,
     if (resizable) {
       if (blob_.valid()) {
         if (serializable) {
-          DEV_WARNING("Destroying a resizable serializable HashTable's underlying "
-                      "StorageBlob.");
+          DEV_WARNING(
+              "Destroying a resizable serializable HashTable's underlying "
+              "StorageBlob.");
         }
         const block_id blob_id = blob_->getID();
         blob_.release();
@@ -212,8 +213,7 @@ class FastHashTable : public HashTableBase<resizable,
    *         resizable is false and storage space for the hash table has been
    *         exhausted.
    **/
-  HashTablePutResult put(const TypedValue &key,
-                         const uint8_t &value);
+  HashTablePutResult put(const TypedValue &key, const std::uint8_t &value);
 
   /**
    * @brief Add a new entry into the hash table (composite key version).
@@ -238,7 +238,7 @@ class FastHashTable : public HashTableBase<resizable,
    **/
 
   HashTablePutResult putCompositeKeyFast(const std::vector<TypedValue> &key,
-                                     const uint8_t *value_ptr);
+                                         const std::uint8_t *value_ptr);
 
   /**
    * @brief Add (multiple) new entries into the hash table from a
@@ -378,7 +378,7 @@ class FastHashTable : public HashTableBase<resizable,
    **/
   template <typename FunctorT>
   bool upsert(const TypedValue &key,
-              const uint8_t *initial_value_ptr,
+              const std::uint8_t *initial_value_ptr,
               FunctorT *functor);
 
   /**
@@ -421,18 +421,18 @@ class FastHashTable : public HashTableBase<resizable,
    **/
   template <typename FunctorT>
   bool upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                          const uint8_t *init_value_ptr,
-                          FunctorT *functor);
+                              const std::uint8_t *init_value_ptr,
+                              FunctorT *functor);
 
   template <typename FunctorT>
   bool upsertCompositeKeyFast(const std::vector<TypedValue> &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<TypedValue> &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 +481,11 @@ class FastHashTable : public HashTableBase<resizable,
    *         accessor's iteration will be left on the first tuple which could
    *         not be inserted).
    **/
-  bool upsertValueAccessorFast(const std::vector<std::vector<attribute_id>> &argument_ids,
-                           ValueAccessor *accessor,
-                           const attribute_id key_attr_id,
-                           const bool check_for_null_keys);
+  bool upsertValueAccessorFast(
+      const std::vector<std::vector<attribute_id>> &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 +583,7 @@ class FastHashTable : public HashTableBase<resizable,
    * @return The value of a matched entry if a matching key is found.
    *         Otherwise, return NULL.
    **/
-  virtual const uint8_t* getSingle(const TypedValue &key) const = 0;
+  virtual const std::uint8_t* getSingle(const TypedValue &key) const = 0;
 
   /**
    * @brief Lookup a composite key against this hash table to find a matching
@@ -607,9 +608,10 @@ class FastHashTable : public HashTableBase<resizable,
    * @return The value of a matched entry if a matching key is found.
    *         Otherwise, return NULL.
    **/
-  virtual const uint8_t* getSingleCompositeKey(const std::vector<TypedValue> &key) const = 0;
-  virtual const uint8_t* getSingleCompositeKey(const std::vector<TypedValue> &key,
-                                               int index) const = 0;
+  virtual const std::uint8_t* getSingleCompositeKey(
+      const std::vector<TypedValue> &key) const = 0;
+  virtual const std::uint8_t *getSingleCompositeKey(
+      const std::vector<TypedValue> &key, int index) const = 0;
 
   /**
    * @brief Lookup a key against this hash table to find matching entries.
@@ -634,7 +636,8 @@ class FastHashTable : public HashTableBase<resizable,
    * @param values A vector to hold values of all matching entries. Matches
    *        will be appended to the vector.
    **/
-  virtual void getAll(const TypedValue &key, std::vector<const uint8_t*> *values) const = 0;
+  virtual void getAll(const TypedValue &key,
+                      std::vector<const std::uint8_t *> *values) const = 0;
 
   /**
    * @brief Lookup a composite key against this hash table to find matching
@@ -659,8 +662,9 @@ class FastHashTable : public HashTableBase<resizable,
    * @param values A vector to hold values of all matching entries. Matches
    *        will be appended to the vector.
    **/
-  virtual void getAllCompositeKey(const std::vector<TypedValue> &key,
-                                  std::vector<const uint8_t*> *values) const = 0;
+  virtual void getAllCompositeKey(
+      const std::vector<TypedValue> &key,
+      std::vector<const std::uint8_t *> *values) const = 0;
 
   /**
    * @brief Lookup (multiple) keys from a ValueAccessor and apply a functor to
@@ -726,7 +730,8 @@ class FastHashTable : public HashTableBase<resizable,
    *        set to true if some of the keys that will be read from accessor may
    *        be null.
    * @param functor A pointer to a functor, which should provide two functions:
-   *        1) An operator that takes 2 arguments: const ValueAccessor& (or better
+   *        1) An operator that takes 2 arguments: const ValueAccessor& (or
+   * better
    *        yet, a templated call operator which takes a const reference to
    *        some subclass of ValueAccessor as its first argument) and
    *        const ValueT&. The operator will be invoked once for each pair of a
@@ -767,7 +772,8 @@ class FastHashTable : public HashTableBase<resizable,
    *        set to true if some of the keys that will be read from accessor may
    *        be null.
    * @param functor A pointer to a functor, which should provide two functions:
-   *        1) An operator that takes 2 arguments: const ValueAccessor& (or better
+   *        1) An operator that takes 2 arguments: const ValueAccessor& (or
+   * better
    *        yet, a templated call operator which takes a const reference to
    *        some subclass of ValueAccessor as its first argument) and
    *        const ValueT&. The operator will be invoked once for each pair of a
@@ -817,10 +823,11 @@ class FastHashTable : public HashTableBase<resizable,
    *        key taken from accessor and matching value.
    **/
   template <typename FunctorT>
-  void getAllFromValueAccessorCompositeKey(ValueAccessor *accessor,
-                                           const std::vector<attribute_id> &key_attr_ids,
-                                           const bool check_for_null_keys,
-                                           FunctorT *functor) const;
+  void getAllFromValueAccessorCompositeKey(
+      ValueAccessor *accessor,
+      const std::vector<attribute_id> &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 +849,8 @@ class FastHashTable : public HashTableBase<resizable,
                                                 const attribute_id key_attr_id,
                                                 const bool check_for_null_keys,
                                                 FunctorT *functor) const {
-    return runOverKeysFromValueAccessor<true>(accessor,
-                                              key_attr_id,
-                                              check_for_null_keys,
-                                              functor);
+    return runOverKeysFromValueAccessor<true>(
+        accessor, key_attr_id, check_for_null_keys, functor);
   }
 
   /**
@@ -869,10 +874,8 @@ class FastHashTable : public HashTableBase<resizable,
       const std::vector<attribute_id> &key_attr_ids,
       const bool check_for_null_keys,
       FunctorT *functor) const {
-    return runOverKeysFromValueAccessorCompositeKey<true>(accessor,
-                                                          key_attr_ids,
-                                                          check_for_null_keys,
-                                                          functor);
+    return runOverKeysFromValueAccessorCompositeKey<true>(
+        accessor, key_attr_ids, check_for_null_keys, functor);
   }
 
   /**
@@ -896,10 +899,8 @@ class FastHashTable : public HashTableBase<resizable,
       const attribute_id key_attr_id,
       const bool check_for_null_keys,
       FunctorT *functor) const {
-    return runOverKeysFromValueAccessor<false>(accessor,
-                                               key_attr_id,
-                                               check_for_null_keys,
-                                               functor);
+    return runOverKeysFromValueAccessor<false>(
+        accessor, key_attr_id, check_for_null_keys, functor);
   }
 
   /**
@@ -923,10 +924,8 @@ class FastHashTable : public HashTableBase<resizable,
       const std::vector<attribute_id> &key_attr_ids,
       const bool check_for_null_keys,
       FunctorT *functor) const {
-    return runOverKeysFromValueAccessorCompositeKey<false>(accessor,
-                                                           key_attr_ids,
-                                                           check_for_null_keys,
-                                                           functor);
+    return runOverKeysFromValueAccessorCompositeKey<false>(
+        accessor, key_attr_ids, check_for_null_keys, functor);
   }
 
   /**
@@ -983,8 +982,7 @@ class FastHashTable : public HashTableBase<resizable,
   std::size_t forEachCompositeKeyFast(FunctorT *functor) const;
 
   template <typename FunctorT>
-  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 +1035,8 @@ class FastHashTable : public HashTableBase<resizable,
    * @param probe_attribute_ids The vector of attribute ids to use for probing
    *        the bloom filter.
    **/
-  inline void addProbeSideAttributeIds(std::vector<attribute_id> &&probe_attribute_ids) {
+  inline void addProbeSideAttributeIds(
+      std::vector<attribute_id> &&probe_attribute_ids) {
     probe_attribute_ids_.push_back(probe_attribute_ids);
   }
 
@@ -1065,30 +1064,32 @@ class FastHashTable : public HashTableBase<resizable,
    *        pass when bulk-inserting entries. If false, resources are allocated
    *        on the fly for each entry.
    **/
-  FastHashTable(const std::vector<const Type*> &key_types,
-            const std::size_t num_entries,
-            const std::vector<AggregationHandle *> &handles,
-            const std::vector<std::size_t> &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<const Type *> &key_types,
+                const std::size_t num_entries,
+                const std::vector<AggregationHandle *> &handles,
+                const std::vector<std::size_t> &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 +1123,14 @@ class FastHashTable : public HashTableBase<resizable,
    *        pass when bulk-inserting entries. If false, resources are allocated
    *        on the fly for each entry.
    **/
-  FastHashTable(const std::vector<const Type*> &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<const Type *> &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 +1170,17 @@ class FastHashTable : public HashTableBase<resizable,
 
   // Helpers for put. If this HashTable is resizable, 'resize_shared_mutex_'
   // should be locked in shared mode before calling either of these methods.
-  virtual HashTablePutResult putInternal(const TypedValue &key,
-                                         const std::size_t variable_key_size,
-                                         const uint8_t &value,
-                                         HashTablePreallocationState *prealloc_state) = 0;
-
-  virtual HashTablePutResult putCompositeKeyInternalFast(const std::vector<TypedValue> &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<TypedValue> &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 +1188,15 @@ class FastHashTable : public HashTableBase<resizable,
   // return NULL if there is not enough space to insert a new key, in which
   // case a resizable HashTable should release the 'resize_shared_mutex_' and
   // call resize(), then try again.
-  virtual uint8_t* upsertInternalFast(const TypedValue &key,
-                                 const std::size_t variable_key_size,
-                                 const std::uint8_t *init_value_ptr) = 0;
+  virtual std::uint8_t *upsertInternalFast(
+      const TypedValue &key,
+      const std::size_t variable_key_size,
+      const std::uint8_t *init_value_ptr) = 0;
 
-  virtual uint8_t* upsertCompositeKeyInternalFast(const std::vector<TypedValue> &key,
-                                                  const std::uint8_t *init_value_ptr,
-                                                  const std::size_t variable_key_size) = 0;
+  virtual std::uint8_t *upsertCompositeKeyInternalFast(
+      const std::vector<TypedValue> &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 +1204,10 @@ class FastHashTable : public HashTableBase<resizable,
   // '*entry_num' is incremented to the next (implementation defined) entry to
   // check ('*entry_num' should initially be set to zero).
   virtual bool getNextEntry(TypedValue *key,
-                            const uint8_t **value,
+                            const std::uint8_t **value,
                             std::size_t *entry_num) const = 0;
   virtual bool getNextEntryCompositeKey(std::vector<TypedValue> *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 +1217,11 @@ class FastHashTable : public HashTableBase<resizable,
   // initially be set to zero).
   virtual bool getNextEntryForKey(const TypedValue &key,
                                   const std::size_t hash_code,
-                                  const uint8_t **value,
+                                  const std::uint8_t **value,
                                   std::size_t *entry_num) const = 0;
   virtual bool getNextEntryForCompositeKey(const std::vector<TypedValue> &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 +1254,17 @@ class FastHashTable : public HashTableBase<resizable,
   // method is intended to support that. Returns true and fills in
   // '*prealloc_state' if pre-allocation was successful. Returns false if a
   // resize() is needed.
-  virtual bool preallocateForBulkInsert(const std::size_t total_entries,
-                                        const std::size_t total_variable_key_size,
-                                        HashTablePreallocationState *prealloc_state) {
-    FATAL_ERROR("Called HashTable::preallocateForBulkInsert() on a HashTable "
-                "implementation that does not support preallocation.");
+  virtual bool preallocateForBulkInsert(
+      const std::size_t total_entries,
+      const std::size_t total_variable_key_size,
+      HashTablePreallocationState *prealloc_state) {
+    FATAL_ERROR(
+        "Called HashTable::preallocateForBulkInsert() on a HashTable "
+        "implementation that does not support preallocation.");
   }
 
   // Type(s) of keys.
-  const std::vector<const Type*> key_types_;
+  const std::vector<const Type *> 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 +1281,7 @@ class FastHashTable : public HashTableBase<resizable,
   const bool preallocate_supported_;
 
   const std::vector<AggregationHandle *> handles_;
+  const unsigned int num_handles_;
   const std::size_t total_payload_size_;
   std::vector<std::size_t> payload_offsets_;
 
@@ -1341,12 +1348,11 @@ class FastHashTable : public HashTableBase<resizable,
   bool has_build_side_bloom_filter_ = false;
   bool has_probe_side_bloom_filter_ = false;
   BloomFilter *build_bloom_filter_;
-  std::vector<const BloomFilter*> probe_bloom_filters_;
+  std::vector<const BloomFilter *> probe_bloom_filters_;
   std::vector<std::vector<attribute_id>> 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 +1369,11 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-HashTablePutResult FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::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<resizable, serializable, force_key_copy, allow_duplicate_keys>::
+    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 +1395,19 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-HashTablePutResult FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::putCompositeKeyFast(const std::vector<TypedValue> &key,
-                      const std::uint8_t* init_value_ptr) {
-  const std::size_t variable_size = calculateVariableLengthCompositeKeyCopySize(key);
+HashTablePutResult
+FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
+    putCompositeKeyFast(const std::vector<TypedValue> &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<true> 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 +1415,22 @@ HashTablePutResult FastHashTable<resizable, serializable, force_key_copy, allow_
     }
     return result;
   } else {
-    return putCompositeKeyInternalFast(key, variable_size, init_value_ptr, nullptr);
+    return putCompositeKeyInternalFast(
+        key, variable_size, init_value_ptr, nullptr);
   }
 }
 
-
 template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-HashTablePutResult FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::putValueAccessor(ValueAccessor *accessor,
-                       const attribute_id key_attr_id,
-                       const bool check_for_null_keys,
-                       FunctorT *functor) {
+HashTablePutResult
+FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
+    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 +1438,120 @@ HashTablePutResult FastHashTable<resizable, serializable, force_key_copy, allow_
   return InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> 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<true> 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<true> 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<BloomFilter> 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<true> lock(resize_shared_mutex_);
+        std::unique_ptr<BloomFilter> 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<true> 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<const std::uint8_t *>(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<const std::uint8_t *>(key.getDataPtr()),
-                                                      key.getDataSize());
+              thread_local_bloom_filter->insertUnSafe(
+                  static_cast<const std::uint8_t *>(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<const std::uint8_t *>(key.getDataPtr()),
-                                                  key.getDataSize());
+          build_bloom_filter_->bitwiseOr(thread_local_bloom_filter.get());
         }
-        if (result != HashTablePutResult::kOK) {
-          return result;
-        }
-      }
-    }
-    // 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 <bool resizable,
@@ -1540,11 +1559,12 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-HashTablePutResult FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::putValueAccessorCompositeKey(ValueAccessor *accessor,
-                                   const std::vector<attribute_id> &key_attr_ids,
-                                   const bool check_for_null_keys,
-                                   FunctorT *functor) {
+HashTablePutResult
+FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
+    putValueAccessorCompositeKey(ValueAccessor *accessor,
+                                 const std::vector<attribute_id> &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 +1575,79 @@ HashTablePutResult FastHashTable<resizable, serializable, force_key_copy, allow_
   return InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> 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<true> 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<true> 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<true> lock(resize_shared_mutex_);
+        if (resizable) {
+          while (result == HashTablePutResult::kOutOfSpace) {
+            {
+              result = HashTablePutResult::kOK;
+              SpinSharedMutexSharedLock<true> 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 +1655,21 @@ HashTablePutResult FastHashTable<resizable, serializable, force_key_copy, allow_
                                                        &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);
+            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 <bool resizable,
@@ -1653,17 +1677,22 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsert(const TypedValue &key,
-             const uint8_t *initial_value_ptr,
-             FunctorT *functor) {
+bool FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::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<true> 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 +1701,8 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
       resize(0, force_key_copy && !scalar_key_inline_ ? key.getDataSize() : 0);
     }
   } else {
-    uint8_t *value = upsertInternalFast(key, variable_size, initial_value_ptr);
+    std::uint8_t *value =
+        upsertInternalFast(key, variable_size, initial_value_ptr);
     if (value == nullptr) {
       return false;
     } else {
@@ -1691,8 +1721,11 @@ class HashTableMergerFast {
    * @param destination_hash_table The destination hash table to which other
    *        hash tables will be merged.
    **/
-  explicit HashTableMergerFast(AggregationStateHashTableBase *destination_hash_table)
-      : destination_hash_table_(static_cast<FastHashTable<true, false, true, false> *>(destination_hash_table)) {}
+  explicit HashTableMergerFast(
+      AggregationStateHashTableBase *destination_hash_table)
+      : destination_hash_table_(
+            static_cast<FastHashTable<true, false, true, false> *>(
+                destination_hash_table)) {}
 
   /**
    * @brief The operator for the functor.
@@ -1702,8 +1735,8 @@ class HashTableMergerFast {
    *        aggregation hash table.
    **/
   inline void operator()(const std::vector<TypedValue> &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
@@ -1722,23 +1755,27 @@ class HashTableMergerFast {
   DISALLOW_COPY_AND_ASSIGN(HashTableMergerFast);
 };
 
-
 template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                         const std::uint8_t *init_value_ptr,
-                         FunctorT *functor) {
+bool FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
+    upsertCompositeKeyFast(const std::vector<TypedValue> &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<true> 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 +1784,8 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
       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 {
@@ -1762,70 +1800,83 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                         const std::uint8_t *init_value_ptr,
-                         FunctorT *functor,
-                         int index) {
+bool FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
+    upsertCompositeKeyFast(const std::vector<TypedValue> &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<true> 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 resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                         const std::uint8_t *init_value_ptr,
-                         const std::uint8_t *source_state) {
+bool FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
+    upsertCompositeKeyFast(const std::vector<TypedValue> &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<true> 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<SpinMutex *>(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<SpinMutex *>(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<SpinMutex *>(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 +1887,102 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsertValueAccessorFast(const std::vector<std::vector<attribute_id>> &argument_ids,
-                          ValueAccessor *accessor,
-                          const attribute_id key_attr_id,
-                          const bool check_for_null_keys) {
+bool FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
+    upsertValueAccessorFast(
+        const std::vector<std::vector<attribute_id>> &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<TypedValue> local;
   return InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
-    if (resizable) {
-      bool continuing = true;
-      while (continuing) {
-        {
-          continuing = false;
-          SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
+        if (resizable) {
+          bool continuing = true;
+          while (continuing) {
+            {
+              continuing = false;
+              SpinSharedMutexSharedLock<true> 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<SpinMutex *>(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<SpinMutex *>(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<SpinMutex *>(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 resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsertValueAccessorCompositeKeyFast(const std::vector<std::vector<attribute_id>> &argument_ids,
-                                      ValueAccessor *accessor,
-                                      const std::vector<attribute_id> &key_attr_ids,
-                                      const bool check_for_null_keys) {
+bool FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
+    upsertValueAccessorCompositeKeyFast(
+        const std::vector<std::vector<attribute_id>> &argument_ids,
+        ValueAccessor *accessor,
+        const std::vector<attribute_id> &key_attr_ids,
+        const bool check_for_null_keys) {
   DEBUG_ASSERT(!allow_duplicate_keys);
   std::size_t variable_size;
   std::vector<TypedValue> key_vector;
@@ -1924,12 +1991,47 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
   return InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
-    if (resizable) {
-      bool continuing = true;
-      while (continuing) {
-        {
-          continuing = false;
-          SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
+        if (resizable) {
+          bool continuing = true;
+          while (continuing) {
+            {
+              continuing = false;
+              SpinSharedMutexSharedLock<true> 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<SpinMutex *>(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 +2039,28 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
                                                        &key_vector)) {
               continue;
             }
-            variable_size = this->calculateVariableLengthCompositeKeyCopySize(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<SpinMutex *>(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<SpinMutex *>(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 resizable,
@@ -1997,11 +2068,14 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-void FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getAllFromValueAccessor(ValueAccessor *accessor,
-                              const attribute_id key_attr_id,
-                              const bool check_for_null_keys,
-                              FunctorT *functor) const {
+void FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
+    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 +2122,52 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-void FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getAllFromValueAccessorCompositeKey(ValueAccessor *accessor,
-                                          const std::vector<attribute_id> &key_attr_ids,
-                                          const bool check_for_null_keys,
-                                          FunctorT *functor) const {
+void FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
+    getAllFromValueAccessorCompositeKey(
+        ValueAccessor *accessor,
+        const std::vector<attribute_id> &key_attr_ids,
+        const bool check_for_null_keys,
+        FunctorT *functor) const {
   DEBUG_ASSERT(key_types_.size() == key_attr_ids.size());
   std::vector<TypedValue> 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<attribute_id>::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<attribute_id>::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 <bool resizable,
@@ -2095,9 +2176,9 @@ template <bool resizable,
           bool allow_duplicate_keys>
 template <typename FunctorT>
 void FastHashTable<resizable,
-               serializable,
-               force_key_copy,
-               allow_duplicate_keys>::
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
     getAllFromValueAccessorWithExtraWorkForFirstMatch(
         ValueAccessor *accessor,
         const attribute_id key_attr_id,
@@ -2106,29 +2187,34 @@ void FastHashTable<resizable,
   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()) {
-        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 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 <bool resizable,
@@ -2136,8 +2222,11 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-void FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getAllFromValueAccessorCompositeKeyWithExtraWorkForFirstMatch(
+void FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
+    getAllFromValueAccessorCompositeKeyWithExtraWorkForFirstMatch(
         ValueAccessor *accessor,
         const std::vector<attribute_id> &key_attr_ids,
         const bool check_for_null_keys,
@@ -2148,39 +2237,46 @@ void FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
   InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-    while (accessor->next()) {
-      bool null_key = false;
-      for (std::vector<attribute_id>::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<attribute_id>::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<resizable, serializable, force_key_copy, allow_duplicate_keys>
-                               ::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<resizable,
+                                  serializable,
+                                  force_key_copy,
+                                  allow_duplicate_keys>::
+                        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 <bool resizable,
@@ -2189,35 +2285,35 @@ template <bool resizable,
           bool allow_duplicate_keys>
 template <bool run_if_match_found, typename FunctorT>
 void FastHashTable<resizable,
-               serializable,
-               force_key_copy,
-               allow_duplicate_keys>::
+                   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 <bool resizable,
@@ -2225,44 +2321,49 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <bool run_if_match_found, typename FunctorT>
-void FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::runOverKeysFromValueAccessorCompositeKey(ValueAccessor *accessor,
-                                               const std::vector<attribute_id> &key_attr_ids,
-                                               const bool check_for_null_keys,
-                                               FunctorT *functor) const {
+void FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
+    runOverKeysFromValueAccessorCompositeKey(
+        ValueAccessor *accessor,
+        const std::vector<attribute_id> &key_attr_ids,
+        const bool check_for_null_keys,
+        FunctorT *functor) const {
   DEBUG_ASSERT(key_types_.size() == key_attr_ids.size());
   std::vector<TypedValue> 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<attribute_id>::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<attribute_id>::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 <bool resizable,
@@ -2270,12 +2371,13 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-std::size_t FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::forEach(FunctorT *functor) const {
+std::size_t
+FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
+    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 +2390,13 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-std::size_t FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::forEachCompositeKeyFast(FunctorT *functor) const {
+std::size_t
+FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
+    forEachCompositeKeyFast(FunctorT *functor) const {
   std::size_t entries_visited = 0;
   std::size_t entry_num = 0;
   std::vector<TypedValue> 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 +2405,21 @@ std::size_t FastHashTable<resizable, serializable, force_key_copy, allow_duplica
   return entries_visited;
 }
 
-
 template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 template <typename FunctorT>
-std::size_t FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::forEachCompositeKeyFast(FunctorT *functor,
-                              int index) const {
+std::size_t
+FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
+    forEachCompositeKeyFast(FunctorT *functor, int index) const {
   std::size_t entries_visited = 0;
   std::size_t entry_num = 0;
   std::vector<TypedValue> 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 +2429,9 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-inline std::size_t FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::hashCompositeKey(const std::vector<TypedValue> &key) const {
+inline std::size_t
+FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
+    hashCompositeKey(const std::vector<TypedValue> &key) const {
   DEBUG_ASSERT(!key.empty());
   DEBUG_ASSERT(key.size() == key_types_.size());
   std::size_t hash = key.front().getHash();
@@ -2344,15 +2447,15 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-inline std::size_t FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::calculateVariableLengthCompositeKeyCopySize(const std::vector<TypedValue> &key) const {
+inline std::size_t
+FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
+    calculateVariableLengthCompositeKeyCopySize(
+        const std::vector<TypedValue> &key) const {
   DEBUG_ASSERT(!key.empty());
   DEBUG_ASSERT(key.size() == key_types_.size());
   if (force_key_copy) {
     std::size_t total = 0;
-    for (std::vector<TypedValue>::size_type idx = 0;
-         idx < key.size();
-         ++idx) {
+    for (std::vector<TypedValue>::size_type idx = 0; idx < key.size(); ++idx) {
       if (!(*key_inline_)[idx]) {
         total += key[idx].getDataSize();
       }
@@ -2371,54 +2474,62 @@ template <typename FunctorT,
           bool check_for_null_keys,
           bool adjust_hashes_template,
           bool use_scalar_literal_hash_template>
-void FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getAllFromValueAccessorImpl(
-        ValueAccessor *accessor,
-        const attribute_id key_attr_id,
-        FunctorT *functor) const {
+void FastHashTable<resizable,
+                   serializable,
+                   force_key_copy,
+                   allow_duplicate_keys>::
+    getAllFromValueAccessorImpl(ValueAccessor *accessor,
+                                const attribute_id key_attr_id,
+                                FunctorT *functor) const {
   InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-    while (accessor->next()) {
-      // Probe any bloom filters, if enabled.
-      if (has_probe_side_bloom_filter_) {
-        DCHECK_EQ(probe_bloom_filters_.size(), probe_attribute_ids_.size());
-        // Check if the key is contained in the BloomFilters or not.
-        bool bloom_miss = false;
-        for (std::size_t i = 0; i < probe_bloom_filters_.size() && !bloom_miss; ++i) {
-          const BloomFilter *bloom_filter = probe_bloom_filters_[i];
-          for (const attribute_id &attr_id : probe_attribute_ids_[i]) {
-            TypedValue bloom_key = accessor->getTypedValue(attr_id);
-            if (!bloom_filter->contains(static_cast<const std::uint8_t*>(bloom_key.getDataPtr()),
-                                        bloom_key.getDataSize())) {
-              bloom_miss = true;
-              break;
+        while (accessor->next()) {
+          // Probe any bloom filters, if enabled.
+          if (has_probe_side_bloom_filter_) {
+            DCHECK_EQ(probe_bloom_filters_.size(), probe_attribute_ids_.size());
+            // Check if the key is contained in the BloomFilters or not.
+            bool bloom_miss = false;
+            for (std::size_t i = 0;
+                 i < probe_bloom_filters_.size() && !bloom_miss;
+                 ++i) {
+              const BloomFilter *bloom_filter = probe_bloom_filters_[i];
+              for (const attribute_id &attr_id : probe_attribute_ids_[i]) {
+                TypedValue bloom_key = accessor->getTypedValue(attr_id);
+                if (!bloom_filter->contains(static_cast<const std::uint8_t *>(
+                                                bloom_key.getDataPtr()),
+                                            bloom_key.getDataSize())) {
+                  bloom_miss = true;
+                  break;
+                }
+              }
+            }
+            if (bloom_miss) {
+              continue;  

<TRUNCATED>