You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by hb...@apache.org on 2016/09/20 17:57:44 UTC

[01/28] incubator-quickstep git commit: QUICKSTEP-36 fixed. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/partitioned-aggregation be43958fc -> 402c99a46 (forced update)


QUICKSTEP-36 fixed.

Added read access check in TextScanOperator file before opening a file


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/43c7a42d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/43c7a42d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/43c7a42d

Branch: refs/heads/partitioned-aggregation
Commit: 43c7a42db736aa6e61e6f4db12721ded6e646e13
Parents: 590ba4d
Author: Tarun Bansal <ta...@gmail.com>
Authored: Tue Sep 6 23:40:28 2016 -0500
Committer: tarunbansal <ta...@gmail.com>
Committed: Wed Sep 14 11:18:25 2016 -0500

----------------------------------------------------------------------
 relational_operators/CMakeLists.txt             |  7 +++++++
 .../RelationalOperatorsConfig.h.in              | 20 ++++++++++++++++++++
 relational_operators/TextScanOperator.cpp       | 14 ++++++++++++++
 3 files changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/43c7a42d/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index cdfe309..a9645b4 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -15,6 +15,13 @@
 # specific language governing permissions and limitations
 # under the License.
 
+include(CheckIncludeFiles)
+check_include_files("unistd.h" QUICKSTEP_HAVE_UNISTD)
+configure_file (
+  "${CMAKE_CURRENT_SOURCE_DIR}/RelationalOperatorsConfig.h.in"
+  "${CMAKE_CURRENT_BINARY_DIR}/RelationalOperatorsConfig.h"
+)
+
 QS_PROTOBUF_GENERATE_CPP(relationaloperators_SortMergeRunOperator_proto_srcs
                          relationaloperators_SortMergeRunOperator_proto_hdrs
                          SortMergeRunOperator.proto)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/43c7a42d/relational_operators/RelationalOperatorsConfig.h.in
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperatorsConfig.h.in b/relational_operators/RelationalOperatorsConfig.h.in
new file mode 100644
index 0000000..879d5b3
--- /dev/null
+++ b/relational_operators/RelationalOperatorsConfig.h.in
@@ -0,0 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#cmakedefine QUICKSTEP_HAVE_UNISTD

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/43c7a42d/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index 1a0b715..4151bac 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -19,6 +19,12 @@
 
 #include "relational_operators/TextScanOperator.hpp"
 
+#include "relational_operators/RelationalOperatorsConfig.h"  // For QUICKSTEP_HAVE_UNISTD.
+
+#ifdef QUICKSTEP_HAVE_UNISTD
+#include <unistd.h>
+#endif  // QUICKSTEP_HAVE_UNISTD
+
 #include <algorithm>
 #include <cctype>
 #include <cstddef>
@@ -91,6 +97,14 @@ bool TextScanOperator::getAllWorkOrders(
   if (blocking_dependencies_met_ && !work_generated_) {
     for (const std::string &file : files) {
       // Use standard C libary to retrieve the file size.
+
+#ifdef QUICKSTEP_HAVE_UNISTD
+      // Check file permissions before trying to open it.
+      const int access_result = access(file.c_str(), R_OK);
+      CHECK_EQ(0, access_result)
+          << "File " << file << " is not readable due to permission issues.";
+#endif  // QUICKSTEP_HAVE_UNISTD
+
       FILE *fp = std::fopen(file.c_str(), "rb");
       std::fseek(fp, 0, SEEK_END);
       const std::size_t file_size = std::ftell(fp);


[03/28] incubator-quickstep git commit: Initial commit for QUICKSTEP-28 and QUICKSTEP-29. Code refactoring and cleanup, some more optimizations are pending.

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/storage/FastSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastSeparateChainingHashTable.hpp b/storage/FastSeparateChainingHashTable.hpp
new file mode 100644
index 0000000..64c4979
--- /dev/null
+++ b/storage/FastSeparateChainingHashTable.hpp
@@ -0,0 +1,1761 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015-2016 Pivotal Software, Inc.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_STORAGE_FAST_SEPARATE_CHAINING_HASH_TABLE_HPP_
+#define QUICKSTEP_STORAGE_FAST_SEPARATE_CHAINING_HASH_TABLE_HPP_
+
+#include <algorithm>
+#include <atomic>
+#include <cstddef>
+#include <cstring>
+#include <limits>
+#include <memory>
+#include <utility>
+#include <vector>
+
+#include "storage/HashTable.hpp"
+#include "storage/FastHashTable.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/HashTableKeyManager.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/StorageManager.hpp"
+#include "threading/SpinSharedMutex.hpp"
+#include "types/Type.hpp"
+#include "types/TypedValue.hpp"
+#include "utility/Alignment.hpp"
+#include "utility/Macros.hpp"
+#include "utility/PrimeNumber.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+/**
+ * @brief A hash table implementation which uses separate chaining for buckets.
+ **/
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+class FastSeparateChainingHashTable : public FastHashTable<resizable,
+                                                   serializable,
+                                                   force_key_copy,
+                                                   allow_duplicate_keys> {
+ public:
+  FastSeparateChainingHashTable(const std::vector<const Type*> &key_types,
+                            const std::size_t num_entries,
+                            const std::vector<std::size_t> &payload_sizes,
+                            const std::vector<AggregationHandle *> &handles,
+                            StorageManager *storage_manager);
+
+  FastSeparateChainingHashTable(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);
+
+  // Delegating constructors for single scalar keys.
+  FastSeparateChainingHashTable(const Type &key_type,
+                            const std::size_t num_entries,
+                            StorageManager *storage_manager)
+      : FastSeparateChainingHashTable(std::vector<const Type*>(1, &key_type),
+                                  num_entries,
+                                  storage_manager) {
+  }
+
+  FastSeparateChainingHashTable(const Type &key_type,
+                            void *hash_table_memory,
+                            const std::size_t hash_table_memory_size,
+                            const bool new_hash_table,
+                            const bool hash_table_memory_zeroed)
+      : FastSeparateChainingHashTable(std::vector<const Type*>(1, &key_type),
+                                  hash_table_memory,
+                                  hash_table_memory_size,
+                                  new_hash_table,
+                                  hash_table_memory_zeroed) {
+  }
+
+  ~FastSeparateChainingHashTable() override {
+    DestroyValues(buckets_,
+                  header_->buckets_allocated.load(std::memory_order_relaxed),
+                  bucket_size_);
+    std::free(init_payload_);
+  }
+
+  void clear() override;
+
+  std::size_t numEntries() const override {
+    return header_->buckets_allocated.load(std::memory_order_relaxed);
+  }
+
+  const uint8_t* getSingle(const TypedValue &key) const override;
+  const uint8_t* getSingleCompositeKey(const std::vector<TypedValue> &key) const override;
+  const uint8_t* getSingleCompositeKey(const std::vector<TypedValue> &key, int index) const override;
+
+  void getAll(const TypedValue &key,
+              std::vector<const uint8_t*> *values) const override;
+  void getAllCompositeKey(const std::vector<TypedValue> &key,
+                          std::vector<const uint8_t*> *values) const override;
+
+ protected:
+  HashTablePutResult putInternal(const TypedValue &key,
+                                 const std::size_t variable_key_size,
+                                 const uint8_t &value,
+                                 HashTablePreallocationState *prealloc_state) override;
+  HashTablePutResult putCompositeKeyInternal(const std::vector<TypedValue> &key,
+                                             const std::size_t variable_key_size,
+                                             const uint8_t &value,
+                                             HashTablePreallocationState *prealloc_state) override;
+  HashTablePutResult putCompositeKeyInternalFast(const std::vector<TypedValue> &key,
+                                             const std::size_t variable_key_size,
+                                             const std::uint8_t *init_value_ptr,
+                                             HashTablePreallocationState *prealloc_state) override;
+
+  uint8_t* upsertInternal(const TypedValue &key,
+                         const std::size_t variable_key_size,
+                         const uint8_t &initial_value) override;
+  uint8_t* upsertInternalFast(const TypedValue &key,
+                         const std::uint8_t *init_value_ptr,
+                         const std::size_t variable_key_size) override;
+
+  uint8_t* upsertCompositeKeyInternal(const std::vector<TypedValue> &key,
+                                     const std::size_t variable_key_size,
+                                     const uint8_t &initial_value) override;
+
+  uint8_t* upsertCompositeKeyInternalFast(const std::vector<TypedValue> &key,
+                                     const std::uint8_t *init_value_ptr,
+                                     const std::size_t variable_key_size) override;
+
+  bool getNextEntry(TypedValue *key,
+                    const uint8_t **value,
+                    std::size_t *entry_num) const override;
+  bool getNextEntryCompositeKey(std::vector<TypedValue> *key,
+                                const uint8_t **value,
+                                std::size_t *entry_num) const override;
+
+  bool getNextEntryForKey(const TypedValue &key,
+                          const std::size_t hash_code,
+                          const uint8_t **value,
+                          std::size_t *entry_num) const override;
+  bool getNextEntryForCompositeKey(const std::vector<TypedValue> &key,
+                                   const std::size_t hash_code,
+                                   const uint8_t **value,
+                                   std::size_t *entry_num) const override;
+
+  bool hasKey(const TypedValue &key) const override;
+  bool hasCompositeKey(const std::vector<TypedValue> &key) const override;
+
+  void resize(const std::size_t extra_buckets,
+              const std::size_t extra_variable_storage,
+              const std::size_t retry_num = 0) override;
+
+  bool preallocateForBulkInsert(const std::size_t total_entries,
+                                const std::size_t total_variable_key_size,
+                                HashTablePreallocationState *prealloc_state) override;
+
+  size_t get_buckets_allocated() const override {return header_->buckets_allocated;}
+
+ private:
+  struct Header {
+    std::size_t num_slots;
+    std::size_t num_buckets;
+    alignas(kCacheLineBytes)
+        std::atomic<std::size_t> buckets_allocated;
+    alignas(kCacheLineBytes)
+        std::atomic<std::size_t> variable_length_bytes_allocated;
+  };
+
+  std::uint8_t *init_payload_;
+  std::size_t kBucketAlignment;
+
+  // Value's offset in a bucket is the first alignof(ValueT) boundary after the
+  // next pointer and hash code.
+  std::size_t kValueOffset;
+
+  // Round bucket size up to a multiple of kBucketAlignment.
+  constexpr std::size_t ComputeBucketSize(const std::size_t fixed_key_size) {
+    return (((kValueOffset + this->total_payload_size_ + fixed_key_size - 1) / kBucketAlignment) + 1)
+           * kBucketAlignment;
+  }
+  // If ValueT is not trivially destructible, invoke its destructor for all
+  // values held in the specified buckets (including those in "empty" buckets
+  // that were default constructed). If ValueT is trivially destructible, this
+  // is a no-op.
+  void DestroyValues(void *buckets,
+                            const std::size_t num_buckets,
+                            const std::size_t bucket_size);
+
+  // Attempt to find an empty bucket to insert 'hash_code' into, starting after
+  // '*bucket' in the chain (or, if '*bucket' is NULL, starting from the slot
+  // array). Returns true and stores SIZE_T_MAX in '*pending_chain_ptr' if an
+  // empty bucket is found. Returns false if 'allow_duplicate_keys' is false
+  // and a hash collision is found (caller should then check whether there is a
+  // genuine key collision or the hash collision is spurious). Returns false
+  // and sets '*bucket' to NULL if there are no more empty buckets in the hash
+  // table. If 'variable_key_allocation_required' is nonzero, this method will
+  // attempt to allocate storage for a variable-length key BEFORE allocating a
+  // bucket, so that no bucket number below 'header_->num_buckets' is ever
+  // deallocated after being allocated.
+  inline bool locateBucketForInsertion(const std::size_t hash_code,
+                                       const std::size_t variable_key_allocation_required,
+                                       void **bucket,
+                                       std::atomic<std::size_t> **pending_chain_ptr,
+                                       std::size_t *pending_chain_ptr_finish_value,
+                                       HashTablePreallocationState *prealloc_state);
+
+  // Write a scalar 'key' and its 'hash_code' into the '*bucket', which was
+  // found by locateBucketForInsertion(). Assumes that storage for a
+  // variable-length key copy (if any) was already allocated by a successful
+  // call to allocateVariableLengthKeyStorage().
+  inline void writeScalarKeyToBucket(const TypedValue &key,
+                                     const std::size_t hash_code,
+                                     void *bucket,
+                                     HashTablePreallocationState *prealloc_state);
+
+  // Write a composite 'key' and its 'hash_code' into the '*bucket', which was
+  // found by locateBucketForInsertion(). Assumes that storage for
+  // variable-length key copies (if any) was already allocated by a successful
+  // call to allocateVariableLengthKeyStorage().
+  inline void writeCompositeKeyToBucket(const std::vector<TypedValue> &key,
+                                        const std::size_t hash_code,
+                                        void *bucket,
+                                        HashTablePreallocationState *prealloc_state);
+
+  // Determine whether it is actually necessary to resize this hash table.
+  // Checks that there is at least one unallocated bucket, and that there is
+  // at least 'extra_variable_storage' bytes of variable-length storage free.
+  bool isFull(const std::size_t extra_variable_storage) const;
+
+  // Helper object to manage key storage.
+  HashTableKeyManager<serializable, force_key_copy> key_manager_;
+
+  // In-memory structure is as follows:
+  //   - SeparateChainingHashTable::Header
+  //   - Array of slots, interpreted as follows:
+  //       - 0 = Points to nothing (empty)
+  //       - SIZE_T_MAX = Pending (some thread is starting a chain from this
+  //         slot and will overwrite it soon)
+  //       - Anything else = The number of the first bucket in the chain for
+  //         this slot PLUS ONE (i.e. subtract one to get the actual bucket
+  //         number).
+  //   - Array of buckets, each of which is:
+  //       - atomic size_t "next" pointer, interpreted the same as slots above.
+  //       - size_t hash value
+  //       - possibly some unused bytes as needed so that ValueT's alignment
+  //         requirement is met
+  //       - ValueT value slot
+  //       - fixed-length key storage (which may include pointers to external
+  //         memory or offsets of variable length keys stored within this hash
+  //         table)
+  //       - possibly some additional unused bytes so that bucket size is a
+  //         multiple of both alignof(std::atomic<std::size_t>) and
+  //         alignof(ValueT)
+  //   - Variable-length key storage region (referenced by offsets stored in
+  //     fixed-length keys).
+  Header *header_;
+
+  std::atomic<std::size_t> *slots_;
+  void *buckets_;
+  const std::size_t bucket_size_;
+
+  DISALLOW_COPY_AND_ASSIGN(FastSeparateChainingHashTable);
+};
+
+/** @} */
+
+// ----------------------------------------------------------------------------
+// Implementations of template class methods follow.
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::FastSeparateChainingHashTable(const std::vector<const Type*> &key_types,
+                                const std::size_t num_entries,
+                                const std::vector<std::size_t> &payload_sizes,
+                                const std::vector<AggregationHandle *> &handles,
+                                StorageManager *storage_manager)
+        : FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>(
+              key_types,
+              num_entries,
+              handles,
+              payload_sizes,
+              storage_manager,
+              false,
+              false,
+              true),
+          kBucketAlignment(alignof(std::atomic<std::size_t>)),
+          kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
+          key_manager_(this->key_types_, kValueOffset + this->total_payload_size_),
+          bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
+  init_payload_ = static_cast<std::uint8_t *>(calloc(this->total_payload_size_, 1));
+  for (auto handle : handles)
+      handle->initPayload(init_payload_);
+  // Bucket size always rounds up to the alignment requirement of the atomic
+  // size_t "next" pointer at the front or a ValueT, whichever is larger.
+  //
+  // Give base HashTable information about what key components are stored
+  // inline from 'key_manager_'.
+  this->setKeyInline(key_manager_.getKeyInline());
+
+  // Pick out a prime number of slots and calculate storage requirements.
+  std::size_t num_slots_tmp = get_next_prime_number(num_entries * kHashTableLoadFactor);
+  std::size_t required_memory = sizeof(Header)
+                                + num_slots_tmp * sizeof(std::atomic<std::size_t>)
+                                + (num_slots_tmp / kHashTableLoadFactor)
+                                    * (bucket_size_ + key_manager_.getEstimatedVariableKeySize());
+  std::size_t num_storage_slots = this->storage_manager_->SlotsNeededForBytes(required_memory);
+  if (num_storage_slots == 0) {
+    FATAL_ERROR("Storage requirement for SeparateChainingHashTable "
+                "exceeds maximum allocation size.");
+  }
+
+  // Get a StorageBlob to hold the hash table.
+  const block_id blob_id = this->storage_manager_->createBlob(num_storage_slots);
+  this->blob_ = this->storage_manager_->getBlobMutable(blob_id);
+
+  void *aligned_memory_start = this->blob_->getMemoryMutable();
+  std::size_t available_memory = num_storage_slots * kSlotSizeBytes;
+  if (align(alignof(Header),
+            sizeof(Header),
+            aligned_memory_start,
+            available_memory)
+          == nullptr) {
+    // With current values from StorageConstants.hpp, this should be
+    // impossible. A blob is at least 1 MB, while a Header has alignment
+    // requirement of just kCacheLineBytes (64 bytes).
+    FATAL_ERROR("StorageBlob used to hold resizable "
+                "SeparateChainingHashTable is too small to meet alignment "
+                "requirements of SeparateChainingHashTable::Header.");
+  } else if (aligned_memory_start != this->blob_->getMemoryMutable()) {
+    // This should also be impossible, since the StorageManager allocates slots
+    // aligned to kCacheLineBytes.
+    DEV_WARNING("StorageBlob memory adjusted by "
+                << (num_storage_slots * kSlotSizeBytes - available_memory)
+                << " bytes to meet alignment requirement for "
+                << "SeparateChainingHashTable::Header.");
+  }
+
+  // Locate the header.
+  header_ = static_cast<Header*>(aligned_memory_start);
+  aligned_memory_start = static_cast<char*>(aligned_memory_start) + sizeof(Header);
+  available_memory -= sizeof(Header);
+
+  // Recompute the number of slots & buckets using the actual available memory.
+  // Most likely, we got some extra free bucket space due to "rounding up" to
+  // the storage blob's size. It's also possible (though very unlikely) that we
+  // will wind up with fewer buckets than we initially wanted because of screwy
+  // alignment requirements for ValueT.
+  std::size_t num_buckets_tmp
+      = available_memory / (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>)
+                            + bucket_size_
+                            + key_manager_.getEstimatedVariableKeySize());
+  num_slots_tmp = get_previous_prime_number(num_buckets_tmp * kHashTableLoadFactor);
+  num_buckets_tmp = num_slots_tmp / kHashTableLoadFactor;
+  DEBUG_ASSERT(num_slots_tmp > 0);
+  DEBUG_ASSERT(num_buckets_tmp > 0);
+
+  // Locate the slot array.
+  slots_ = static_cast<std::atomic<std::size_t>*>(aligned_memory_start);
+  aligned_memory_start = static_cast<char*>(aligned_memory_start)
+                         + sizeof(std::atomic<std::size_t>) * num_slots_tmp;
+  available_memory -= sizeof(std::atomic<std::size_t>) * num_slots_tmp;
+
+  // Locate the buckets.
+  buckets_ = aligned_memory_start;
+  // Extra-paranoid: If ValueT has an alignment requirement greater than that
+  // of std::atomic<std::size_t>, we may need to adjust the start of the bucket
+  // array.
+  if (align(kBucketAlignment,
+            bucket_size_,
+            buckets_,
+            available_memory)
+          == nullptr) {
+    FATAL_ERROR("StorageBlob used to hold resizable "
+                "SeparateChainingHashTable is too small to meet "
+                "alignment requirements of buckets.");
+  } else if (buckets_ != aligned_memory_start) {
+    DEV_WARNING("Bucket array start position adjusted to meet alignment "
+                "requirement for SeparateChainingHashTable's value type.");
+    if (num_buckets_tmp * bucket_size_ > available_memory) {
+      --num_buckets_tmp;
+    }
+  }
+
+  // Fill in the header.
+  header_->num_slots = num_slots_tmp;
+  header_->num_buckets = num_buckets_tmp;
+  header_->buckets_allocated.store(0, std::memory_order_relaxed);
+  header_->variable_length_bytes_allocated.store(0, std::memory_order_relaxed);
+  available_memory -= bucket_size_ * (header_->num_buckets);
+
+  // Locate variable-length key storage region, and give it all the remaining
+  // bytes in the blob.
+  key_manager_.setVariableLengthStorageInfo(
+      static_cast<char*>(buckets_) + header_->num_buckets * bucket_size_,
+      available_memory,
+      &(header_->variable_length_bytes_allocated));
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::FastSeparateChainingHashTable(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)
+        : FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>(
+              key_types,
+              hash_table_memory,
+              hash_table_memory_size,
+              new_hash_table,
+              hash_table_memory_zeroed,
+              false,
+              false,
+              true),
+          kBucketAlignment(alignof(std::atomic<std::size_t>) < alignof(uint8_t) ? alignof(uint8_t)
+                                                  : alignof(std::atomic<std::size_t>)),
+          kValueOffset((((sizeof(std::atomic<std::size_t>) + sizeof(std::size_t) - 1) /
+                                           alignof(uint8_t)) + 1) * alignof(uint8_t)),
+          key_manager_(this->key_types_, kValueOffset + sizeof(uint8_t)),
+          bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
+  // Bucket size always rounds up to the alignment requirement of the atomic
+  // size_t "next" pointer at the front or a ValueT, whichever is larger.
+  //
+  // Make sure that the larger of the two alignment requirements also satisfies
+  // the smaller.
+  static_assert(alignof(std::atomic<std::size_t>) < alignof(uint8_t)
+                    ? alignof(uint8_t) % alignof(std::atomic<std::size_t>) == 0
+                    : alignof(std::atomic<std::size_t>) % alignof(uint8_t) == 0,
+                "Alignment requirement of std::atomic<std::size_t> does not "
+                "evenly divide with alignment requirement of ValueT.");
+
+  // Give base HashTable information about what key components are stored
+  // inline from 'key_manager_'.
+  this->setKeyInline(key_manager_.getKeyInline());
+
+  // FIXME(chasseur): If we are reconstituting a HashTable using a block of
+  // memory whose start was aligned differently than the memory block that was
+  // originally used (modulo alignof(Header)), we could wind up with all of our
+  // data structures misaligned. If memory is inside a
+  // StorageBlock/StorageBlob, this will never occur, since the StorageManager
+  // always allocates slots aligned to kCacheLineBytes. Similarly, this isn't
+  // a problem for memory inside any other allocation aligned to at least
+  // alignof(Header) == kCacheLineBytes.
+
+  void *aligned_memory_start = this->hash_table_memory_;
+  std::size_t available_memory = this->hash_table_memory_size_;
+
+  if (align(alignof(Header),
+            sizeof(Header),
+            aligned_memory_start,
+            available_memory)
+          == nullptr) {
+    FATAL_ERROR("Attempted to create a non-resizable "
+                << "SeparateChainingHashTable with "
+                << available_memory << " bytes of memory at "
+                << aligned_memory_start << " which either can not fit a "
+                << "SeparateChainingHashTable::Header or meet its alignement "
+                << "requirement.");
+  } else if (aligned_memory_start != this->hash_table_memory_) {
+    // In general, we could get memory of any alignment, although at least
+    // cache-line aligned would be nice.
+    DEV_WARNING("StorageBlob memory adjusted by "
+                << (this->hash_table_memory_size_ - available_memory)
+                << " bytes to meet alignment requirement for "
+                << "SeparateChainingHashTable::Header.");
+  }
+
+  header_ = static_cast<Header*>(aligned_memory_start);
+  aligned_memory_start = static_cast<char*>(aligned_memory_start) + sizeof(Header);
+  available_memory -= sizeof(Header);
+
+  if (new_hash_table) {
+    std::size_t estimated_bucket_capacity
+        = available_memory / (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>)
+                              + bucket_size_
+                              + key_manager_.getEstimatedVariableKeySize());
+    std::size_t num_slots = get_previous_prime_number(estimated_bucket_capacity * kHashTableLoadFactor);
+
+    // Fill in the header.
+    header_->num_slots = num_slots;
+    header_->num_buckets = num_slots / kHashTableLoadFactor;
+    header_->buckets_allocated.store(0, std::memory_order_relaxed);
+    header_->variable_length_bytes_allocated.store(0, std::memory_order_relaxed);
+  }
+
+  // Locate the slot array.
+  slots_ = static_cast<std::atomic<std::size_t>*>(aligned_memory_start);
+  aligned_memory_start = static_cast<char*>(aligned_memory_start)
+                         + sizeof(std::atomic<std::size_t>) * header_->num_slots;
+  available_memory -= sizeof(std::atomic<std::size_t>) * header_->num_slots;
+
+  if (new_hash_table && !hash_table_memory_zeroed) {
+    std::memset(slots_, 0x0, sizeof(std::atomic<std::size_t>) * header_->num_slots);
+  }
+
+  // Locate the buckets.
+  buckets_ = aligned_memory_start;
+  // Extra-paranoid: sizeof(Header) should almost certainly be a multiple of
+  // kBucketAlignment, unless ValueT has some members with seriously big
+  // (> kCacheLineBytes) alignment requirements specified using alignas().
+  if (align(kBucketAlignment,
+            bucket_size_,
+            buckets_,
+            available_memory)
+          == nullptr) {
+    FATAL_ERROR("Attempted to create a non-resizable "
+                << "SeparateChainingHashTable with "
+                << this->hash_table_memory_size_ << " bytes of memory at "
+                << this->hash_table_memory_ << ", which can hold an aligned "
+                << "SeparateChainingHashTable::Header but does not have "
+                << "enough remaining space for even a single hash bucket.");
+  } else if (buckets_ != aligned_memory_start) {
+    DEV_WARNING("Bucket array start position adjusted to meet alignment "
+                "requirement for SeparateChainingHashTable's value type.");
+    if (header_->num_buckets * bucket_size_ > available_memory) {
+      DEBUG_ASSERT(new_hash_table);
+      --(header_->num_buckets);
+    }
+  }
+  available_memory -= bucket_size_ * header_->num_buckets;
+
+  // Make sure "next" pointers in buckets are zeroed-out.
+  if (new_hash_table && !hash_table_memory_zeroed) {
+    std::memset(buckets_, 0x0, header_->num_buckets * bucket_size_);
+  }
+
+  // Locate variable-length key storage region.
+  key_manager_.setVariableLengthStorageInfo(
+      static_cast<char*>(buckets_) + header_->num_buckets * bucket_size_,
+      available_memory,
+      &(header_->variable_length_bytes_allocated));
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::clear() {
+  const std::size_t used_buckets = header_->buckets_allocated.load(std::memory_order_relaxed);
+  // Destroy existing values, if necessary.
+  DestroyValues(buckets_,
+                used_buckets,
+                bucket_size_);
+
+  // Zero-out slot array.
+  std::memset(slots_, 0x0, sizeof(std::atomic<std::size_t>) * header_->num_slots);
+
+  // Zero-out used buckets.
+  std::memset(buckets_, 0x0, used_buckets * bucket_size_);
+
+  header_->buckets_allocated.store(0, std::memory_order_relaxed);
+  header_->variable_length_bytes_allocated.store(0, std::memory_order_relaxed);
+  key_manager_.zeroNextVariableLengthKeyOffset();
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+const uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::getSingle(const TypedValue &key) const {
+  DEBUG_ASSERT(!allow_duplicate_keys);
+  DEBUG_ASSERT(this->key_types_.size() == 1);
+  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+
+  const std::size_t hash_code = key.getHash();
+  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  while (bucket_ref != 0) {
+    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
+    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) && key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+      // Match located.
+      return reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+    }
+    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+  }
+
+  // Reached the end of the chain and didn't find a match.
+  return nullptr;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+const uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::getSingleCompositeKey(const std::vector<TypedValue> &key) const {
+  DEBUG_ASSERT(!allow_duplicate_keys);
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  while (bucket_ref != 0) {
+    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
+    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) && key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+      // Match located.
+      return reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+    }
+    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+  }
+
+  // Reached the end of the chain and didn't find a match.
+  return nullptr;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+const uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::getSingleCompositeKey(const std::vector<TypedValue> &key, int index) const {
+  DEBUG_ASSERT(!allow_duplicate_keys);
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  while (bucket_ref != 0) {
+    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
+    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) && key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+      // Match located.
+      return reinterpret_cast<const uint8_t*>(bucket + kValueOffset)+this->payload_offsets_[index];
+    }
+    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+  }
+
+  // Reached the end of the chain and didn't find a match.
+  return nullptr;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::getAll(const TypedValue &key, std::vector<const uint8_t*> *values) const {
+  DEBUG_ASSERT(this->key_types_.size() == 1);
+  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+
+  const std::size_t hash_code = key.getHash();
+  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  while (bucket_ref != 0) {
+    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
+    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) && key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+      // Match located.
+      values->push_back(reinterpret_cast<const uint8_t*>(bucket + kValueOffset));
+      if (!allow_duplicate_keys) {
+        return;
+      }
+    }
+    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+  }
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::getAllCompositeKey(const std::vector<TypedValue> &key, std::vector<const uint8_t*> *values) const {
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  while (bucket_ref != 0) {
+    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
+    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) && key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+      // Match located.
+      values->push_back(reinterpret_cast<const uint8_t*>(bucket + kValueOffset));
+      if (!allow_duplicate_keys) {
+        return;
+      }
+    }
+    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+  }
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+HashTablePutResult
+    FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+        ::putInternal(const TypedValue &key,
+                      const std::size_t variable_key_size,
+                      const uint8_t &value,
+                      HashTablePreallocationState *prealloc_state) {
+  DEBUG_ASSERT(this->key_types_.size() == 1);
+  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+
+  if (prealloc_state == nullptr) {
+    // Early check for a free bucket.
+    if (header_->buckets_allocated.load(std::memory_order_relaxed) >= header_->num_buckets) {
+      return HashTablePutResult::kOutOfSpace;
+    }
+
+    // TODO(chasseur): If allow_duplicate_keys is true, avoid storing more than
+    // one copy of the same variable-length key.
+    if (!key_manager_.allocateVariableLengthKeyStorage(variable_key_size)) {
+      // Ran out of variable-length key storage space.
+      return HashTablePutResult::kOutOfSpace;
+    }
+  }
+
+  const std::size_t hash_code = key.getHash();
+  void *bucket = nullptr;
+  std::atomic<std::size_t> *pending_chain_ptr;
+  std::size_t pending_chain_ptr_finish_value;
+  for (;;) {
+    if (locateBucketForInsertion(hash_code,
+                                 0,
+                                 &bucket,
+                                 &pending_chain_ptr,
+                                 &pending_chain_ptr_finish_value,
+                                 prealloc_state)) {
+      // Found an empty bucket.
+      break;
+    } else if (bucket == nullptr) {
+      // Ran out of buckets. Deallocate any variable space that we were unable
+      // to use.
+      DEBUG_ASSERT(prealloc_state == nullptr);
+      key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
+      return HashTablePutResult::kOutOfSpace;
+    } else {
+      // Hash collision found, and duplicates aren't allowed.
+      DEBUG_ASSERT(!allow_duplicate_keys);
+      DEBUG_ASSERT(prealloc_state == nullptr);
+      if (key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+        // Duplicate key. Deallocate any variable storage space and return.
+        key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
+        return HashTablePutResult::kDuplicateKey;
+      }
+    }
+  }
+
+  // Write the key and hash.
+  writeScalarKeyToBucket(key, hash_code, bucket, prealloc_state);
+
+  // Store the value by using placement new with ValueT's copy constructor.
+  new(static_cast<char*>(bucket) + kValueOffset) uint8_t(value);
+
+  // Update the previous chain pointer to point to the new bucket.
+  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+
+  // We're all done.
+  return HashTablePutResult::kOK;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+HashTablePutResult
+    FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+        ::putCompositeKeyInternal(const std::vector<TypedValue> &key,
+                                  const std::size_t variable_key_size,
+                                  const uint8_t &value,
+                                  HashTablePreallocationState *prealloc_state) {
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  if (prealloc_state == nullptr) {
+    // Early check for a free bucket.
+    if (header_->buckets_allocated.load(std::memory_order_relaxed) >= header_->num_buckets) {
+      return HashTablePutResult::kOutOfSpace;
+    }
+
+    // TODO(chasseur): If allow_duplicate_keys is true, avoid storing more than
+    // one copy of the same variable-length key.
+    if (!key_manager_.allocateVariableLengthKeyStorage(variable_key_size)) {
+      // Ran out of variable-length key storage space.
+      return HashTablePutResult::kOutOfSpace;
+    }
+  }
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  void *bucket = nullptr;
+  std::atomic<std::size_t> *pending_chain_ptr;
+  std::size_t pending_chain_ptr_finish_value;
+  for (;;) {
+    if (locateBucketForInsertion(hash_code,
+                                 0,
+                                 &bucket,
+                                 &pending_chain_ptr,
+                                 &pending_chain_ptr_finish_value,
+                                 prealloc_state)) {
+      // Found an empty bucket.
+      break;
+    } else if (bucket == nullptr) {
+      // Ran out of buckets. Deallocate any variable space that we were unable
+      // to use.
+      DEBUG_ASSERT(prealloc_state == nullptr);
+      key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
+      return HashTablePutResult::kOutOfSpace;
+    } else {
+      // Hash collision found, and duplicates aren't allowed.
+      DEBUG_ASSERT(!allow_duplicate_keys);
+      DEBUG_ASSERT(prealloc_state == nullptr);
+      if (key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+        // Duplicate key. Deallocate any variable storage space and return.
+        key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
+        return HashTablePutResult::kDuplicateKey;
+      }
+    }
+  }
+
+  // Write the key and hash.
+  writeCompositeKeyToBucket(key, hash_code, bucket, prealloc_state);
+
+  // Store the value by using placement new with ValueT's copy constructor.
+  new(static_cast<char*>(bucket) + kValueOffset) uint8_t(value);
+
+  // Update the previous chain pointer to point to the new bucket.
+  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+
+  // We're all done.
+  return HashTablePutResult::kOK;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+HashTablePutResult
+    FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+        ::putCompositeKeyInternalFast(const std::vector<TypedValue> &key,
+                                  const std::size_t variable_key_size,
+                                  const uint8_t *init_value_ptr,
+                                  HashTablePreallocationState *prealloc_state) {
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  if (prealloc_state == nullptr) {
+    // Early check for a free bucket.
+    if (header_->buckets_allocated.load(std::memory_order_relaxed) >= header_->num_buckets) {
+      return HashTablePutResult::kOutOfSpace;
+    }
+
+    // TODO(chasseur): If allow_duplicate_keys is true, avoid storing more than
+    // one copy of the same variable-length key.
+    if (!key_manager_.allocateVariableLengthKeyStorage(variable_key_size)) {
+      // Ran out of variable-length key storage space.
+      return HashTablePutResult::kOutOfSpace;
+    }
+  }
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  void *bucket = nullptr;
+  std::atomic<std::size_t> *pending_chain_ptr;
+  std::size_t pending_chain_ptr_finish_value;
+  for (;;) {
+    if (locateBucketForInsertion(hash_code,
+                                 0,
+                                 &bucket,
+                                 &pending_chain_ptr,
+                                 &pending_chain_ptr_finish_value,
+                                 prealloc_state)) {
+      // Found an empty bucket.
+      break;
+    } else if (bucket == nullptr) {
+      // Ran out of buckets. Deallocate any variable space that we were unable
+      // to use.
+      DEBUG_ASSERT(prealloc_state == nullptr);
+      key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
+      return HashTablePutResult::kOutOfSpace;
+    } else {
+      // Hash collision found, and duplicates aren't allowed.
+      DEBUG_ASSERT(!allow_duplicate_keys);
+      DEBUG_ASSERT(prealloc_state == nullptr);
+      if (key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+        // Duplicate key. Deallocate any variable storage space and return.
+        key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
+        return HashTablePutResult::kDuplicateKey;
+      }
+    }
+  }
+
+  // Write the key and hash.
+  writeCompositeKeyToBucket(key, hash_code, bucket, prealloc_state);
+
+  // Store the value by using placement new with ValueT's copy constructor.
+//  new(static_cast<char*>(bucket) + kValueOffset) uint8_t(value);
+    uint8_t *value = static_cast<uint8_t*>(bucket) + kValueOffset;
+        memcpy(value, init_value_ptr, this->total_payload_size_);
+  // Update the previous chain pointer to point to the new bucket.
+  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+
+  // We're all done.
+  return HashTablePutResult::kOK;
+}
+
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::upsertInternal(const TypedValue &key,
+                     const std::size_t variable_key_size,
+                     const uint8_t &initial_value) {
+  DEBUG_ASSERT(!allow_duplicate_keys);
+  DEBUG_ASSERT(this->key_types_.size() == 1);
+  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+
+  if (variable_key_size > 0) {
+    // Don't allocate yet, since the key may already be present. However, we
+    // do check if either the allocated variable storage space OR the free
+    // space is big enough to hold the key (at least one must be true: either
+    // the key is already present and allocated, or we need to be able to
+    // allocate enough space for it).
+    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
+    if ((allocated_bytes < variable_key_size)
+        && (allocated_bytes + variable_key_size > key_manager_.getVariableLengthKeyStorageSize())) {
+      return nullptr;
+    }
+  }
+
+  const std::size_t hash_code = key.getHash();
+  void *bucket = nullptr;
+  std::atomic<std::size_t> *pending_chain_ptr;
+  std::size_t pending_chain_ptr_finish_value;
+  for (;;) {
+    if (locateBucketForInsertion(hash_code,
+                                 variable_key_size,
+                                 &bucket,
+                                 &pending_chain_ptr,
+                                 &pending_chain_ptr_finish_value,
+                                 nullptr)) {
+      // Found an empty bucket.
+      break;
+    } else if (bucket == nullptr) {
+      // Ran out of buckets or variable-key space.
+      return nullptr;
+    } else if (key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+      // Found an already-existing entry for this key.
+      return reinterpret_cast<uint8_t*>(static_cast<char*>(bucket) + kValueOffset);
+    }
+  }
+
+  // We are now writing to an empty bucket.
+  // Write the key and hash.
+  writeScalarKeyToBucket(key, hash_code, bucket, nullptr);
+
+  // Copy the supplied 'initial_value' into place.
+  uint8_t *value = new(static_cast<char*>(bucket) + kValueOffset) uint8_t(initial_value);
+
+  // Update the previous chain pointer to point to the new bucket.
+  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+
+  // Return the value.
+  return value;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::upsertInternalFast(const TypedValue &key,
+                     const std::uint8_t *init_value_ptr,
+                     const std::size_t variable_key_size) {
+  DEBUG_ASSERT(!allow_duplicate_keys);
+  DEBUG_ASSERT(this->key_types_.size() == 1);
+  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+
+  if (variable_key_size > 0) {
+    // Don't allocate yet, since the key may already be present. However, we
+    // do check if either the allocated variable storage space OR the free
+    // space is big enough to hold the key (at least one must be true: either
+    // the key is already present and allocated, or we need to be able to
+    // allocate enough space for it).
+    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
+    if ((allocated_bytes < variable_key_size)
+        && (allocated_bytes + variable_key_size > key_manager_.getVariableLengthKeyStorageSize())) {
+      return nullptr;
+    }
+  }
+
+  const std::size_t hash_code = key.getHash();
+  void *bucket = nullptr;
+  std::atomic<std::size_t> *pending_chain_ptr;
+  std::size_t pending_chain_ptr_finish_value;
+  for (;;) {
+    if (locateBucketForInsertion(hash_code,
+                                 variable_key_size,
+                                 &bucket,
+                                 &pending_chain_ptr,
+                                 &pending_chain_ptr_finish_value,
+                                 nullptr)) {
+      // Found an empty bucket.
+      break;
+    } else if (bucket == nullptr) {
+      // Ran out of buckets or variable-key space.
+      return nullptr;
+    } else if (key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+      // Found an already-existing entry for this key.
+      return reinterpret_cast<uint8_t*>(static_cast<char*>(bucket) + kValueOffset);
+    }
+  }
+
+  // We are now writing to an empty bucket.
+  // Write the key and hash.
+  writeScalarKeyToBucket(key, hash_code, bucket, nullptr);
+
+  // Copy the supplied 'initial_value' into place.
+//  uint8_t *value = new(static_cast<char*>(bucket) + kValueOffset) uint8_t(initial_value);
+
+    uint8_t *value = static_cast<unsigned char*>(bucket) + kValueOffset;
+    if (init_value_ptr == nullptr)
+        memcpy(value, init_payload_, this->total_payload_size_);
+    else
+        memcpy(value, init_value_ptr, this->total_payload_size_);
+
+
+  // Update the previous chain pointer to point to the new bucket.
+  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+
+  // Return the value.
+  return value;
+}
+
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::upsertCompositeKeyInternal(const std::vector<TypedValue> &key,
+                                 const std::size_t variable_key_size,
+                                 const uint8_t &initial_value) {
+  DEBUG_ASSERT(!allow_duplicate_keys);
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  if (variable_key_size > 0) {
+    // Don't allocate yet, since the key may already be present. However, we
+    // do check if either the allocated variable storage space OR the free
+    // space is big enough to hold the key (at least one must be true: either
+    // the key is already present and allocated, or we need to be able to
+    // allocate enough space for it).
+    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
+    if ((allocated_bytes < variable_key_size)
+        && (allocated_bytes + variable_key_size > key_manager_.getVariableLengthKeyStorageSize())) {
+      return nullptr;
+    }
+  }
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  void *bucket = nullptr;
+  std::atomic<std::size_t> *pending_chain_ptr;
+  std::size_t pending_chain_ptr_finish_value;
+  for (;;) {
+    if (locateBucketForInsertion(hash_code,
+                                 variable_key_size,
+                                 &bucket,
+                                 &pending_chain_ptr,
+                                 &pending_chain_ptr_finish_value,
+                                 nullptr)) {
+      // Found an empty bucket.
+      break;
+    } else if (bucket == nullptr) {
+      // Ran out of buckets or variable-key space.
+      return nullptr;
+    } else if (key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+      // Found an already-existing entry for this key.
+      return reinterpret_cast<uint8_t*>(static_cast<char*>(bucket) + kValueOffset);
+    }
+  }
+
+  // We are now writing to an empty bucket.
+  // Write the key and hash.
+  writeCompositeKeyToBucket(key, hash_code, bucket, nullptr);
+
+  // Copy the supplied 'initial_value' into place.
+  uint8_t *value = new(static_cast<char*>(bucket) + kValueOffset) uint8_t(initial_value);
+
+  // Update the previous chaing pointer to point to the new bucket.
+  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+
+  // Return the value.
+  return value;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::upsertCompositeKeyInternalFast(const std::vector<TypedValue> &key,
+                                 const std::uint8_t *init_value_ptr,
+                                 const std::size_t variable_key_size) {
+  DEBUG_ASSERT(!allow_duplicate_keys);
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  if (variable_key_size > 0) {
+    // Don't allocate yet, since the key may already be present. However, we
+    // do check if either the allocated variable storage space OR the free
+    // space is big enough to hold the key (at least one must be true: either
+    // the key is already present and allocated, or we need to be able to
+    // allocate enough space for it).
+    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
+    if ((allocated_bytes < variable_key_size)
+        && (allocated_bytes + variable_key_size > key_manager_.getVariableLengthKeyStorageSize())) {
+      return nullptr;
+    }
+  }
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  void *bucket = nullptr;
+  std::atomic<std::size_t> *pending_chain_ptr;
+  std::size_t pending_chain_ptr_finish_value;
+  for (;;) {
+    if (locateBucketForInsertion(hash_code,
+                                 variable_key_size,
+                                 &bucket,
+                                 &pending_chain_ptr,
+                                 &pending_chain_ptr_finish_value,
+                                 nullptr)) {
+      // Found an empty bucket.
+      break;
+    } else if (bucket == nullptr) {
+      // Ran out of buckets or variable-key space.
+      return nullptr;
+    } else if (key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+      // Found an already-existing entry for this key.
+      return reinterpret_cast<uint8_t*>(static_cast<char*>(bucket) + kValueOffset);
+    }
+  }
+
+  // We are now writing to an empty bucket.
+  // Write the key and hash.
+  writeCompositeKeyToBucket(key, hash_code, bucket, nullptr);
+
+//  uint8_t *value;
+//  value = static_cast<unsigned char*>(bucket) + kValueOffset;
+    uint8_t *value = static_cast<unsigned char*>(bucket) + kValueOffset;
+    if (init_value_ptr == nullptr)
+        memcpy(value, init_payload_, this->total_payload_size_);
+    else
+        memcpy(value, init_value_ptr, this->total_payload_size_);
+
+  // Update the previous chaing pointer to point to the new bucket.
+  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+
+  // Return the value.
+  return value;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::getNextEntry(TypedValue *key, const uint8_t **value, std::size_t *entry_num) const {
+  DEBUG_ASSERT(this->key_types_.size() == 1);
+  if (*entry_num < header_->buckets_allocated.load(std::memory_order_relaxed)) {
+    const char *bucket = static_cast<const char*>(buckets_) + (*entry_num) * bucket_size_;
+    *key = key_manager_.getKeyComponentTyped(bucket, 0);
+    *value = reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+    ++(*entry_num);
+    return true;
+  } else {
+    return false;
+  }
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::getNextEntryCompositeKey(std::vector<TypedValue> *key,
+                               const uint8_t **value,
+                               std::size_t *entry_num) const {
+  if (*entry_num < header_->buckets_allocated.load(std::memory_order_relaxed)) {
+    const char *bucket = static_cast<const char*>(buckets_) + (*entry_num) * bucket_size_;
+    for (std::vector<const Type*>::size_type key_idx = 0;
+         key_idx < this->key_types_.size();
+         ++key_idx) {
+      key->emplace_back(key_manager_.getKeyComponentTyped(bucket, key_idx));
+    }
+    *value = reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+    ++(*entry_num);
+    return true;
+  } else {
+    return false;
+  }
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::getNextEntryForKey(const TypedValue &key,
+                         const std::size_t hash_code,
+                         const uint8_t **value,
+                         std::size_t *entry_num) const {
+  DEBUG_ASSERT(this->key_types_.size() == 1);
+  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+
+  if (*entry_num == 0) {
+    *entry_num = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  } else if (*entry_num == std::numeric_limits<std::size_t>::max()) {
+    return false;
+  }
+
+  while (*entry_num != 0) {
+    DEBUG_ASSERT(*entry_num != std::numeric_limits<std::size_t>::max());
+    const char *bucket = static_cast<const char*>(buckets_) + (*entry_num - 1) * bucket_size_;
+    *entry_num = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) && key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+      // Match located.
+      *value = reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+      if (*entry_num == 0) {
+        // If this is the last bucket in the chain, prevent the next call from
+        // starting over again.
+        *entry_num = std::numeric_limits<std::size_t>::max();
+      }
+      return true;
+    }
+  }
+
+  // Reached the end of the chain.
+  return false;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::getNextEntryForCompositeKey(const std::vector<TypedValue> &key,
+                                  const std::size_t hash_code,
+                                  const uint8_t **value,
+                                  std::size_t *entry_num) const {
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  if (*entry_num == 0) {
+    *entry_num = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  } else if (*entry_num == std::numeric_limits<std::size_t>::max()) {
+    return false;
+  }
+
+  while (*entry_num != 0) {
+    DEBUG_ASSERT(*entry_num != std::numeric_limits<std::size_t>::max());
+    const char *bucket = static_cast<const char*>(buckets_) + (*entry_num - 1) * bucket_size_;
+    *entry_num = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) && key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+      // Match located.
+      *value = reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+      if (*entry_num == 0) {
+        // If this is the last bucket in the chain, prevent the next call from
+        // starting over again.
+        *entry_num = std::numeric_limits<std::size_t>::max();
+      }
+      return true;
+    }
+  }
+
+  // Reached the end of the chain.
+  return false;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::hasKey(const TypedValue &key) const {
+  DEBUG_ASSERT(this->key_types_.size() == 1);
+  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+
+  const std::size_t hash_code = key.getHash();
+  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  while (bucket_ref != 0) {
+    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
+    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) && key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+      // Find a match.
+      return true;
+    }
+    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+  }
+  return false;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::hasCompositeKey(const std::vector<TypedValue> &key) const {
+  DEBUG_ASSERT(this->key_types_.size() == key.size());
+
+  const std::size_t hash_code = this->hashCompositeKey(key);
+  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  while (bucket_ref != 0) {
+    DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
+    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+        bucket + sizeof(std::atomic<std::size_t>));
+    if ((bucket_hash == hash_code) && key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+      // Find a match.
+      return true;
+    }
+    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+  }
+  return false;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::resize(const std::size_t extra_buckets,
+             const std::size_t extra_variable_storage,
+             const std::size_t retry_num) {
+  DEBUG_ASSERT(resizable);
+
+  // A retry should never be necessary with this implementation of HashTable.
+  // Separate chaining ensures that any resized hash table with more buckets
+  // than the original table will be able to hold more entries than the
+  // original.
+  DEBUG_ASSERT(retry_num == 0);
+
+  SpinSharedMutexExclusiveLock<true> write_lock(this->resize_shared_mutex_);
+
+  // Recheck whether the hash table is still full. Note that multiple threads
+  // might wait to rebuild this hash table simultaneously. Only the first one
+  // should do the rebuild.
+  if (!isFull(extra_variable_storage)) {
+    return;
+  }
+
+  // Approximately double the number of buckets and slots.
+  //
+  // TODO(chasseur): It may be worth it to more than double the number of
+  // buckets here so that we can maintain a good, sparse fill factor for a
+  // longer time as more values are inserted. Such behavior should take into
+  // account kHashTableLoadFactor.
+  std::size_t resized_num_slots = get_next_prime_number(
+      (header_->num_buckets + extra_buckets / 2) * kHashTableLoadFactor * 2);
+  std::size_t variable_storage_required
+      = (resized_num_slots / kHashTableLoadFactor) * key_manager_.getEstimatedVariableKeySize();
+  const std::size_t original_variable_storage_used
+      = header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
+  // If this resize was triggered by a too-large variable-length key, bump up
+  // the variable-length storage requirement.
+  if ((extra_variable_storage > 0)
+      && (extra_variable_storage + original_variable_storage_used
+          > key_manager_.getVariableLengthKeyStorageSize())) {
+    variable_storage_required += extra_variable_storage;
+  }
+
+  const std::size_t resized_memory_required
+      = sizeof(Header)
+        + resized_num_slots * sizeof(std::atomic<std::size_t>)
+        + (resized_num_slots / kHashTableLoadFactor) * bucket_size_
+        + variable_storage_required;
+  const std::size_t resized_storage_slots
+      = this->storage_manager_->SlotsNeededForBytes(resized_memory_required);
+  if (resized_storage_slots == 0) {
+    FATAL_ERROR("Storage requirement for resized SeparateChainingHashTable "
+                "exceeds maximum allocation size.");
+  }
+
+  // Get a new StorageBlob to hold the resized hash table.
+  const block_id resized_blob_id = this->storage_manager_->createBlob(resized_storage_slots);
+  MutableBlobReference resized_blob = this->storage_manager_->getBlobMutable(resized_blob_id);
+
+  // Locate data structures inside the new StorageBlob.
+  void *aligned_memory_start = resized_blob->getMemoryMutable();
+  std::size_t available_memory = resized_storage_slots * kSlotSizeBytes;
+  if (align(alignof(Header),
+            sizeof(Header),
+            aligned_memory_start,
+            available_memory)
+          == nullptr) {
+    // Should be impossible, as noted in constructor.
+    FATAL_ERROR("StorageBlob used to hold resized SeparateChainingHashTable "
+                "is too small to meet alignment requirements of "
+                "LinearOpenAddressingHashTable::Header.");
+  } else if (aligned_memory_start != resized_blob->getMemoryMutable()) {
+    // Again, should be impossible.
+    DEV_WARNING("In SeparateChainingHashTable::resize(), StorageBlob "
+                << "memory adjusted by "
+                << (resized_num_slots * kSlotSizeBytes - available_memory)
+                << " bytes to meet alignment requirement for "
+                << "LinearOpenAddressingHashTable::Header.");
+  }
+
+  Header *resized_header = static_cast<Header*>(aligned_memory_start);
+  aligned_memory_start = static_cast<char*>(aligned_memory_start) + sizeof(Header);
+  available_memory -= sizeof(Header);
+
+  // As in constructor, recompute the number of slots and buckets using the
+  // actual available memory.
+  std::size_t resized_num_buckets
+      = (available_memory - extra_variable_storage)
+        / (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>)
+           + bucket_size_
+           + key_manager_.getEstimatedVariableKeySize());
+  resized_num_slots = get_previous_prime_number(resized_num_buckets * kHashTableLoadFactor);
+  resized_num_buckets = resized_num_slots / kHashTableLoadFactor;
+
+  // Locate slot array.
+  std::atomic<std::size_t> *resized_slots = static_cast<std::atomic<std::size_t>*>(aligned_memory_start);
+  aligned_memory_start = static_cast<char*>(aligned_memory_start)
+                         + sizeof(std::atomic<std::size_t>) * resized_num_slots;
+  available_memory -= sizeof(std::atomic<std::size_t>) * resized_num_slots;
+
+  // As in constructor, we will be extra paranoid and use align() to locate the
+  // start of the array of buckets, as well.
+  void *resized_buckets = aligned_memory_start;
+  if (align(kBucketAlignment,
+            bucket_size_,
+            resized_buckets,
+            available_memory)
+          == nullptr) {
+    FATAL_ERROR("StorageBlob used to hold resized SeparateChainingHashTable "
+                "is too small to meet alignment requirements of buckets.");
+  } else if (resized_buckets != aligned_memory_start) {
+    DEV_WARNING("Bucket array start position adjusted to meet alignment "
+                "requirement for SeparateChainingHashTable's value type.");
+    if (resized_num_buckets * bucket_size_ + variable_storage_required > available_memory) {
+      --resized_num_buckets;
+    }
+  }
+  aligned_memory_start = static_cast<char*>(aligned_memory_start)
+                         + resized_num_buckets * bucket_size_;
+  available_memory -= resized_num_buckets * bucket_size_;
+
+  void *resized_variable_length_key_storage = aligned_memory_start;
+  const std::size_t resized_variable_length_key_storage_size = available_memory;
+
+  const std::size_t original_buckets_used = header_->buckets_allocated.load(std::memory_order_relaxed);
+
+  // Initialize the header.
+  resized_header->num_slots = resized_num_slots;
+  resized_header->num_buckets = resized_num_buckets;
+  resized_header->buckets_allocated.store(original_buckets_used, std::memory_order_relaxed);
+  resized_header->variable_length_bytes_allocated.store(
+      original_variable_storage_used,
+      std::memory_order_relaxed);
+
+  // Bulk-copy buckets. This is safe because:
+  //     1. The "next" pointers will be adjusted when rebuilding chains below.
+  //     2. The hash codes will stay the same.
+  //     3. For key components:
+  //       a. Inline keys will stay exactly the same.
+  //       b. Offsets into variable-length storage will remain valid, because
+  //          we also do a byte-for-byte copy of variable-length storage below.
+  //       c. Absolute external pointers will still point to the same address.
+  //       d. Relative pointers are not used with resizable hash tables.
+  //     4. If values are not trivially copyable, then we invoke ValueT's copy
+  //        or move constructor with placement new.
+  std::memcpy(resized_buckets, buckets_, original_buckets_used * bucket_size_);
+
+  // TODO(chasseur): std::is_trivially_copyable is not yet implemented in
+  // GCC 4.8.3, so we assume we need to invoke ValueT's copy or move
+  // constructor, even though the plain memcpy above could suffice for many
+  // possible ValueTs.
+  void *current_value_original = static_cast<char*>(buckets_) + kValueOffset;
+  void *current_value_resized = static_cast<char*>(resized_buckets) + kValueOffset;
+  for (std::size_t bucket_num = 0; bucket_num < original_buckets_used; ++bucket_num) {
+    // Use a move constructor if available to avoid a deep-copy, since resizes
+    // always succeed.
+    new (current_value_resized) uint8_t(std::move(*static_cast<uint8_t*>(current_value_original)));
+    current_value_original = static_cast<char*>(current_value_original) + bucket_size_;
+    current_value_resized = static_cast<char*>(current_value_resized) + bucket_size_;
+  }
+
+  // Copy over variable-length key components, if any.
+  if (original_variable_storage_used > 0) {
+    DEBUG_ASSERT(original_variable_storage_used
+                 == key_manager_.getNextVariableLengthKeyOffset());
+    DEBUG_ASSERT(original_variable_storage_used <= resized_variable_length_key_storage_size);
+    std::memcpy(resized_variable_length_key_storage,
+                key_manager_.getVariableLengthKeyStorage(),
+                original_variable_storage_used);
+  }
+
+  // Destroy values in the original hash table, if neccesary,
+  DestroyValues(buckets_,
+                original_buckets_used,
+                bucket_size_);
+
+  // Make resized structures active.
+  std::swap(this->blob_, resized_blob);
+  header_ = resized_header;
+  slots_ = resized_slots;
+  buckets_ = resized_buckets;
+  key_manager_.setVariableLengthStorageInfo(
+      resized_variable_length_key_storage,
+      resized_variable_length_key_storage_size,
+      &(resized_header->variable_length_bytes_allocated));
+
+  // Drop the old blob.
+  const block_id old_blob_id = resized_blob->getID();
+  resized_blob.release();
+  this->storage_manager_->deleteBlockOrBlobFile(old_blob_id);
+
+  // Rebuild chains.
+  void *current_bucket = buckets_;
+  for (std::size_t bucket_num = 0; bucket_num < original_buckets_used; ++bucket_num) {
+    std::atomic<std::size_t> *next_ptr
+        = static_cast<std::atomic<std::size_t>*>(current_bucket);
+    const std::size_t hash_code = *reinterpret_cast<const std::size_t*>(
+        static_cast<const char*>(current_bucket) + sizeof(std::atomic<std::size_t>));
+
+    const std::size_t slot_number = hash_code % header_->num_slots;
+    std::size_t slot_ptr_value = 0;
+    if (slots_[slot_number].compare_exchange_strong(slot_ptr_value,
+                                                    bucket_num + 1,
+                                                    std::memory_order_relaxed)) {
+      // This bucket is the first in the chain for this block, so reset its
+      // next pointer to 0.
+      next_ptr->store(0, std::memory_order_relaxed);
+    } else {
+      // A chain already exists starting from this slot, so put this bucket at
+      // the head.
+      next_ptr->store(slot_ptr_value, std::memory_order_relaxed);
+      slots_[slot_number].store(bucket_num + 1, std::memory_order_relaxed);
+    }
+    current_bucket = static_cast<char*>(current_bucket) + bucket_size_;
+  }
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::preallocateForBulkInsert(const std::size_t total_entries,
+                               const std::size_t total_variable_key_size,
+                               HashTablePreallocationState *prealloc_state) {
+  DEBUG_ASSERT(allow_duplicate_keys);
+  if (!key_manager_.allocateVariableLengthKeyStorage(total_variable_key_size)) {
+    return false;
+  }
+
+  // We use load then compare-exchange here instead of simply fetch-add,
+  // because if multiple threads are simultaneously trying to allocate more
+  // than one bucket and exceed 'header_->num_buckets', their respective
+  // rollbacks might happen in such an order that some bucket ranges get
+  // skipped, while others might get double-allocated later.
+  std::size_t original_buckets_allocated = header_->buckets_allocated.load(std::memory_order_relaxed);
+  std::size_t buckets_post_allocation = original_buckets_allocated + total_entries;
+  while ((buckets_post_allocation <= header_->num_buckets)
+         && !header_->buckets_allocated.compare_exchange_weak(original_buckets_allocated,
+                                                              buckets_post_allocation,
+                                                              std::memory_order_relaxed)) {
+    buckets_post_allocation = original_buckets_allocated + total_entries;
+  }
+
+  if (buckets_post_allocation > header_->num_buckets) {
+    key_manager_.deallocateVariableLengthKeyStorage(total_variable_key_size);
+    return false;
+  }
+
+  prealloc_state->bucket_position = original_buckets_allocated;
+  if (total_variable_key_size != 0) {
+    prealloc_state->variable_length_key_position
+        = key_manager_.incrementNextVariableLengthKeyOffset(total_variable_key_size);
+  }
+  return true;
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::DestroyValues(void *hash_buckets,
+                    const std::size_t num_buckets,
+                    const std::size_t bucket_size) {
+  if (!std::is_trivially_destructible<uint8_t>::value) {
+    void *value_ptr = static_cast<char*>(hash_buckets) + kValueOffset;
+    for (std::size_t bucket_num = 0;
+         bucket_num < num_buckets;
+         ++bucket_num) {
+     static_cast<uint8_t*>(value_ptr)->~uint8_t();
+      value_ptr = static_cast<char*>(value_ptr) + bucket_size;
+    }
+  }
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+inline bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::locateBucketForInsertion(const std::size_t hash_code,
+                               const std::size_t variable_key_allocation_required,
+                               void **bucket,
+                               std::atomic<std::size_t> **pending_chain_ptr,
+                               std::size_t *pending_chain_ptr_finish_value,
+                               HashTablePreallocationState *prealloc_state) {
+  DEBUG_ASSERT((prealloc_state == nullptr) || allow_duplicate_keys);
+  if (*bucket == nullptr) {
+    *pending_chain_ptr = &(slots_[hash_code % header_->num_slots]);
+  } else {
+    *pending_chain_ptr = static_cast<std::atomic<std::size_t>*>(*bucket);
+  }
+  for (;;) {
+    std::size_t existing_chain_ptr = 0;
+    if ((*pending_chain_ptr)->compare_exchange_strong(existing_chain_ptr,
+                                                      std::numeric_limits<std::size_t>::max(),
+                                                      std::memory_order_acq_rel)) {
+      // Got to the end of the chain. Allocate a new bucket.
+
+      // First, allocate variable-length key storage, if needed (i.e. if this
+      // is an upsert and we didn't allocate up-front).
+      if ((prealloc_state == nullptr)
+          && !key_manager_.allocateVariableLengthKeyStorage(variable_key_allocation_required)) {
+        // Ran out of variable-length storage.
+        (*pending_chain_ptr)->store(0, std::memory_order_release);
+        *bucket = nullptr;
+        return false;
+      }
+
+      const std::size_t allocated_bucket_num
+          = (prealloc_state == nullptr)
+              ? header_->buckets_allocated.fetch_add(1, std::memory_order_relaxed)
+              : (prealloc_state->bucket_position)++;
+      if (allocated_bucket_num >= header_->num_buckets) {
+        // Ran out of buckets.
+        DEBUG_ASSERT(prealloc_state == nullptr);
+        header_->buckets_allocated.fetch_sub(1, std::memory_order_relaxed);
+        (*pending_chain_ptr)->store(0, std::memory_order_release);
+        *bucket = nullptr;
+        return false;
+      } else {
+        *bucket = static_cast<char*>(buckets_) + allocated_bucket_num * bucket_size_;
+        *pending_chain_ptr_finish_value = allocated_bucket_num + 1;
+        return true;
+      }
+    }
+    // Spin until the real "next" pointer is available.
+    while (existing_chain_ptr == std::numeric_limits<std::size_t>::max()) {
+      existing_chain_ptr = (*pending_chain_ptr)->load(std::memory_order_acquire);
+    }
+    if (existing_chain_ptr == 0) {
+      // Other thread had to roll back, so try again.
+      continue;
+    }
+    // Chase the next pointer.
+    *bucket = static_cast<char*>(buckets_) + (existing_chain_ptr - 1) * bucket_size_;
+    *pending_chain_ptr = static_cast<std::atomic<std::size_t>*>(*bucket);
+    if (!allow_duplicate_keys) {
+      const std::size_t hash_in_bucket
+          = *reinterpret_cast<const std::size_t*>(static_cast<const char*>(*bucket)
+                                                  + sizeof(std::atomic<std::size_t>));
+      if (hash_in_bucket == hash_code) {
+        return false;
+      }
+    }
+  }
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+inline void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::writeScalarKeyToBucket(const TypedValue &key,
+                             const std::size_t hash_code,
+                             void *bucket,
+                             HashTablePreallocationState *prealloc_state) {
+  *reinterpret_cast<std::size_t*>(static_cast<char*>(bucket) + sizeof(std::atomic<std::size_t>))
+      = hash_code;
+  key_manager_.writeKeyComponentToBucket(key, 0, bucket, prealloc_state);
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+inline void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::writeCompositeKeyToBucket(const std::vector<TypedValue> &key,
+                                const std::size_t hash_code,
+                                void *bucket,
+                                HashTablePreallocationState *prealloc_state) {
+  DEBUG_ASSERT(key.size() == this->key_types_.size());
+  *reinterpret_cast<std::size_t*>(static_cast<char*>(bucket) + sizeof(std::atomic<std::size_t>))
+      = hash_code;
+  for (std::size_t idx = 0;
+       idx < this->key_types_.size();
+       ++idx) {
+    key_manager_.writeKeyComponentToBucket(key[idx], idx, bucket, prealloc_state);
+  }
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::isFull(const std::size_t extra_variable_storage) const {
+  if (header_->buckets_allocated.load(std::memory_order_relaxed) >= header_->num_buckets) {
+    // All buckets are allocated.
+    return true;
+  }
+
+  if (extra_variable_storage > 0) {
+    if (extra_variable_storage
+            + header_->variable_length_bytes_allocated.load(std::memory_order_relaxed)
+        > key_manager_.getVariableLengthKeyStorageSize()) {
+      // Not enough variable-length key storage space.
+      return true;
+    }
+  }
+
+  return false;
+}
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_SEPARATE_CHAINING_HASH_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index f1594e3..7eadae9 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -66,7 +66,7 @@ class HashTableBase {
  public:
   virtual ~HashTableBase() {
   }
-
+virtual size_t get_buckets_allocated() const {return 0;}
  protected:
   HashTableBase() {
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/storage/HashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTablePool.hpp b/storage/HashTablePool.hpp
index 53fe514..17578de 100644
--- a/storage/HashTablePool.hpp
+++ b/storage/HashTablePool.hpp
@@ -27,6 +27,8 @@
 
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/FastHashTable.hpp"
+#include "storage/FastHashTableFactory.hpp"
 #include "threading/SpinMutex.hpp"
 #include "utility/Macros.hpp"
 #include "utility/StringUtil.hpp"
@@ -81,6 +83,19 @@ class HashTablePool {
         agg_handle_(DCHECK_NOTNULL(agg_handle)),
         storage_manager_(DCHECK_NOTNULL(storage_manager)) {}
 
+  HashTablePool(const std::size_t estimated_num_entries,
+                const HashTableImplType hash_table_impl_type,
+                const std::vector<const Type *> &group_by_types,
+                const std::vector<std::size_t> &payload_sizes,
+                const std::vector<AggregationHandle *> &handles,
+                StorageManager *storage_manager)
+      : estimated_num_entries_(reduceEstimatedCardinality(estimated_num_entries)),
+        hash_table_impl_type_(hash_table_impl_type),
+        group_by_types_(group_by_types),
+        payload_sizes_(payload_sizes),
+        handles_(handles),
+        storage_manager_(DCHECK_NOTNULL(storage_manager)) {}
+
   /**
    * @brief Check out a hash table for insertion.
    *
@@ -100,6 +115,20 @@ class HashTablePool {
     return createNewHashTable();
   }
 
+  AggregationStateHashTableBase* getHashTableFast() {
+    {
+      SpinMutexLock lock(mutex_);
+      if (!hash_tables_.empty()) {
+        std::unique_ptr<AggregationStateHashTableBase> ret_hash_table(
+            std::move(hash_tables_.back()));
+        hash_tables_.pop_back();
+        DCHECK(ret_hash_table != nullptr);
+        return ret_hash_table.release();
+      }
+    }
+    return createNewHashTableFast();
+  }
+
   /**
    * @brief Return a previously checked out hash table.
    *
@@ -134,6 +163,16 @@ class HashTablePool {
                                                storage_manager_);
   }
 
+  AggregationStateHashTableBase* createNewHashTableFast() {
+    return AggregationStateFastHashTableFactory::CreateResizable(
+                hash_table_impl_type_,
+                group_by_types_,
+                estimated_num_entries_,
+                payload_sizes_,
+                handles_,
+                storage_manager_);
+  }
+
   inline std::size_t reduceEstimatedCardinality(
       const std::size_t original_estimate) const {
     if (original_estimate < kEstimateReductionFactor) {
@@ -153,7 +192,10 @@ class HashTablePool {
 
   const std::vector<const Type *> group_by_types_;
 
+  std::vector<std::size_t> payload_sizes_;
+
   AggregationHandle *agg_handle_;
+  const std::vector<AggregationHandle *> handles_;
   StorageManager *storage_manager_;
 
   SpinMutex mutex_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 21aa12c..50732fd 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -38,6 +38,7 @@
 #include "storage/CompressedPackedRowStoreTupleStorageSubBlock.hpp"
 #include "storage/CountedReference.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/FastHashTable.hpp"
 #include "storage/IndexSubBlock.hpp"
 #include "storage/InsertDestinationInterface.hpp"
 #include "storage/PackedRowStoreTupleStorageSubBlock.hpp"
@@ -494,6 +495,92 @@ void StorageBlock::aggregateGroupBy(
                                              hash_table);
 }
 
+
+void StorageBlock::aggregateGroupByFast(
+    const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
+    const std::vector<std::unique_ptr<const Scalar>> &group_by,
+    const Predicate *predicate,
+    AggregationStateHashTableBase *hash_table,
+    std::unique_ptr<TupleIdSequence> *reuse_matches,
+    std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
+  DCHECK_GT(group_by.size(), 0u)
+      << "Called aggregateGroupBy() with zero GROUP BY expressions";
+
+  SubBlocksReference sub_blocks_ref(*tuple_store_,
+                                    indices_,
+                                    indices_consistent_);
+
+  // IDs of 'arguments' as attributes in the ValueAccessor we create below.
+  std::vector<attribute_id> arg_ids;
+  std::vector<std::vector<attribute_id>> argument_ids;
+
+  // IDs of GROUP BY key element(s) in the ValueAccessor we create below.
+  std::vector<attribute_id> key_ids;
+
+  // An intermediate ValueAccessor that stores the materialized 'arguments' for
+  // this aggregate, as well as the GROUP BY expression values.
+  ColumnVectorsValueAccessor temp_result;
+  {
+    std::unique_ptr<ValueAccessor> accessor;
+    if (predicate) {
+      if (!*reuse_matches) {
+        // If there is a filter predicate that hasn't already been evaluated,
+        // evaluate it now and save the results for other aggregates on this
+        // same block.
+        reuse_matches->reset(getMatchesForPredicate(predicate));
+      }
+
+      // Create a filtered ValueAccessor that only iterates over predicate
+      // matches.
+      accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));
+    } else {
+      // Create a ValueAccessor that iterates over all tuples in this block
+      accessor.reset(tuple_store_->createValueAccessor());
+    }
+
+    attribute_id attr_id = 0;
+
+    // First, put GROUP BY keys into 'temp_result'.
+    if (reuse_group_by_vectors->empty()) {
+      // Compute GROUP BY values from group_by Scalars, and store them in
+      // reuse_group_by_vectors for reuse by other aggregates on this same
+      // block.
+      reuse_group_by_vectors->reserve(group_by.size());
+      for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
+        reuse_group_by_vectors->emplace_back(
+            group_by_element->getAllValues(accessor.get(), &sub_blocks_ref));
+        temp_result.addColumn(reuse_group_by_vectors->back().get(), false);
+        key_ids.push_back(attr_id++);
+      }
+    } else {
+      // Reuse precomputed GROUP BY values from reuse_group_by_vectors.
+      DCHECK_EQ(group_by.size(), reuse_group_by_vectors->size())
+          << "Wrong number of reuse_group_by_vectors";
+      for (const std::unique_ptr<ColumnVector> &reuse_cv : *reuse_group_by_vectors) {
+        temp_result.addColumn(reuse_cv.get(), false);
+        key_ids.push_back(attr_id++);
+      }
+    }
+
+    // Compute argument vectors and add them to 'temp_result'.
+    for (const std::vector<std::unique_ptr<const Scalar>> &argument : arguments) {
+        arg_ids.clear();
+        for (const std::unique_ptr<const Scalar> &args : argument) {
+          temp_result.addColumn(args->getAllValues(accessor.get(), &sub_blocks_ref));
+          arg_ids.push_back(attr_id++);
+        }
+        argument_ids.push_back(arg_ids);
+     }
+  }
+
+  static_cast<AggregationStateFastHashTable *>(hash_table)->upsertValueAccessorCompositeKeyFast(
+      argument_ids,
+      &temp_result,
+      key_ids,
+      true);
+}
+
+
 void StorageBlock::aggregateDistinct(
     const AggregationHandle &handle,
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
@@ -582,7 +669,6 @@ void StorageBlock::aggregateDistinct(
       &temp_result, key_ids, distinctify_hash_table);
 }
 
-
 // TODO(chasseur): Vectorization for updates.
 StorageBlock::UpdateResult StorageBlock::update(
     const unordered_map<attribute_id, unique_ptr<const Scalar>> &assignments,



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

Posted by hb...@apache.org.
Modified Aggregation unit test. Ran clang-format.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/c123bd49
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/c123bd49
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/c123bd49

Branch: refs/heads/partitioned-aggregation
Commit: c123bd494fd98428a488d349b9ecb2114092a33f
Parents: 3606000
Author: rathijit <ra...@node-2.aggregation-pr.quickstep-pg0.wisc.cloudlab.us>
Authored: Sun Aug 21 05:33:40 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:56:07 2016 -0500

----------------------------------------------------------------------
 .../aggregation/AggregationConcreteHandle.cpp   |   15 +-
 .../aggregation/AggregationConcreteHandle.hpp   |  153 +-
 expressions/aggregation/AggregationHandle.hpp   |   48 +-
 .../aggregation/AggregationHandleAvg.cpp        |   96 +-
 .../aggregation/AggregationHandleAvg.hpp        |  130 +-
 .../aggregation/AggregationHandleCount.cpp      |  150 +-
 .../aggregation/AggregationHandleCount.hpp      |  118 +-
 .../aggregation/AggregationHandleDistinct.hpp   |   28 +-
 .../aggregation/AggregationHandleMax.cpp        |   71 +-
 .../aggregation/AggregationHandleMax.hpp        |   98 +-
 .../aggregation/AggregationHandleMin.cpp        |   73 +-
 .../aggregation/AggregationHandleMin.hpp        |  101 +-
 .../aggregation/AggregationHandleSum.cpp        |   87 +-
 .../aggregation/AggregationHandleSum.hpp        |  113 +-
 expressions/aggregation/CMakeLists.txt          |   85 +-
 .../tests/AggregationHandleAvg_unittest.cpp     |  255 +--
 .../tests/AggregationHandleCount_unittest.cpp   |  311 ++--
 .../tests/AggregationHandleMax_unittest.cpp     |  382 +++--
 .../tests/AggregationHandleMin_unittest.cpp     |  378 +++--
 .../tests/AggregationHandleSum_unittest.cpp     |  291 ++--
 storage/AggregationOperationState.cpp           |  263 ++--
 storage/AggregationOperationState.hpp           |   42 +-
 storage/FastHashTable.hpp                       | 1461 +++++++++---------
 storage/FastSeparateChainingHashTable.hpp       | 1171 ++++++++------
 storage/HashTableBase.hpp                       |   20 +-
 25 files changed, 3281 insertions(+), 2659 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationConcreteHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.cpp b/expressions/aggregation/AggregationConcreteHandle.cpp
index ac5148b..ae677d9 100644
--- a/expressions/aggregation/AggregationConcreteHandle.cpp
+++ b/expressions/aggregation/AggregationConcreteHandle.cpp
@@ -23,8 +23,8 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
 #include "storage/FastHashTable.hpp"
+#include "storage/HashTable.hpp"
 #include "storage/HashTableFactory.hpp"
 
 namespace quickstep {
@@ -52,24 +52,19 @@ void AggregationConcreteHandle::insertValueAccessorIntoDistinctifyHashTable(
     AggregationStateHashTableBase *distinctify_hash_table) const {
   // If the key-value pair is already there, we don't need to update the value,
   // which should always be "true". I.e. the value is just a placeholder.
-  //  const auto noop_upserter = [](const auto &accessor, const bool *value) -> void {};
 
   AggregationStateFastHashTable *hash_table =
       static_cast<AggregationStateFastHashTable *>(distinctify_hash_table);
   if (key_ids.size() == 1) {
     std::vector<std::vector<attribute_id>> args;
     args.emplace_back(key_ids);
-    hash_table->upsertValueAccessorFast(args,
-                                    accessor,
-                                    key_ids[0],
-                                    true /* check_for_null_keys */);
+    hash_table->upsertValueAccessorFast(
+        args, accessor, key_ids[0], true /* check_for_null_keys */);
   } else {
     std::vector<std::vector<attribute_id>> empty_args;
     empty_args.resize(1);
-    hash_table->upsertValueAccessorCompositeKeyFast(empty_args,
-                                                    accessor,
-                                                    key_ids,
-                                                    true /* check_for_null_keys */);
+    hash_table->upsertValueAccessorCompositeKeyFast(
+        empty_args, accessor, key_ids, true /* check_for_null_keys */);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationConcreteHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.hpp b/expressions/aggregation/AggregationConcreteHandle.hpp
index 5b47e93..ac37bae 100644
--- a/expressions/aggregation/AggregationConcreteHandle.hpp
+++ b/expressions/aggregation/AggregationConcreteHandle.hpp
@@ -21,18 +21,18 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_
 
 #include <cstddef>
-#include <vector>
 #include <utility>
+#include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/HashTable.hpp"
 #include "storage/FastHashTable.hpp"
+#include "storage/HashTable.hpp"
 #include "storage/HashTableBase.hpp"
+#include "threading/SpinMutex.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
 #include "utility/Macros.hpp"
-#include "threading/SpinMutex.hpp"
 
 #include "glog/logging.h"
 
@@ -61,7 +61,8 @@ class HashTableStateUpserterFast {
    *        table. The corresponding state (for the same key) in the destination
    *        hash table will be upserted.
    **/
-  HashTableStateUpserterFast(const HandleT &handle, const uint8_t *source_state)
+  HashTableStateUpserterFast(const HandleT &handle,
+                             const std::uint8_t *source_state)
       : handle_(handle), source_state_(source_state) {}
 
   /**
@@ -70,13 +71,13 @@ class HashTableStateUpserterFast {
    * @param destination_state The aggregation state in the aggregation hash
    *        table that is being upserted.
    **/
-  void operator()(uint8_t *destination_state) {
+  void operator()(std::uint8_t *destination_state) {
     handle_.mergeStatesFast(source_state_, destination_state);
   }
 
  private:
   const HandleT &handle_;
-  const uint8_t *source_state_;
+  const std::uint8_t *source_state_;
 
   DISALLOW_COPY_AND_ASSIGN(HashTableStateUpserterFast);
 };
@@ -108,13 +109,15 @@ class AggregationConcreteHandle : public AggregationHandle {
    */
   AggregationStateHashTableBase* createDistinctifyHashTable(
       const HashTableImplType hash_table_impl,
-      const std::vector<const Type*> &key_types,
+      const std::vector<const Type *> &key_types,
       const std::size_t estimated_num_distinct_keys,
       StorageManager *storage_manager) const override;
 
   /**
-   * @brief Implementaion for AggregationHandle::insertValueAccessorIntoDistinctifyHashTable()
-   *        that inserts the GROUP BY expressions and aggregation arguments together
+   * @brief Implementaion for
+   * AggregationHandle::insertValueAccessorIntoDistinctifyHashTable()
+   *        that inserts the GROUP BY expressions and aggregation arguments
+   * together
    *        as keys into the distinctify hash table.
    */
   void insertValueAccessorIntoDistinctifyHashTable(
@@ -123,23 +126,19 @@ class AggregationConcreteHandle : public AggregationHandle {
       AggregationStateHashTableBase *distinctify_hash_table) const override;
 
  protected:
-  AggregationConcreteHandle() {
-  }
+  AggregationConcreteHandle() {}
 
-  template <typename HandleT,
-            typename StateT>
+  template <typename HandleT, typename StateT>
   StateT* aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
       const AggregationStateHashTableBase &distinctify_hash_table) const;
 
-  template <typename HandleT,
-            typename HashTableT>
+  template <typename HandleT, typename HashTableT>
   void aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
       const AggregationStateHashTableBase &distinctify_hash_table,
       AggregationStateHashTableBase *hash_table,
-      int index) const;
+      std::size_t index) const;
 
-  template <typename HandleT,
-            typename HashTableT>
+  template <typename HandleT, typename HashTableT>
   ColumnVector* finalizeHashTableHelperFast(
       const Type &result_type,
       const AggregationStateHashTableBase &hash_table,
@@ -151,11 +150,13 @@ class AggregationConcreteHandle : public AggregationHandle {
       const AggregationStateHashTableBase &hash_table,
       const std::vector<TypedValue> &group_key,
       int index) const {
-    const std::uint8_t *group_state
-        = static_cast<const HashTableT&>(hash_table).getSingleCompositeKey(group_key, index);
+    const std::uint8_t *group_state =
+        static_cast<const HashTableT &>(hash_table)
+            .getSingleCompositeKey(group_key, index);
     DCHECK(group_state != nullptr)
         << "Could not find entry for specified group_key in HashTable";
-    return static_cast<const HandleT*>(this)->finalizeHashTableEntryFast(group_state);
+    return static_cast<const HandleT *>(this)->finalizeHashTableEntryFast(
+        group_state);
   }
 
   template <typename HandleT, typename HashTableT>
@@ -177,24 +178,26 @@ class AggregationConcreteHandle : public AggregationHandle {
 template <typename HandleT, typename ColumnVectorT>
 class HashTableAggregateFinalizer {
  public:
-  HashTableAggregateFinalizer(const HandleT &handle,
-                              std::vector<std::vector<TypedValue>> *group_by_keys,
-                              ColumnVectorT *output_column_vector)
+  HashTableAggregateFinalizer(
+      const HandleT &handle,
+      std::vector<std::vector<TypedValue>> *group_by_keys,
+      ColumnVectorT *output_column_vector)
       : handle_(handle),
         group_by_keys_(group_by_keys),
-        output_column_vector_(output_column_vector) {
-  }
+        output_column_vector_(output_column_vector) {}
 
   inline void operator()(const std::vector<TypedValue> &group_by_key,
                          const AggregationState &group_state) {
     group_by_keys_->emplace_back(group_by_key);
-    output_column_vector_->appendTypedValue(handle_.finalizeHashTableEntry(group_state));
+    output_column_vector_->appendTypedValue(
+        handle_.finalizeHashTableEntry(group_state));
   }
 
   inline void operator()(const std::vector<TypedValue> &group_by_key,
                          const unsigned char *byte_ptr) {
     group_by_keys_->emplace_back(group_by_key);
-    output_column_vector_->appendTypedValue(handle_.finalizeHashTableEntryFast(byte_ptr));
+    output_column_vector_->appendTypedValue(
+        handle_.finalizeHashTableEntryFast(byte_ptr));
   }
 
  private:
@@ -208,47 +211,51 @@ class HashTableAggregateFinalizer {
 // ----------------------------------------------------------------------------
 // Implementations of templated methods follow:
 
-template <typename HandleT,
-          typename StateT>
-StateT* AggregationConcreteHandle::aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
-    const AggregationStateHashTableBase &distinctify_hash_table) const {
-  const HandleT& handle = static_cast<const HandleT&>(*this);
-  StateT *state = static_cast<StateT*>(createInitialState());
+template <typename HandleT, typename StateT>
+StateT* AggregationConcreteHandle::
+    aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
+        const AggregationStateHashTableBase &distinctify_hash_table) const {
+  const HandleT &handle = static_cast<const HandleT &>(*this);
+  StateT *state = static_cast<StateT *>(createInitialState());
 
   // A lambda function which will be called on each key from the distinctify
   // hash table.
-  const auto aggregate_functor = [&handle, &state](const TypedValue &key,
-                                                   const std::uint8_t &dumb_placeholder) {
+  const auto aggregate_functor = [&handle, &state](
+      const TypedValue &key, const std::uint8_t &dumb_placeholder) {
     // For each (unary) key in the distinctify hash table, aggregate the key
     // into "state".
     handle.iterateUnaryInl(state, key);
   };
 
   const AggregationStateFastHashTable &hash_table =
-      static_cast<const AggregationStateFastHashTable &>(distinctify_hash_table);
-  // Invoke the lambda function "aggregate_functor" on each key from the distinctify
+      static_cast<const AggregationStateFastHashTable &>(
+          distinctify_hash_table);
+  // Invoke the lambda function "aggregate_functor" on each key from the
+  // distinctify
   // hash table.
   hash_table.forEach(&aggregate_functor);
 
   return state;
 }
 
-template <typename HandleT,
-          typename HashTableT>
-void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
-    const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    int index) const {
-  const HandleT& handle = static_cast<const HandleT&>(*this);
-  HashTableT *target_hash_table = static_cast<HashTableT*>(aggregation_hash_table);
+template <typename HandleT, typename HashTableT>
+void AggregationConcreteHandle::
+    aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
+        const AggregationStateHashTableBase &distinctify_hash_table,
+        AggregationStateHashTableBase *aggregation_hash_table,
+        std::size_t index) const {
+  const HandleT &handle = static_cast<const HandleT &>(*this);
+  HashTableT *target_hash_table =
+      static_cast<HashTableT *>(aggregation_hash_table);
 
   // A lambda function which will be called on each key-value pair from the
   // distinctify hash table.
   const auto aggregate_functor = [&handle, &target_hash_table, &index](
-      std::vector<TypedValue> &key,
-      const bool &dumb_placeholder) {
-    // For each (composite) key vector in the distinctify hash table with size N.
-    // The first N-1 entries are GROUP BY columns and the last entry is the argument
+      std::vector<TypedValue> &key, const bool &dumb_placeholder) {
+    // For each (composite) key vector in the distinctify hash table with size
+    // N.
+    // The first N-1 entries are GROUP BY columns and the last entry is the
+    // argument
     // to be aggregated on.
     const TypedValue argument(std::move(key.back()));
     key.pop_back();
@@ -263,59 +270,55 @@ void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHe
   };
 
   const HashTableT &source_hash_table =
-      static_cast<const HashTableT&>(distinctify_hash_table);
+      static_cast<const HashTableT &>(distinctify_hash_table);
   // Invoke the lambda function "aggregate_functor" on each composite key vector
   // from the distinctify hash table.
   source_hash_table.forEachCompositeKeyFast(&aggregate_functor);
 }
 
-template <typename HandleT,
-          typename HashTableT>
+template <typename HandleT, typename HashTableT>
 ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
     const Type &result_type,
     const AggregationStateHashTableBase &hash_table,
     std::vector<std::vector<TypedValue>> *group_by_keys,
     int index) const {
-  const HandleT &handle = static_cast<const HandleT&>(*this);
-  const HashTableT &hash_table_concrete = static_cast<const HashTableT&>(hash_table);
+  const HandleT &handle = static_cast<const HandleT &>(*this);
+  const HashTableT &hash_table_concrete =
+      static_cast<const HashTableT &>(hash_table);
 
   if (group_by_keys->empty()) {
     if (NativeColumnVector::UsableForType(result_type)) {
-      NativeColumnVector *result = new NativeColumnVector(result_type,
-                                                          hash_table_concrete.numEntries());
+      NativeColumnVector *result =
+          new NativeColumnVector(result_type, hash_table_concrete.numEntries());
       HashTableAggregateFinalizer<HandleT, NativeColumnVector> finalizer(
-          handle,
-          group_by_keys,
-          result);
+          handle, group_by_keys, result);
       hash_table_concrete.forEachCompositeKeyFast(&finalizer, index);
       return result;
     } else {
-      IndirectColumnVector *result = new IndirectColumnVector(result_type,
-                                                              hash_table_concrete.numEntries());
+      IndirectColumnVector *result = new IndirectColumnVector(
+          result_type, hash_table_concrete.numEntries());
       HashTableAggregateFinalizer<HandleT, IndirectColumnVector> finalizer(
-          handle,
-          group_by_keys,
-          result);
+          handle, group_by_keys, result);
       hash_table_concrete.forEachCompositeKeyFast(&finalizer, index);
       return result;
     }
   } else {
     if (NativeColumnVector::UsableForType(result_type)) {
-      NativeColumnVector *result = new NativeColumnVector(result_type,
-                                                          group_by_keys->size());
+      NativeColumnVector *result =
+          new NativeColumnVector(result_type, group_by_keys->size());
       for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
-        result->appendTypedValue(finalizeGroupInHashTableFast<HandleT, HashTableT>(hash_table,
-                                                                                   group_by_key,
-                                                                                   index));
+        result->appendTypedValue(
+            finalizeGroupInHashTableFast<HandleT, HashTableT>(
+                hash_table, group_by_key, index));
       }
       return result;
     } else {
-      IndirectColumnVector *result = new IndirectColumnVector(result_type,
-                                                              hash_table_concrete.numEntries());
+      IndirectColumnVector *result = new IndirectColumnVector(
+          result_type, hash_table_concrete.numEntries());
       for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
-        result->appendTypedValue(finalizeGroupInHashTableFast<HandleT, HashTableT>(hash_table,
-                                                                                   group_by_key,
-                                                                                   index));
+        result->appendTypedValue(
+            finalizeGroupInHashTableFast<HandleT, HashTableT>(
+                hash_table, group_by_key, index));
       }
       return result;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index 01e3d20..d2cee6d 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -40,7 +40,6 @@ class ValueAccessor;
  *  @{
  */
 
-
 /**
  * @brief Abstract base class for aggregation state.
  **/
@@ -107,8 +106,7 @@ class AggregationHandle {
    * @brief Virtual destructor.
    *
    **/
-  virtual ~AggregationHandle() {
-  }
+  virtual ~AggregationHandle() {}
 
   /**
    * @brief Create an initial "blank" state for this aggregation.
@@ -136,7 +134,7 @@ class AggregationHandle {
    **/
   virtual AggregationStateHashTableBase* createGroupByHashTable(
       const HashTableImplType hash_table_impl,
-      const std::vector<const Type*> &group_by_types,
+      const std::vector<const Type *> &group_by_types,
       const std::size_t estimated_num_groups,
       StorageManager *storage_manager) const = 0;
 
@@ -167,7 +165,8 @@ class AggregationHandle {
    *         for deleting the returned AggregationState.
    **/
   virtual AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const = 0;
+      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
+      const = 0;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   /**
@@ -269,7 +268,8 @@ class AggregationHandle {
       int index) const = 0;
 
   /**
-   * @brief Create a new HashTable for the distinctify step for DISTINCT aggregation.
+   * @brief Create a new HashTable for the distinctify step for DISTINCT
+   * aggregation.
    *
    * Distinctify is the first step for DISTINCT aggregation. This step inserts
    * the GROUP BY expression values and aggregation arguments together as keys
@@ -282,7 +282,8 @@ class AggregationHandle {
    * we simply treat it as a special GROUP BY case that the GROUP BY expression
    * vector is empty.
    *
-   * @param hash_table_impl The choice of which concrete HashTable implementation
+   * @param hash_table_impl The choice of which concrete HashTable
+   * implementation
    *        to use.
    * @param key_types The types of the GROUP BY expressions together with the
    *        types of the aggregation arguments.
@@ -292,14 +293,15 @@ class AggregationHandle {
    *        This is an estimate only, and the HashTable will be resized if it
    *        becomes over-full.
    * @param storage_manager The StorageManager to use to create the HashTable.
-   *        A StorageBlob will be allocated to serve as the HashTable's in-memory
+   *        A StorageBlob will be allocated to serve as the HashTable's
+   * in-memory
    *        storage.
    * @return A new HashTable instance with the appropriate state type for this
    *         aggregate as the ValueT.
    */
   virtual AggregationStateHashTableBase* createDistinctifyHashTable(
       const HashTableImplType hash_table_impl,
-      const std::vector<const Type*> &key_types,
+      const std::vector<const Type *> &key_types,
       const std::size_t estimated_num_distinct_keys,
       StorageManager *storage_manager) const = 0;
 
@@ -307,11 +309,13 @@ class AggregationHandle {
    * @brief Inserts the GROUP BY expressions and aggregation arguments together
    * as keys into the distinctify hash table.
    *
-   * @param accessor The ValueAccessor that will be iterated over to read tuples.
+   * @param accessor The ValueAccessor that will be iterated over to read
+   * tuples.
    * @param key_ids The attribute_ids of the GROUP BY expressions in accessor
    *        together with the attribute_ids of the arguments to this aggregate
    *        in accessor, in order.
-   * @param distinctify_hash_table The HashTable to store the GROUP BY expressions
+   * @param distinctify_hash_table The HashTable to store the GROUP BY
+   * expressions
    *        and the aggregation arguments together as hash table keys and a bool
    *        constant \c true as hash table value (So the hash table actually
    *        serves as a hash set). This should have been created by calling
@@ -340,7 +344,8 @@ class AggregationHandle {
    * @brief Perform GROUP BY aggregation on the keys from the distinctify hash
    * table and upserts states into the aggregation hash table.
    *
-   * @param distinctify_hash_table Hash table which stores the GROUP BY expression
+   * @param distinctify_hash_table Hash table which stores the GROUP BY
+   * expression
    *        values and aggregation arguments together as hash table keys.
    * @param aggregation_hash_table The HashTable to upsert AggregationStates in.
    *        This should have been created by calling createGroupByHashTable() on
@@ -349,18 +354,19 @@ class AggregationHandle {
   virtual void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
       AggregationStateHashTableBase *aggregation_hash_table,
-      int index) const = 0;
+      std::size_t index) const = 0;
 
-  virtual size_t getPayloadSize() const {return 1;}
-  virtual void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const {}
-  virtual void mergeStatesFast(const uint8_t *src, uint8_t *dst) const {}
-  virtual void initPayload(uint8_t *byte_ptr) const {}
-  virtual void BlockUpdate() {}
-  virtual void AllowUpdate() {}
+  virtual std::size_t getPayloadSize() const { return 1; }
+  virtual void updateState(const std::vector<TypedValue> &arguments,
+                           std::uint8_t *byte_ptr) const {}
+  virtual void mergeStatesFast(const std::uint8_t *src,
+                               std::uint8_t *dst) const {}
+  virtual void initPayload(std::uint8_t *byte_ptr) const {}
+  virtual void blockUpdate() {}
+  virtual void allowUpdate() {}
 
  protected:
-  AggregationHandle() {
-  }
+  AggregationHandle() {}
 
  private:
   DISALLOW_COPY_AND_ASSIGN(AggregationHandle);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index b27888f..2481092 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.cpp
@@ -42,7 +42,7 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleAvg::AggregationHandleAvg(const Type &type)
-    : argument_type_(type), block_update(false) {
+    : argument_type_(type), block_update_(false) {
   // We sum Int as Long and Float as Double so that we have more headroom when
   // adding many values.
   TypeID type_precision_id;
@@ -76,26 +76,24 @@ AggregationHandleAvg::AggregationHandleAvg(const Type &type)
   // Divide operator for dividing sum by count to get final average.
   divide_operator_.reset(
       BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
-          .makeUncheckedBinaryOperatorForTypes(sum_type, TypeFactory::GetType(kDouble)));
+          .makeUncheckedBinaryOperatorForTypes(sum_type,
+                                               TypeFactory::GetType(kDouble)));
 
   // Result is nullable, because AVG() over 0 values (or all NULL values) is
   // NULL.
-  result_type_
-      = &(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
-              .resultTypeForArgumentTypes(sum_type, TypeFactory::GetType(kDouble))
-                  ->getNullableVersion());
+  result_type_ =
+      &(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
+            .resultTypeForArgumentTypes(sum_type, TypeFactory::GetType(kDouble))
+            ->getNullableVersion());
 }
 
 AggregationStateHashTableBase* AggregationHandleAvg::createGroupByHashTable(
     const HashTableImplType hash_table_impl,
-    const std::vector<const Type*> &group_by_types,
+    const std::vector<const Type *> &group_by_types,
     const std::size_t estimated_num_groups,
     StorageManager *storage_manager) const {
   return AggregationStateHashTableFactory<AggregationStateAvg>::CreateResizable(
-      hash_table_impl,
-      group_by_types,
-      estimated_num_groups,
-      storage_manager);
+      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
 }
 
 AggregationState* AggregationHandleAvg::accumulateColumnVectors(
@@ -105,9 +103,8 @@ AggregationState* AggregationHandleAvg::accumulateColumnVectors(
 
   AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
   std::size_t count = 0;
-  state->sum_ = fast_add_operator_->accumulateColumnVector(state->sum_,
-                                                           *column_vectors.front(),
-                                                           &count);
+  state->sum_ = fast_add_operator_->accumulateColumnVector(
+      state->sum_, *column_vectors.front(), &count);
   state->count_ = count;
   return state;
 }
@@ -121,10 +118,8 @@ AggregationState* AggregationHandleAvg::accumulateValueAccessor(
 
   AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
   std::size_t count = 0;
-  state->sum_ = fast_add_operator_->accumulateValueAccessor(state->sum_,
-                                                            accessor,
-                                                            accessor_ids.front(),
-                                                            &count);
+  state->sum_ = fast_add_operator_->accumulateValueAccessor(
+      state->sum_, accessor, accessor_ids.front(), &count);
   state->count_ = count;
   return state;
 }
@@ -139,40 +134,44 @@ void AggregationHandleAvg::aggregateValueAccessorIntoHashTable(
       << "Got wrong number of arguments for AVG: " << argument_ids.size();
 }
 
-void AggregationHandleAvg::mergeStates(
-    const AggregationState &source,
-    AggregationState *destination) const {
-  const AggregationStateAvg &avg_source = static_cast<const AggregationStateAvg&>(source);
-  AggregationStateAvg *avg_destination = static_cast<AggregationStateAvg*>(destination);
+void AggregationHandleAvg::mergeStates(const AggregationState &source,
+                                       AggregationState *destination) const {
+  const AggregationStateAvg &avg_source =
+      static_cast<const AggregationStateAvg &>(source);
+  AggregationStateAvg *avg_destination =
+      static_cast<AggregationStateAvg *>(destination);
 
   SpinMutexLock lock(avg_destination->mutex_);
   avg_destination->count_ += avg_source.count_;
-  avg_destination->sum_ = merge_add_operator_->applyToTypedValues(avg_destination->sum_,
-                                                                  avg_source.sum_);
+  avg_destination->sum_ = merge_add_operator_->applyToTypedValues(
+      avg_destination->sum_, avg_source.sum_);
 }
 
-void AggregationHandleAvg::mergeStatesFast(
-    const uint8_t *source,
-    uint8_t *destination) const {
-    const TypedValue *src_sum_ptr = reinterpret_cast<const TypedValue *>(source + blank_state_.sum_offset);
-    const std::int64_t *src_count_ptr = reinterpret_cast<const std::int64_t *>(source + blank_state_.count_offset);
-    TypedValue *dst_sum_ptr = reinterpret_cast<TypedValue *>(destination+blank_state_.sum_offset);
-    std::int64_t *dst_count_ptr = reinterpret_cast<std::int64_t *>(destination + blank_state_.count_offset);
-    (*dst_count_ptr) += (*src_count_ptr);
-    *dst_sum_ptr = merge_add_operator_->applyToTypedValues(*dst_sum_ptr, *src_sum_ptr);
+void AggregationHandleAvg::mergeStatesFast(const std::uint8_t *source,
+                                           std::uint8_t *destination) const {
+  const TypedValue *src_sum_ptr =
+      reinterpret_cast<const TypedValue *>(source + blank_state_.sum_offset_);
+  const std::int64_t *src_count_ptr = reinterpret_cast<const std::int64_t *>(
+      source + blank_state_.count_offset_);
+  TypedValue *dst_sum_ptr =
+      reinterpret_cast<TypedValue *>(destination + blank_state_.sum_offset_);
+  std::int64_t *dst_count_ptr = reinterpret_cast<std::int64_t *>(
+      destination + blank_state_.count_offset_);
+  (*dst_count_ptr) += (*src_count_ptr);
+  *dst_sum_ptr =
+      merge_add_operator_->applyToTypedValues(*dst_sum_ptr, *src_sum_ptr);
 }
 
-
-
 TypedValue AggregationHandleAvg::finalize(const AggregationState &state) const {
-  const AggregationStateAvg &agg_state = static_cast<const AggregationStateAvg&>(state);
+  const AggregationStateAvg &agg_state =
+      static_cast<const AggregationStateAvg &>(state);
   if (agg_state.count_ == 0) {
     // AVG() over no values is NULL.
     return result_type_->makeNullValue();
   } else {
     // Divide sum by count to get final average.
-    return divide_operator_->applyToTypedValues(agg_state.sum_,
-                                                TypedValue(static_cast<double>(agg_state.count_)));
+    return divide_operator_->applyToTypedValues(
+        agg_state.sum_, TypedValue(static_cast<double>(agg_state.count_)));
   }
 }
 
@@ -181,31 +180,26 @@ ColumnVector* AggregationHandleAvg::finalizeHashTable(
     std::vector<std::vector<TypedValue>> *group_by_keys,
     int index) const {
   return finalizeHashTableHelperFast<AggregationHandleAvg,
-                                 AggregationStateFastHashTable>(
-      *result_type_,
-      hash_table,
-      group_by_keys,
-      index);
+                                     AggregationStateFastHashTable>(
+      *result_type_, hash_table, group_by_keys, index);
 }
 
-AggregationState* AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
+AggregationState*
+AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
   return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
       AggregationHandleAvg,
-      AggregationStateAvg>(
-          distinctify_hash_table);
+      AggregationStateAvg>(distinctify_hash_table);
 }
 
 void AggregationHandleAvg::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
     AggregationStateHashTableBase *aggregation_hash_table,
-    int index) const {
+    std::size_t index) const {
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleAvg,
       AggregationStateFastHashTable>(
-          distinctify_hash_table,
-          aggregation_hash_table,
-          index);
+      distinctify_hash_table, aggregation_hash_table, index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index d134620..3e49213 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -28,8 +28,8 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/HashTableBase.hpp"
 #include "storage/FastHashTable.hpp"
+#include "storage/HashTableBase.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -59,30 +59,36 @@ class AggregationStateAvg : public AggregationState {
   AggregationStateAvg(const AggregationStateAvg &orig)
       : sum_(orig.sum_),
         count_(orig.count_),
-        sum_offset(orig.sum_offset),
-        count_offset(orig.count_offset),
-        mutex_offset(orig.mutex_offset) {
-  }
+        sum_offset_(orig.sum_offset_),
+        count_offset_(orig.count_offset_),
+        mutex_offset_(orig.mutex_offset_) {}
 
   /**
    * @brief Destructor.
    */
   ~AggregationStateAvg() override {}
 
-  size_t getPayloadSize() const {
-     size_t p1 = reinterpret_cast<size_t>(&sum_);
-     size_t p2 = reinterpret_cast<size_t>(&mutex_);
-     return (p2-p1);
+  std::size_t getPayloadSize() const {
+    std::size_t p1 = reinterpret_cast<std::size_t>(&sum_);
+    std::size_t p2 = reinterpret_cast<std::size_t>(&mutex_);
+    return (p2 - p1);
+  }
+
+  const std::uint8_t *getPayloadAddress() const {
+    return reinterpret_cast<const uint8_t *>(&sum_);
   }
 
  private:
   friend class AggregationHandleAvg;
 
   AggregationStateAvg()
-      : sum_(0), count_(0), sum_offset(0),
-        count_offset(reinterpret_cast<uint8_t *>(&count_)-reinterpret_cast<uint8_t *>(&sum_)),
-        mutex_offset(reinterpret_cast<uint8_t *>(&mutex_)-reinterpret_cast<uint8_t *>(&sum_)) {
-  }
+      : sum_(0),
+        count_(0),
+        sum_offset_(0),
+        count_offset_(reinterpret_cast<std::uint8_t *>(&count_) -
+                      reinterpret_cast<std::uint8_t *>(&sum_)),
+        mutex_offset_(reinterpret_cast<std::uint8_t *>(&mutex_) -
+                      reinterpret_cast<std::uint8_t *>(&sum_)) {}
 
   // TODO(shoban): We might want to specialize sum_ and count_ to use atomics
   // for int types similar to in AggregationStateCount.
@@ -90,7 +96,7 @@ class AggregationStateAvg : public AggregationState {
   std::int64_t count_;
   SpinMutex mutex_;
 
-  int sum_offset, count_offset, mutex_offset;
+  int sum_offset_, count_offset_, mutex_offset_;
 };
 
 /**
@@ -98,8 +104,7 @@ class AggregationStateAvg : public AggregationState {
  **/
 class AggregationHandleAvg : public AggregationConcreteHandle {
  public:
-  ~AggregationHandleAvg() override {
-  }
+  ~AggregationHandleAvg() override {}
 
   AggregationState* createInitialState() const override {
     return new AggregationStateAvg(blank_state_);
@@ -107,14 +112,15 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
 
   AggregationStateHashTableBase* createGroupByHashTable(
       const HashTableImplType hash_table_impl,
-      const std::vector<const Type*> &group_by_types,
+      const std::vector<const Type *> &group_by_types,
       const std::size_t estimated_num_groups,
       StorageManager *storage_manager) const override;
 
   /**
    * @brief Iterate method with average aggregation state.
    **/
-  inline void iterateUnaryInl(AggregationStateAvg *state, const TypedValue &value) const {
+  inline void iterateUnaryInl(AggregationStateAvg *state,
+                              const TypedValue &value) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
     if (value.isNull()) return;
 
@@ -123,37 +129,41 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     ++state->count_;
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) const {
+  inline void iterateUnaryInlFast(const TypedValue &value,
+                                  std::uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
     if (value.isNull()) return;
-    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
-    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr + blank_state_.count_offset);
+    TypedValue *sum_ptr =
+        reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset_);
+    std::int64_t *count_ptr =
+        reinterpret_cast<std::int64_t *>(byte_ptr + blank_state_.count_offset_);
     *sum_ptr = fast_add_operator_->applyToTypedValues(*sum_ptr, value);
     ++(*count_ptr);
   }
 
-  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const override {
-     if (block_update) return;
-     iterateUnaryInlFast(arguments.front(), byte_ptr);
+  inline void updateState(const std::vector<TypedValue> &arguments,
+                          std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateUnaryInlFast(arguments.front(), byte_ptr);
+    }
   }
 
-  void BlockUpdate() override {
-      block_update = true;
-  }
+  void blockUpdate() override { block_update_ = true; }
 
-  void AllowUpdate() override {
-      block_update = false;
-  }
+  void allowUpdate() override { block_update_ = false; }
 
-  void initPayload(uint8_t *byte_ptr) const override {
-    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
-    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr + blank_state_.count_offset);
+  void initPayload(std::uint8_t *byte_ptr) const override {
+    TypedValue *sum_ptr =
+        reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset_);
+    std::int64_t *count_ptr =
+        reinterpret_cast<std::int64_t *>(byte_ptr + blank_state_.count_offset_);
     *sum_ptr = blank_state_.sum_;
     *count_ptr = blank_state_.count_;
   }
 
   AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
+      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
+      const override;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   AggregationState* accumulateValueAccessor(
@@ -170,33 +180,32 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override;
 
-  void mergeStatesFast(const uint8_t *source,
-                   uint8_t *destination) const override;
+  void mergeStatesFast(const std::uint8_t *source,
+                       std::uint8_t *destination) const override;
 
   TypedValue finalize(const AggregationState &state) const override;
 
-  inline TypedValue finalizeHashTableEntry(const AggregationState &state) const {
-    const AggregationStateAvg &agg_state = static_cast<const AggregationStateAvg&>(state);
+  inline TypedValue finalizeHashTableEntry(
+      const AggregationState &state) const {
+    const AggregationStateAvg &agg_state =
+        static_cast<const AggregationStateAvg &>(state);
     // TODO(chasseur): Could improve performance further if we made a special
     // version of finalizeHashTable() that collects all the sums into one
     // ColumnVector and all the counts into another and then applies
     // '*divide_operator_' to them in bulk.
-    return divide_operator_->applyToTypedValues(agg_state.sum_,
-                                                TypedValue(static_cast<double>(agg_state.count_)));
+    return divide_operator_->applyToTypedValues(
+        agg_state.sum_, TypedValue(static_cast<double>(agg_state.count_)));
   }
 
-  inline TypedValue finalizeHashTableEntryFast(const uint8_t *byte_ptr) const {
-//    const AggregationStateAvg &agg_state = static_cast<const AggregationStateAvg&>(state);
-    // TODO(chasseur): Could improve performance further if we made a special
-    // version of finalizeHashTable() that collects all the sums into one
-    // ColumnVector and all the counts into another and then applies
-    // '*divide_operator_' to them in bulk.
-
-    uint8_t *value_ptr = const_cast<uint8_t*>(byte_ptr);
-    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(value_ptr + blank_state_.sum_offset);
-    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(value_ptr + blank_state_.count_offset);
-    return divide_operator_->applyToTypedValues(*sum_ptr,
-                                                TypedValue(static_cast<double>(*count_ptr)));
+  inline TypedValue finalizeHashTableEntryFast(
+      const std::uint8_t *byte_ptr) const {
+    std::uint8_t *value_ptr = const_cast<std::uint8_t *>(byte_ptr);
+    TypedValue *sum_ptr =
+        reinterpret_cast<TypedValue *>(value_ptr + blank_state_.sum_offset_);
+    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(
+        value_ptr + blank_state_.count_offset_);
+    return divide_operator_->applyToTypedValues(
+        *sum_ptr, TypedValue(static_cast<double>(*count_ptr)));
   }
 
   ColumnVector* finalizeHashTable(
@@ -205,23 +214,26 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
       int index) const override;
 
   /**
-   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
+   * @brief Implementation of
+   * AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
    *        for AVG aggregation.
    */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table) const override;
+      const AggregationStateHashTableBase &distinctify_hash_table)
+      const override;
 
   /**
-   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
+   * @brief Implementation of
+   * AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
    *        for AVG aggregation.
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
       AggregationStateHashTableBase *aggregation_hash_table,
-      int index) const override;
+      std::size_t index) const override;
 
-  size_t getPayloadSize() const override {
-      return blank_state_.getPayloadSize();
+  std::size_t getPayloadSize() const override {
+    return blank_state_.getPayloadSize();
   }
 
  private:
@@ -241,7 +253,7 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
   std::unique_ptr<UncheckedBinaryOperator> merge_add_operator_;
   std::unique_ptr<UncheckedBinaryOperator> divide_operator_;
 
-  bool block_update;
+  bool block_update_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleAvg);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.cpp b/expressions/aggregation/AggregationHandleCount.cpp
index aff3c28..d690f71 100644
--- a/expressions/aggregation/AggregationHandleCount.cpp
+++ b/expressions/aggregation/AggregationHandleCount.cpp
@@ -49,48 +49,50 @@ class ValueAccessor;
 
 template <bool count_star, bool nullable_type>
 AggregationStateHashTableBase*
-    AggregationHandleCount<count_star, nullable_type>::createGroupByHashTable(
-        const HashTableImplType hash_table_impl,
-        const std::vector<const Type*> &group_by_types,
-        const std::size_t estimated_num_groups,
-        StorageManager *storage_manager) const {
-  return AggregationStateHashTableFactory<AggregationStateCount>::CreateResizable(
-      hash_table_impl,
-      group_by_types,
-      estimated_num_groups,
-      storage_manager);
+AggregationHandleCount<count_star, nullable_type>::createGroupByHashTable(
+    const HashTableImplType hash_table_impl,
+    const std::vector<const Type *> &group_by_types,
+    const std::size_t estimated_num_groups,
+    StorageManager *storage_manager) const {
+  return AggregationStateHashTableFactory<
+      AggregationStateCount>::CreateResizable(hash_table_impl,
+                                              group_by_types,
+                                              estimated_num_groups,
+                                              storage_manager);
 }
 
 template <bool count_star, bool nullable_type>
 AggregationState*
-    AggregationHandleCount<count_star, nullable_type>::accumulateColumnVectors(
-        const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
+AggregationHandleCount<count_star, nullable_type>::accumulateColumnVectors(
+    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
   DCHECK(!count_star)
       << "Called non-nullary accumulation method on an AggregationHandleCount "
       << "set up for nullary COUNT(*)";
 
   DCHECK_EQ(1u, column_vectors.size())
-      << "Got wrong number of ColumnVectors for COUNT: " << column_vectors.size();
+      << "Got wrong number of ColumnVectors for COUNT: "
+      << column_vectors.size();
 
   std::size_t count = 0;
   InvokeOnColumnVector(
       *column_vectors.front(),
       [&](const auto &column_vector) -> void {  // NOLINT(build/c++11)
-    if (nullable_type) {
-      // TODO(shoban): Iterating over the ColumnVector is a rather slow way to
-      // do this. We should look at extending the ColumnVector interface to do
-      // a quick count of the non-null values (i.e. the length minus the
-      // population count of the null bitmap). We should do something similar
-      // for ValueAccessor too.
-      for (std::size_t pos = 0;
-           pos < column_vector.size();
-           ++pos) {
-        count += !column_vector.getTypedValue(pos).isNull();
-      }
-    } else {
-      count = column_vector.size();
-    }
-  });
+        if (nullable_type) {
+          // TODO(shoban): Iterating over the ColumnVector is a rather slow way
+          // to
+          // do this. We should look at extending the ColumnVector interface to
+          // do
+          // a quick count of the non-null values (i.e. the length minus the
+          // population count of the null bitmap). We should do something
+          // similar
+          // for ValueAccessor too.
+          for (std::size_t pos = 0; pos < column_vector.size(); ++pos) {
+            count += !column_vector.getTypedValue(pos).isNull();
+          }
+        } else {
+          count = column_vector.size();
+        }
+      });
 
   return new AggregationStateCount(count);
 }
@@ -98,9 +100,9 @@ AggregationState*
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 template <bool count_star, bool nullable_type>
 AggregationState*
-    AggregationHandleCount<count_star, nullable_type>::accumulateValueAccessor(
-        ValueAccessor *accessor,
-        const std::vector<attribute_id> &accessor_ids) const {
+AggregationHandleCount<count_star, nullable_type>::accumulateValueAccessor(
+    ValueAccessor *accessor,
+    const std::vector<attribute_id> &accessor_ids) const {
   DCHECK(!count_star)
       << "Called non-nullary accumulation method on an AggregationHandleCount "
       << "set up for nullary COUNT(*)";
@@ -113,28 +115,30 @@ AggregationState*
   InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
       accessor,
       [&accessor_id, &count](auto *accessor) -> void {  // NOLINT(build/c++11)
-    if (nullable_type) {
-      while (accessor->next()) {
-        count += !accessor->getTypedValue(accessor_id).isNull();
-      }
-    } else {
-      count = accessor->getNumTuples();
-    }
-  });
+        if (nullable_type) {
+          while (accessor->next()) {
+            count += !accessor->getTypedValue(accessor_id).isNull();
+          }
+        } else {
+          count = accessor->getNumTuples();
+        }
+      });
 
   return new AggregationStateCount(count);
 }
 #endif
 
 template <bool count_star, bool nullable_type>
-    void AggregationHandleCount<count_star, nullable_type>::aggregateValueAccessorIntoHashTable(
+void AggregationHandleCount<count_star, nullable_type>::
+    aggregateValueAccessorIntoHashTable(
         ValueAccessor *accessor,
         const std::vector<attribute_id> &argument_ids,
         const std::vector<attribute_id> &group_by_key_ids,
         AggregationStateHashTableBase *hash_table) const {
   if (count_star) {
     DCHECK_EQ(0u, argument_ids.size())
-        << "Got wrong number of arguments for COUNT(*): " << argument_ids.size();
+        << "Got wrong number of arguments for COUNT(*): "
+        << argument_ids.size();
   } else {
     DCHECK_EQ(1u, argument_ids.size())
         << "Got wrong number of arguments for COUNT: " << argument_ids.size();
@@ -142,62 +146,60 @@ template <bool count_star, bool nullable_type>
 }
 
 template <bool count_star, bool nullable_type>
-    void AggregationHandleCount<count_star, nullable_type>::mergeStates(
-        const AggregationState &source,
-        AggregationState *destination) const {
-  const AggregationStateCount &count_source = static_cast<const AggregationStateCount&>(source);
-  AggregationStateCount *count_destination = static_cast<AggregationStateCount*>(destination);
-
-  count_destination->count_.fetch_add(count_source.count_.load(std::memory_order_relaxed),
-                                      std::memory_order_relaxed);
+void AggregationHandleCount<count_star, nullable_type>::mergeStates(
+    const AggregationState &source, AggregationState *destination) const {
+  const AggregationStateCount &count_source =
+      static_cast<const AggregationStateCount &>(source);
+  AggregationStateCount *count_destination =
+      static_cast<AggregationStateCount *>(destination);
+
+  count_destination->count_.fetch_add(
+      count_source.count_.load(std::memory_order_relaxed),
+      std::memory_order_relaxed);
 }
 
 template <bool count_star, bool nullable_type>
 void AggregationHandleCount<count_star, nullable_type>::mergeStatesFast(
-    const uint8_t *source,
-    uint8_t *destination) const {
-    const std::int64_t *src_count_ptr = reinterpret_cast<const std::int64_t *>(source);
-    std::int64_t *dst_count_ptr = reinterpret_cast<std::int64_t *>(destination);
-    (*dst_count_ptr) += (*src_count_ptr);
+    const std::uint8_t *source, std::uint8_t *destination) const {
+  const std::int64_t *src_count_ptr =
+      reinterpret_cast<const std::int64_t *>(source);
+  std::int64_t *dst_count_ptr = reinterpret_cast<std::int64_t *>(destination);
+  (*dst_count_ptr) += (*src_count_ptr);
 }
 
 template <bool count_star, bool nullable_type>
-    ColumnVector* AggregationHandleCount<count_star, nullable_type>::finalizeHashTable(
-        const AggregationStateHashTableBase &hash_table,
-        std::vector<std::vector<TypedValue>> *group_by_keys,
-        int index) const {
-  return finalizeHashTableHelperFast<AggregationHandleCount<count_star, nullable_type>,
-                                 AggregationStateFastHashTable>(
-      TypeFactory::GetType(kLong),
-      hash_table,
-      group_by_keys,
-      index);
+ColumnVector*
+AggregationHandleCount<count_star, nullable_type>::finalizeHashTable(
+    const AggregationStateHashTableBase &hash_table,
+    std::vector<std::vector<TypedValue>> *group_by_keys,
+    int index) const {
+  return finalizeHashTableHelperFast<
+      AggregationHandleCount<count_star, nullable_type>,
+      AggregationStateFastHashTable>(
+      TypeFactory::GetType(kLong), hash_table, group_by_keys, index);
 }
 
 template <bool count_star, bool nullable_type>
-AggregationState* AggregationHandleCount<count_star, nullable_type>
-    ::aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleCount<count_star, nullable_type>::
+    aggregateOnDistinctifyHashTableForSingle(
         const AggregationStateHashTableBase &distinctify_hash_table) const {
   DCHECK_EQ(count_star, false);
   return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
       AggregationHandleCount<count_star, nullable_type>,
-      AggregationStateCount>(
-          distinctify_hash_table);
+      AggregationStateCount>(distinctify_hash_table);
 }
 
 template <bool count_star, bool nullable_type>
-void AggregationHandleCount<count_star, nullable_type>
-    ::aggregateOnDistinctifyHashTableForGroupBy(
+void AggregationHandleCount<count_star, nullable_type>::
+    aggregateOnDistinctifyHashTableForGroupBy(
         const AggregationStateHashTableBase &distinctify_hash_table,
         AggregationStateHashTableBase *aggregation_hash_table,
-        int index) const {
+        std::size_t index) const {
   DCHECK_EQ(count_star, false);
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleCount<count_star, nullable_type>,
       AggregationStateFastHashTable>(
-          distinctify_hash_table,
-          aggregation_hash_table,
-          index);
+      distinctify_hash_table, aggregation_hash_table, index);
 }
 
 // Explicitly instantiate and compile in the different versions of

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index 2beb0e2..2c6d717 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.hpp
@@ -29,8 +29,8 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/HashTableBase.hpp"
 #include "storage/FastHashTable.hpp"
+#include "storage/HashTableBase.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
@@ -41,7 +41,8 @@ class StorageManager;
 class Type;
 class ValueAccessor;
 
-template <bool, bool> class AggregationHandleCount;
+template <bool, bool>
+class AggregationHandleCount;
 
 /** \addtogroup Expressions
  *  @{
@@ -63,8 +64,10 @@ class AggregationStateCount : public AggregationState {
    */
   ~AggregationStateCount() override {}
 
-  size_t getPayloadSize() const {
-     return sizeof(count_);
+  std::size_t getPayloadSize() const { return sizeof(count_); }
+
+  const std::uint8_t* getPayloadAddress() const {
+    return reinterpret_cast<const uint8_t *>(&count_);
   }
 
  private:
@@ -73,13 +76,10 @@ class AggregationStateCount : public AggregationState {
   friend class AggregationHandleCount<true, false>;
   friend class AggregationHandleCount<true, true>;
 
-  AggregationStateCount()
-      : count_(0) {
-  }
+  AggregationStateCount() : count_(0) {}
 
   explicit AggregationStateCount(const std::int64_t initial_count)
-      : count_(initial_count) {
-  }
+      : count_(initial_count) {}
 
   std::atomic<std::int64_t> count_;
 };
@@ -96,16 +96,15 @@ class AggregationStateCount : public AggregationState {
 template <bool count_star, bool nullable_type>
 class AggregationHandleCount : public AggregationConcreteHandle {
  public:
-  ~AggregationHandleCount() override {
-  }
+  ~AggregationHandleCount() override {}
 
   AggregationState* createInitialState() const override {
     return new AggregationStateCount();
   }
 
-  AggregationStateHashTableBase* createGroupByHashTable(
+  AggregationStateHashTableBase *createGroupByHashTable(
       const HashTableImplType hash_table_impl,
-      const std::vector<const Type*> &group_by_types,
+      const std::vector<const Type *> &group_by_types,
       const std::size_t estimated_num_groups,
       StorageManager *storage_manager) const override;
 
@@ -113,54 +112,56 @@ class AggregationHandleCount : public AggregationConcreteHandle {
     state->count_.fetch_add(1, std::memory_order_relaxed);
   }
 
-  inline void iterateNullaryInlFast(uint8_t *byte_ptr) const {
-      std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
-      (*count_ptr)++;
+  inline void iterateNullaryInlFast(std::uint8_t *byte_ptr) const {
+    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
+    (*count_ptr)++;
   }
 
   /**
    * @brief Iterate with count aggregation state.
    */
-  inline void iterateUnaryInl(AggregationStateCount *state, const TypedValue &value) const {
+  inline void iterateUnaryInl(AggregationStateCount *state,
+                              const TypedValue &value) const {
     if ((!nullable_type) || (!value.isNull())) {
       state->count_.fetch_add(1, std::memory_order_relaxed);
     }
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) const {
+  inline void iterateUnaryInlFast(const TypedValue &value,
+                                  std::uint8_t *byte_ptr) const {
     if ((!nullable_type) || (!value.isNull())) {
       std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
       (*count_ptr)++;
     }
   }
 
-  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const override {
-     if (block_update) return;
-     if (arguments.size())
-         iterateUnaryInlFast(arguments.front(), byte_ptr);
-     else
-         iterateNullaryInlFast(byte_ptr);
+  inline void updateState(const std::vector<TypedValue> &arguments,
+                          std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      if (arguments.size())
+        iterateUnaryInlFast(arguments.front(), byte_ptr);
+      else
+        iterateNullaryInlFast(byte_ptr);
+    }
   }
 
-  void BlockUpdate() override {
-     block_update = true;
-  }
+  void blockUpdate() override { block_update_ = true; }
 
-  void AllowUpdate() override {
-     block_update = false;
-  }
+  void allowUpdate() override { block_update_ = false; }
 
-  void initPayload(uint8_t *byte_ptr) const override {
-     std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
-     *count_ptr = 0;
+  void initPayload(std::uint8_t *byte_ptr) const override {
+    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
+    *count_ptr = 0;
   }
 
-  AggregationState* accumulateNullary(const std::size_t num_tuples) const override {
+  AggregationState* accumulateNullary(
+      const std::size_t num_tuples) const override {
     return new AggregationStateCount(num_tuples);
   }
 
   AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
+      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
+      const override;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   AggregationState* accumulateValueAccessor(
@@ -177,25 +178,26 @@ class AggregationHandleCount : public AggregationConcreteHandle {
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override;
 
-  void mergeStatesFast(const uint8_t *source,
-                   uint8_t *destination) const override;
+  void mergeStatesFast(const std::uint8_t *source,
+                       std::uint8_t *destination) const override;
 
   TypedValue finalize(const AggregationState &state) const override {
-    return TypedValue(static_cast<const AggregationStateCount&>(state).count_.load(std::memory_order_relaxed));
+    return TypedValue(
+        static_cast<const AggregationStateCount &>(state).count_.load(
+            std::memory_order_relaxed));
   }
 
-  inline TypedValue finalizeHashTableEntry(const AggregationState &state) const {
-    return TypedValue(static_cast<const AggregationStateCount&>(state).count_.load(std::memory_order_relaxed));
+  inline TypedValue finalizeHashTableEntry(
+      const AggregationState &state) const {
+    return TypedValue(
+        static_cast<const AggregationStateCount &>(state).count_.load(
+            std::memory_order_relaxed));
   }
 
-  inline TypedValue finalizeHashTableEntryFast(const uint8_t *byte_ptr) const {
-//    const AggregationStateAvg &agg_state = static_cast<const AggregationStateAvg&>(state);
-    // TODO(chasseur): Could improve performance further if we made a special
-    // version of finalizeHashTable() that collects all the sums into one
-    // ColumnVector and all the counts into another and then applies
-    // '*divide_operator_' to them in bulk.
-
-    const std::int64_t *count_ptr = reinterpret_cast<const std::int64_t *>(byte_ptr);
+  inline TypedValue finalizeHashTableEntryFast(
+      const std::uint8_t *byte_ptr) const {
+    const std::int64_t *count_ptr =
+        reinterpret_cast<const std::int64_t *>(byte_ptr);
     return TypedValue(*count_ptr);
   }
 
@@ -205,24 +207,25 @@ class AggregationHandleCount : public AggregationConcreteHandle {
       int index) const override;
 
   /**
-   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
+   * @brief Implementation of
+   * AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
    *        for SUM aggregation.
    */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table) const override;
+      const AggregationStateHashTableBase &distinctify_hash_table)
+      const override;
 
   /**
-   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
+   * @brief Implementation of
+   * AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
    *        for SUM aggregation.
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
       AggregationStateHashTableBase *aggregation_hash_table,
-      int index) const override;
+      std::size_t index) const override;
 
-  size_t getPayloadSize() const override {
-      return sizeof(std::int64_t);
-  }
+  std::size_t getPayloadSize() const override { return sizeof(std::int64_t); }
 
  private:
   friend class AggregateFunctionCount;
@@ -230,10 +233,9 @@ class AggregationHandleCount : public AggregationConcreteHandle {
   /**
    * @brief Constructor.
    **/
-  AggregationHandleCount() : block_update(false) {
-  }
+  AggregationHandleCount() : block_update_(false) {}
 
-  bool block_update;
+  bool block_update_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleCount);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleDistinct.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.hpp b/expressions/aggregation/AggregationHandleDistinct.hpp
index 978319b..838bfdd 100644
--- a/expressions/aggregation/AggregationHandleDistinct.hpp
+++ b/expressions/aggregation/AggregationHandleDistinct.hpp
@@ -49,27 +49,32 @@ class AggregationHandleDistinct : public AggregationConcreteHandle {
   /**
    * @brief Constructor.
    **/
-  AggregationHandleDistinct() {
-  }
+  AggregationHandleDistinct() {}
 
   AggregationState* createInitialState() const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support createInitialState().";
+    LOG(FATAL)
+        << "AggregationHandleDistinct does not support createInitialState().";
   }
 
-  AggregationState* accumulateNullary(const std::size_t num_tuples) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support accumulateNullary().";
+  AggregationState* accumulateNullary(
+      const std::size_t num_tuples) const override {
+    LOG(FATAL)
+        << "AggregationHandleDistinct does not support accumulateNullary().";
   }
 
   AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support accumulateColumnVectors().";
+      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
+      const override {
+    LOG(FATAL) << "AggregationHandleDistinct does not support "
+                  "accumulateColumnVectors().";
   }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   AggregationState* accumulateValueAccessor(
       ValueAccessor *accessor,
       const std::vector<attribute_id> &accessor_ids) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support accumulateValueAccessor().";
+    LOG(FATAL) << "AggregationHandleDistinct does not support "
+                  "accumulateValueAccessor().";
   }
 #endif
 
@@ -83,7 +88,8 @@ class AggregationHandleDistinct : public AggregationConcreteHandle {
   }
 
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table) const override {
+      const AggregationStateHashTableBase &distinctify_hash_table)
+      const override {
     LOG(FATAL) << "AggregationHandleDistinct does not support "
                << "aggregateOnDistinctifyHashTableForSingle().";
   }
@@ -91,14 +97,14 @@ class AggregationHandleDistinct : public AggregationConcreteHandle {
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
       AggregationStateHashTableBase *groupby_hash_table,
-      int index) const override {
+      std::size_t index) const override {
     LOG(FATAL) << "AggregationHandleDistinct does not support "
                << "aggregateOnDistinctifyHashTableForGroupBy().";
   }
 
   AggregationStateHashTableBase* createGroupByHashTable(
       const HashTableImplType hash_table_impl,
-      const std::vector<const Type*> &group_by_types,
+      const std::vector<const Type *> &group_by_types,
       const std::size_t estimated_num_groups,
       StorageManager *storage_manager) const override;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.cpp b/expressions/aggregation/AggregationHandleMax.cpp
index ec3e671..c2d571b 100644
--- a/expressions/aggregation/AggregationHandleMax.cpp
+++ b/expressions/aggregation/AggregationHandleMax.cpp
@@ -39,22 +39,19 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleMax::AggregationHandleMax(const Type &type)
-    : type_(type), block_update(false) {
-  fast_comparator_.reset(ComparisonFactory::GetComparison(ComparisonID::kGreater)
-                         .makeUncheckedComparatorForTypes(type,
-                                                          type.getNonNullableVersion()));
+    : type_(type), block_update_(false) {
+  fast_comparator_.reset(
+      ComparisonFactory::GetComparison(ComparisonID::kGreater)
+          .makeUncheckedComparatorForTypes(type, type.getNonNullableVersion()));
 }
 
 AggregationStateHashTableBase* AggregationHandleMax::createGroupByHashTable(
     const HashTableImplType hash_table_impl,
-    const std::vector<const Type*> &group_by_types,
+    const std::vector<const Type *> &group_by_types,
     const std::size_t estimated_num_groups,
     StorageManager *storage_manager) const {
   return AggregationStateHashTableFactory<AggregationStateMax>::CreateResizable(
-      hash_table_impl,
-      group_by_types,
-      estimated_num_groups,
-      storage_manager);
+      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
 }
 
 AggregationState* AggregationHandleMax::accumulateColumnVectors(
@@ -62,9 +59,8 @@ AggregationState* AggregationHandleMax::accumulateColumnVectors(
   DCHECK_EQ(1u, column_vectors.size())
       << "Got wrong number of ColumnVectors for MAX: " << column_vectors.size();
 
-  return new AggregationStateMax(
-      fast_comparator_->accumulateColumnVector(type_.getNullableVersion().makeNullValue(),
-                                               *column_vectors.front()));
+  return new AggregationStateMax(fast_comparator_->accumulateColumnVector(
+      type_.getNullableVersion().makeNullValue(), *column_vectors.front()));
 }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -74,10 +70,10 @@ AggregationState* AggregationHandleMax::accumulateValueAccessor(
   DCHECK_EQ(1u, accessor_ids.size())
       << "Got wrong number of attributes for MAX: " << accessor_ids.size();
 
-  return new AggregationStateMax(
-      fast_comparator_->accumulateValueAccessor(type_.getNullableVersion().makeNullValue(),
-                                                accessor,
-                                                accessor_ids.front()));
+  return new AggregationStateMax(fast_comparator_->accumulateValueAccessor(
+      type_.getNullableVersion().makeNullValue(),
+      accessor,
+      accessor_ids.front()));
 }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
@@ -90,24 +86,24 @@ void AggregationHandleMax::aggregateValueAccessorIntoHashTable(
       << "Got wrong number of arguments for MAX: " << argument_ids.size();
 }
 
-void AggregationHandleMax::mergeStates(
-    const AggregationState &source,
-    AggregationState *destination) const {
-  const AggregationStateMax &max_source = static_cast<const AggregationStateMax&>(source);
-  AggregationStateMax *max_destination = static_cast<AggregationStateMax*>(destination);
+void AggregationHandleMax::mergeStates(const AggregationState &source,
+                                       AggregationState *destination) const {
+  const AggregationStateMax &max_source =
+      static_cast<const AggregationStateMax &>(source);
+  AggregationStateMax *max_destination =
+      static_cast<AggregationStateMax *>(destination);
 
   if (!max_source.max_.isNull()) {
     compareAndUpdate(max_destination, max_source.max_);
   }
 }
 
-void AggregationHandleMax::mergeStatesFast(
-    const std::uint8_t *source,
-    std::uint8_t *destination) const {
-    const TypedValue *src_max_ptr = reinterpret_cast<const TypedValue *>(source);
-    TypedValue *dst_max_ptr = reinterpret_cast<TypedValue *>(destination);
-    if (!(src_max_ptr->isNull())) {
-      compareAndUpdateFast(dst_max_ptr, *src_max_ptr);
+void AggregationHandleMax::mergeStatesFast(const std::uint8_t *source,
+                                           std::uint8_t *destination) const {
+  const TypedValue *src_max_ptr = reinterpret_cast<const TypedValue *>(source);
+  TypedValue *dst_max_ptr = reinterpret_cast<TypedValue *>(destination);
+  if (!(src_max_ptr->isNull())) {
+    compareAndUpdateFast(dst_max_ptr, *src_max_ptr);
   }
 }
 
@@ -116,31 +112,26 @@ ColumnVector* AggregationHandleMax::finalizeHashTable(
     std::vector<std::vector<TypedValue>> *group_by_keys,
     int index) const {
   return finalizeHashTableHelperFast<AggregationHandleMax,
-                                 AggregationStateFastHashTable>(
-      type_.getNullableVersion(),
-      hash_table,
-      group_by_keys,
-      index);
+                                     AggregationStateFastHashTable>(
+      type_.getNullableVersion(), hash_table, group_by_keys, index);
 }
 
-AggregationState* AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
+AggregationState*
+AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
   return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
       AggregationHandleMax,
-      AggregationStateMax>(
-          distinctify_hash_table);
+      AggregationStateMax>(distinctify_hash_table);
 }
 
 void AggregationHandleMax::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
     AggregationStateHashTableBase *aggregation_hash_table,
-    int index) const {
+    std::size_t index) const {
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleMax,
       AggregationStateFastHashTable>(
-          distinctify_hash_table,
-          aggregation_hash_table,
-          index);
+      distinctify_hash_table, aggregation_hash_table, index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index c1a3a36..de173c9 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -28,8 +28,8 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/HashTableBase.hpp"
 #include "storage/FastHashTable.hpp"
+#include "storage/HashTableBase.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -56,25 +56,24 @@ class AggregationStateMax : public AggregationState {
   /**
    * @brief Copy constructor (ignores mutex).
    */
-  AggregationStateMax(const AggregationStateMax &orig)
-      : max_(orig.max_) {
-  }
+  AggregationStateMax(const AggregationStateMax &orig) : max_(orig.max_) {}
 
   /**
    * @brief Destructor.
    */
-  ~AggregationStateMax() override {};
+  ~AggregationStateMax() override{};
+
+  const std::uint8_t* getPayloadAddress() const {
+    return reinterpret_cast<const uint8_t *>(&max_);
+  }
 
  private:
   friend class AggregationHandleMax;
 
   explicit AggregationStateMax(const Type &type)
-      : max_(type.getNullableVersion().makeNullValue()) {
-  }
+      : max_(type.getNullableVersion().makeNullValue()) {}
 
-  explicit AggregationStateMax(TypedValue &&value)
-      : max_(std::move(value)) {
-  }
+  explicit AggregationStateMax(TypedValue &&value) : max_(std::move(value)) {}
 
   TypedValue max_;
   SpinMutex mutex_;
@@ -85,8 +84,7 @@ class AggregationStateMax : public AggregationState {
  **/
 class AggregationHandleMax : public AggregationConcreteHandle {
  public:
-  ~AggregationHandleMax() override {
-  }
+  ~AggregationHandleMax() override {}
 
   AggregationState* createInitialState() const override {
     return new AggregationStateMax(type_);
@@ -94,45 +92,46 @@ class AggregationHandleMax : public AggregationConcreteHandle {
 
   AggregationStateHashTableBase* createGroupByHashTable(
       const HashTableImplType hash_table_impl,
-      const std::vector<const Type*> &group_by_types,
+      const std::vector<const Type *> &group_by_types,
       const std::size_t estimated_num_groups,
       StorageManager *storage_manager) const override;
 
   /**
    * @brief Iterate with max aggregation state.
    */
-  inline void iterateUnaryInl(AggregationStateMax *state, const TypedValue &value) const {
+  inline void iterateUnaryInl(AggregationStateMax *state,
+                              const TypedValue &value) const {
     DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
-    compareAndUpdate(static_cast<AggregationStateMax*>(state), value);
+    compareAndUpdate(static_cast<AggregationStateMax *>(state), value);
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value, std::uint8_t *byte_ptr) const {
+  inline void iterateUnaryInlFast(const TypedValue &value,
+                                  std::uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
     TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
     compareAndUpdateFast(max_ptr, value);
   }
 
-  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const override {
-    if (block_update) return;
-    iterateUnaryInlFast(arguments.front(), byte_ptr);
+  inline void updateState(const std::vector<TypedValue> &arguments,
+                          std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateUnaryInlFast(arguments.front(), byte_ptr);
+    }
   }
 
-  void BlockUpdate() override {
-      block_update = true;
-  }
+  void blockUpdate() override { block_update_ = true; }
 
-  void AllowUpdate() override {
-      block_update = false;
-  }
+  void allowUpdate() override { block_update_ = false; }
 
-  void initPayload(uint8_t *byte_ptr) const override {
+  void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
     TypedValue t1 = (type_.getNullableVersion().makeNullValue());
     *max_ptr = t1;
   }
 
   AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
+      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
+      const override;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   AggregationState* accumulateValueAccessor(
@@ -150,17 +149,19 @@ class AggregationHandleMax : public AggregationConcreteHandle {
                    AggregationState *destination) const override;
 
   void mergeStatesFast(const std::uint8_t *source,
-                   std::uint8_t *destination) const override;
+                       std::uint8_t *destination) const override;
 
   TypedValue finalize(const AggregationState &state) const override {
-    return TypedValue(static_cast<const AggregationStateMax&>(state).max_);
+    return TypedValue(static_cast<const AggregationStateMax &>(state).max_);
   }
 
-  inline TypedValue finalizeHashTableEntry(const AggregationState &state) const {
-    return TypedValue(static_cast<const AggregationStateMax&>(state).max_);
+  inline TypedValue finalizeHashTableEntry(
+      const AggregationState &state) const {
+    return TypedValue(static_cast<const AggregationStateMax &>(state).max_);
   }
 
-  inline TypedValue finalizeHashTableEntryFast(const std::uint8_t *byte_ptr) const {
+  inline TypedValue finalizeHashTableEntryFast(
+      const std::uint8_t *byte_ptr) const {
     const TypedValue *max_ptr = reinterpret_cast<const TypedValue *>(byte_ptr);
     return TypedValue(*max_ptr);
   }
@@ -171,25 +172,25 @@ class AggregationHandleMax : public AggregationConcreteHandle {
       int index) const override;
 
   /**
-   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
+   * @brief Implementation of
+   * AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
    *        for MAX aggregation.
    */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table) const override;
-
+      const AggregationStateHashTableBase &distinctify_hash_table)
+      const override;
 
   /**
-   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
+   * @brief Implementation of
+   * AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
    *        for MAX aggregation.
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
       AggregationStateHashTableBase *aggregation_hash_table,
-      int index) const override;
+      std::size_t index) const override;
 
-  size_t getPayloadSize() const override {
-      return sizeof(TypedValue);
-  }
+  std::size_t getPayloadSize() const override { return sizeof(TypedValue); }
 
  private:
   friend class AggregateFunctionMax;
@@ -202,24 +203,29 @@ class AggregationHandleMax : public AggregationConcreteHandle {
   explicit AggregationHandleMax(const Type &type);
 
   /**
-   * @brief compare the value with max_ and update it if the value is larger than
+   * @brief compare the value with max_ and update it if the value is larger
+   *than
    *        current maximum. NULLs are ignored.
    *
    * @param value A TypedValue to compare
    **/
-  inline void compareAndUpdate(AggregationStateMax *state, const TypedValue &value) const {
+  inline void compareAndUpdate(AggregationStateMax *state,
+                               const TypedValue &value) const {
     // TODO(chasseur): Avoid null-checks when aggregating a non-nullable Type.
     if (value.isNull()) return;
 
     SpinMutexLock lock(state->mutex_);
-    if (state->max_.isNull() || fast_comparator_->compareTypedValues(value, state->max_)) {
+    if (state->max_.isNull() ||
+        fast_comparator_->compareTypedValues(value, state->max_)) {
       state->max_ = value;
     }
   }
 
-  inline void compareAndUpdateFast(TypedValue *max_ptr, const TypedValue &value) const {
+  inline void compareAndUpdateFast(TypedValue *max_ptr,
+                                   const TypedValue &value) const {
     if (value.isNull()) return;
-    if (max_ptr->isNull() || fast_comparator_->compareTypedValues(value, *max_ptr)) {
+    if (max_ptr->isNull() ||
+        fast_comparator_->compareTypedValues(value, *max_ptr)) {
       *max_ptr = value;
     }
   }
@@ -227,7 +233,7 @@ class AggregationHandleMax : public AggregationConcreteHandle {
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 
-  bool block_update;
+  bool block_update_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMax);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.cpp b/expressions/aggregation/AggregationHandleMin.cpp
index cc714a5..a07f299 100644
--- a/expressions/aggregation/AggregationHandleMin.cpp
+++ b/expressions/aggregation/AggregationHandleMin.cpp
@@ -39,22 +39,19 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleMin::AggregationHandleMin(const Type &type)
-    : type_(type), block_update(false) {
-  fast_comparator_.reset(ComparisonFactory::GetComparison(ComparisonID::kLess)
-                         .makeUncheckedComparatorForTypes(type,
-                                                          type.getNonNullableVersion()));
+    : type_(type), block_update_(false) {
+  fast_comparator_.reset(
+      ComparisonFactory::GetComparison(ComparisonID::kLess)
+          .makeUncheckedComparatorForTypes(type, type.getNonNullableVersion()));
 }
 
 AggregationStateHashTableBase* AggregationHandleMin::createGroupByHashTable(
     const HashTableImplType hash_table_impl,
-    const std::vector<const Type*> &group_by_types,
+    const std::vector<const Type *> &group_by_types,
     const std::size_t estimated_num_groups,
     StorageManager *storage_manager) const {
   return AggregationStateHashTableFactory<AggregationStateMin>::CreateResizable(
-      hash_table_impl,
-      group_by_types,
-      estimated_num_groups,
-      storage_manager);
+      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
 }
 
 AggregationState* AggregationHandleMin::accumulateColumnVectors(
@@ -62,9 +59,8 @@ AggregationState* AggregationHandleMin::accumulateColumnVectors(
   DCHECK_EQ(1u, column_vectors.size())
       << "Got wrong number of ColumnVectors for MIN: " << column_vectors.size();
 
-  return new AggregationStateMin(
-      fast_comparator_->accumulateColumnVector(type_.getNullableVersion().makeNullValue(),
-                                               *column_vectors.front()));
+  return new AggregationStateMin(fast_comparator_->accumulateColumnVector(
+      type_.getNullableVersion().makeNullValue(), *column_vectors.front()));
 }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -74,10 +70,10 @@ AggregationState* AggregationHandleMin::accumulateValueAccessor(
   DCHECK_EQ(1u, accessor_ids.size())
       << "Got wrong number of attributes for MIN: " << accessor_ids.size();
 
-  return new AggregationStateMin(
-      fast_comparator_->accumulateValueAccessor(type_.getNullableVersion().makeNullValue(),
-                                                accessor,
-                                                accessor_ids.front()));
+  return new AggregationStateMin(fast_comparator_->accumulateValueAccessor(
+      type_.getNullableVersion().makeNullValue(),
+      accessor,
+      accessor_ids.front()));
 }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
@@ -90,26 +86,26 @@ void AggregationHandleMin::aggregateValueAccessorIntoHashTable(
       << "Got wrong number of arguments for MIN: " << argument_ids.size();
 }
 
-void AggregationHandleMin::mergeStates(
-    const AggregationState &source,
-    AggregationState *destination) const {
-  const AggregationStateMin &min_source = static_cast<const AggregationStateMin&>(source);
-  AggregationStateMin *min_destination = static_cast<AggregationStateMin*>(destination);
+void AggregationHandleMin::mergeStates(const AggregationState &source,
+                                       AggregationState *destination) const {
+  const AggregationStateMin &min_source =
+      static_cast<const AggregationStateMin &>(source);
+  AggregationStateMin *min_destination =
+      static_cast<AggregationStateMin *>(destination);
 
   if (!min_source.min_.isNull()) {
     compareAndUpdate(min_destination, min_source.min_);
   }
 }
 
-void AggregationHandleMin::mergeStatesFast(
-    const std::uint8_t *source,
-    std::uint8_t *destination) const {
-    const TypedValue *src_min_ptr = reinterpret_cast<const TypedValue *>(source);
-    TypedValue *dst_min_ptr = reinterpret_cast<TypedValue *>(destination);
+void AggregationHandleMin::mergeStatesFast(const std::uint8_t *source,
+                                           std::uint8_t *destination) const {
+  const TypedValue *src_min_ptr = reinterpret_cast<const TypedValue *>(source);
+  TypedValue *dst_min_ptr = reinterpret_cast<TypedValue *>(destination);
 
-    if (!(src_min_ptr->isNull())) {
-      compareAndUpdateFast(dst_min_ptr, *src_min_ptr);
-    }
+  if (!(src_min_ptr->isNull())) {
+    compareAndUpdateFast(dst_min_ptr, *src_min_ptr);
+  }
 }
 
 ColumnVector* AggregationHandleMin::finalizeHashTable(
@@ -117,31 +113,26 @@ ColumnVector* AggregationHandleMin::finalizeHashTable(
     std::vector<std::vector<TypedValue>> *group_by_keys,
     int index) const {
   return finalizeHashTableHelperFast<AggregationHandleMin,
-                                 AggregationStateFastHashTable>(
-      type_.getNonNullableVersion(),
-      hash_table,
-      group_by_keys,
-      index);
+                                     AggregationStateFastHashTable>(
+      type_.getNonNullableVersion(), hash_table, group_by_keys, index);
 }
 
-AggregationState* AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
+AggregationState*
+AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
   return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
       AggregationHandleMin,
-      AggregationStateMin>(
-          distinctify_hash_table);
+      AggregationStateMin>(distinctify_hash_table);
 }
 
 void AggregationHandleMin::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
     AggregationStateHashTableBase *aggregation_hash_table,
-    int index) const {
+    std::size_t index) const {
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleMin,
       AggregationStateFastHashTable>(
-          distinctify_hash_table,
-          aggregation_hash_table,
-          index);
+      distinctify_hash_table, aggregation_hash_table, index);
 }
 
 }  // namespace quickstep



[25/28] incubator-quickstep git commit: Created method for partition aware finalize aggregate.

Posted by hb...@apache.org.
Created method for partition aware finalize aggregate.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/157ce784
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/157ce784
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/157ce784

Branch: refs/heads/partitioned-aggregation
Commit: 157ce784123dc514c4a8f4894cb1e6ad3570f98f
Parents: d28f99a
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Thu Aug 18 16:54:38 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:57:06 2016 -0500

----------------------------------------------------------------------
 storage/AggregationOperationState.cpp | 69 +++++++++++++++++++++++++++++-
 storage/AggregationOperationState.hpp | 33 +++++++++++++-
 2 files changed, 98 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/157ce784/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 7f63004..faf2c0b 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -565,8 +565,14 @@ void AggregationOperationState::finalizeHashTable(
     }
     AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
     DCHECK(agg_hash_table != nullptr);
-    ColumnVector *agg_result_col = handles_[agg_idx]->finalizeHashTable(
-        *agg_hash_table, &group_by_keys, agg_idx);
+    // TODO(harshad) - Modify the finalizeHashTable() function called below such
+    // that group_by_keys is a single ColumnVectorValueAccessor in which there
+    // is one ColumnVector per group by key. If we do that, the code below
+    // for reorganizing group_by_keys can be removed.
+    ColumnVector* agg_result_col =
+        handles_[agg_idx]->finalizeHashTable(*agg_hash_table,
+                                             &group_by_keys,
+                                              agg_idx);
     if (agg_result_col != nullptr) {
       final_values.emplace_back(agg_result_col);
     }
@@ -617,4 +623,63 @@ void AggregationOperationState::finalizeHashTable(
   output_destination->bulkInsertTuples(&complete_result);
 }
 
+void AggregationOperationState::finalizeAggregatePartitioned(
+    const std::size_t partition_id, InsertDestination *output_destination) {
+  // Each element of 'group_by_keys' is a vector of values for a particular
+  // group (which is also the prefix of the finalized Tuple for that group).
+  std::vector<std::vector<TypedValue>> group_by_keys;
+
+  // Collect per-aggregate finalized values.
+  std::vector<std::unique_ptr<ColumnVector>> final_values;
+  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
+    AggregationStateHashTableBase *hash_table =
+        partitioned_group_by_hashtable_pool_->getHashTable(partition_id);
+    ColumnVector *agg_result_col = handles_[agg_idx]->finalizeHashTable(
+        *hash_table, &group_by_keys, agg_idx);
+    if (agg_result_col != nullptr) {
+      final_values.emplace_back(agg_result_col);
+    }
+  }
+
+  // Reorganize 'group_by_keys' in column-major order so that we can make a
+  // ColumnVectorsValueAccessor to bulk-insert results.
+  //
+  // TODO(chasseur): Shuffling around the GROUP BY keys like this is suboptimal
+  // if there is only one aggregate. The need to do this should hopefully go
+  // away when we work out storing composite structures for multiple aggregates
+  // in a single HashTable.
+  std::vector<std::unique_ptr<ColumnVector>> group_by_cvs;
+  std::size_t group_by_element_idx = 0;
+  for (const std::unique_ptr<const Scalar> &group_by_element : group_by_list_) {
+    const Type &group_by_type = group_by_element->getType();
+    if (NativeColumnVector::UsableForType(group_by_type)) {
+      NativeColumnVector *element_cv = new NativeColumnVector(group_by_type, group_by_keys.size());
+      group_by_cvs.emplace_back(element_cv);
+      for (std::vector<TypedValue> &group_key : group_by_keys) {
+        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
+      }
+    } else {
+      IndirectColumnVector *element_cv = new IndirectColumnVector(group_by_type, group_by_keys.size());
+      group_by_cvs.emplace_back(element_cv);
+      for (std::vector<TypedValue> &group_key : group_by_keys) {
+        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
+      }
+    }
+    ++group_by_element_idx;
+  }
+
+  // Stitch together a ColumnVectorsValueAccessor combining the GROUP BY keys
+  // and the finalized aggregates.
+  ColumnVectorsValueAccessor complete_result;
+  for (std::unique_ptr<ColumnVector> &group_by_cv : group_by_cvs) {
+    complete_result.addColumn(group_by_cv.release());
+  }
+  for (std::unique_ptr<ColumnVector> &final_value_cv : final_values) {
+    complete_result.addColumn(final_value_cv.release());
+  }
+
+  // Bulk-insert the complete result.
+  output_destination->bulkInsertTuples(&complete_result);
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/157ce784/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 384d943..880375f 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -168,8 +168,37 @@ class AggregationOperationState {
    **/
   void finalizeAggregate(InsertDestination *output_destination);
 
-  static void mergeGroupByHashTables(AggregationStateHashTableBase *src,
-                                     AggregationStateHashTableBase *dst);
+  /**
+   * @brief Generate the final results for the aggregates managed by this
+   *        AggregationOperationState, for the given partition and write them
+   *        out to StorageBlock(s).
+   *
+   * @param partition_id The Partition ID for which the finalize has to be
+   *        performed.
+   * @param output_destination An InsertDestination where the finalized output
+   *        tuple(s) from this aggregate are to be written.
+   **/
+  void finalizeAggregatePartitioned(const std::size_t partition_id,
+                                    InsertDestination *output_destination);
+
+  bool isAggregatePartitioned() const {
+    return is_aggregate_partitioned_;
+  }
+
+  /**
+   * @brief Get the number of partitions used for the aggregation.
+   *
+   * @note This is relevant only when is_aggregate_partitioned_ is true.
+   *
+   * @return The number of partitions used for the aggregation. Default is 1.
+   **/
+  std::size_t getNumPartitions() const {
+    if (is_aggregate_partitioned_) {
+      return partitioned_group_by_hashtable_pool_->getNumPartitions();
+    } else {
+      return 1;
+    }
+  }
 
   int dflag;
 


[21/28] incubator-quickstep git commit: Added function to aggregate group by with partition

Posted by hb...@apache.org.
Added function to aggregate group by with partition

- Added a function to compute a composite hash function for multiple
  attributes.
- Changes to Tuple class to support partitioning.
- A function in StorageBlock to compute aggregate with group by that
  supports partitioning.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/d28f99a7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/d28f99a7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/d28f99a7

Branch: refs/heads/partitioned-aggregation
Commit: d28f99a719a8c767197fec6113c96f1017d1a034
Parents: 8f3e1d0
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Thu Aug 18 12:30:18 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:57:06 2016 -0500

----------------------------------------------------------------------
 storage/AggregationOperationState.cpp |  68 +++++++++++------
 storage/AggregationOperationState.hpp |   7 ++
 storage/PartitionedHashTablePool.hpp  |   4 +
 storage/StorageBlock.cpp              | 113 +++++++++++++++++++++++++++++
 storage/StorageBlock.hpp              |   9 +++
 types/containers/CMakeLists.txt       |   1 +
 types/containers/Tuple.hpp            |   8 ++
 utility/CMakeLists.txt                |   6 ++
 utility/CompositeHash.hpp             |  52 +++++++++++++
 9 files changed, 245 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d28f99a7/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 073b813..7f63004 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -68,7 +68,8 @@ AggregationOperationState::AggregationOperationState(
     const HashTableImplType hash_table_impl_type,
     const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
     StorageManager *storage_manager)
-    : input_relation_(input_relation),
+    : is_aggregate_partitioned_(estimated_num_entries > kPartitionedAggregateThreshold),
+      input_relation_(input_relation),
       predicate_(predicate),
       group_by_list_(std::move(group_by)),
       arguments_(std::move(arguments)),
@@ -193,14 +194,26 @@ AggregationOperationState::AggregationOperationState(
     }
 
     if (!group_by_handles.empty()) {
-      // Aggregation with GROUP BY: create a HashTable pool for per-group
-      // states.
-      group_by_hashtable_pool_.reset(new HashTablePool(estimated_num_entries,
-                                                       hash_table_impl_type,
-                                                       group_by_types,
-                                                       payload_sizes,
-                                                       group_by_handles,
-                                                       storage_manager));
+      // Aggregation with GROUP BY: create a HashTable pool for per-group states.
+      if (!is_aggregate_partitioned_) {
+        group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
+              new HashTablePool(estimated_num_entries,
+                                hash_table_impl_type,
+                                group_by_types,
+                                payload_sizes,
+                                group_by_handles,
+                                storage_manager)));
+        }
+      else {
+        partitioned_group_by_hashtable_pool_.reset(
+            new PartitionedHashTablePool(estimated_num_entries,
+                                         kNumPartitionsForAggregate,
+                                         hash_table_impl_type,
+                                         group_by_types,
+                                         payload_sizes,
+                                         group_by_handles,
+                                         storage_manager));
+      }
     }
   }
 }
@@ -439,20 +452,29 @@ void AggregationOperationState::aggregateBlockHashTable(
     }
   }
 
-  // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
-  // directly into the (threadsafe) shared global HashTable for this
-  // aggregate.
-  DCHECK(group_by_hashtable_pool_ != nullptr);
-  AggregationStateHashTableBase *agg_hash_table =
-      group_by_hashtable_pool_->getHashTableFast();
-  DCHECK(agg_hash_table != nullptr);
-  block->aggregateGroupBy(arguments_,
-                          group_by_list_,
-                          predicate_.get(),
-                          agg_hash_table,
-                          &reuse_matches,
-                          &reuse_group_by_vectors);
-  group_by_hashtable_pool_->returnHashTable(agg_hash_table);
+  if (!is_aggregate_partitioned_) {
+    // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
+    // directly into the (threadsafe) shared global HashTable for this
+    // aggregate.
+    DCHECK(group_by_hashtable_pools_[0] != nullptr);
+    AggregationStateHashTableBase *agg_hash_table = group_by_hashtable_pools_[0]->getHashTableFast();
+    DCHECK(agg_hash_table != nullptr);
+    block->aggregateGroupByFast(arguments_,
+                                group_by_list_,
+                                predicate_.get(),
+                                agg_hash_table,
+                                &reuse_matches,
+                                &reuse_group_by_vectors);
+    group_by_hashtable_pools_[0]->returnHashTable(agg_hash_table);
+  } else {
+    block->aggregateGroupByPartitioned(
+        arguments_,
+        group_by_list_,
+        predicate_.get(),
+        &reuse_matches,
+        &reuse_group_by_vectors,
+        partitioned_group_by_hashtable_pool_.get());
+  }
 }
 
 void AggregationOperationState::finalizeSingleState(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d28f99a7/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 805a347..384d943 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -174,6 +174,11 @@ class AggregationOperationState {
   int dflag;
 
  private:
+  static constexpr std::size_t kPartitionedAggregateThreshold = 100;
+  static constexpr std::size_t kNumPartitionsForAggregate = 40;
+
+  const bool is_aggregate_partitioned_;
+
   // Merge locally (per storage block) aggregated states with global aggregation
   // states.
   void mergeSingleState(
@@ -225,6 +230,8 @@ class AggregationOperationState {
   // A vector of group by hash table pools.
   std::unique_ptr<HashTablePool> group_by_hashtable_pool_;
 
+  std::unique_ptr<PartitionedHashTablePool> partitioned_group_by_hashtable_pool_;
+
   StorageManager *storage_manager_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationOperationState);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d28f99a7/storage/PartitionedHashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/PartitionedHashTablePool.hpp b/storage/PartitionedHashTablePool.hpp
index a71af44..7f58fa9 100644
--- a/storage/PartitionedHashTablePool.hpp
+++ b/storage/PartitionedHashTablePool.hpp
@@ -143,6 +143,10 @@ class PartitionedHashTablePool {
     return &hash_tables_;
   }
 
+  inline std::size_t getNumPartitions() const {
+    return num_partitions_;
+  }
+
  private:
   void initializeAllHashTables() {
     for (std::size_t part_num = 0; part_num < num_partitions_; ++part_num) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d28f99a7/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index ec5990f..bb43630 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -41,6 +41,7 @@
 #include "storage/IndexSubBlock.hpp"
 #include "storage/InsertDestinationInterface.hpp"
 #include "storage/PackedRowStoreTupleStorageSubBlock.hpp"
+#include "storage/PartitionedHashTablePool.hpp"
 #include "storage/SMAIndexSubBlock.hpp"
 #include "storage/SplitRowStoreTupleStorageSubBlock.hpp"
 #include "storage/StorageBlockBase.hpp"
@@ -1369,4 +1370,116 @@ const std::size_t StorageBlock::getNumTuples() const {
   return tuple_store_->numTuples();
 }
 
+void StorageBlock::aggregateGroupByPartitioned(
+    const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
+    const std::vector<std::unique_ptr<const Scalar>> &group_by,
+    const Predicate *predicate,
+    std::unique_ptr<TupleIdSequence> *reuse_matches,
+    std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors,
+    PartitionedHashTablePool *hashtable_pool) const {
+  DCHECK_EQ(group_by.size(), 0u)
+      << "Called aggregateGroupByPartitioned() with zero GROUP BY expressions";
+
+  SubBlocksReference sub_blocks_ref(*tuple_store_,
+                                    indices_,
+                                    indices_consistent_);
+
+  // IDs of 'arguments' as attributes in the ValueAccessor we create below.
+  std::vector<attribute_id> arg_ids;
+  std::vector<std::vector<attribute_id>> argument_ids;
+
+  // IDs of GROUP BY key element(s) in the ValueAccessor we create below.
+  std::vector<attribute_id> key_ids;
+
+  // An intermediate ValueAccessor that stores the materialized 'arguments' for
+  // this aggregate, as well as the GROUP BY expression values.
+  ColumnVectorsValueAccessor temp_result;
+  std::unique_ptr<ValueAccessor> accessor;
+  if (predicate) {
+    if (!*reuse_matches) {
+      // If there is a filter predicate that hasn't already been evaluated,
+      // evaluate it now and save the results for other aggregates on this
+      // same block.
+      reuse_matches->reset(getMatchesForPredicate(predicate));
+    }
+
+    // Create a filtered ValueAccessor that only iterates over predicate
+    // matches.
+    accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));
+  } else {
+    // Create a ValueAccessor that iterates over all tuples in this block
+    accessor.reset(tuple_store_->createValueAccessor());
+  }
+
+  attribute_id attr_id = 0;
+
+  // First, put GROUP BY keys into 'temp_result'.
+  if (reuse_group_by_vectors->empty()) {
+    // Compute GROUP BY values from group_by Scalars, and store them in
+    // reuse_group_by_vectors for reuse by other aggregates on this same
+    // block.
+    reuse_group_by_vectors->reserve(group_by.size());
+    for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
+      reuse_group_by_vectors->emplace_back(
+          group_by_element->getAllValues(accessor.get(), &sub_blocks_ref));
+      temp_result.addColumn(reuse_group_by_vectors->back().get(), false);
+      key_ids.push_back(attr_id++);
+    }
+  } else {
+    // Reuse precomputed GROUP BY values from reuse_group_by_vectors.
+    DCHECK_EQ(group_by.size(), reuse_group_by_vectors->size())
+        << "Wrong number of reuse_group_by_vectors";
+    for (const std::unique_ptr<ColumnVector> &reuse_cv : *reuse_group_by_vectors) {
+      temp_result.addColumn(reuse_cv.get(), false);
+      key_ids.push_back(attr_id++);
+    }
+  }
+
+  // Compute argument vectors and add them to 'temp_result'.
+  for (const std::vector<std::unique_ptr<const Scalar>> &argument : arguments) {
+    arg_ids.clear();
+    for (const std::unique_ptr<const Scalar> &args : argument) {
+      temp_result.addColumn(args->getAllValues(accessor.get(), &sub_blocks_ref));
+      arg_ids.push_back(attr_id++);
+    }
+    argument_ids.push_back(arg_ids);
+  }
+
+  // Compute the partitions for the tuple formed by group by values.
+  std::vector<std::unique_ptr<TupleIdSequence>> partition_membership;
+  partition_membership.resize(hashtable_pool->getNumPartitions());
+
+  // Create a tuple-id sequence for each partition.
+  for (std::size_t partition = 0;
+       partition < hashtable_pool->getNumPartitions();
+       ++partition) {
+    partition_membership[partition].reset(new TupleIdSequence(temp_result.getEndPosition()));
+  }
+
+  // Iterate over ValueAccessor for each tuple,
+  // set a bit in the appropriate TupleIdSequence.
+  temp_result.beginIteration();
+  while (temp_result.next()) {
+    const std::size_t curr_tuple_partition_id =
+        temp_result.getTupleWithAttributes(key_ids)->getTupleHash() %
+        hashtable_pool->getNumPartitions();
+    partition_membership[curr_tuple_partition_id]->set(
+        temp_result.getCurrentPosition(), true);
+  }
+  // For each partition, create an adapter around Value Accessor and
+  // TupleIdSequence.
+  std::vector<std::unique_ptr<
+      TupleIdSequenceAdapterValueAccessor<ColumnVectorsValueAccessor>>> adapter;
+  adapter.resize(hashtable_pool->getNumPartitions());
+  for (std::size_t partition = 0;
+       partition < hashtable_pool->getNumPartitions();
+       ++partition) {
+    adapter[partition].reset(temp_result.createSharedTupleIdSequenceAdapter(
+        *partition_membership[partition]));
+    hashtable_pool->getHashTable(partition)
+        ->upsertValueAccessorCompositeKeyFast(
+            argument_ids, adapter[partition].get(), key_ids, true);
+  }
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d28f99a7/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 398008e..4be91c6 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -44,6 +44,7 @@ class AggregationState;
 class CatalogRelationSchema;
 class ColumnVector;
 class InsertDestinationInterface;
+class PartitionedHashTablePool;
 class Predicate;
 class Scalar;
 class StorageBlockLayout;
@@ -467,6 +468,14 @@ class StorageBlock : public StorageBlockBase {
       std::unique_ptr<TupleIdSequence> *reuse_matches,
       std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
 
+  void aggregateGroupByPartitioned(
+      const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
+      const std::vector<std::unique_ptr<const Scalar>> &group_by,
+      const Predicate *predicate,
+      std::unique_ptr<TupleIdSequence> *reuse_matches,
+      std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors,
+      PartitionedHashTablePool *hashtable_pool) const;
+
   /**
    * @brief Inserts the GROUP BY expressions and aggregation arguments together
    *        as keys into the distinctify hash table.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d28f99a7/types/containers/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/containers/CMakeLists.txt b/types/containers/CMakeLists.txt
index aacb63a..c2a6623 100644
--- a/types/containers/CMakeLists.txt
+++ b/types/containers/CMakeLists.txt
@@ -49,6 +49,7 @@ target_link_libraries(quickstep_types_containers_Tuple
                       quickstep_catalog_CatalogTypedefs
                       quickstep_types_TypedValue
                       quickstep_types_containers_Tuple_proto
+                      quickstep_utility_CompositeHash
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_types_containers_Tuple_proto
                       quickstep_types_TypedValue_proto

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d28f99a7/types/containers/Tuple.hpp
----------------------------------------------------------------------
diff --git a/types/containers/Tuple.hpp b/types/containers/Tuple.hpp
index 60f832c..6237d54 100644
--- a/types/containers/Tuple.hpp
+++ b/types/containers/Tuple.hpp
@@ -28,6 +28,7 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/Tuple.pb.h"
+#include "utility/CompositeHash.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -218,6 +219,13 @@ class Tuple {
     return attribute_values_.size();
   }
 
+  /**
+   * @brief Get the hash value of the tuple.
+   **/
+  std::size_t getTupleHash() const {
+    return HashCompositeKey(attribute_values_);
+  }
+
  private:
   /**
    * @brief Constructor which does not create any attributes, nor pre-reserve

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d28f99a7/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index ddaae45..4fb6e5b 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -167,6 +167,7 @@ add_library(quickstep_utility_BloomFilter_proto
 add_library(quickstep_utility_CalculateInstalledMemory CalculateInstalledMemory.cpp CalculateInstalledMemory.hpp)
 add_library(quickstep_utility_Cast ../empty_src.cpp Cast.hpp)
 add_library(quickstep_utility_CheckSnprintf ../empty_src.cpp CheckSnprintf.hpp)
+add_library(quickstep_utility_CompositeHash ../empty_src.cpp CompositeHash.hpp)
 add_library(quickstep_utility_DAG ../empty_src.cpp DAG.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)
 add_library(quickstep_utility_ExecutionDAGVisualizer
@@ -227,6 +228,10 @@ target_link_libraries(quickstep_utility_CalculateInstalledMemory
                       glog)
 target_link_libraries(quickstep_utility_CheckSnprintf
                       glog)
+target_link_libraries(quickstep_utility_CompositeHash
+                      quickstep_types_TypedValue
+                      quickstep_utility_HashPair
+                      glog)
 target_link_libraries(quickstep_utility_DAG
                       glog
                       quickstep_utility_Macros)
@@ -318,6 +323,7 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_CalculateInstalledMemory
                       quickstep_utility_Cast
                       quickstep_utility_CheckSnprintf
+                      quickstep_utility_CompositeHash
                       quickstep_utility_DAG
                       quickstep_utility_EqualsAnyConstant
                       quickstep_utility_ExecutionDAGVisualizer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d28f99a7/utility/CompositeHash.hpp
----------------------------------------------------------------------
diff --git a/utility/CompositeHash.hpp b/utility/CompositeHash.hpp
new file mode 100644
index 0000000..517bc96
--- /dev/null
+++ b/utility/CompositeHash.hpp
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_COMPOSITE_HASH_HPP_
+#define QUICKSTEP_UTILITY_COMPOSITE_HASH_HPP_
+
+#include <cstddef>
+#include <vector>
+
+#include "types/TypedValue.hpp"
+#include "utility/HashPair.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/**
+ * @brief Compute the hash value of a composite key.
+ *
+ * @param key A vector of TypedValues which together form the composite key.
+ * @return The hash value.
+ **/
+static std::size_t HashCompositeKey(const std::vector<TypedValue> &key) {
+  DCHECK(!key.empty());
+  std::size_t hash = key.front().getHash();
+  for (std::vector<TypedValue>::const_iterator key_it = key.begin() + 1;
+       key_it != key.end();
+       ++key_it) {
+    hash = CombineHashes(hash, key_it->getHash());
+  }
+  return hash;
+}
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_COMPOSITE_HASH_HPP_


[11/28] incubator-quickstep git commit: Removed some dead code and made minor updates.

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/storage/FastSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastSeparateChainingHashTable.hpp b/storage/FastSeparateChainingHashTable.hpp
index 49cea5b..0670993 100644
--- a/storage/FastSeparateChainingHashTable.hpp
+++ b/storage/FastSeparateChainingHashTable.hpp
@@ -120,25 +120,15 @@ class FastSeparateChainingHashTable : public FastHashTable<resizable,
                                  const std::size_t variable_key_size,
                                  const uint8_t &value,
                                  HashTablePreallocationState *prealloc_state) override;
-  HashTablePutResult putCompositeKeyInternal(const std::vector<TypedValue> &key,
-                                             const std::size_t variable_key_size,
-                                             const uint8_t &value,
-                                             HashTablePreallocationState *prealloc_state) override;
+
   HashTablePutResult putCompositeKeyInternalFast(const std::vector<TypedValue> &key,
                                              const std::size_t variable_key_size,
                                              const std::uint8_t *init_value_ptr,
                                              HashTablePreallocationState *prealloc_state) override;
 
-  uint8_t* upsertInternal(const TypedValue &key,
-                         const std::size_t variable_key_size,
-                         const uint8_t &initial_value) override;
   uint8_t* upsertInternalFast(const TypedValue &key,
-                         const std::uint8_t *init_value_ptr,
-                         const std::size_t variable_key_size) override;
-
-  uint8_t* upsertCompositeKeyInternal(const std::vector<TypedValue> &key,
-                                     const std::size_t variable_key_size,
-                                     const uint8_t &initial_value) override;
+                         const std::size_t variable_key_size,
+                         const std::uint8_t *init_value_ptr) override;
 
   uint8_t* upsertCompositeKeyInternalFast(const std::vector<TypedValue> &key,
                                      const std::uint8_t *init_value_ptr,
@@ -788,76 +778,6 @@ template <bool resizable,
           bool allow_duplicate_keys>
 HashTablePutResult
     FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-        ::putCompositeKeyInternal(const std::vector<TypedValue> &key,
-                                  const std::size_t variable_key_size,
-                                  const uint8_t &value,
-                                  HashTablePreallocationState *prealloc_state) {
-  DEBUG_ASSERT(this->key_types_.size() == key.size());
-
-  if (prealloc_state == nullptr) {
-    // Early check for a free bucket.
-    if (header_->buckets_allocated.load(std::memory_order_relaxed) >= header_->num_buckets) {
-      return HashTablePutResult::kOutOfSpace;
-    }
-
-    // TODO(chasseur): If allow_duplicate_keys is true, avoid storing more than
-    // one copy of the same variable-length key.
-    if (!key_manager_.allocateVariableLengthKeyStorage(variable_key_size)) {
-      // Ran out of variable-length key storage space.
-      return HashTablePutResult::kOutOfSpace;
-    }
-  }
-
-  const std::size_t hash_code = this->hashCompositeKey(key);
-  void *bucket = nullptr;
-  std::atomic<std::size_t> *pending_chain_ptr;
-  std::size_t pending_chain_ptr_finish_value;
-  for (;;) {
-    if (locateBucketForInsertion(hash_code,
-                                 0,
-                                 &bucket,
-                                 &pending_chain_ptr,
-                                 &pending_chain_ptr_finish_value,
-                                 prealloc_state)) {
-      // Found an empty bucket.
-      break;
-    } else if (bucket == nullptr) {
-      // Ran out of buckets. Deallocate any variable space that we were unable
-      // to use.
-      DEBUG_ASSERT(prealloc_state == nullptr);
-      key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
-      return HashTablePutResult::kOutOfSpace;
-    } else {
-      // Hash collision found, and duplicates aren't allowed.
-      DEBUG_ASSERT(!allow_duplicate_keys);
-      DEBUG_ASSERT(prealloc_state == nullptr);
-      if (key_manager_.compositeKeyCollisionCheck(key, bucket)) {
-        // Duplicate key. Deallocate any variable storage space and return.
-        key_manager_.deallocateVariableLengthKeyStorage(variable_key_size);
-        return HashTablePutResult::kDuplicateKey;
-      }
-    }
-  }
-
-  // Write the key and hash.
-  writeCompositeKeyToBucket(key, hash_code, bucket, prealloc_state);
-
-  // Store the value by using placement new with ValueT's copy constructor.
-  new(static_cast<char*>(bucket) + kValueOffset) uint8_t(value);
-
-  // Update the previous chain pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
-
-  // We're all done.
-  return HashTablePutResult::kOK;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-HashTablePutResult
-    FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
         ::putCompositeKeyInternalFast(const std::vector<TypedValue> &key,
                                   const std::size_t variable_key_size,
                                   const uint8_t *init_value_ptr,
@@ -923,76 +843,14 @@ HashTablePutResult
   return HashTablePutResult::kOK;
 }
 
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsertInternal(const TypedValue &key,
-                     const std::size_t variable_key_size,
-                     const uint8_t &initial_value) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
-
-  if (variable_key_size > 0) {
-    // Don't allocate yet, since the key may already be present. However, we
-    // do check if either the allocated variable storage space OR the free
-    // space is big enough to hold the key (at least one must be true: either
-    // the key is already present and allocated, or we need to be able to
-    // allocate enough space for it).
-    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
-    if ((allocated_bytes < variable_key_size)
-        && (allocated_bytes + variable_key_size > key_manager_.getVariableLengthKeyStorageSize())) {
-      return nullptr;
-    }
-  }
-
-  const std::size_t hash_code = key.getHash();
-  void *bucket = nullptr;
-  std::atomic<std::size_t> *pending_chain_ptr;
-  std::size_t pending_chain_ptr_finish_value;
-  for (;;) {
-    if (locateBucketForInsertion(hash_code,
-                                 variable_key_size,
-                                 &bucket,
-                                 &pending_chain_ptr,
-                                 &pending_chain_ptr_finish_value,
-                                 nullptr)) {
-      // Found an empty bucket.
-      break;
-    } else if (bucket == nullptr) {
-      // Ran out of buckets or variable-key space.
-      return nullptr;
-    } else if (key_manager_.scalarKeyCollisionCheck(key, bucket)) {
-      // Found an already-existing entry for this key.
-      return reinterpret_cast<uint8_t*>(static_cast<char*>(bucket) + kValueOffset);
-    }
-  }
-
-  // We are now writing to an empty bucket.
-  // Write the key and hash.
-  writeScalarKeyToBucket(key, hash_code, bucket, nullptr);
-
-  // Copy the supplied 'initial_value' into place.
-  uint8_t *value = new(static_cast<char*>(bucket) + kValueOffset) uint8_t(initial_value);
-
-  // Update the previous chain pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
-
-  // Return the value.
-  return value;
-}
-
 template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
     ::upsertInternalFast(const TypedValue &key,
-                     const std::uint8_t *init_value_ptr,
-                     const std::size_t variable_key_size) {
+                     const std::size_t variable_key_size,
+                     const std::uint8_t *init_value_ptr) {
   DEBUG_ASSERT(!allow_duplicate_keys);
   DEBUG_ASSERT(this->key_types_.size() == 1);
   DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
@@ -1052,67 +910,6 @@ uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy,
   return value;
 }
 
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsertCompositeKeyInternal(const std::vector<TypedValue> &key,
-                                 const std::size_t variable_key_size,
-                                 const uint8_t &initial_value) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  DEBUG_ASSERT(this->key_types_.size() == key.size());
-
-  if (variable_key_size > 0) {
-    // Don't allocate yet, since the key may already be present. However, we
-    // do check if either the allocated variable storage space OR the free
-    // space is big enough to hold the key (at least one must be true: either
-    // the key is already present and allocated, or we need to be able to
-    // allocate enough space for it).
-    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
-    if ((allocated_bytes < variable_key_size)
-        && (allocated_bytes + variable_key_size > key_manager_.getVariableLengthKeyStorageSize())) {
-      return nullptr;
-    }
-  }
-
-  const std::size_t hash_code = this->hashCompositeKey(key);
-  void *bucket = nullptr;
-  std::atomic<std::size_t> *pending_chain_ptr;
-  std::size_t pending_chain_ptr_finish_value;
-  for (;;) {
-    if (locateBucketForInsertion(hash_code,
-                                 variable_key_size,
-                                 &bucket,
-                                 &pending_chain_ptr,
-                                 &pending_chain_ptr_finish_value,
-                                 nullptr)) {
-      // Found an empty bucket.
-      break;
-    } else if (bucket == nullptr) {
-      // Ran out of buckets or variable-key space.
-      return nullptr;
-    } else if (key_manager_.compositeKeyCollisionCheck(key, bucket)) {
-      // Found an already-existing entry for this key.
-      return reinterpret_cast<uint8_t*>(static_cast<char*>(bucket) + kValueOffset);
-    }
-  }
-
-  // We are now writing to an empty bucket.
-  // Write the key and hash.
-  writeCompositeKeyToBucket(key, hash_code, bucket, nullptr);
-
-  // Copy the supplied 'initial_value' into place.
-  uint8_t *value = new(static_cast<char*>(bucket) + kValueOffset) uint8_t(initial_value);
-
-  // Update the previous chaing pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
-
-  // Return the value.
-  return value;
-}
-
 template <bool resizable,
           bool serializable,
           bool force_key_copy,


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

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/storage/FastSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastSeparateChainingHashTable.hpp b/storage/FastSeparateChainingHashTable.hpp
index 0670993..886a8ca 100644
--- a/storage/FastSeparateChainingHashTable.hpp
+++ b/storage/FastSeparateChainingHashTable.hpp
@@ -27,8 +27,8 @@
 #include <utility>
 #include <vector>
 
-#include "storage/HashTable.hpp"
 #include "storage/FastHashTable.hpp"
+#include "storage/HashTable.hpp"
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTableKeyManager.hpp"
 #include "storage/StorageBlob.hpp"
@@ -55,43 +55,42 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-class FastSeparateChainingHashTable : public FastHashTable<resizable,
-                                                   serializable,
-                                                   force_key_copy,
-                                                   allow_duplicate_keys> {
+class FastSeparateChainingHashTable
+    : public FastHashTable<resizable,
+                           serializable,
+                           force_key_copy,
+                           allow_duplicate_keys> {
  public:
-  FastSeparateChainingHashTable(const std::vector<const Type*> &key_types,
-                            const std::size_t num_entries,
-                            const std::vector<std::size_t> &payload_sizes,
-                            const std::vector<AggregationHandle *> &handles,
-                            StorageManager *storage_manager);
-
-  FastSeparateChainingHashTable(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);
+  FastSeparateChainingHashTable(const std::vector<const Type *> &key_types,
+                                const std::size_t num_entries,
+                                const std::vector<std::size_t> &payload_sizes,
+                                const std::vector<AggregationHandle *> &handles,
+                                StorageManager *storage_manager);
+
+  FastSeparateChainingHashTable(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);
 
   // Delegating constructors for single scalar keys.
   FastSeparateChainingHashTable(const Type &key_type,
-                            const std::size_t num_entries,
-                            StorageManager *storage_manager)
-      : FastSeparateChainingHashTable(std::vector<const Type*>(1, &key_type),
-                                  num_entries,
-                                  storage_manager) {
-  }
+                                const std::size_t num_entries,
+                                StorageManager *storage_manager)
+      : FastSeparateChainingHashTable(std::vector<const Type *>(1, &key_type),
+                                      num_entries,
+                                      storage_manager) {}
 
   FastSeparateChainingHashTable(const Type &key_type,
-                            void *hash_table_memory,
-                            const std::size_t hash_table_memory_size,
-                            const bool new_hash_table,
-                            const bool hash_table_memory_zeroed)
-      : FastSeparateChainingHashTable(std::vector<const Type*>(1, &key_type),
-                                  hash_table_memory,
-                                  hash_table_memory_size,
-                                  new_hash_table,
-                                  hash_table_memory_zeroed) {
-  }
+                                void *hash_table_memory,
+                                const std::size_t hash_table_memory_size,
+                                const bool new_hash_table,
+                                const bool hash_table_memory_zeroed)
+      : FastSeparateChainingHashTable(std::vector<const Type *>(1, &key_type),
+                                      hash_table_memory,
+                                      hash_table_memory_size,
+                                      new_hash_table,
+                                      hash_table_memory_zeroed) {}
 
   ~FastSeparateChainingHashTable() override {
     DestroyValues(buckets_,
@@ -106,48 +105,54 @@ class FastSeparateChainingHashTable : public FastHashTable<resizable,
     return header_->buckets_allocated.load(std::memory_order_relaxed);
   }
 
-  const uint8_t* getSingle(const TypedValue &key) const override;
-  const uint8_t* getSingleCompositeKey(const std::vector<TypedValue> &key) const override;
-  const uint8_t* getSingleCompositeKey(const std::vector<TypedValue> &key, int index) const override;
+  const std::uint8_t* getSingle(const TypedValue &key) const override;
+  const std::uint8_t* getSingleCompositeKey(
+      const std::vector<TypedValue> &key) const override;
+  const std::uint8_t* getSingleCompositeKey(const std::vector<TypedValue> &key,
+                                            int index) const override;
 
   void getAll(const TypedValue &key,
-              std::vector<const uint8_t*> *values) const override;
-  void getAllCompositeKey(const std::vector<TypedValue> &key,
-                          std::vector<const uint8_t*> *values) const override;
+              std::vector<const std::uint8_t *> *values) const override;
+  void getAllCompositeKey(
+      const std::vector<TypedValue> &key,
+      std::vector<const std::uint8_t *> *values) const override;
 
  protected:
-  HashTablePutResult putInternal(const TypedValue &key,
-                                 const std::size_t variable_key_size,
-                                 const uint8_t &value,
-                                 HashTablePreallocationState *prealloc_state) override;
-
-  HashTablePutResult putCompositeKeyInternalFast(const std::vector<TypedValue> &key,
-                                             const std::size_t variable_key_size,
-                                             const std::uint8_t *init_value_ptr,
-                                             HashTablePreallocationState *prealloc_state) override;
-
-  uint8_t* upsertInternalFast(const TypedValue &key,
-                         const std::size_t variable_key_size,
-                         const std::uint8_t *init_value_ptr) override;
-
-  uint8_t* upsertCompositeKeyInternalFast(const std::vector<TypedValue> &key,
-                                     const std::uint8_t *init_value_ptr,
-                                     const std::size_t variable_key_size) override;
+  HashTablePutResult putInternal(
+      const TypedValue &key,
+      const std::size_t variable_key_size,
+      const std::uint8_t &value,
+      HashTablePreallocationState *prealloc_state) override;
+
+  HashTablePutResult putCompositeKeyInternalFast(
+      const std::vector<TypedValue> &key,
+      const std::size_t variable_key_size,
+      const std::uint8_t *init_value_ptr,
+      HashTablePreallocationState *prealloc_state) override;
+
+  std::uint8_t* upsertInternalFast(const TypedValue &key,
+                                   const std::size_t variable_key_size,
+                                   const std::uint8_t *init_value_ptr) override;
+
+  std::uint8_t* upsertCompositeKeyInternalFast(
+      const std::vector<TypedValue> &key,
+      const std::uint8_t *init_value_ptr,
+      const std::size_t variable_key_size) override;
 
   bool getNextEntry(TypedValue *key,
-                    const uint8_t **value,
+                    const std::uint8_t **value,
                     std::size_t *entry_num) const override;
   bool getNextEntryCompositeKey(std::vector<TypedValue> *key,
-                                const uint8_t **value,
+                                const std::uint8_t **value,
                                 std::size_t *entry_num) const override;
 
   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 override;
   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 override;
 
   bool hasKey(const TypedValue &key) const override;
@@ -157,15 +162,16 @@ class FastSeparateChainingHashTable : public FastHashTable<resizable,
               const std::size_t extra_variable_storage,
               const std::size_t retry_num = 0) override;
 
-  bool preallocateForBulkInsert(const std::size_t total_entries,
-                                const std::size_t total_variable_key_size,
-                                HashTablePreallocationState *prealloc_state) override;
+  bool preallocateForBulkInsert(
+      const std::size_t total_entries,
+      const std::size_t total_variable_key_size,
+      HashTablePreallocationState *prealloc_state) override;
+
  private:
   struct Header {
     std::size_t num_slots;
     std::size_t num_buckets;
-    alignas(kCacheLineBytes)
-        std::atomic<std::size_t> buckets_allocated;
+    alignas(kCacheLineBytes) std::atomic<std::size_t> buckets_allocated;
     alignas(kCacheLineBytes)
         std::atomic<std::size_t> variable_length_bytes_allocated;
   };
@@ -179,16 +185,18 @@ class FastSeparateChainingHashTable : public FastHashTable<resizable,
 
   // Round bucket size up to a multiple of kBucketAlignment.
   constexpr std::size_t ComputeBucketSize(const std::size_t fixed_key_size) {
-    return (((kValueOffset + this->total_payload_size_ + fixed_key_size - 1) / kBucketAlignment) + 1)
-           * kBucketAlignment;
+    return (((kValueOffset + this->total_payload_size_ + fixed_key_size - 1) /
+             kBucketAlignment) +
+            1) *
+           kBucketAlignment;
   }
   // If ValueT is not trivially destructible, invoke its destructor for all
   // values held in the specified buckets (including those in "empty" buckets
   // that were default constructed). If ValueT is trivially destructible, this
   // is a no-op.
   void DestroyValues(void *buckets,
-                            const std::size_t num_buckets,
-                            const std::size_t bucket_size);
+                     const std::size_t num_buckets,
+                     const std::size_t bucket_size);
 
   // Attempt to find an empty bucket to insert 'hash_code' into, starting after
   // '*bucket' in the chain (or, if '*bucket' is NULL, starting from the slot
@@ -201,30 +209,33 @@ class FastSeparateChainingHashTable : public FastHashTable<resizable,
   // attempt to allocate storage for a variable-length key BEFORE allocating a
   // bucket, so that no bucket number below 'header_->num_buckets' is ever
   // deallocated after being allocated.
-  inline bool locateBucketForInsertion(const std::size_t hash_code,
-                                       const std::size_t variable_key_allocation_required,
-                                       void **bucket,
-                                       std::atomic<std::size_t> **pending_chain_ptr,
-                                       std::size_t *pending_chain_ptr_finish_value,
-                                       HashTablePreallocationState *prealloc_state);
+  inline bool locateBucketForInsertion(
+      const std::size_t hash_code,
+      const std::size_t variable_key_allocation_required,
+      void **bucket,
+      std::atomic<std::size_t> **pending_chain_ptr,
+      std::size_t *pending_chain_ptr_finish_value,
+      HashTablePreallocationState *prealloc_state);
 
   // Write a scalar 'key' and its 'hash_code' into the '*bucket', which was
   // found by locateBucketForInsertion(). Assumes that storage for a
   // variable-length key copy (if any) was already allocated by a successful
   // call to allocateVariableLengthKeyStorage().
-  inline void writeScalarKeyToBucket(const TypedValue &key,
-                                     const std::size_t hash_code,
-                                     void *bucket,
-                                     HashTablePreallocationState *prealloc_state);
+  inline void writeScalarKeyToBucket(
+      const TypedValue &key,
+      const std::size_t hash_code,
+      void *bucket,
+      HashTablePreallocationState *prealloc_state);
 
   // Write a composite 'key' and its 'hash_code' into the '*bucket', which was
   // found by locateBucketForInsertion(). Assumes that storage for
   // variable-length key copies (if any) was already allocated by a successful
   // call to allocateVariableLengthKeyStorage().
-  inline void writeCompositeKeyToBucket(const std::vector<TypedValue> &key,
-                                        const std::size_t hash_code,
-                                        void *bucket,
-                                        HashTablePreallocationState *prealloc_state);
+  inline void writeCompositeKeyToBucket(
+      const std::vector<TypedValue> &key,
+      const std::size_t hash_code,
+      void *bucket,
+      HashTablePreallocationState *prealloc_state);
 
   // Determine whether it is actually necessary to resize this hash table.
   // Checks that there is at least one unallocated bucket, and that there is
@@ -275,30 +286,37 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::FastSeparateChainingHashTable(const std::vector<const Type*> &key_types,
-                                const std::size_t num_entries,
-                                const std::vector<std::size_t> &payload_sizes,
-                                const std::vector<AggregationHandle *> &handles,
-                                StorageManager *storage_manager)
-        : FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>(
-              key_types,
-              num_entries,
-              handles,
-              payload_sizes,
-              storage_manager,
-              false,
-              false,
-              true),
-          kBucketAlignment(alignof(std::atomic<std::size_t>)),
-          kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
-          key_manager_(this->key_types_, kValueOffset + this->total_payload_size_),
-          bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
-  init_payload_ = static_cast<std::uint8_t *>(calloc(this->total_payload_size_, 1));
+FastSeparateChainingHashTable<resizable,
+                              serializable,
+                              force_key_copy,
+                              allow_duplicate_keys>::
+    FastSeparateChainingHashTable(
+        const std::vector<const Type *> &key_types,
+        const std::size_t num_entries,
+        const std::vector<std::size_t> &payload_sizes,
+        const std::vector<AggregationHandle *> &handles,
+        StorageManager *storage_manager)
+    : FastHashTable<resizable,
+                    serializable,
+                    force_key_copy,
+                    allow_duplicate_keys>(key_types,
+                                          num_entries,
+                                          handles,
+                                          payload_sizes,
+                                          storage_manager,
+                                          false,
+                                          false,
+                                          true),
+      kBucketAlignment(alignof(std::atomic<std::size_t>)),
+      kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
+      key_manager_(this->key_types_, kValueOffset + this->total_payload_size_),
+      bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
+  init_payload_ =
+      static_cast<std::uint8_t *>(calloc(this->total_payload_size_, 1));
   int k = 0;
   for (auto handle : handles) {
-      handle->initPayload(init_payload_+this->payload_offsets_[k]);
-      k++;
+    handle->initPayload(init_payload_ + this->payload_offsets_[k]);
+    k++;
   }
   // Bucket size always rounds up to the alignment requirement of the atomic
   // size_t "next" pointer at the front or a ValueT, whichever is larger.
@@ -308,19 +326,23 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
   this->setKeyInline(key_manager_.getKeyInline());
 
   // Pick out a prime number of slots and calculate storage requirements.
-  std::size_t num_slots_tmp = get_next_prime_number(num_entries * kHashTableLoadFactor);
-  std::size_t required_memory = sizeof(Header)
-                                + num_slots_tmp * sizeof(std::atomic<std::size_t>)
-                                + (num_slots_tmp / kHashTableLoadFactor)
-                                    * (bucket_size_ + key_manager_.getEstimatedVariableKeySize());
-  std::size_t num_storage_slots = this->storage_manager_->SlotsNeededForBytes(required_memory);
+  std::size_t num_slots_tmp =
+      get_next_prime_number(num_entries * kHashTableLoadFactor);
+  std::size_t required_memory =
+      sizeof(Header) + num_slots_tmp * sizeof(std::atomic<std::size_t>) +
+      (num_slots_tmp / kHashTableLoadFactor) *
+          (bucket_size_ + key_manager_.getEstimatedVariableKeySize());
+  std::size_t num_storage_slots =
+      this->storage_manager_->SlotsNeededForBytes(required_memory);
   if (num_storage_slots == 0) {
-    FATAL_ERROR("Storage requirement for SeparateChainingHashTable "
-                "exceeds maximum allocation size.");
+    FATAL_ERROR(
+        "Storage requirement for SeparateChainingHashTable "
+        "exceeds maximum allocation size.");
   }
 
   // Get a StorageBlob to hold the hash table.
-  const block_id blob_id = this->storage_manager_->createBlob(num_storage_slots);
+  const block_id blob_id =
+      this->storage_manager_->createBlob(num_storage_slots);
   this->blob_ = this->storage_manager_->getBlobMutable(blob_id);
 
   void *aligned_memory_start = this->blob_->getMemoryMutable();
@@ -328,14 +350,14 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
   if (align(alignof(Header),
             sizeof(Header),
             aligned_memory_start,
-            available_memory)
-          == nullptr) {
+            available_memory) == nullptr) {
     // With current values from StorageConstants.hpp, this should be
     // impossible. A blob is at least 1 MB, while a Header has alignment
     // requirement of just kCacheLineBytes (64 bytes).
-    FATAL_ERROR("StorageBlob used to hold resizable "
-                "SeparateChainingHashTable is too small to meet alignment "
-                "requirements of SeparateChainingHashTable::Header.");
+    FATAL_ERROR(
+        "StorageBlob used to hold resizable "
+        "SeparateChainingHashTable is too small to meet alignment "
+        "requirements of SeparateChainingHashTable::Header.");
   } else if (aligned_memory_start != this->blob_->getMemoryMutable()) {
     // This should also be impossible, since the StorageManager allocates slots
     // aligned to kCacheLineBytes.
@@ -346,8 +368,9 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
   }
 
   // Locate the header.
-  header_ = static_cast<Header*>(aligned_memory_start);
-  aligned_memory_start = static_cast<char*>(aligned_memory_start) + sizeof(Header);
+  header_ = static_cast<Header *>(aligned_memory_start);
+  aligned_memory_start =
+      static_cast<char *>(aligned_memory_start) + sizeof(Header);
   available_memory -= sizeof(Header);
 
   // Recompute the number of slots & buckets using the actual available memory.
@@ -355,19 +378,20 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
   // the storage blob's size. It's also possible (though very unlikely) that we
   // will wind up with fewer buckets than we initially wanted because of screwy
   // alignment requirements for ValueT.
-  std::size_t num_buckets_tmp
-      = available_memory / (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>)
-                            + bucket_size_
-                            + key_manager_.getEstimatedVariableKeySize());
-  num_slots_tmp = get_previous_prime_number(num_buckets_tmp * kHashTableLoadFactor);
+  std::size_t num_buckets_tmp =
+      available_memory /
+      (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>) + bucket_size_ +
+       key_manager_.getEstimatedVariableKeySize());
+  num_slots_tmp =
+      get_previous_prime_number(num_buckets_tmp * kHashTableLoadFactor);
   num_buckets_tmp = num_slots_tmp / kHashTableLoadFactor;
   DEBUG_ASSERT(num_slots_tmp > 0);
   DEBUG_ASSERT(num_buckets_tmp > 0);
 
   // Locate the slot array.
-  slots_ = static_cast<std::atomic<std::size_t>*>(aligned_memory_start);
-  aligned_memory_start = static_cast<char*>(aligned_memory_start)
-                         + sizeof(std::atomic<std::size_t>) * num_slots_tmp;
+  slots_ = static_cast<std::atomic<std::size_t> *>(aligned_memory_start);
+  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
+                         sizeof(std::atomic<std::size_t>) * num_slots_tmp;
   available_memory -= sizeof(std::atomic<std::size_t>) * num_slots_tmp;
 
   // Locate the buckets.
@@ -375,17 +399,16 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
   // Extra-paranoid: If ValueT has an alignment requirement greater than that
   // of std::atomic<std::size_t>, we may need to adjust the start of the bucket
   // array.
-  if (align(kBucketAlignment,
-            bucket_size_,
-            buckets_,
-            available_memory)
-          == nullptr) {
-    FATAL_ERROR("StorageBlob used to hold resizable "
-                "SeparateChainingHashTable is too small to meet "
-                "alignment requirements of buckets.");
+  if (align(kBucketAlignment, bucket_size_, buckets_, available_memory) ==
+      nullptr) {
+    FATAL_ERROR(
+        "StorageBlob used to hold resizable "
+        "SeparateChainingHashTable is too small to meet "
+        "alignment requirements of buckets.");
   } else if (buckets_ != aligned_memory_start) {
-    DEV_WARNING("Bucket array start position adjusted to meet alignment "
-                "requirement for SeparateChainingHashTable's value type.");
+    DEV_WARNING(
+        "Bucket array start position adjusted to meet alignment "
+        "requirement for SeparateChainingHashTable's value type.");
     if (num_buckets_tmp * bucket_size_ > available_memory) {
       --num_buckets_tmp;
     }
@@ -401,7 +424,7 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
   // Locate variable-length key storage region, and give it all the remaining
   // bytes in the blob.
   key_manager_.setVariableLengthStorageInfo(
-      static_cast<char*>(buckets_) + header_->num_buckets * bucket_size_,
+      static_cast<char *>(buckets_) + header_->num_buckets * bucket_size_,
       available_memory,
       &(header_->variable_length_bytes_allocated));
 }
@@ -410,36 +433,43 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::FastSeparateChainingHashTable(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)
-        : FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>(
-              key_types,
-              hash_table_memory,
-              hash_table_memory_size,
-              new_hash_table,
-              hash_table_memory_zeroed,
-              false,
-              false,
-              true),
-          kBucketAlignment(alignof(std::atomic<std::size_t>) < alignof(uint8_t) ? alignof(uint8_t)
-                                                  : alignof(std::atomic<std::size_t>)),
-          kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
-          key_manager_(this->key_types_, kValueOffset + sizeof(uint8_t)),
-          bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
+FastSeparateChainingHashTable<resizable,
+                              serializable,
+                              force_key_copy,
+                              allow_duplicate_keys>::
+    FastSeparateChainingHashTable(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)
+    : FastHashTable<resizable,
+                    serializable,
+                    force_key_copy,
+                    allow_duplicate_keys>(key_types,
+                                          hash_table_memory,
+                                          hash_table_memory_size,
+                                          new_hash_table,
+                                          hash_table_memory_zeroed,
+                                          false,
+                                          false,
+                                          true),
+      kBucketAlignment(alignof(std::atomic<std::size_t>) < alignof(std::uint8_t)
+                           ? alignof(std::uint8_t)
+                           : alignof(std::atomic<std::size_t>)),
+      kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
+      key_manager_(this->key_types_, kValueOffset + sizeof(std::uint8_t)),
+      bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
   // Bucket size always rounds up to the alignment requirement of the atomic
   // size_t "next" pointer at the front or a ValueT, whichever is larger.
   //
   // Make sure that the larger of the two alignment requirements also satisfies
   // the smaller.
-  static_assert(alignof(std::atomic<std::size_t>) < alignof(uint8_t)
-                    ? alignof(uint8_t) % alignof(std::atomic<std::size_t>) == 0
-                    : alignof(std::atomic<std::size_t>) % alignof(uint8_t) == 0,
-                "Alignment requirement of std::atomic<std::size_t> does not "
-                "evenly divide with alignment requirement of ValueT.");
+  static_assert(
+      alignof(std::atomic<std::size_t>) < alignof(std::uint8_t)
+          ? alignof(std::uint8_t) % alignof(std::atomic<std::size_t>) == 0
+          : alignof(std::atomic<std::size_t>) % alignof(std::uint8_t) == 0,
+      "Alignment requirement of std::atomic<std::size_t> does not "
+      "evenly divide with alignment requirement of ValueT.");
 
   // Give base HashTable information about what key components are stored
   // inline from 'key_manager_'.
@@ -460,12 +490,13 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
   if (align(alignof(Header),
             sizeof(Header),
             aligned_memory_start,
-            available_memory)
-          == nullptr) {
+            available_memory) == nullptr) {
     FATAL_ERROR("Attempted to create a non-resizable "
                 << "SeparateChainingHashTable with "
-                << available_memory << " bytes of memory at "
-                << aligned_memory_start << " which either can not fit a "
+                << available_memory
+                << " bytes of memory at "
+                << aligned_memory_start
+                << " which either can not fit a "
                 << "SeparateChainingHashTable::Header or meet its alignement "
                 << "requirement.");
   } else if (aligned_memory_start != this->hash_table_memory_) {
@@ -477,32 +508,36 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
                 << "SeparateChainingHashTable::Header.");
   }
 
-  header_ = static_cast<Header*>(aligned_memory_start);
-  aligned_memory_start = static_cast<char*>(aligned_memory_start) + sizeof(Header);
+  header_ = static_cast<Header *>(aligned_memory_start);
+  aligned_memory_start =
+      static_cast<char *>(aligned_memory_start) + sizeof(Header);
   available_memory -= sizeof(Header);
 
   if (new_hash_table) {
-    std::size_t estimated_bucket_capacity
-        = available_memory / (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>)
-                              + bucket_size_
-                              + key_manager_.getEstimatedVariableKeySize());
-    std::size_t num_slots = get_previous_prime_number(estimated_bucket_capacity * kHashTableLoadFactor);
+    std::size_t estimated_bucket_capacity =
+        available_memory /
+        (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>) +
+         bucket_size_ + key_manager_.getEstimatedVariableKeySize());
+    std::size_t num_slots = get_previous_prime_number(
+        estimated_bucket_capacity * kHashTableLoadFactor);
 
     // Fill in the header.
     header_->num_slots = num_slots;
     header_->num_buckets = num_slots / kHashTableLoadFactor;
     header_->buckets_allocated.store(0, std::memory_order_relaxed);
-    header_->variable_length_bytes_allocated.store(0, std::memory_order_relaxed);
+    header_->variable_length_bytes_allocated.store(0,
+                                                   std::memory_order_relaxed);
   }
 
   // Locate the slot array.
-  slots_ = static_cast<std::atomic<std::size_t>*>(aligned_memory_start);
-  aligned_memory_start = static_cast<char*>(aligned_memory_start)
-                         + sizeof(std::atomic<std::size_t>) * header_->num_slots;
+  slots_ = static_cast<std::atomic<std::size_t> *>(aligned_memory_start);
+  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
+                         sizeof(std::atomic<std::size_t>) * header_->num_slots;
   available_memory -= sizeof(std::atomic<std::size_t>) * header_->num_slots;
 
   if (new_hash_table && !hash_table_memory_zeroed) {
-    std::memset(slots_, 0x0, sizeof(std::atomic<std::size_t>) * header_->num_slots);
+    std::memset(
+        slots_, 0x0, sizeof(std::atomic<std::size_t>) * header_->num_slots);
   }
 
   // Locate the buckets.
@@ -510,20 +545,20 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
   // Extra-paranoid: sizeof(Header) should almost certainly be a multiple of
   // kBucketAlignment, unless ValueT has some members with seriously big
   // (> kCacheLineBytes) alignment requirements specified using alignas().
-  if (align(kBucketAlignment,
-            bucket_size_,
-            buckets_,
-            available_memory)
-          == nullptr) {
+  if (align(kBucketAlignment, bucket_size_, buckets_, available_memory) ==
+      nullptr) {
     FATAL_ERROR("Attempted to create a non-resizable "
                 << "SeparateChainingHashTable with "
-                << this->hash_table_memory_size_ << " bytes of memory at "
-                << this->hash_table_memory_ << ", which can hold an aligned "
+                << this->hash_table_memory_size_
+                << " bytes of memory at "
+                << this->hash_table_memory_
+                << ", which can hold an aligned "
                 << "SeparateChainingHashTable::Header but does not have "
                 << "enough remaining space for even a single hash bucket.");
   } else if (buckets_ != aligned_memory_start) {
-    DEV_WARNING("Bucket array start position adjusted to meet alignment "
-                "requirement for SeparateChainingHashTable's value type.");
+    DEV_WARNING(
+        "Bucket array start position adjusted to meet alignment "
+        "requirement for SeparateChainingHashTable's value type.");
     if (header_->num_buckets * bucket_size_ > available_memory) {
       DEBUG_ASSERT(new_hash_table);
       --(header_->num_buckets);
@@ -538,7 +573,7 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
 
   // Locate variable-length key storage region.
   key_manager_.setVariableLengthStorageInfo(
-      static_cast<char*>(buckets_) + header_->num_buckets * bucket_size_,
+      static_cast<char *>(buckets_) + header_->num_buckets * bucket_size_,
       available_memory,
       &(header_->variable_length_bytes_allocated));
 }
@@ -547,16 +582,18 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::clear() {
-  const std::size_t used_buckets = header_->buckets_allocated.load(std::memory_order_relaxed);
+void FastSeparateChainingHashTable<resizable,
+                                   serializable,
+                                   force_key_copy,
+                                   allow_duplicate_keys>::clear() {
+  const std::size_t used_buckets =
+      header_->buckets_allocated.load(std::memory_order_relaxed);
   // Destroy existing values, if necessary.
-  DestroyValues(buckets_,
-                used_buckets,
-                bucket_size_);
+  DestroyValues(buckets_, used_buckets, bucket_size_);
 
   // Zero-out slot array.
-  std::memset(slots_, 0x0, sizeof(std::atomic<std::size_t>) * header_->num_slots);
+  std::memset(
+      slots_, 0x0, sizeof(std::atomic<std::size_t>) * header_->num_slots);
 
   // Zero-out used buckets.
   std::memset(buckets_, 0x0, used_buckets * bucket_size_);
@@ -570,24 +607,33 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-const uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getSingle(const TypedValue &key) const {
+const std::uint8_t* FastSeparateChainingHashTable<
+    resizable,
+    serializable,
+    force_key_copy,
+    allow_duplicate_keys>::getSingle(const TypedValue &key) const {
   DEBUG_ASSERT(!allow_duplicate_keys);
   DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+  DEBUG_ASSERT(
+      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
 
   const std::size_t hash_code = key.getHash();
-  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  std::size_t bucket_ref =
+      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
   while (bucket_ref != 0) {
     DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
         bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) && key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+    if ((bucket_hash == hash_code) &&
+        key_manager_.scalarKeyCollisionCheck(key, bucket)) {
       // Match located.
-      return reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+      return reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
     }
-    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+    bucket_ref =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
   }
 
   // Reached the end of the chain and didn't find a match.
@@ -598,23 +644,31 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-const uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getSingleCompositeKey(const std::vector<TypedValue> &key) const {
+const std::uint8_t* FastSeparateChainingHashTable<resizable,
+                                                  serializable,
+                                                  force_key_copy,
+                                                  allow_duplicate_keys>::
+    getSingleCompositeKey(const std::vector<TypedValue> &key) const {
   DEBUG_ASSERT(!allow_duplicate_keys);
   DEBUG_ASSERT(this->key_types_.size() == key.size());
 
   const std::size_t hash_code = this->hashCompositeKey(key);
-  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  std::size_t bucket_ref =
+      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
   while (bucket_ref != 0) {
     DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
         bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) && key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+    if ((bucket_hash == hash_code) &&
+        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
       // Match located.
-      return reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+      return reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
     }
-    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+    bucket_ref =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
   }
 
   // Reached the end of the chain and didn't find a match.
@@ -625,23 +679,32 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-const uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getSingleCompositeKey(const std::vector<TypedValue> &key, int index) const {
+const std::uint8_t* FastSeparateChainingHashTable<resizable,
+                                                  serializable,
+                                                  force_key_copy,
+                                                  allow_duplicate_keys>::
+    getSingleCompositeKey(const std::vector<TypedValue> &key, int index) const {
   DEBUG_ASSERT(!allow_duplicate_keys);
   DEBUG_ASSERT(this->key_types_.size() == key.size());
 
   const std::size_t hash_code = this->hashCompositeKey(key);
-  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  std::size_t bucket_ref =
+      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
   while (bucket_ref != 0) {
     DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
         bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) && key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+    if ((bucket_hash == hash_code) &&
+        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
       // Match located.
-      return reinterpret_cast<const uint8_t*>(bucket + kValueOffset)+this->payload_offsets_[index];
+      return reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset) +
+             this->payload_offsets_[index];
     }
-    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+    bucket_ref =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
   }
 
   // Reached the end of the chain and didn't find a match.
@@ -652,26 +715,38 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getAll(const TypedValue &key, std::vector<const uint8_t*> *values) const {
+void FastSeparateChainingHashTable<
+    resizable,
+    serializable,
+    force_key_copy,
+    allow_duplicate_keys>::getAll(const TypedValue &key,
+                                  std::vector<const std::uint8_t *> *values)
+    const {
   DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+  DEBUG_ASSERT(
+      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
 
   const std::size_t hash_code = key.getHash();
-  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  std::size_t bucket_ref =
+      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
   while (bucket_ref != 0) {
     DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
         bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) && key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+    if ((bucket_hash == hash_code) &&
+        key_manager_.scalarKeyCollisionCheck(key, bucket)) {
       // Match located.
-      values->push_back(reinterpret_cast<const uint8_t*>(bucket + kValueOffset));
+      values->push_back(
+          reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset));
       if (!allow_duplicate_keys) {
         return;
       }
     }
-    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+    bucket_ref =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
   }
 }
 
@@ -679,25 +754,35 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getAllCompositeKey(const std::vector<TypedValue> &key, std::vector<const uint8_t*> *values) const {
+void FastSeparateChainingHashTable<resizable,
+                                   serializable,
+                                   force_key_copy,
+                                   allow_duplicate_keys>::
+    getAllCompositeKey(const std::vector<TypedValue> &key,
+                       std::vector<const std::uint8_t *> *values) const {
   DEBUG_ASSERT(this->key_types_.size() == key.size());
 
   const std::size_t hash_code = this->hashCompositeKey(key);
-  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  std::size_t bucket_ref =
+      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
   while (bucket_ref != 0) {
     DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
         bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) && key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+    if ((bucket_hash == hash_code) &&
+        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
       // Match located.
-      values->push_back(reinterpret_cast<const uint8_t*>(bucket + kValueOffset));
+      values->push_back(
+          reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset));
       if (!allow_duplicate_keys) {
         return;
       }
     }
-    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+    bucket_ref =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
   }
 }
 
@@ -705,18 +790,22 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-HashTablePutResult
-    FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-        ::putInternal(const TypedValue &key,
-                      const std::size_t variable_key_size,
-                      const uint8_t &value,
-                      HashTablePreallocationState *prealloc_state) {
+HashTablePutResult FastSeparateChainingHashTable<resizable,
+                                                 serializable,
+                                                 force_key_copy,
+                                                 allow_duplicate_keys>::
+    putInternal(const TypedValue &key,
+                const std::size_t variable_key_size,
+                const std::uint8_t &value,
+                HashTablePreallocationState *prealloc_state) {
   DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+  DEBUG_ASSERT(
+      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
 
   if (prealloc_state == nullptr) {
     // Early check for a free bucket.
-    if (header_->buckets_allocated.load(std::memory_order_relaxed) >= header_->num_buckets) {
+    if (header_->buckets_allocated.load(std::memory_order_relaxed) >=
+        header_->num_buckets) {
       return HashTablePutResult::kOutOfSpace;
     }
 
@@ -763,10 +852,11 @@ HashTablePutResult
   writeScalarKeyToBucket(key, hash_code, bucket, prealloc_state);
 
   // Store the value by using placement new with ValueT's copy constructor.
-  new(static_cast<char*>(bucket) + kValueOffset) uint8_t(value);
+  new (static_cast<char *>(bucket) + kValueOffset) std::uint8_t(value);
 
   // Update the previous chain pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+  pending_chain_ptr->store(pending_chain_ptr_finish_value,
+                           std::memory_order_release);
 
   // We're all done.
   return HashTablePutResult::kOK;
@@ -776,17 +866,20 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-HashTablePutResult
-    FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-        ::putCompositeKeyInternalFast(const std::vector<TypedValue> &key,
-                                  const std::size_t variable_key_size,
-                                  const uint8_t *init_value_ptr,
-                                  HashTablePreallocationState *prealloc_state) {
+HashTablePutResult FastSeparateChainingHashTable<resizable,
+                                                 serializable,
+                                                 force_key_copy,
+                                                 allow_duplicate_keys>::
+    putCompositeKeyInternalFast(const std::vector<TypedValue> &key,
+                                const std::size_t variable_key_size,
+                                const std::uint8_t *init_value_ptr,
+                                HashTablePreallocationState *prealloc_state) {
   DEBUG_ASSERT(this->key_types_.size() == key.size());
 
   if (prealloc_state == nullptr) {
     // Early check for a free bucket.
-    if (header_->buckets_allocated.load(std::memory_order_relaxed) >= header_->num_buckets) {
+    if (header_->buckets_allocated.load(std::memory_order_relaxed) >=
+        header_->num_buckets) {
       return HashTablePutResult::kOutOfSpace;
     }
 
@@ -832,12 +925,11 @@ HashTablePutResult
   // Write the key and hash.
   writeCompositeKeyToBucket(key, hash_code, bucket, prealloc_state);
 
-  // Store the value by using placement new with ValueT's copy constructor.
-//  new(static_cast<char*>(bucket) + kValueOffset) uint8_t(value);
-    uint8_t *value = static_cast<uint8_t*>(bucket) + kValueOffset;
-        memcpy(value, init_value_ptr, this->total_payload_size_);
+  std::uint8_t *value = static_cast<std::uint8_t *>(bucket) + kValueOffset;
+  memcpy(value, init_value_ptr, this->total_payload_size_);
   // Update the previous chain pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+  pending_chain_ptr->store(pending_chain_ptr_finish_value,
+                           std::memory_order_release);
 
   // We're all done.
   return HashTablePutResult::kOK;
@@ -847,13 +939,17 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsertInternalFast(const TypedValue &key,
-                     const std::size_t variable_key_size,
-                     const std::uint8_t *init_value_ptr) {
+std::uint8_t* FastSeparateChainingHashTable<resizable,
+                                            serializable,
+                                            force_key_copy,
+                                            allow_duplicate_keys>::
+    upsertInternalFast(const TypedValue &key,
+                       const std::size_t variable_key_size,
+                       const std::uint8_t *init_value_ptr) {
   DEBUG_ASSERT(!allow_duplicate_keys);
   DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+  DEBUG_ASSERT(
+      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
 
   if (variable_key_size > 0) {
     // Don't allocate yet, since the key may already be present. However, we
@@ -861,9 +957,11 @@ uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy,
     // space is big enough to hold the key (at least one must be true: either
     // the key is already present and allocated, or we need to be able to
     // allocate enough space for it).
-    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
-    if ((allocated_bytes < variable_key_size)
-        && (allocated_bytes + variable_key_size > key_manager_.getVariableLengthKeyStorageSize())) {
+    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(
+        std::memory_order_relaxed);
+    if ((allocated_bytes < variable_key_size) &&
+        (allocated_bytes + variable_key_size >
+         key_manager_.getVariableLengthKeyStorageSize())) {
       return nullptr;
     }
   }
@@ -886,7 +984,8 @@ uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy,
       return nullptr;
     } else if (key_manager_.scalarKeyCollisionCheck(key, bucket)) {
       // Found an already-existing entry for this key.
-      return reinterpret_cast<uint8_t*>(static_cast<char*>(bucket) + kValueOffset);
+      return reinterpret_cast<std::uint8_t *>(static_cast<char *>(bucket) +
+                                              kValueOffset);
     }
   }
 
@@ -895,16 +994,15 @@ uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy,
   writeScalarKeyToBucket(key, hash_code, bucket, nullptr);
 
   // Copy the supplied 'initial_value' into place.
-//  uint8_t *value = new(static_cast<char*>(bucket) + kValueOffset) uint8_t(initial_value);
-
-    uint8_t *value = static_cast<unsigned char*>(bucket) + kValueOffset;
-    if (init_value_ptr == nullptr)
-        memcpy(value, init_payload_, this->total_payload_size_);
-    else
-        memcpy(value, init_value_ptr, this->total_payload_size_);
+  std::uint8_t *value = static_cast<unsigned char *>(bucket) + kValueOffset;
+  if (init_value_ptr == nullptr)
+    memcpy(value, init_payload_, this->total_payload_size_);
+  else
+    memcpy(value, init_value_ptr, this->total_payload_size_);
 
   // Update the previous chain pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+  pending_chain_ptr->store(pending_chain_ptr_finish_value,
+                           std::memory_order_release);
 
   // Return the value.
   return value;
@@ -914,10 +1012,13 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsertCompositeKeyInternalFast(const std::vector<TypedValue> &key,
-                                 const std::uint8_t *init_value_ptr,
-                                 const std::size_t variable_key_size) {
+std::uint8_t* FastSeparateChainingHashTable<resizable,
+                                            serializable,
+                                            force_key_copy,
+                                            allow_duplicate_keys>::
+    upsertCompositeKeyInternalFast(const std::vector<TypedValue> &key,
+                                   const std::uint8_t *init_value_ptr,
+                                   const std::size_t variable_key_size) {
   DEBUG_ASSERT(!allow_duplicate_keys);
   DEBUG_ASSERT(this->key_types_.size() == key.size());
 
@@ -927,9 +1028,11 @@ uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy,
     // space is big enough to hold the key (at least one must be true: either
     // the key is already present and allocated, or we need to be able to
     // allocate enough space for it).
-    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
-    if ((allocated_bytes < variable_key_size)
-        && (allocated_bytes + variable_key_size > key_manager_.getVariableLengthKeyStorageSize())) {
+    std::size_t allocated_bytes = header_->variable_length_bytes_allocated.load(
+        std::memory_order_relaxed);
+    if ((allocated_bytes < variable_key_size) &&
+        (allocated_bytes + variable_key_size >
+         key_manager_.getVariableLengthKeyStorageSize())) {
       return nullptr;
     }
   }
@@ -952,7 +1055,8 @@ uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy,
       return nullptr;
     } else if (key_manager_.compositeKeyCollisionCheck(key, bucket)) {
       // Found an already-existing entry for this key.
-      return reinterpret_cast<uint8_t*>(static_cast<char*>(bucket) + kValueOffset);
+      return reinterpret_cast<std::uint8_t *>(static_cast<char *>(bucket) +
+                                              kValueOffset);
     }
   }
 
@@ -960,17 +1064,16 @@ uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy,
   // Write the key and hash.
   writeCompositeKeyToBucket(key, hash_code, bucket, nullptr);
 
-//  uint8_t *value;
-//  value = static_cast<unsigned char*>(bucket) + kValueOffset;
-    uint8_t *value = static_cast<unsigned char*>(bucket) + kValueOffset;
-    if (init_value_ptr == nullptr) {
-        memcpy(value, init_payload_, this->total_payload_size_);
-    } else {
-        memcpy(value, init_value_ptr, this->total_payload_size_);
-    }
+  std::uint8_t *value = static_cast<unsigned char *>(bucket) + kValueOffset;
+  if (init_value_ptr == nullptr) {
+    memcpy(value, init_payload_, this->total_payload_size_);
+  } else {
+    memcpy(value, init_value_ptr, this->total_payload_size_);
+  }
 
   // Update the previous chaing pointer to point to the new bucket.
-  pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
+  pending_chain_ptr->store(pending_chain_ptr_finish_value,
+                           std::memory_order_release);
 
   // Return the value.
   return value;
@@ -980,13 +1083,19 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getNextEntry(TypedValue *key, const uint8_t **value, std::size_t *entry_num) const {
+bool FastSeparateChainingHashTable<
+    resizable,
+    serializable,
+    force_key_copy,
+    allow_duplicate_keys>::getNextEntry(TypedValue *key,
+                                        const std::uint8_t **value,
+                                        std::size_t *entry_num) const {
   DEBUG_ASSERT(this->key_types_.size() == 1);
   if (*entry_num < header_->buckets_allocated.load(std::memory_order_relaxed)) {
-    const char *bucket = static_cast<const char*>(buckets_) + (*entry_num) * bucket_size_;
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (*entry_num) * bucket_size_;
     *key = key_manager_.getKeyComponentTyped(bucket, 0);
-    *value = reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+    *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
     ++(*entry_num);
     return true;
   } else {
@@ -998,18 +1107,22 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getNextEntryCompositeKey(std::vector<TypedValue> *key,
-                               const uint8_t **value,
-                               std::size_t *entry_num) const {
+bool FastSeparateChainingHashTable<resizable,
+                                   serializable,
+                                   force_key_copy,
+                                   allow_duplicate_keys>::
+    getNextEntryCompositeKey(std::vector<TypedValue> *key,
+                             const std::uint8_t **value,
+                             std::size_t *entry_num) const {
   if (*entry_num < header_->buckets_allocated.load(std::memory_order_relaxed)) {
-    const char *bucket = static_cast<const char*>(buckets_) + (*entry_num) * bucket_size_;
-    for (std::vector<const Type*>::size_type key_idx = 0;
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (*entry_num) * bucket_size_;
+    for (std::vector<const Type *>::size_type key_idx = 0;
          key_idx < this->key_types_.size();
          ++key_idx) {
       key->emplace_back(key_manager_.getKeyComponentTyped(bucket, key_idx));
     }
-    *value = reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+    *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
     ++(*entry_num);
     return true;
   } else {
@@ -1021,29 +1134,38 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getNextEntryForKey(const TypedValue &key,
-                         const std::size_t hash_code,
-                         const uint8_t **value,
-                         std::size_t *entry_num) const {
+bool FastSeparateChainingHashTable<
+    resizable,
+    serializable,
+    force_key_copy,
+    allow_duplicate_keys>::getNextEntryForKey(const TypedValue &key,
+                                              const std::size_t hash_code,
+                                              const std::uint8_t **value,
+                                              std::size_t *entry_num) const {
   DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+  DEBUG_ASSERT(
+      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
 
   if (*entry_num == 0) {
-    *entry_num = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+    *entry_num =
+        slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
   } else if (*entry_num == std::numeric_limits<std::size_t>::max()) {
     return false;
   }
 
   while (*entry_num != 0) {
     DEBUG_ASSERT(*entry_num != std::numeric_limits<std::size_t>::max());
-    const char *bucket = static_cast<const char*>(buckets_) + (*entry_num - 1) * bucket_size_;
-    *entry_num = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (*entry_num - 1) * bucket_size_;
+    *entry_num =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
         bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) && key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+    if ((bucket_hash == hash_code) &&
+        key_manager_.scalarKeyCollisionCheck(key, bucket)) {
       // Match located.
-      *value = reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+      *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
       if (*entry_num == 0) {
         // If this is the last bucket in the chain, prevent the next call from
         // starting over again.
@@ -1061,28 +1183,36 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::getNextEntryForCompositeKey(const std::vector<TypedValue> &key,
-                                  const std::size_t hash_code,
-                                  const uint8_t **value,
-                                  std::size_t *entry_num) const {
+bool FastSeparateChainingHashTable<resizable,
+                                   serializable,
+                                   force_key_copy,
+                                   allow_duplicate_keys>::
+    getNextEntryForCompositeKey(const std::vector<TypedValue> &key,
+                                const std::size_t hash_code,
+                                const std::uint8_t **value,
+                                std::size_t *entry_num) const {
   DEBUG_ASSERT(this->key_types_.size() == key.size());
 
   if (*entry_num == 0) {
-    *entry_num = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+    *entry_num =
+        slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
   } else if (*entry_num == std::numeric_limits<std::size_t>::max()) {
     return false;
   }
 
   while (*entry_num != 0) {
     DEBUG_ASSERT(*entry_num != std::numeric_limits<std::size_t>::max());
-    const char *bucket = static_cast<const char*>(buckets_) + (*entry_num - 1) * bucket_size_;
-    *entry_num = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (*entry_num - 1) * bucket_size_;
+    *entry_num =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
         bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) && key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+    if ((bucket_hash == hash_code) &&
+        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
       // Match located.
-      *value = reinterpret_cast<const uint8_t*>(bucket + kValueOffset);
+      *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
       if (*entry_num == 0) {
         // If this is the last bucket in the chain, prevent the next call from
         // starting over again.
@@ -1100,23 +1230,32 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::hasKey(const TypedValue &key) const {
+bool FastSeparateChainingHashTable<
+    resizable,
+    serializable,
+    force_key_copy,
+    allow_duplicate_keys>::hasKey(const TypedValue &key) const {
   DEBUG_ASSERT(this->key_types_.size() == 1);
-  DEBUG_ASSERT(key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
+  DEBUG_ASSERT(
+      key.isPlausibleInstanceOf(this->key_types_.front()->getSignature()));
 
   const std::size_t hash_code = key.getHash();
-  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  std::size_t bucket_ref =
+      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
   while (bucket_ref != 0) {
     DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
         bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) && key_manager_.scalarKeyCollisionCheck(key, bucket)) {
+    if ((bucket_hash == hash_code) &&
+        key_manager_.scalarKeyCollisionCheck(key, bucket)) {
       // Find a match.
       return true;
     }
-    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+    bucket_ref =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
   }
   return false;
 }
@@ -1125,22 +1264,31 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::hasCompositeKey(const std::vector<TypedValue> &key) const {
+bool FastSeparateChainingHashTable<
+    resizable,
+    serializable,
+    force_key_copy,
+    allow_duplicate_keys>::hasCompositeKey(const std::vector<TypedValue> &key)
+    const {
   DEBUG_ASSERT(this->key_types_.size() == key.size());
 
   const std::size_t hash_code = this->hashCompositeKey(key);
-  std::size_t bucket_ref = slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
+  std::size_t bucket_ref =
+      slots_[hash_code % header_->num_slots].load(std::memory_order_relaxed);
   while (bucket_ref != 0) {
     DEBUG_ASSERT(bucket_ref != std::numeric_limits<std::size_t>::max());
-    const char *bucket = static_cast<const char*>(buckets_) + (bucket_ref - 1) * bucket_size_;
-    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t*>(
+    const char *bucket =
+        static_cast<const char *>(buckets_) + (bucket_ref - 1) * bucket_size_;
+    const std::size_t bucket_hash = *reinterpret_cast<const std::size_t *>(
         bucket + sizeof(std::atomic<std::size_t>));
-    if ((bucket_hash == hash_code) && key_manager_.compositeKeyCollisionCheck(key, bucket)) {
+    if ((bucket_hash == hash_code) &&
+        key_manager_.compositeKeyCollisionCheck(key, bucket)) {
       // Find a match.
       return true;
     }
-    bucket_ref = reinterpret_cast<const std::atomic<std::size_t>*>(bucket)->load(std::memory_order_relaxed);
+    bucket_ref =
+        reinterpret_cast<const std::atomic<std::size_t> *>(bucket)->load(
+            std::memory_order_relaxed);
   }
   return false;
 }
@@ -1149,10 +1297,13 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::resize(const std::size_t extra_buckets,
-             const std::size_t extra_variable_storage,
-             const std::size_t retry_num) {
+void FastSeparateChainingHashTable<
+    resizable,
+    serializable,
+    force_key_copy,
+    allow_duplicate_keys>::resize(const std::size_t extra_buckets,
+                                  const std::size_t extra_variable_storage,
+                                  const std::size_t retry_num) {
   DEBUG_ASSERT(resizable);
 
   // A retry should never be necessary with this implementation of HashTable.
@@ -1178,33 +1329,36 @@ void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allo
   // account kHashTableLoadFactor.
   std::size_t resized_num_slots = get_next_prime_number(
       (header_->num_buckets + extra_buckets / 2) * kHashTableLoadFactor * 2);
-  std::size_t variable_storage_required
-      = (resized_num_slots / kHashTableLoadFactor) * key_manager_.getEstimatedVariableKeySize();
-  const std::size_t original_variable_storage_used
-      = header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
+  std::size_t variable_storage_required =
+      (resized_num_slots / kHashTableLoadFactor) *
+      key_manager_.getEstimatedVariableKeySize();
+  const std::size_t original_variable_storage_used =
+      header_->variable_length_bytes_allocated.load(std::memory_order_relaxed);
   // If this resize was triggered by a too-large variable-length key, bump up
   // the variable-length storage requirement.
-  if ((extra_variable_storage > 0)
-      && (extra_variable_storage + original_variable_storage_used
-          > key_manager_.getVariableLengthKeyStorageSize())) {
+  if ((extra_variable_storage > 0) &&
+      (extra_variable_storage + original_variable_storage_used >
+       key_manager_.getVariableLengthKeyStorageSize())) {
     variable_storage_required += extra_variable_storage;
   }
 
-  const std::size_t resized_memory_required
-      = sizeof(Header)
-        + resized_num_slots * sizeof(std::atomic<std::size_t>)
-        + (resized_num_slots / kHashTableLoadFactor) * bucket_size_
-        + variable_storage_required;
-  const std::size_t resized_storage_slots
-      = this->storage_manager_->SlotsNeededForBytes(resized_memory_required);
+  const std::size_t resized_memory_required =
+      sizeof(Header) + resized_num_slots * sizeof(std::atomic<std::size_t>) +
+      (resized_num_slots / kHashTableLoadFactor) * bucket_size_ +
+      variable_storage_required;
+  const std::size_t resized_storage_slots =
+      this->storage_manager_->SlotsNeededForBytes(resized_memory_required);
   if (resized_storage_slots == 0) {
-    FATAL_ERROR("Storage requirement for resized SeparateChainingHashTable "
-                "exceeds maximum allocation size.");
+    FATAL_ERROR(
+        "Storage requirement for resized SeparateChainingHashTable "
+        "exceeds maximum allocation size.");
   }
 
   // Get a new StorageBlob to hold the resized hash table.
-  const block_id resized_blob_id = this->storage_manager_->createBlob(resized_storage_slots);
-  MutableBlobReference resized_blob = this->storage_manager_->getBlobMutable(resized_blob_id);
+  const block_id resized_blob_id =
+      this->storage_manager_->createBlob(resized_storage_slots);
+  MutableBlobReference resized_blob =
+      this->storage_manager_->getBlobMutable(resized_blob_id);
 
   // Locate data structures inside the new StorageBlob.
   void *aligned_memory_start = resized_blob->getMemoryMutable();
@@ -1212,12 +1366,12 @@ void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allo
   if (align(alignof(Header),
             sizeof(Header),
             aligned_memory_start,
-            available_memory)
-          == nullptr) {
+            available_memory) == nullptr) {
     // Should be impossible, as noted in constructor.
-    FATAL_ERROR("StorageBlob used to hold resized SeparateChainingHashTable "
-                "is too small to meet alignment requirements of "
-                "LinearOpenAddressingHashTable::Header.");
+    FATAL_ERROR(
+        "StorageBlob used to hold resized SeparateChainingHashTable "
+        "is too small to meet alignment requirements of "
+        "LinearOpenAddressingHashTable::Header.");
   } else if (aligned_memory_start != resized_blob->getMemoryMutable()) {
     // Again, should be impossible.
     DEV_WARNING("In SeparateChainingHashTable::resize(), StorageBlob "
@@ -1227,59 +1381,63 @@ void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allo
                 << "LinearOpenAddressingHashTable::Header.");
   }
 
-  Header *resized_header = static_cast<Header*>(aligned_memory_start);
-  aligned_memory_start = static_cast<char*>(aligned_memory_start) + sizeof(Header);
+  Header *resized_header = static_cast<Header *>(aligned_memory_start);
+  aligned_memory_start =
+      static_cast<char *>(aligned_memory_start) + sizeof(Header);
   available_memory -= sizeof(Header);
 
   // As in constructor, recompute the number of slots and buckets using the
   // actual available memory.
-  std::size_t resized_num_buckets
-      = (available_memory - extra_variable_storage)
-        / (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>)
-           + bucket_size_
-           + key_manager_.getEstimatedVariableKeySize());
-  resized_num_slots = get_previous_prime_number(resized_num_buckets * kHashTableLoadFactor);
+  std::size_t resized_num_buckets =
+      (available_memory - extra_variable_storage) /
+      (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>) + bucket_size_ +
+       key_manager_.getEstimatedVariableKeySize());
+  resized_num_slots =
+      get_previous_prime_number(resized_num_buckets * kHashTableLoadFactor);
   resized_num_buckets = resized_num_slots / kHashTableLoadFactor;
 
   // Locate slot array.
-  std::atomic<std::size_t> *resized_slots = static_cast<std::atomic<std::size_t>*>(aligned_memory_start);
-  aligned_memory_start = static_cast<char*>(aligned_memory_start)
-                         + sizeof(std::atomic<std::size_t>) * resized_num_slots;
+  std::atomic<std::size_t> *resized_slots =
+      static_cast<std::atomic<std::size_t> *>(aligned_memory_start);
+  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
+                         sizeof(std::atomic<std::size_t>) * resized_num_slots;
   available_memory -= sizeof(std::atomic<std::size_t>) * resized_num_slots;
 
   // As in constructor, we will be extra paranoid and use align() to locate the
   // start of the array of buckets, as well.
   void *resized_buckets = aligned_memory_start;
-  if (align(kBucketAlignment,
-            bucket_size_,
-            resized_buckets,
-            available_memory)
-          == nullptr) {
-    FATAL_ERROR("StorageBlob used to hold resized SeparateChainingHashTable "
-                "is too small to meet alignment requirements of buckets.");
+  if (align(
+          kBucketAlignment, bucket_size_, resized_buckets, available_memory) ==
+      nullptr) {
+    FATAL_ERROR(
+        "StorageBlob used to hold resized SeparateChainingHashTable "
+        "is too small to meet alignment requirements of buckets.");
   } else if (resized_buckets != aligned_memory_start) {
-    DEV_WARNING("Bucket array start position adjusted to meet alignment "
-                "requirement for SeparateChainingHashTable's value type.");
-    if (resized_num_buckets * bucket_size_ + variable_storage_required > available_memory) {
+    DEV_WARNING(
+        "Bucket array start position adjusted to meet alignment "
+        "requirement for SeparateChainingHashTable's value type.");
+    if (resized_num_buckets * bucket_size_ + variable_storage_required >
+        available_memory) {
       --resized_num_buckets;
     }
   }
-  aligned_memory_start = static_cast<char*>(aligned_memory_start)
-                         + resized_num_buckets * bucket_size_;
+  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
+                         resized_num_buckets * bucket_size_;
   available_memory -= resized_num_buckets * bucket_size_;
 
   void *resized_variable_length_key_storage = aligned_memory_start;
   const std::size_t resized_variable_length_key_storage_size = available_memory;
 
-  const std::size_t original_buckets_used = header_->buckets_allocated.load(std::memory_order_relaxed);
+  const std::size_t original_buckets_used =
+      header_->buckets_allocated.load(std::memory_order_relaxed);
 
   // Initialize the header.
   resized_header->num_slots = resized_num_slots;
   resized_header->num_buckets = resized_num_buckets;
-  resized_header->buckets_allocated.store(original_buckets_used, std::memory_order_relaxed);
+  resized_header->buckets_allocated.store(original_buckets_used,
+                                          std::memory_order_relaxed);
   resized_header->variable_length_bytes_allocated.store(
-      original_variable_storage_used,
-      std::memory_order_relaxed);
+      original_variable_storage_used, std::memory_order_relaxed);
 
   // Bulk-copy buckets. This is safe because:
   //     1. The "next" pointers will be adjusted when rebuilding chains below.
@@ -1298,30 +1456,34 @@ void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allo
   // GCC 4.8.3, so we assume we need to invoke ValueT's copy or move
   // constructor, even though the plain memcpy above could suffice for many
   // possible ValueTs.
-  void *current_value_original = static_cast<char*>(buckets_) + kValueOffset;
-  void *current_value_resized = static_cast<char*>(resized_buckets) + kValueOffset;
-  for (std::size_t bucket_num = 0; bucket_num < original_buckets_used; ++bucket_num) {
+  void *current_value_original = static_cast<char *>(buckets_) + kValueOffset;
+  void *current_value_resized =
+      static_cast<char *>(resized_buckets) + kValueOffset;
+  for (std::size_t bucket_num = 0; bucket_num < original_buckets_used;
+       ++bucket_num) {
     // Use a move constructor if available to avoid a deep-copy, since resizes
     // always succeed.
-    new (current_value_resized) uint8_t(std::move(*static_cast<uint8_t*>(current_value_original)));
-    current_value_original = static_cast<char*>(current_value_original) + bucket_size_;
-    current_value_resized = static_cast<char*>(current_value_resized) + bucket_size_;
+    new (current_value_resized) std::uint8_t(
+        std::move(*static_cast<std::uint8_t *>(current_value_original)));
+    current_value_original =
+        static_cast<char *>(current_value_original) + bucket_size_;
+    current_value_resized =
+        static_cast<char *>(current_value_resized) + bucket_size_;
   }
 
   // Copy over variable-length key components, if any.
   if (original_variable_storage_used > 0) {
-    DEBUG_ASSERT(original_variable_storage_used
-                 == key_manager_.getNextVariableLengthKeyOffset());
-    DEBUG_ASSERT(original_variable_storage_used <= resized_variable_length_key_storage_size);
+    DEBUG_ASSERT(original_variable_storage_used ==
+                 key_manager_.getNextVariableLengthKeyOffset());
+    DEBUG_ASSERT(original_variable_storage_used <=
+                 resized_variable_length_key_storage_size);
     std::memcpy(resized_variable_length_key_storage,
                 key_manager_.getVariableLengthKeyStorage(),
                 original_variable_storage_used);
   }
 
   // Destroy values in the original hash table, if neccesary,
-  DestroyValues(buckets_,
-                original_buckets_used,
-                bucket_size_);
+  DestroyValues(buckets_, original_buckets_used, bucket_size_);
 
   // Make resized structures active.
   std::swap(this->blob_, resized_blob);
@@ -1340,17 +1502,18 @@ void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allo
 
   // Rebuild chains.
   void *current_bucket = buckets_;
-  for (std::size_t bucket_num = 0; bucket_num < original_buckets_used; ++bucket_num) {
-    std::atomic<std::size_t> *next_ptr
-        = static_cast<std::atomic<std::size_t>*>(current_bucket);
-    const std::size_t hash_code = *reinterpret_cast<const std::size_t*>(
-        static_cast<const char*>(current_bucket) + sizeof(std::atomic<std::size_t>));
+  for (std::size_t bucket_num = 0; bucket_num < original_buckets_used;
+       ++bucket_num) {
+    std::atomic<std::size_t> *next_ptr =
+        static_cast<std::atomic<std::size_t> *>(current_bucket);
+    const std::size_t hash_code = *reinterpret_cast<const std::size_t *>(
+        static_cast<const char *>(current_bucket) +
+        sizeof(std::atomic<std::size_t>));
 
     const std::size_t slot_number = hash_code % header_->num_slots;
     std::size_t slot_ptr_value = 0;
-    if (slots_[slot_number].compare_exchange_strong(slot_ptr_value,
-                                                    bucket_num + 1,
-                                                    std::memory_order_relaxed)) {
+    if (slots_[slot_number].compare_exchange_strong(
+            slot_ptr_value, bucket_num + 1, std::memory_order_relaxed)) {
       // This bucket is the first in the chain for this block, so reset its
       // next pointer to 0.
       next_ptr->store(0, std::memory_order_relaxed);
@@ -1360,7 +1523,7 @@ void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allo
       next_ptr->store(slot_ptr_value, std::memory_order_relaxed);
       slots_[slot_number].store(bucket_num + 1, std::memory_order_relaxed);
     }
-    current_bucket = static_cast<char*>(current_bucket) + bucket_size_;
+    current_bucket = static_cast<char *>(current_bucket) + bucket_size_;
   }
 }
 
@@ -1368,10 +1531,13 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::preallocateForBulkInsert(const std::size_t total_entries,
-                               const std::size_t total_variable_key_size,
-                               HashTablePreallocationState *prealloc_state) {
+bool FastSeparateChainingHashTable<resizable,
+                                   serializable,
+                                   force_key_copy,
+                                   allow_duplicate_keys>::
+    preallocateForBulkInsert(const std::size_t total_entries,
+                             const std::size_t total_variable_key_size,
+                             HashTablePreallocationState *prealloc_state) {
   DEBUG_ASSERT(allow_duplicate_keys);
   if (!key_manager_.allocateVariableLengthKeyStorage(total_variable_key_size)) {
     return false;
@@ -1382,12 +1548,15 @@ bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allo
   // than one bucket and exceed 'header_->num_buckets', their respective
   // rollbacks might happen in such an order that some bucket ranges get
   // skipped, while others might get double-allocated later.
-  std::size_t original_buckets_allocated = header_->buckets_allocated.load(std::memory_order_relaxed);
-  std::size_t buckets_post_allocation = original_buckets_allocated + total_entries;
-  while ((buckets_post_allocation <= header_->num_buckets)
-         && !header_->buckets_allocated.compare_exchange_weak(original_buckets_allocated,
-                                                              buckets_post_allocation,
-                                                              std::memory_order_relaxed)) {
+  std::size_t original_buckets_allocated =
+      header_->buckets_allocated.load(std::memory_order_relaxed);
+  std::size_t buckets_post_allocation =
+      original_buckets_allocated + total_entries;
+  while ((buckets_post_allocation <= header_->num_buckets) &&
+         !header_->buckets_allocated.compare_exchange_weak(
+             original_buckets_allocated,
+             buckets_post_allocation,
+             std::memory_order_relaxed)) {
     buckets_post_allocation = original_buckets_allocated + total_entries;
   }
 
@@ -1398,8 +1567,9 @@ bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allo
 
   prealloc_state->bucket_position = original_buckets_allocated;
   if (total_variable_key_size != 0) {
-    prealloc_state->variable_length_key_position
-        = key_manager_.incrementNextVariableLengthKeyOffset(total_variable_key_size);
+    prealloc_state->variable_length_key_position =
+        key_manager_.incrementNextVariableLengthKeyOffset(
+            total_variable_key_size);
   }
   return true;
 }
@@ -1408,17 +1578,18 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-void FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::DestroyValues(void *hash_buckets,
-                    const std::size_t num_buckets,
-                    const std::size_t bucket_size) {
-  if (!std::is_trivially_destructible<uint8_t>::value) {
-    void *value_ptr = static_cast<char*>(hash_buckets) + kValueOffset;
-    for (std::size_t bucket_num = 0;
-         bucket_num < num_buckets;
-         ++bucket_num) {
-     static_cast<uint8_t*>(value_ptr)->~uint8_t();
-      value_ptr = static_cast<char*>(value_ptr) + bucket_size;
+void FastSeparateChainingHashTable<
+    resizable,
+    serializable,
+    force_key_copy,
+    allow_duplicate_keys>::DestroyValues(void *hash_buckets,
+                                         const std::size_t num_buckets,
+                                         const std::size_t bucket_size) {
+  if (!std::is_trivially_destructible<std::uint8_t>::value) {
+    void *value_ptr = static_cast<char *>(hash_buckets) + kValueOffset;
+    for (std::size_t bucket_num = 0; bucket_num < num_buckets; ++bucket_num) {
+      static_cast<std::uint8_t *>(value_ptr)->~uint8_t();
+      value_ptr = static_cast<char *>(value_ptr) + bucket_size;
     }
   }
 }
@@ -1427,39 +1598,45 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-inline bool FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::locateBucketForInsertion(const std::size_t hash_code,
-                               const std::size_t variable_key_allocation_required,
-                               void **bucket,
-                               std::atomic<std::size_t> **pending_chain_ptr,
-                               std::size_t *pending_chain_ptr_finish_value,
-                               HashTablePreallocationState *prealloc_state) {
+inline bool FastSeparateChainingHashTable<resizable,
+                                          serializable,
+                                          force_key_copy,
+                                          allow_duplicate_keys>::
+    locateBucketForInsertion(const std::size_t hash_code,
+                             const std::size_t variable_key_allocation_required,
+                             void **bucket,
+                             std::atomic<std::size_t> **pending_chain_ptr,
+                             std::size_t *pending_chain_ptr_finish_value,
+                             HashTablePreallocationState *prealloc_state) {
   DEBUG_ASSERT((prealloc_state == nullptr) || allow_duplicate_keys);
   if (*bucket == nullptr) {
     *pending_chain_ptr = &(slots_[hash_code % header_->num_slots]);
   } else {
-    *pending_chain_ptr = static_cast<std::atomic<std::size_t>*>(*bucket);
+    *pending_chain_ptr = static_cast<std::atomic<std::size_t> *>(*bucket);
   }
   for (;;) {
     std::size_t existing_chain_ptr = 0;
-    if ((*pending_chain_ptr)->compare_exchange_strong(existing_chain_ptr,
-                        

<TRUNCATED>


[23/28] incubator-quickstep git commit: New operator to destroy aggregation state.

Posted by hb...@apache.org.
New operator to destroy aggregation state.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/1db46d60
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/1db46d60
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/1db46d60

Branch: refs/heads/partitioned-aggregation
Commit: 1db46d605e6256d82eaab6cd11918b3b0b589e5b
Parents: f05036b
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Sun Aug 21 09:32:51 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:57:06 2016 -0500

----------------------------------------------------------------------
 query_execution/QueryContext.hpp    | 13 +++++++++++++
 relational_operators/CMakeLists.txt | 11 +++++++++++
 2 files changed, 24 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1db46d60/query_execution/QueryContext.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.hpp b/query_execution/QueryContext.hpp
index 393b55e..f6c08ba 100644
--- a/query_execution/QueryContext.hpp
+++ b/query_execution/QueryContext.hpp
@@ -185,6 +185,19 @@ class QueryContext {
    * @brief Destroy the given aggregation state.
    *
    * @param id The ID of the AggregationOperationState to destroy.
+   *
+   * @return The AggregationOperationState, alreadly created in the constructor.
+   **/
+  inline void destroyAggregationState(const aggregation_state_id id) {
+    DCHECK_LT(id, aggregation_states_.size());
+    DCHECK(aggregation_states_[id]);
+    aggregation_states_[id].reset(nullptr);
+  }
+
+  /**
+   * @brief Destroy the given aggregation state.
+   *
+   * @param id The ID of the AggregationOperationState to destroy.
    **/
   inline void destroyAggregationState(const aggregation_state_id id) {
     DCHECK_LT(id, aggregation_states_.size());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1db46d60/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index a9645b4..2632995 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -44,6 +44,7 @@ add_library(quickstep_relationaloperators_DestroyAggregationStateOperator
             DestroyAggregationStateOperator.cpp 
             DestroyAggregationStateOperator.hpp)
 add_library(quickstep_relationaloperators_DeleteOperator DeleteOperator.cpp DeleteOperator.hpp)
+add_library(quickstep_relationaloperators_DestroyAggregationStateOperator DestroyAggregationStateOperator.cpp DestroyAggregationStateOperator.hpp)
 add_library(quickstep_relationaloperators_DestroyHashOperator DestroyHashOperator.cpp DestroyHashOperator.hpp)
 add_library(quickstep_relationaloperators_DropTableOperator DropTableOperator.cpp DropTableOperator.hpp)
 add_library(quickstep_relationaloperators_FinalizeAggregationOperator
@@ -166,6 +167,16 @@ target_link_libraries(quickstep_relationaloperators_DestroyHashOperator
                       quickstep_relationaloperators_WorkOrder_proto
                       quickstep_utility_Macros
                       tmb)
+target_link_libraries(quickstep_relationaloperators_DestroyAggregationStateOperator
+                      glog
+                      quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
+                      quickstep_queryexecution_WorkOrdersContainer
+                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
+                      quickstep_utility_Macros
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_DropTableOperator
                       glog
                       quickstep_catalog_CatalogDatabase


[24/28] incubator-quickstep git commit: Changed the bool flag

Posted by hb...@apache.org.
Changed the bool flag


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/f05036be
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/f05036be
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/f05036be

Branch: refs/heads/partitioned-aggregation
Commit: f05036bed3ef510d46f7c5b714f774e3710c4a12
Parents: bc524a9
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Fri Aug 19 15:00:28 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:57:06 2016 -0500

----------------------------------------------------------------------
 storage/AggregationOperationState.cpp | 3 ++-
 storage/AggregationOperationState.hpp | 5 +++++
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f05036be/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index faf2c0b..66c4a83 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -68,7 +68,7 @@ AggregationOperationState::AggregationOperationState(
     const HashTableImplType hash_table_impl_type,
     const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
     StorageManager *storage_manager)
-    : is_aggregate_partitioned_(estimated_num_entries > kPartitionedAggregateThreshold),
+    : is_aggregate_partitioned_(estimated_num_entries > kPartitionedAggregateThreshold && !group_by.empty()),
       input_relation_(input_relation),
       predicate_(predicate),
       group_by_list_(std::move(group_by)),
@@ -76,6 +76,7 @@ AggregationOperationState::AggregationOperationState(
       is_distinct_(std::move(is_distinct)),
       storage_manager_(storage_manager) {
   // Sanity checks: each aggregate has a corresponding list of arguments.
+  LOG(INFO) << "Aggregate partitioned: " << is_aggregate_partitioned_ << " with estimated # entries: " << estimated_num_entries;
   DCHECK(aggregate_functions.size() == arguments_.size());
 
   // Get the types of GROUP BY expressions for creating HashTables below.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f05036be/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 880375f..ab82c2e 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -181,6 +181,11 @@ class AggregationOperationState {
   void finalizeAggregatePartitioned(const std::size_t partition_id,
                                     InsertDestination *output_destination);
 
+  /**
+   * @brief Find if the aggregation has been performed in a partitoned way.
+   *
+   * @note At present the partitioning is enabled only with GROUP BY.
+   **/
   bool isAggregatePartitioned() const {
     return is_aggregate_partitioned_;
   }


[12/28] incubator-quickstep git commit: Removed some dead code and made minor updates.

Posted by hb...@apache.org.
Removed some dead code and made minor updates.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/3606000b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/3606000b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/3606000b

Branch: refs/heads/partitioned-aggregation
Commit: 3606000bb477fe8ad2c77ee483975ea82c708e1e
Parents: ae789f9
Author: rathijit <ra...@node-2.aggregation.quickstep-pg0.wisc.cloudlab.us>
Authored: Mon Aug 15 06:28:36 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:56:07 2016 -0500

----------------------------------------------------------------------
 .../aggregation/AggregationConcreteHandle.hpp   | 418 +------------------
 expressions/aggregation/AggregationHandle.hpp   |  24 +-
 .../aggregation/AggregationHandleAvg.cpp        |  30 +-
 .../aggregation/AggregationHandleAvg.hpp        |  11 +-
 .../aggregation/AggregationHandleCount.cpp      |  30 +-
 .../aggregation/AggregationHandleCount.hpp      |  13 +-
 .../aggregation/AggregationHandleDistinct.cpp   |   5 +-
 .../aggregation/AggregationHandleDistinct.hpp   |  13 +-
 .../aggregation/AggregationHandleMax.cpp        |  24 +-
 .../aggregation/AggregationHandleMax.hpp        |  11 +-
 .../aggregation/AggregationHandleMin.cpp        |  24 +-
 .../aggregation/AggregationHandleMin.hpp        |  11 +-
 .../aggregation/AggregationHandleSum.cpp        |  21 +-
 .../aggregation/AggregationHandleSum.hpp        |  14 +-
 expressions/aggregation/CMakeLists.txt          |  82 ++--
 storage/AggregationOperationState.cpp           |  21 +-
 storage/FastHashTable.hpp                       | 293 ++-----------
 storage/FastSeparateChainingHashTable.hpp       | 213 +---------
 18 files changed, 136 insertions(+), 1122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationConcreteHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.hpp b/expressions/aggregation/AggregationConcreteHandle.hpp
index 609937a..5b47e93 100644
--- a/expressions/aggregation/AggregationConcreteHandle.hpp
+++ b/expressions/aggregation/AggregationConcreteHandle.hpp
@@ -50,37 +50,6 @@ class ValueAccessor;
  * @brief An upserter class for modifying the destination hash table while
  *        merging two group by hash tables.
  **/
-template <typename HandleT, typename StateT>
-class HashTableStateUpserter {
- public:
-  /**
-   * @brief Constructor.
-   *
-   * @param handle The aggregation handle being used.
-   * @param source_state The aggregation state in the source aggregation hash
-   *        table. The corresponding state (for the same key) in the destination
-   *        hash table will be upserted.
-   **/
-  HashTableStateUpserter(const HandleT &handle, const StateT &source_state)
-      : handle_(handle), source_state_(source_state) {}
-
-  /**
-   * @brief The operator for the functor required for the upsert.
-   *
-   * @param destination_state The aggregation state in the aggregation hash
-   *        table that is being upserted.
-   **/
-  void operator()(StateT *destination_state) {
-    handle_.mergeStates(source_state_, destination_state);
-  }
-
- private:
-  const HandleT &handle_;
-  const StateT &source_state_;
-
-  DISALLOW_COPY_AND_ASSIGN(HashTableStateUpserter);
-};
-
 template <typename HandleT>
 class HashTableStateUpserterFast {
  public:
@@ -113,103 +82,6 @@ class HashTableStateUpserterFast {
 };
 
 /**
- * @brief A class to support the functor for merging group by hash tables.
- **/
-template <typename HandleT, typename StateT, typename HashTableT>
-class HashTableMerger {
- public:
-  /**
-   * @brief Constructor
-   *
-   * @param handle The Aggregation handle being used.
-   * @param destination_hash_table The destination hash table to which other
-   *        hash tables will be merged.
-   **/
-  HashTableMerger(const HandleT &handle,
-                  AggregationStateHashTableBase *destination_hash_table)
-      : handle_(handle),
-        destination_hash_table_(
-            static_cast<HashTableT *>(destination_hash_table)) {}
-
-  /**
-   * @brief The operator for the functor.
-   *
-   * @param group_by_key The group by key being merged.
-   * @param source_state The aggregation state for the given key in the source
-   *        aggregation hash table.
-   **/
-  inline void operator()(const std::vector<TypedValue> &group_by_key,
-                         const StateT &source_state) {
-    const StateT *original_state =
-        destination_hash_table_->getSingleCompositeKey(group_by_key);
-    if (original_state != nullptr) {
-      HashTableStateUpserter<HandleT, StateT> upserter(
-          handle_, source_state);
-      // The CHECK is required as upsertCompositeKey can return false if the
-      // hash table runs out of space during the upsert process. The ideal
-      // solution will be to retry again if the upsert fails.
-      CHECK(destination_hash_table_->upsertCompositeKey(
-          group_by_key, *original_state, &upserter));
-    } else {
-      destination_hash_table_->putCompositeKey(group_by_key, source_state);
-    }
-  }
-
- private:
-  const HandleT &handle_;
-  HashTableT *destination_hash_table_;
-
-  DISALLOW_COPY_AND_ASSIGN(HashTableMerger);
-};
-
-template <typename HandleT, typename HashTableT>
-class HashTableMergerFast {
- public:
-  /**
-   * @brief Constructor
-   *
-   * @param handle The Aggregation handle being used.
-   * @param destination_hash_table The destination hash table to which other
-   *        hash tables will be merged.
-   **/
-  HashTableMergerFast(const HandleT &handle,
-                  AggregationStateHashTableBase *destination_hash_table)
-      : handle_(handle),
-        destination_hash_table_(
-            static_cast<HashTableT *>(destination_hash_table)) {}
-
-  /**
-   * @brief The operator for the functor.
-   *
-   * @param group_by_key The group by key being merged.
-   * @param source_state The aggregation state for the given key in the source
-   *        aggregation hash table.
-   **/
-  inline void operator()(const std::vector<TypedValue> &group_by_key,
-                         const uint8_t *source_state) {
-    const uint8_t *original_state =
-        destination_hash_table_->getSingleCompositeKey(group_by_key);
-    if (original_state != nullptr) {
-      HashTableStateUpserterFast<HandleT> upserter(
-          handle_, source_state);
-      // The CHECK is required as upsertCompositeKey can return false if the
-      // hash table runs out of space during the upsert process. The ideal
-      // solution will be to retry again if the upsert fails.
-      CHECK(destination_hash_table_->upsertCompositeKeyFast(
-          group_by_key, original_state, &upserter));
-    } else {
-      destination_hash_table_->putCompositeKeyFast(group_by_key, source_state);
-    }
-  }
-
- private:
-  const HandleT &handle_;
-  HashTableT *destination_hash_table_;
-
-  DISALLOW_COPY_AND_ASSIGN(HashTableMergerFast);
-};
-
-/**
  * @brief The helper intermediate subclass of AggregationHandle that provides
  *        virtual method implementations as well as helper methods that are
  *        shared among all its subclasses.
@@ -255,55 +127,16 @@ class AggregationConcreteHandle : public AggregationHandle {
   }
 
   template <typename HandleT,
-            typename StateT,
-            typename HashTableT>
-  void aggregateValueAccessorIntoHashTableNullaryHelper(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &group_by_key_ids,
-      const StateT &default_state,
-      AggregationStateHashTableBase *hash_table) const;
-
-  template <typename HandleT,
-            typename StateT,
-            typename HashTableT>
-  void aggregateValueAccessorIntoHashTableUnaryHelper(
-      ValueAccessor *accessor,
-      const attribute_id argument_id,
-      const std::vector<attribute_id> &group_by_key_ids,
-      const StateT &default_state,
-      AggregationStateHashTableBase *hash_table) const;
-
-  template <typename HandleT,
-            typename StateT>
-  StateT* aggregateOnDistinctifyHashTableForSingleUnaryHelper(
-      const AggregationStateHashTableBase &distinctify_hash_table) const;
-
-  template <typename HandleT,
             typename StateT>
   StateT* aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
       const AggregationStateHashTableBase &distinctify_hash_table) const;
 
   template <typename HandleT,
-            typename StateT,
-            typename HashTableT>
-  void aggregateOnDistinctifyHashTableForGroupByUnaryHelper(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      const StateT &default_state,
-      AggregationStateHashTableBase *hash_table) const;
-
-  template <typename HandleT,
             typename HashTableT>
   void aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *hash_table, int index) const;
-
-
-  template <typename HandleT,
-            typename HashTableT>
-  ColumnVector* finalizeHashTableHelper(
-      const Type &result_type,
-      const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys) const;
+      AggregationStateHashTableBase *hash_table,
+      int index) const;
 
   template <typename HandleT,
             typename HashTableT>
@@ -314,17 +147,6 @@ class AggregationConcreteHandle : public AggregationHandle {
       int index) const;
 
   template <typename HandleT, typename HashTableT>
-  inline TypedValue finalizeGroupInHashTable(
-      const AggregationStateHashTableBase &hash_table,
-      const std::vector<TypedValue> &group_key) const {
-    const AggregationState *group_state
-        = static_cast<const HashTableT&>(hash_table).getSingleCompositeKey(group_key);
-    DCHECK(group_state != nullptr)
-        << "Could not find entry for specified group_key in HashTable";
-    return static_cast<const HandleT*>(this)->finalizeHashTableEntry(*group_state);
-  }
-
-  template <typename HandleT, typename HashTableT>
   inline TypedValue finalizeGroupInHashTableFast(
       const AggregationStateHashTableBase &hash_table,
       const std::vector<TypedValue> &group_key,
@@ -336,67 +158,16 @@ class AggregationConcreteHandle : public AggregationHandle {
     return static_cast<const HandleT*>(this)->finalizeHashTableEntryFast(group_state);
   }
 
-  template <typename HandleT, typename StateT, typename HashTableT>
-  void mergeGroupByHashTablesHelper(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const;
-
   template <typename HandleT, typename HashTableT>
   void mergeGroupByHashTablesHelperFast(
       const AggregationStateHashTableBase &source_hash_table,
       AggregationStateHashTableBase *destination_hash_table) const;
 
-
  private:
   DISALLOW_COPY_AND_ASSIGN(AggregationConcreteHandle);
 };
 
 /**
- * @brief Templated class to implement value-accessor-based upserter for each
- *        aggregation state payload type. This version is for nullary
- *        aggregates (those that take no arguments).
- **/
-template <typename HandleT, typename StateT>
-class NullaryAggregationStateValueAccessorUpserter {
- public:
-  explicit NullaryAggregationStateValueAccessorUpserter(const HandleT &handle)
-      : handle_(handle) {
-  }
-
-  template <typename ValueAccessorT>
-  inline void operator()(const ValueAccessorT &accessor, StateT *state) {
-    handle_.iterateNullaryInl(state);
-  }
-
- private:
-  const HandleT &handle_;
-};
-
-/**
- * @brief Templated class to implement value-accessor-based upserter for each
- *        aggregation state payload type. This version is for unary aggregates
- *        (those that take a single argument).
- **/
-template <typename HandleT, typename StateT>
-class UnaryAggregationStateValueAccessorUpserter {
- public:
-  UnaryAggregationStateValueAccessorUpserter(const HandleT &handle,
-                                             attribute_id value_id)
-    : handle_(handle),
-      value_id_(value_id) {
-  }
-
-  template <typename ValueAccessorT>
-  inline void operator()(const ValueAccessorT &accessor, StateT *state) {
-    handle_.iterateUnaryInl(state, accessor.getTypedValue(value_id_));
-  }
-
- private:
-  const HandleT &handle_;
-  const attribute_id value_id_;
-};
-
-/**
  * @brief Templated helper class used to implement
  *        AggregationHandle::finalizeHashTable() by visiting each entry (i.e.
  *        GROUP) in a HashTable, finalizing the aggregation for the GROUP, and
@@ -438,68 +209,6 @@ class HashTableAggregateFinalizer {
 // Implementations of templated methods follow:
 
 template <typename HandleT,
-          typename StateT,
-          typename HashTableT>
-void AggregationConcreteHandle::aggregateValueAccessorIntoHashTableNullaryHelper(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &group_by_key_ids,
-    const StateT &default_state,
-    AggregationStateHashTableBase *hash_table) const {
-  NullaryAggregationStateValueAccessorUpserter<HandleT, StateT>
-      upserter(static_cast<const HandleT&>(*this));
-  static_cast<HashTableT*>(hash_table)->upsertValueAccessorCompositeKey(
-      accessor,
-      group_by_key_ids,
-      true,
-      default_state,
-      &upserter);
-}
-
-template <typename HandleT,
-          typename StateT,
-          typename HashTableT>
-void AggregationConcreteHandle::aggregateValueAccessorIntoHashTableUnaryHelper(
-    ValueAccessor *accessor,
-    const attribute_id argument_id,
-    const std::vector<attribute_id> &group_by_key_ids,
-    const StateT &default_state,
-    AggregationStateHashTableBase *hash_table) const {
-  UnaryAggregationStateValueAccessorUpserter<HandleT, StateT>
-      upserter(static_cast<const HandleT&>(*this), argument_id);
-  static_cast<HashTableT*>(hash_table)->upsertValueAccessorCompositeKey(
-      accessor,
-      group_by_key_ids,
-      true,
-      default_state,
-      &upserter);
-}
-
-template <typename HandleT,
-          typename StateT>
-StateT* AggregationConcreteHandle::aggregateOnDistinctifyHashTableForSingleUnaryHelper(
-    const AggregationStateHashTableBase &distinctify_hash_table) const {
-  const HandleT& handle = static_cast<const HandleT&>(*this);
-  StateT *state = static_cast<StateT*>(createInitialState());
-
-  // A lambda function which will be called on each key from the distinctify
-  // hash table.
-  const auto aggregate_functor = [&handle, &state](const TypedValue &key,
-                                                   const bool &dumb_placeholder) {
-    // For each (unary) key in the distinctify hash table, aggregate the key
-    // into "state".
-    handle.iterateUnaryInl(state, key);
-  };
-
-  const AggregationStateHashTable<bool> &hash_table =
-      static_cast<const AggregationStateHashTable<bool>&>(distinctify_hash_table);
-  // Invoke the lambda function "aggregate_functor" on each key from the distinctify
-  // hash table.
-  hash_table.forEach(&aggregate_functor);
-
-  return state;
-}
-
-template <typename HandleT,
           typename StateT>
 StateT* AggregationConcreteHandle::aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
@@ -525,47 +234,11 @@ StateT* AggregationConcreteHandle::aggregateOnDistinctifyHashTableForSingleUnary
 }
 
 template <typename HandleT,
-          typename StateT,
-          typename HashTableT>
-void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHelper(
-    const AggregationStateHashTableBase &distinctify_hash_table,
-    const StateT &default_state,
-    AggregationStateHashTableBase *aggregation_hash_table) const {
-  const HandleT& handle = static_cast<const HandleT&>(*this);
-  HashTableT *target_hash_table = static_cast<HashTableT*>(aggregation_hash_table);
-
-  // A lambda function which will be called on each key-value pair from the
-  // distinctify hash table.
-  const auto aggregate_functor = [&handle, &target_hash_table, &default_state](
-      std::vector<TypedValue> &key,
-      const bool &dumb_placeholder) {
-    // For each (composite) key vector in the distinctify hash table with size N.
-    // The first N-1 entries are GROUP BY columns and the last entry is the argument
-    // to be aggregated on.
-    const TypedValue argument(std::move(key.back()));
-    key.pop_back();
-
-    // An upserter as lambda function for aggregating the argument into its
-    // GROUP BY group's entry inside aggregation_hash_table.
-    const auto upserter = [&handle, &argument](StateT *state) {
-      handle.iterateUnaryInl(state, argument);
-    };
-
-    target_hash_table->upsertCompositeKey(key, default_state, &upserter);
-  };
-
-  const AggregationStateHashTable<bool> &source_hash_table =
-      static_cast<const AggregationStateHashTable<bool>&>(distinctify_hash_table);
-  // Invoke the lambda function "aggregate_functor" on each composite key vector
-  // from the distinctify hash table.
-  source_hash_table.forEachCompositeKey(&aggregate_functor);
-}
-
-template <typename HandleT,
           typename HashTableT>
 void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table, int index) const {
+    AggregationStateHashTableBase *aggregation_hash_table,
+    int index) const {
   const HandleT& handle = static_cast<const HandleT&>(*this);
   HashTableT *target_hash_table = static_cast<HashTableT*>(aggregation_hash_table);
 
@@ -596,57 +269,6 @@ void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHe
   source_hash_table.forEachCompositeKeyFast(&aggregate_functor);
 }
 
-
-template <typename HandleT,
-          typename HashTableT>
-ColumnVector* AggregationConcreteHandle::finalizeHashTableHelper(
-    const Type &result_type,
-    const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys) const {
-  const HandleT &handle = static_cast<const HandleT&>(*this);
-  const HashTableT &hash_table_concrete = static_cast<const HashTableT&>(hash_table);
-
-  if (group_by_keys->empty()) {
-    if (NativeColumnVector::UsableForType(result_type)) {
-      NativeColumnVector *result = new NativeColumnVector(result_type,
-                                                          hash_table_concrete.numEntries());
-      HashTableAggregateFinalizer<HandleT, NativeColumnVector> finalizer(
-          handle,
-          group_by_keys,
-          result);
-      hash_table_concrete.forEachCompositeKey(&finalizer);
-      return result;
-    } else {
-      IndirectColumnVector *result = new IndirectColumnVector(result_type,
-                                                              hash_table_concrete.numEntries());
-      HashTableAggregateFinalizer<HandleT, IndirectColumnVector> finalizer(
-          handle,
-          group_by_keys,
-          result);
-      hash_table_concrete.forEachCompositeKey(&finalizer);
-      return result;
-    }
-  } else {
-    if (NativeColumnVector::UsableForType(result_type)) {
-      NativeColumnVector *result = new NativeColumnVector(result_type,
-                                                          group_by_keys->size());
-      for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
-        result->appendTypedValue(finalizeGroupInHashTable<HandleT, HashTableT>(hash_table,
-                                                                               group_by_key));
-      }
-      return result;
-    } else {
-      IndirectColumnVector *result = new IndirectColumnVector(result_type,
-                                                              hash_table_concrete.numEntries());
-      for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
-        result->appendTypedValue(finalizeGroupInHashTable<HandleT, HashTableT>(hash_table,
-                                                                               group_by_key));
-      }
-      return result;
-    }
-  }
-}
-
 template <typename HandleT,
           typename HashTableT>
 ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
@@ -700,38 +322,6 @@ ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
   }
 }
 
-template <typename HandleT,
-          typename StateT,
-          typename HashTableT>
-void AggregationConcreteHandle::mergeGroupByHashTablesHelper(
-    const AggregationStateHashTableBase &source_hash_table,
-    AggregationStateHashTableBase *destination_hash_table) const {
-  const HandleT &handle = static_cast<const HandleT &>(*this);
-  const HashTableT &source_hash_table_concrete =
-      static_cast<const HashTableT &>(source_hash_table);
-
-  HashTableMerger<HandleT, StateT, HashTableT> merger(handle,
-                                                      destination_hash_table);
-
-  source_hash_table_concrete.forEachCompositeKey(&merger);
-}
-
-template <typename HandleT,
-          typename HashTableT>
-void AggregationConcreteHandle::mergeGroupByHashTablesHelperFast(
-    const AggregationStateHashTableBase &source_hash_table,
-    AggregationStateHashTableBase *destination_hash_table) const {
-  const HandleT &handle = static_cast<const HandleT &>(*this);
-  const HashTableT &source_hash_table_concrete =
-      static_cast<const HashTableT &>(source_hash_table);
-
-  HashTableMergerFast<HandleT, HashTableT> merger(handle,
-                                              destination_hash_table);
-
-  source_hash_table_concrete.forEachCompositeKeyFast(&merger);
-}
-
-
 }  // namespace quickstep
 
 #endif  // QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index 7c9e544..01e3d20 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -265,7 +265,8 @@ class AggregationHandle {
    **/
   virtual ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys, int index) const = 0;
+      std::vector<std::vector<TypedValue>> *group_by_keys,
+      int index) const = 0;
 
   /**
    * @brief Create a new HashTable for the distinctify step for DISTINCT aggregation.
@@ -347,26 +348,13 @@ class AggregationHandle {
    */
   virtual void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table, int index) const = 0;
-
-  /**
-   * @brief Merge two GROUP BY hash tables in one.
-   *
-   * @note Both the hash tables should have the same structure.
-   *
-   * @param source_hash_table The hash table which will get merged.
-   * @param destination_hash_table The hash table to which we will merge the
-   *        other hash table.
-   **/
-  virtual void mergeGroupByHashTables(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const = 0;
+      AggregationStateHashTableBase *aggregation_hash_table,
+      int index) const = 0;
 
   virtual size_t getPayloadSize() const {return 1;}
-  virtual void setPayloadOffset(std::size_t) {}
-  virtual void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) {}
+  virtual void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const {}
   virtual void mergeStatesFast(const uint8_t *src, uint8_t *dst) const {}
-  virtual void initPayload(uint8_t *byte_ptr) {}
+  virtual void initPayload(uint8_t *byte_ptr) const {}
   virtual void BlockUpdate() {}
   virtual void AllowUpdate() {}
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index 383a81f..b27888f 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.cpp
@@ -137,22 +137,6 @@ void AggregationHandleAvg::aggregateValueAccessorIntoHashTable(
     AggregationStateHashTableBase *hash_table) const {
   DCHECK_EQ(1u, argument_ids.size())
       << "Got wrong number of arguments for AVG: " << argument_ids.size();
-/*  aggregateValueAccessorIntoHashTableUnaryHelper<
-      AggregationHandleAvg,
-      AggregationStateAvg,
-      AggregationStateHashTable<AggregationStateAvg>>(
-          accessor,
-          argument_ids.front(),
-          group_by_key_ids,
-          blank_state_,
-          hash_table); */
-
-/*     static_cast<AggregationStateFastHashTable *>(hash_table)->upsertValueAccessorCompositeKeyFast(
-      argument_ids.front(),
-      accessor,
-      group_by_key_ids,
-      true,
-      const_cast<AggregationHandleAvg *>(this));*/
 }
 
 void AggregationHandleAvg::mergeStates(
@@ -214,20 +198,14 @@ AggregationState* AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle
 
 void AggregationHandleAvg::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table, int index) const {
+    AggregationStateHashTableBase *aggregation_hash_table,
+    int index) const {
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleAvg,
       AggregationStateFastHashTable>(
           distinctify_hash_table,
-          aggregation_hash_table, index);
-}
-
-void AggregationHandleAvg::mergeGroupByHashTables(
-    const AggregationStateHashTableBase &source_hash_table,
-    AggregationStateHashTableBase *destination_hash_table) const {
-  mergeGroupByHashTablesHelperFast<AggregationHandleAvg,
-                               AggregationStateFastHashTable>(
-      source_hash_table, destination_hash_table);
+          aggregation_hash_table,
+          index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index 15835e0..d134620 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -132,7 +132,7 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     ++(*count_ptr);
   }
 
-  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const override {
      if (block_update) return;
      iterateUnaryInlFast(arguments.front(), byte_ptr);
   }
@@ -145,7 +145,7 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
       block_update = false;
   }
 
-  void initPayload(uint8_t *byte_ptr) override {
+  void initPayload(uint8_t *byte_ptr) const override {
     TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
     std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr + blank_state_.count_offset);
     *sum_ptr = blank_state_.sum_;
@@ -217,11 +217,8 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table, int index) const override;
-
-  void mergeGroupByHashTables(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const override;
+      AggregationStateHashTableBase *aggregation_hash_table,
+      int index) const override;
 
   size_t getPayloadSize() const override {
       return blank_state_.getPayloadSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.cpp b/expressions/aggregation/AggregationHandleCount.cpp
index 3a333ef..aff3c28 100644
--- a/expressions/aggregation/AggregationHandleCount.cpp
+++ b/expressions/aggregation/AggregationHandleCount.cpp
@@ -135,26 +135,9 @@ template <bool count_star, bool nullable_type>
   if (count_star) {
     DCHECK_EQ(0u, argument_ids.size())
         << "Got wrong number of arguments for COUNT(*): " << argument_ids.size();
-/*    aggregateValueAccessorIntoHashTableNullaryHelper<
-        AggregationHandleCount<count_star, nullable_type>,
-        AggregationStateCount,
-        AggregationStateHashTable<AggregationStateCount>>(
-            accessor,
-            group_by_key_ids,
-            AggregationStateCount(),
-            hash_table);*/
   } else {
     DCHECK_EQ(1u, argument_ids.size())
         << "Got wrong number of arguments for COUNT: " << argument_ids.size();
-/*    aggregateValueAccessorIntoHashTableUnaryHelper<
-        AggregationHandleCount<count_star, nullable_type>,
-        AggregationStateCount,
-        AggregationStateHashTable<AggregationStateCount>>(
-            accessor,
-            argument_ids.front(),
-            group_by_key_ids,
-            AggregationStateCount(),
-            hash_table); */
   }
 }
 
@@ -206,7 +189,8 @@ template <bool count_star, bool nullable_type>
 void AggregationHandleCount<count_star, nullable_type>
     ::aggregateOnDistinctifyHashTableForGroupBy(
         const AggregationStateHashTableBase &distinctify_hash_table,
-        AggregationStateHashTableBase *aggregation_hash_table, int index) const {
+        AggregationStateHashTableBase *aggregation_hash_table,
+        int index) const {
   DCHECK_EQ(count_star, false);
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleCount<count_star, nullable_type>,
@@ -216,16 +200,6 @@ void AggregationHandleCount<count_star, nullable_type>
           index);
 }
 
-template <bool count_star, bool nullable_type>
-void AggregationHandleCount<count_star, nullable_type>::mergeGroupByHashTables(
-    const AggregationStateHashTableBase &source_hash_table,
-    AggregationStateHashTableBase *destination_hash_table) const {
-  mergeGroupByHashTablesHelperFast<
-      AggregationHandleCount,
-      AggregationStateFastHashTable>(source_hash_table,
-                                                        destination_hash_table);
-}
-
 // Explicitly instantiate and compile in the different versions of
 // AggregationHandleCount we need. Note that we do not compile a version with
 // 'count_star == true' and 'nullable_type == true', as that combination is

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index a95cae5..2beb0e2 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.hpp
@@ -113,7 +113,7 @@ class AggregationHandleCount : public AggregationConcreteHandle {
     state->count_.fetch_add(1, std::memory_order_relaxed);
   }
 
-  inline void iterateNullaryInlFast(uint8_t *byte_ptr) {
+  inline void iterateNullaryInlFast(uint8_t *byte_ptr) const {
       std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
       (*count_ptr)++;
   }
@@ -134,7 +134,7 @@ class AggregationHandleCount : public AggregationConcreteHandle {
     }
   }
 
-  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const override {
      if (block_update) return;
      if (arguments.size())
          iterateUnaryInlFast(arguments.front(), byte_ptr);
@@ -150,7 +150,7 @@ class AggregationHandleCount : public AggregationConcreteHandle {
      block_update = false;
   }
 
-  void initPayload(uint8_t *byte_ptr) override {
+  void initPayload(uint8_t *byte_ptr) const override {
      std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
      *count_ptr = 0;
   }
@@ -217,11 +217,8 @@ class AggregationHandleCount : public AggregationConcreteHandle {
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table, int index) const override;
-
-  void mergeGroupByHashTables(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const override;
+      AggregationStateHashTableBase *aggregation_hash_table,
+      int index) const override;
 
   size_t getPayloadSize() const override {
       return sizeof(std::int64_t);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleDistinct.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.cpp b/expressions/aggregation/AggregationHandleDistinct.cpp
index a5fc095..0dc8b56 100644
--- a/expressions/aggregation/AggregationHandleDistinct.cpp
+++ b/expressions/aggregation/AggregationHandleDistinct.cpp
@@ -65,14 +65,15 @@ void AggregationHandleDistinct::aggregateValueAccessorIntoHashTable(
 
 ColumnVector* AggregationHandleDistinct::finalizeHashTable(
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys, int index) const {
+    std::vector<std::vector<TypedValue>> *group_by_keys,
+    int index) const {
   DCHECK(group_by_keys->empty());
 
   const auto keys_retriever = [&group_by_keys](std::vector<TypedValue> &group_by_key,
                                                const bool &dumb_placeholder) -> void {
     group_by_keys->emplace_back(std::move(group_by_key));
   };
-  static_cast<const AggregationStateFastHashTable&>(hash_table).forEachCompositeKey(&keys_retriever);
+  static_cast<const AggregationStateFastHashTable&>(hash_table).forEachCompositeKeyFast(&keys_retriever);
 
   return nullptr;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleDistinct.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.hpp b/expressions/aggregation/AggregationHandleDistinct.hpp
index f6ef0c7..978319b 100644
--- a/expressions/aggregation/AggregationHandleDistinct.hpp
+++ b/expressions/aggregation/AggregationHandleDistinct.hpp
@@ -90,7 +90,8 @@ class AggregationHandleDistinct : public AggregationConcreteHandle {
 
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *groupby_hash_table, int index) const override {
+      AggregationStateHashTableBase *groupby_hash_table,
+      int index) const override {
     LOG(FATAL) << "AggregationHandleDistinct does not support "
                << "aggregateOnDistinctifyHashTableForGroupBy().";
   }
@@ -109,14 +110,8 @@ class AggregationHandleDistinct : public AggregationConcreteHandle {
 
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys, int index) const override;
-
-  void mergeGroupByHashTables(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const override {
-    LOG(FATAL)
-        << "AggregationHandleDistinct does not support mergeGroupByHashTables";
-  }
+      std::vector<std::vector<TypedValue>> *group_by_keys,
+      int index) const override;
 
  private:
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleDistinct);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.cpp b/expressions/aggregation/AggregationHandleMax.cpp
index c11fcc8..ec3e671 100644
--- a/expressions/aggregation/AggregationHandleMax.cpp
+++ b/expressions/aggregation/AggregationHandleMax.cpp
@@ -88,16 +88,6 @@ void AggregationHandleMax::aggregateValueAccessorIntoHashTable(
     AggregationStateHashTableBase *hash_table) const {
   DCHECK_EQ(1u, argument_ids.size())
       << "Got wrong number of arguments for MAX: " << argument_ids.size();
-
-/*  aggregateValueAccessorIntoHashTableUnaryHelper<
-      AggregationHandleMax,
-      AggregationStateMax,
-      AggregationStateHashTable<AggregationStateMax>>(
-          accessor,
-          argument_ids.front(),
-          group_by_key_ids,
-          AggregationStateMax(type_),
-          hash_table);*/
 }
 
 void AggregationHandleMax::mergeStates(
@@ -143,20 +133,14 @@ AggregationState* AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle
 
 void AggregationHandleMax::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table, int index) const {
+    AggregationStateHashTableBase *aggregation_hash_table,
+    int index) const {
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleMax,
       AggregationStateFastHashTable>(
           distinctify_hash_table,
-          aggregation_hash_table, index);
-}
-
-void AggregationHandleMax::mergeGroupByHashTables(
-    const AggregationStateHashTableBase &source_hash_table,
-    AggregationStateHashTableBase *destination_hash_table) const {
-  mergeGroupByHashTablesHelperFast<AggregationHandleMax,
-                               AggregationStateFastHashTable>(
-      source_hash_table, destination_hash_table);
+          aggregation_hash_table,
+          index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index 82d6ebb..c1a3a36 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -112,7 +112,7 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     compareAndUpdateFast(max_ptr, value);
   }
 
-  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const override {
     if (block_update) return;
     iterateUnaryInlFast(arguments.front(), byte_ptr);
   }
@@ -125,7 +125,7 @@ class AggregationHandleMax : public AggregationConcreteHandle {
       block_update = false;
   }
 
-  void initPayload(uint8_t *byte_ptr) override {
+  void initPayload(uint8_t *byte_ptr) const override {
     TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
     TypedValue t1 = (type_.getNullableVersion().makeNullValue());
     *max_ptr = t1;
@@ -184,11 +184,8 @@ class AggregationHandleMax : public AggregationConcreteHandle {
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table, int index) const override;
-
-  void mergeGroupByHashTables(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const override;
+      AggregationStateHashTableBase *aggregation_hash_table,
+      int index) const override;
 
   size_t getPayloadSize() const override {
       return sizeof(TypedValue);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.cpp b/expressions/aggregation/AggregationHandleMin.cpp
index 70d6c1b..cc714a5 100644
--- a/expressions/aggregation/AggregationHandleMin.cpp
+++ b/expressions/aggregation/AggregationHandleMin.cpp
@@ -88,16 +88,6 @@ void AggregationHandleMin::aggregateValueAccessorIntoHashTable(
     AggregationStateHashTableBase *hash_table) const {
   DCHECK_EQ(1u, argument_ids.size())
       << "Got wrong number of arguments for MIN: " << argument_ids.size();
-
-/*  aggregateValueAccessorIntoHashTableUnaryHelper<
-      AggregationHandleMin,
-      AggregationStateMin,
-      AggregationStateHashTable<AggregationStateMin>>(
-          accessor,
-          argument_ids.front(),
-          group_by_key_ids,
-          AggregationStateMin(type_),
-          hash_table);*/
 }
 
 void AggregationHandleMin::mergeStates(
@@ -144,20 +134,14 @@ AggregationState* AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle
 
 void AggregationHandleMin::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table, int index) const {
+    AggregationStateHashTableBase *aggregation_hash_table,
+    int index) const {
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleMin,
       AggregationStateFastHashTable>(
           distinctify_hash_table,
-          aggregation_hash_table, index);
-}
-
-void AggregationHandleMin::mergeGroupByHashTables(
-    const AggregationStateHashTableBase &source_hash_table,
-    AggregationStateHashTableBase *destination_hash_table) const {
-  mergeGroupByHashTablesHelperFast<AggregationHandleMin,
-                               AggregationStateFastHashTable>(
-      source_hash_table, destination_hash_table);
+          aggregation_hash_table,
+          index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index 0f5e3a1..4e0c72b 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -116,7 +116,7 @@ class AggregationHandleMin : public AggregationConcreteHandle {
       compareAndUpdateFast(min_ptr, value);
   }
 
-  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const override {
     if (block_update) return;
     iterateUnaryInlFast(arguments.front(), byte_ptr);
   }
@@ -129,7 +129,7 @@ class AggregationHandleMin : public AggregationConcreteHandle {
       block_update = false;
   }
 
-  void initPayload(uint8_t *byte_ptr) override {
+  void initPayload(uint8_t *byte_ptr) const override {
     TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
     TypedValue t1 = (type_.getNullableVersion().makeNullValue());
     *min_ptr = t1;
@@ -187,11 +187,8 @@ class AggregationHandleMin : public AggregationConcreteHandle {
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table, int index) const override;
-
-  void mergeGroupByHashTables(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const override;
+      AggregationStateHashTableBase *aggregation_hash_table,
+      int index) const override;
 
   size_t getPayloadSize() const override {
       return sizeof(TypedValue);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 534db30..1b0bbcd 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -125,16 +125,6 @@ void AggregationHandleSum::aggregateValueAccessorIntoHashTable(
     AggregationStateHashTableBase *hash_table) const {
   DCHECK_EQ(1u, argument_ids.size())
       << "Got wrong number of arguments for SUM: " << argument_ids.size();
-
-/*  aggregateValueAccessorIntoHashTableUnaryHelper<
-      AggregationHandleSum,
-      AggregationStateSum,
-      AggregationStateHashTable<AggregationStateSum>>(
-          accessor,
-          argument_ids.front(),
-          group_by_key_ids,
-          blank_state_,
-          hash_table);*/
 }
 
 void AggregationHandleSum::mergeStates(
@@ -192,7 +182,8 @@ AggregationState* AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle
 
 void AggregationHandleSum::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table, int index) const {
+    AggregationStateHashTableBase *aggregation_hash_table,
+    int index) const {
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleSum,
       AggregationStateFastHashTable>(
@@ -201,12 +192,4 @@ void AggregationHandleSum::aggregateOnDistinctifyHashTableForGroupBy(
           index);
 }
 
-void AggregationHandleSum::mergeGroupByHashTables(
-    const AggregationStateHashTableBase &source_hash_table,
-    AggregationStateHashTableBase *destination_hash_table) const {
-  mergeGroupByHashTablesHelperFast<AggregationHandleSum,
-                               AggregationStateFastHashTable>(
-      source_hash_table, destination_hash_table);
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index 3a2252d..3e1de48 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -127,7 +127,7 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     *null_ptr = false;
   }
 
-  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const override {
      if (block_update) return;
      iterateUnaryInlFast(arguments.front(), byte_ptr);
   }
@@ -140,7 +140,7 @@ class AggregationHandleSum : public AggregationConcreteHandle {
       block_update = false;
   }
 
-  void initPayload(uint8_t *byte_ptr) override {
+  void initPayload(uint8_t *byte_ptr) const override {
     TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
     bool *null_ptr = reinterpret_cast<bool *>(byte_ptr + blank_state_.null_offset);
     *sum_ptr = blank_state_.sum_;
@@ -182,7 +182,8 @@ class AggregationHandleSum : public AggregationConcreteHandle {
 
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys, int index) const override;
+      std::vector<std::vector<TypedValue>> *group_by_keys,
+      int index) const override;
 
   /**
    * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
@@ -197,11 +198,8 @@ class AggregationHandleSum : public AggregationConcreteHandle {
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table, int index) const override;
-
-  void mergeGroupByHashTables(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const override;
+      AggregationStateHashTableBase *aggregation_hash_table,
+      int index) const override;
 
   size_t getPayloadSize() const override {
       return blank_state_.getPayloadSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index 9de6833..30f9784 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -280,45 +280,45 @@ target_link_libraries(quickstep_expressions_aggregation
 # Tests:
 
 # Unified executable to ammortize cost of linking.
-add_executable(AggregationHandle_tests
-               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleAvg_unittest.cpp"
-               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleCount_unittest.cpp"
-               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleMax_unittest.cpp"
-               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleMin_unittest.cpp"
-               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleSum_unittest.cpp")
-target_link_libraries(AggregationHandle_tests
-                      gtest
-                      gtest_main
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregateFunction
-                      quickstep_expressions_aggregation_AggregateFunctionFactory
-                      quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_expressions_aggregation_AggregationHandleAvg
-                      quickstep_expressions_aggregation_AggregationHandleCount
-                      quickstep_expressions_aggregation_AggregationHandleMax
-                      quickstep_expressions_aggregation_AggregationHandleMin
-                      quickstep_expressions_aggregation_AggregationHandleSum
-                      quickstep_expressions_aggregation_AggregationID
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_StorageManager
-                      quickstep_types_CharType
-                      quickstep_types_DateOperatorOverloads
-                      quickstep_types_DatetimeIntervalType
-                      quickstep_types_DatetimeType
-                      quickstep_types_DoubleType
-                      quickstep_types_FloatType
-                      quickstep_types_IntType
-                      quickstep_types_IntervalLit
-                      quickstep_types_LongType
-                      quickstep_types_Type
-                      quickstep_types_TypeFactory
-                      quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_VarCharType
-                      quickstep_types_YearMonthIntervalType
-                      quickstep_types_containers_ColumnVector
-                      quickstep_types_containers_ColumnVectorsValueAccessor
-                      quickstep_types_operations_comparisons_Comparison
-                      quickstep_types_operations_comparisons_ComparisonFactory
-                      quickstep_types_operations_comparisons_ComparisonID)
+# add_executable(AggregationHandle_tests
+#               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleAvg_unittest.cpp"
+#               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleCount_unittest.cpp"
+#               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleMax_unittest.cpp"
+#               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleMin_unittest.cpp"
+#               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleSum_unittest.cpp")
+# target_link_libraries(AggregationHandle_tests
+#                      gtest
+#                      gtest_main
+#                      quickstep_catalog_CatalogTypedefs
+#                      quickstep_expressions_aggregation_AggregateFunction
+#                      quickstep_expressions_aggregation_AggregateFunctionFactory
+#                      quickstep_expressions_aggregation_AggregationHandle
+#                      quickstep_expressions_aggregation_AggregationHandleAvg
+#                      quickstep_expressions_aggregation_AggregationHandleCount
+#                      quickstep_expressions_aggregation_AggregationHandleMax
+#                      quickstep_expressions_aggregation_AggregationHandleMin
+#                      quickstep_expressions_aggregation_AggregationHandleSum
+#                      quickstep_expressions_aggregation_AggregationID
+#                      quickstep_storage_HashTableBase
+#                      quickstep_storage_StorageManager
+#                      quickstep_types_CharType
+#                      quickstep_types_DateOperatorOverloads
+#                      quickstep_types_DatetimeIntervalType
+#                      quickstep_types_DatetimeType
+#                      quickstep_types_DoubleType
+#                      quickstep_types_FloatType
+#                      quickstep_types_IntType
+#                      quickstep_types_IntervalLit
+#                      quickstep_types_LongType
+#                      quickstep_types_Type
+#                      quickstep_types_TypeFactory
+#                      quickstep_types_TypeID
+#                      quickstep_types_TypedValue
+#                      quickstep_types_VarCharType
+#                      quickstep_types_YearMonthIntervalType
+#                      quickstep_types_containers_ColumnVector
+#                      quickstep_types_containers_ColumnVectorsValueAccessor
+#                      quickstep_types_operations_comparisons_Comparison
+#                      quickstep_types_operations_comparisons_ComparisonFactory
+#                      quickstep_types_operations_comparisons_ComparisonID)
 #add_test(AggregationHandle_tests AggregationHandle_tests)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 90b8fcc..05d0636 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -461,7 +461,7 @@ void AggregationOperationState::finalizeSingleState(InsertDestination *output_de
 
 void AggregationOperationState::mergeGroupByHashTables(AggregationStateHashTableBase *src,
                                                        AggregationStateHashTableBase *dst) {
-    HashTableMergerNewFast merger(dst);
+    HashTableMergerFast merger(dst);
     (static_cast<FastHashTable<true, false, true, false> *>(src))->forEachCompositeKeyFast(&merger);
 }
 
@@ -478,18 +478,15 @@ void AggregationOperationState::finalizeHashTable(InsertDestination *output_dest
   // e.g. Keep merging entries from smaller hash tables to larger.
 
   auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
-  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    if (hash_tables->size() > 1) {
-      for (int hash_table_index = 0;
-           hash_table_index < static_cast<int>(hash_tables->size() - 1);
-           ++hash_table_index) {
-        // Merge each hash table to the last hash table.
-        mergeGroupByHashTables(
-            (*hash_tables)[hash_table_index].get(),
-            hash_tables->back().get());
-      }
+  if (hash_tables->size() > 1) {
+    for (int hash_table_index = 0;
+         hash_table_index < static_cast<int>(hash_tables->size() - 1);
+         ++hash_table_index) {
+      // Merge each hash table to the last hash table.
+      mergeGroupByHashTables(
+          (*hash_tables)[hash_table_index].get(),
+          hash_tables->back().get());
     }
-    break;
   }
 
   // Collect per-aggregate finalized values.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3606000b/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
index c659a20..909fcc0 100644
--- a/storage/FastHashTable.hpp
+++ b/storage/FastHashTable.hpp
@@ -236,8 +236,6 @@ class FastHashTable : public HashTableBase<resizable,
    *         resizable is false and storage space for the hash table has been
    *         exhausted.
    **/
-  HashTablePutResult putCompositeKey(const std::vector<TypedValue> &key,
-                                     const uint8_t &value);
 
   HashTablePutResult putCompositeKeyFast(const std::vector<TypedValue> &key,
                                      const uint8_t *value_ptr);
@@ -380,7 +378,7 @@ class FastHashTable : public HashTableBase<resizable,
    **/
   template <typename FunctorT>
   bool upsert(const TypedValue &key,
-              const uint8_t &initial_value,
+              const uint8_t *initial_value_ptr,
               FunctorT *functor);
 
   /**
@@ -422,12 +420,6 @@ class FastHashTable : public HashTableBase<resizable,
    *         enough space to insert a new entry in this HashTable.
    **/
   template <typename FunctorT>
-  bool upsertCompositeKey(const std::vector<TypedValue> &key,
-                          const uint8_t &initial_value,
-                          FunctorT *functor);
-
-
-  template <typename FunctorT>
   bool upsertCompositeKeyFast(const std::vector<TypedValue> &key,
                           const uint8_t *init_value_ptr,
                           FunctorT *functor);
@@ -435,9 +427,10 @@ class FastHashTable : public HashTableBase<resizable,
   template <typename FunctorT>
   bool upsertCompositeKeyFast(const std::vector<TypedValue> &key,
                           const uint8_t *init_value_ptr,
-                          FunctorT *functor, int index);
+                          FunctorT *functor,
+                          int index);
 
-  bool upsertCompositeKeyNewFast(const std::vector<TypedValue> &key,
+  bool upsertCompositeKeyFast(const std::vector<TypedValue> &key,
                           const uint8_t *init_value_ptr,
                           const uint8_t *source_state);
 
@@ -488,14 +481,6 @@ class FastHashTable : public HashTableBase<resizable,
    *         accessor's iteration will be left on the first tuple which could
    *         not be inserted).
    **/
-  template <typename FunctorT>
-  bool upsertValueAccessor(ValueAccessor *accessor,
-                           const attribute_id key_attr_id,
-                           const bool check_for_null_keys,
-                           const uint8_t &initial_value,
-                           FunctorT *functor);
-
-
   bool upsertValueAccessorFast(const std::vector<std::vector<attribute_id>> &argument_ids,
                            ValueAccessor *accessor,
                            const attribute_id key_attr_id,
@@ -548,14 +533,6 @@ class FastHashTable : public HashTableBase<resizable,
    *         accessor's iteration will be left on the first tuple which could
    *         not be inserted).
    **/
-  template <typename FunctorT>
-  bool upsertValueAccessorCompositeKey(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys,
-      const uint8_t &initial_value,
-      FunctorT *functor);
-
   bool upsertValueAccessorCompositeKeyFast(
       const std::vector<std::vector<attribute_id>> &argument,
       ValueAccessor *accessor,
@@ -631,7 +608,8 @@ class FastHashTable : public HashTableBase<resizable,
    *         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 uint8_t* getSingleCompositeKey(const std::vector<TypedValue> &key,
+                                               int index) const = 0;
 
   /**
    * @brief Lookup a key against this hash table to find matching entries.
@@ -1002,13 +980,12 @@ class FastHashTable : public HashTableBase<resizable,
    * @return The number of key-value pairs visited.
    **/
   template <typename FunctorT>
-  std::size_t forEachCompositeKey(FunctorT *functor) const;
-
-  template <typename FunctorT>
   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
    *        during the build phase of this hash table.
@@ -1196,10 +1173,6 @@ class FastHashTable : public HashTableBase<resizable,
                                          const std::size_t variable_key_size,
                                          const uint8_t &value,
                                          HashTablePreallocationState *prealloc_state) = 0;
-  virtual HashTablePutResult putCompositeKeyInternal(const std::vector<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,
@@ -1213,15 +1186,9 @@ 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* upsertInternal(const TypedValue &key,
-                                 const std::size_t variable_key_size,
-                                 const uint8_t &initial_value) = 0;
   virtual uint8_t* upsertInternalFast(const TypedValue &key,
-                                 const std::uint8_t *init_value_ptr,
-                                 const std::size_t variable_key_size) = 0;
-  virtual uint8_t* upsertCompositeKeyInternal(const std::vector<TypedValue> &key,
-                                             const std::size_t variable_key_size,
-                                             const uint8_t &initial_value) = 0;
+                                 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,
@@ -1423,31 +1390,6 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 HashTablePutResult FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::putCompositeKey(const std::vector<TypedValue> &key,
-                      const uint8_t& value) {
-  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 = putCompositeKeyInternal(key, variable_size, value, nullptr);
-      }
-      if (result == HashTablePutResult::kOutOfSpace) {
-        resize(0, variable_size);
-      }
-    }
-    return result;
-  } else {
-    return putCompositeKeyInternal(key, variable_size, value, nullptr);
-  }
-}
-
-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);
@@ -1713,7 +1655,7 @@ template <bool resizable,
 template <typename FunctorT>
 bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
     ::upsert(const TypedValue &key,
-             const uint8_t &initial_value,
+             const 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;
@@ -1721,7 +1663,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
     for (;;) {
       {
         SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
-        uint8_t *value = upsertInternal(key, variable_size, initial_value);
+        uint8_t *value = upsertInternalFast(key, variable_size, initial_value_ptr);
         if (value != nullptr) {
           (*functor)(value);
           return true;
@@ -1730,41 +1672,7 @@ 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 = upsertInternal(key, variable_size, initial_value);
-    if (value == nullptr) {
-      return false;
-    } else {
-      (*functor)(value);
-      return true;
-    }
-  }
-}
-
-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>
-    ::upsertCompositeKey(const std::vector<TypedValue> &key,
-                         const uint8_t &initial_value,
-                         FunctorT *functor) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  const std::size_t variable_size = calculateVariableLengthCompositeKeyCopySize(key);
-  if (resizable) {
-    for (;;) {
-      {
-        SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
-        uint8_t *value = upsertCompositeKeyInternal(key, variable_size, initial_value);
-        if (value != nullptr) {
-          (*functor)(value);
-          return true;
-        }
-      }
-      resize(0, variable_size);
-    }
-  } else {
-    uint8_t *value = upsertCompositeKeyInternal(key, variable_size, initial_value);
+    uint8_t *value = upsertInternalFast(key, variable_size, initial_value_ptr);
     if (value == nullptr) {
       return false;
     } else {
@@ -1774,7 +1682,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
   }
 }
 
-class HashTableMergerNewFast {
+class HashTableMergerFast {
  public:
   /**
    * @brief Constructor
@@ -1783,7 +1691,7 @@ class HashTableMergerNewFast {
    * @param destination_hash_table The destination hash table to which other
    *        hash tables will be merged.
    **/
-  explicit HashTableMergerNewFast(AggregationStateHashTableBase *destination_hash_table)
+  explicit HashTableMergerFast(AggregationStateHashTableBase *destination_hash_table)
       : destination_hash_table_(static_cast<FastHashTable<true, false, true, false> *>(destination_hash_table)) {}
 
   /**
@@ -1801,7 +1709,7 @@ class HashTableMergerNewFast {
       // The CHECK is required as upsertCompositeKey can return false if the
       // hash table runs out of space during the upsert process. The ideal
       // solution will be to retry again if the upsert fails.
-      CHECK(destination_hash_table_->upsertCompositeKeyNewFast(
+      CHECK(destination_hash_table_->upsertCompositeKeyFast(
           group_by_key, original_state, source_state));
     } else {
       destination_hash_table_->putCompositeKeyFast(group_by_key, source_state);
@@ -1811,7 +1719,7 @@ class HashTableMergerNewFast {
  private:
   FastHashTable<true, false, true, false> *destination_hash_table_;
 
-  DISALLOW_COPY_AND_ASSIGN(HashTableMergerNewFast);
+  DISALLOW_COPY_AND_ASSIGN(HashTableMergerFast);
 };
 
 
@@ -1857,7 +1765,8 @@ 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) {
+                         FunctorT *functor,
+                         int index) {
   DEBUG_ASSERT(!allow_duplicate_keys);
   const std::size_t variable_size = calculateVariableLengthCompositeKeyCopySize(key);
   if (resizable) {
@@ -1889,7 +1798,7 @@ template <bool resizable,
           bool force_key_copy,
           bool allow_duplicate_keys>
 bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::upsertCompositeKeyNewFast(const std::vector<TypedValue> &key,
+    ::upsertCompositeKeyFast(const std::vector<TypedValue> &key,
                          const std::uint8_t *init_value_ptr,
                          const std::uint8_t *source_state) {
   DEBUG_ASSERT(!allow_duplicate_keys);
@@ -1927,68 +1836,6 @@ 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>
-    ::upsertValueAccessor(ValueAccessor *accessor,
-                          const attribute_id key_attr_id,
-                          const bool check_for_null_keys,
-                          const uint8_t &initial_value,
-                          FunctorT *functor) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  std::size_t variable_size;
-  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_);
-          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->upsertInternal(key, variable_size, initial_value);
-            if (value == nullptr) {
-              continuing = true;
-              break;
-            } else {
-              (*functor)(*accessor, value);
-            }
-          }
-        }
-        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->upsertInternal(key, variable_size, initial_value);
-        if (value == nullptr) {
-          return false;
-        } else {
-          (*functor)(*accessor, value);
-        }
-      }
-    }
-
-    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>
     ::upsertValueAccessorFast(const std::vector<std::vector<attribute_id>> &argument_ids,
                           ValueAccessor *accessor,
@@ -2012,7 +1859,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
               continue;
             }
             variable_size = (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0;
-            uint8_t *value = this->upsertInternalFast(key, nullptr, variable_size);
+            uint8_t *value = this->upsertInternalFast(key, variable_size, nullptr);
             if (value == nullptr) {
               continuing = true;
               break;
@@ -2040,7 +1887,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
           continue;
         }
         variable_size = (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0;
-        uint8_t *value = this->upsertInternalFast(key, nullptr, variable_size);
+        uint8_t *value = this->upsertInternalFast(key, variable_size, nullptr);
         if (value == nullptr) {
           return false;
         } else {
@@ -2064,78 +1911,6 @@ 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>
-    ::upsertValueAccessorCompositeKey(ValueAccessor *accessor,
-                                      const std::vector<attribute_id> &key_attr_ids,
-                                      const bool check_for_null_keys,
-                                      const uint8_t &initial_value,
-                                      FunctorT *functor) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  std::size_t variable_size;
-  std::vector<TypedValue> key_vector;
-  key_vector.resize(key_attr_ids.size());
-  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_);
-          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->upsertCompositeKeyInternal(key_vector,
-                                                             variable_size,
-                                                             initial_value);
-            if (value == nullptr) {
-              continuing = true;
-              break;
-            } else {
-              (*functor)(*accessor, value);
-            }
-          }
-        }
-        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->upsertCompositeKeyInternal(key_vector,
-                                                         variable_size,
-                                                         initial_value);
-        if (value == nullptr) {
-          return false;
-        } else {
-          (*functor)(*accessor, value);
-        }
-      }
-    }
-
-    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,
@@ -2514,25 +2289,6 @@ template <bool resizable,
           bool allow_duplicate_keys>
 template <typename FunctorT>
 std::size_t FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
-    ::forEachCompositeKey(FunctorT *functor) const {
-  std::size_t entries_visited = 0;
-  std::size_t entry_num = 0;
-  std::vector<TypedValue> key;
-  const uint8_t *value_ptr;
-  while (getNextEntryCompositeKey(&key, &value_ptr, &entry_num)) {
-    ++entries_visited;
-    (*functor)(key, *value_ptr);
-    key.clear();
-  }
-  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) const {
   std::size_t entries_visited = 0;
   std::size_t entry_num = 0;
@@ -2553,7 +2309,8 @@ template <bool resizable,
           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 {
+    ::forEachCompositeKeyFast(FunctorT *functor,
+                              int index) const {
   std::size_t entries_visited = 0;
   std::size_t entry_num = 0;
   std::vector<TypedValue> key;


[22/28] incubator-quickstep git commit: Initial commit for partitioned hash tables.

Posted by hb...@apache.org.
Initial commit for partitioned hash tables.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/8f3e1d0e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/8f3e1d0e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/8f3e1d0e

Branch: refs/heads/partitioned-aggregation
Commit: 8f3e1d0ec507665c24d2a6b08b2516b4946053f3
Parents: 95a46bb
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Mon Aug 15 12:21:19 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:57:06 2016 -0500

----------------------------------------------------------------------
 storage/AggregationOperationState.hpp |   1 +
 storage/CMakeLists.txt                |  11 ++
 storage/PartitionedHashTablePool.hpp  | 201 +++++++++++++++++++++++++++++
 3 files changed, 213 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8f3e1d0e/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index cbbfc22..805a347 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -32,6 +32,7 @@
 #include "storage/AggregationOperationState.pb.h"
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTablePool.hpp"
+#include "storage/PartitionedHashTablePool.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "utility/Macros.hpp"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8f3e1d0e/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index f05cc46..3d6e8d4 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -235,6 +235,7 @@ add_library(quickstep_storage_PackedRowStoreTupleStorageSubBlock
 add_library(quickstep_storage_PackedRowStoreValueAccessor
             ../empty_src.cpp
             PackedRowStoreValueAccessor.hpp)
+add_library(quickstep_storage_PartitionedHashTablePool ../empty_src.cpp PartitionedHashTablePool.hpp)
 add_library(quickstep_storage_PreloaderThread PreloaderThread.cpp PreloaderThread.hpp)
 add_library(quickstep_storage_SMAIndexSubBlock SMAIndexSubBlock.cpp SMAIndexSubBlock.hpp)
 add_library(quickstep_storage_SeparateChainingHashTable ../empty_src.cpp SeparateChainingHashTable.hpp)
@@ -289,6 +290,7 @@ target_link_libraries(quickstep_storage_AggregationOperationState
                       quickstep_storage_HashTableFactory
                       quickstep_storage_HashTablePool
                       quickstep_storage_InsertDestination
+                      quickstep_storage_PartitionedHashTablePool
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
@@ -850,6 +852,14 @@ target_link_libraries(quickstep_storage_PackedRowStoreValueAccessor
                       quickstep_types_TypedValue
                       quickstep_utility_BitVector
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_storage_PartitionedHashTablePool
+                      glog
+                      quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_FastHashTable
+                      quickstep_storage_FastHashTableFactory
+                      quickstep_storage_HashTableBase
+                      quickstep_utility_Macros
+                      quickstep_utility_StringUtil)
 target_link_libraries(quickstep_storage_PreloaderThread
                       glog
                       quickstep_catalog_CatalogDatabase
@@ -1167,6 +1177,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_LinearOpenAddressingHashTable
                       quickstep_storage_PackedRowStoreTupleStorageSubBlock
                       quickstep_storage_PackedRowStoreValueAccessor
+                      quickstep_storage_PartitionedHashTablePool
                       quickstep_storage_PreloaderThread
                       quickstep_storage_SMAIndexSubBlock
                       quickstep_storage_SeparateChainingHashTable

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8f3e1d0e/storage/PartitionedHashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/PartitionedHashTablePool.hpp b/storage/PartitionedHashTablePool.hpp
new file mode 100644
index 0000000..a71af44
--- /dev/null
+++ b/storage/PartitionedHashTablePool.hpp
@@ -0,0 +1,201 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_STORAGE_PARTITIONED_HASH_TABLE_POOL_HPP_
+#define QUICKSTEP_STORAGE_PARTITIONED_HASH_TABLE_POOL_HPP_
+
+#include <chrono>
+#include <memory>
+#include <utility>
+#include <vector>
+
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/FastHashTable.hpp"
+#include "storage/FastHashTableFactory.hpp"
+#include "utility/Macros.hpp"
+#include "utility/StringUtil.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class StorageManager;
+class Type;
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+/**
+ * @brief A pool of HashTables used for a single aggregation handle. Each
+ *        HashTable represents values from a given partition, which is
+ *        determined by the keys in the group by clause.
+ **/
+class PartitionedHashTablePool {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param estimated_num_entries The maximum number of entries in a hash table.
+   * @param num_partitions The number of partitions (i.e. number of HashTables)
+   * @param hash_table_impl_type The type of hash table implementation.
+   * @param group_by_types A vector of pointer of types which form the group by
+   *        key.
+   * @param agg_handle The aggregation handle.
+   * @param storage_manager A pointer to the storage manager.
+   *
+   * @note The estimate of number of entries is quite inaccurate at this time.
+   *       If we go by the current estimate, each hash table demands much
+   *       larger space than it actually needs, which causes the system to
+   *       either trigger evictions or worse - run out of memory. To fix this
+   *       issue, we divide the estimate by 100. The division will not affect
+   *       correctness, however it may allocate some hash tables smaller space
+   *       than their requirement, causing them to be resized during build
+   *       phase, which has a performance penalty.
+   **/
+  PartitionedHashTablePool(const std::size_t estimated_num_entries,
+                           const std::size_t num_partitions,
+                           const HashTableImplType hash_table_impl_type,
+                           const std::vector<const Type *> &group_by_types,
+                           AggregationHandle *agg_handle,
+                           StorageManager *storage_manager)
+      : estimated_num_entries_(
+            reduceEstimatedCardinality(estimated_num_entries, num_partitions)),
+        num_partitions_(num_partitions),
+        hash_table_impl_type_(hash_table_impl_type),
+        group_by_types_(group_by_types),
+        agg_handle_(DCHECK_NOTNULL(agg_handle)),
+        storage_manager_(DCHECK_NOTNULL(storage_manager)) {
+    initializeAllHashTables();
+  }
+
+  PartitionedHashTablePool(const std::size_t estimated_num_entries,
+                           const std::size_t num_partitions,
+                           const HashTableImplType hash_table_impl_type,
+                           const std::vector<const Type *> &group_by_types,
+                           const std::vector<std::size_t> &payload_sizes,
+                           const std::vector<AggregationHandle *> &handles,
+                           StorageManager *storage_manager)
+      : estimated_num_entries_(
+            reduceEstimatedCardinality(estimated_num_entries, num_partitions)),
+        num_partitions_(num_partitions),
+        hash_table_impl_type_(hash_table_impl_type),
+        group_by_types_(group_by_types),
+        payload_sizes_(payload_sizes),
+        handles_(handles),
+        storage_manager_(DCHECK_NOTNULL(storage_manager)) {
+    initializeAllHashTables();
+  }
+
+  /**
+   * @brief Check out a hash table for insertion.
+   *
+   * @param partition_id The ID of the partitioned HashTable.
+   *
+   * @return A hash table pointer for the given HashTable.
+   **/
+  AggregationStateHashTableBase* getHashTable(const std::size_t partition_id) {
+    DCHECK_LT(partition_id, num_partitions_);
+    DCHECK_LT(partition_id, hash_tables_.size());
+    return hash_tables_[partition_id].get();
+  }
+
+  /**
+   * @brief Check out a hash table for insertion.
+   *
+   * @param partition_id The ID of the partitioned HashTable.
+   *
+   * @return A hash table pointer for the given HashTable.
+   **/
+  AggregationStateHashTableBase* getHashTableFast(const std::size_t partition_id) {
+    DCHECK_LT(partition_id, num_partitions_);
+    DCHECK_LT(partition_id, hash_tables_.size());
+    return hash_tables_[partition_id].get();
+  }
+
+  /**
+   * @brief Get all the hash tables from the pool.
+   *
+   * @warning The caller should ensure that this call is made when no hash table
+   *          is being checked in or checked out from the pool. In other words
+   *          the hash table pool is in read-only state.
+   *
+   * @param All the hash tables in the pool.
+   *
+   **/
+  std::vector<std::unique_ptr<AggregationStateHashTableBase>>*
+      getAllHashTables() {
+    return &hash_tables_;
+  }
+
+ private:
+  void initializeAllHashTables() {
+    for (std::size_t part_num = 0; part_num < num_partitions_; ++part_num) {
+      AggregationStateHashTableBase *part_hash_table = createNewHashTableFast();
+      hash_tables_.push_back(
+          std::unique_ptr<AggregationStateHashTableBase>(part_hash_table));
+    }
+  }
+
+  AggregationStateHashTableBase* createNewHashTable() {
+    return agg_handle_->createGroupByHashTable(hash_table_impl_type_,
+                                               group_by_types_,
+                                               estimated_num_entries_,
+                                               storage_manager_);
+  }
+
+  AggregationStateHashTableBase* createNewHashTableFast() {
+    return AggregationStateFastHashTableFactory::CreateResizable(
+                hash_table_impl_type_,
+                group_by_types_,
+                estimated_num_entries_,
+                payload_sizes_,
+                handles_,
+                storage_manager_);
+  }
+
+  inline std::size_t reduceEstimatedCardinality(
+      const std::size_t original_estimate,
+      const std::size_t num_partitions) const {
+    CHECK_NE(num_partitions, 0u);
+    return original_estimate / num_partitions;
+  }
+
+  std::vector<std::unique_ptr<AggregationStateHashTableBase>> hash_tables_;
+
+  const std::size_t estimated_num_entries_;
+  const std::size_t num_partitions_;
+
+  const HashTableImplType hash_table_impl_type_;
+
+  const std::vector<const Type *> group_by_types_;
+
+  std::vector<std::size_t> payload_sizes_;
+
+  AggregationHandle *agg_handle_;
+  const std::vector<AggregationHandle *> handles_;
+  StorageManager *storage_manager_;
+
+  DISALLOW_COPY_AND_ASSIGN(PartitionedHashTablePool);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_HASH_TABLE_POOL_HPP_


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

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index 5526164..b908d6f 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -23,8 +23,8 @@
 #include <cstddef>
 #include <vector>
 
-#include "utility/Macros.hpp"
 #include "ValueAccessor.hpp"
+#include "utility/Macros.hpp"
 
 namespace quickstep {
 
@@ -57,11 +57,7 @@ struct HashTablePreallocationState {
  * @brief Codes which indicate the result of a call to put() or
  *        putCompositeKey().
  **/
-enum class HashTablePutResult {
-  kOK = 0,
-  kDuplicateKey,
-  kOutOfSpace
-};
+enum class HashTablePutResult { kOK = 0, kDuplicateKey, kOutOfSpace };
 
 /**
  * @brief An ultra-minimal base class that HashTables with different ValueT
@@ -76,17 +72,19 @@ template <bool resizable,
           bool allow_duplicate_keys>
 class HashTableBase {
  public:
-  virtual ~HashTableBase() {
-  }
+  virtual ~HashTableBase() {}
+
   virtual bool upsertValueAccessorCompositeKeyFast(
       const std::vector<std::vector<attribute_id>> &argument,
       ValueAccessor *accessor,
       const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys) {return false;}
- protected:
-  HashTableBase() {
+      const bool check_for_null_keys) {
+    return false;
   }
 
+ protected:
+  HashTableBase() {}
+
  private:
   DISALLOW_COPY_AND_ASSIGN(HashTableBase);
 };


[04/28] incubator-quickstep git commit: Initial commit for QUICKSTEP-28 and QUICKSTEP-29. Code refactoring and cleanup, some more optimizations are pending.

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/storage/FastHashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTableFactory.hpp b/storage/FastHashTableFactory.hpp
new file mode 100644
index 0000000..6ad3212
--- /dev/null
+++ b/storage/FastHashTableFactory.hpp
@@ -0,0 +1,300 @@
+/**
+ *   Copyright 2015-2016 Pivotal Software, Inc.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_STORAGE_FAST_HASH_TABLE_FACTORY_HPP_
+#define QUICKSTEP_STORAGE_FAST_HASH_TABLE_FACTORY_HPP_
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+#include "storage/HashTable.hpp"
+#include "storage/FastHashTable.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/HashTableFactory.hpp"
+#include "storage/HashTable.pb.h"
+#include "storage/LinearOpenAddressingHashTable.hpp"
+#include "storage/SeparateChainingHashTable.hpp"
+#include "storage/FastSeparateChainingHashTable.hpp"
+#include "storage/SimpleScalarSeparateChainingHashTable.hpp"
+#include "storage/TupleReference.hpp"
+#include "types/TypeFactory.hpp"
+#include "utility/BloomFilter.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class StorageManager;
+class Type;
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+/**
+ * @brief Templated all-static factory class that makes it easier to
+ *        instantiate HashTables with the particular HashTable implementation
+ *        chosen at runtime. All template parameters are exactly the same as
+ *        those of HashTable.
+ **/
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+class FastHashTableFactory {
+ public:
+  /**
+   * @brief Create a new resizable HashTable, with the type selected by
+   *        hash_table_type. Other parameters are forwarded to the HashTable's
+   *        constructor.
+   *
+   * @param hash_table_type The specific HashTable implementation that should
+   *        be used.
+   * @param key_types A vector of one or more types (>1 indicates a composite
+   *        key). Forwarded as-is to the HashTable's constructor.
+   * @param num_entries The estimated number of entries the HashTable will
+   *        hold. Forwarded as-is to the HashTable's constructor.
+   * @param storage_manager The StorageManager to use (a StorageBlob will be
+   *        allocated to hold the HashTable's contents). Forwarded as-is to the
+   *        HashTable's constructor.
+   * @return A new resizable HashTable.
+   **/
+  static FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>*
+      CreateResizable(const HashTableImplType hash_table_type,
+                      const std::vector<const Type*> &key_types,
+                      const std::size_t num_entries,
+                      const std::vector<std::size_t> &payload_sizes,
+                      const std::vector<AggregationHandle *> &handles,
+                      StorageManager *storage_manager) {
+    DCHECK(resizable);
+
+    switch (hash_table_type) {
+      case HashTableImplType::kSeparateChaining:
+        return new FastSeparateChainingHashTable<
+            resizable,
+            serializable,
+            force_key_copy,
+            allow_duplicate_keys>(key_types, num_entries, payload_sizes, handles, storage_manager);
+      case HashTableImplType::kLinearOpenAddressing:
+/*        return new LinearOpenAddressingHashTable<
+            ValueT,
+            resizable,
+            serializable,
+            force_key_copy,
+            allow_duplicate_keys>(key_types, num_entries, storage_manager);*/
+        return new FastSeparateChainingHashTable<
+            resizable,
+            serializable,
+            force_key_copy,
+            allow_duplicate_keys>(key_types, num_entries, payload_sizes, handles, storage_manager);
+      case HashTableImplType::kSimpleScalarSeparateChaining:
+        return new FastSeparateChainingHashTable<
+            resizable,
+            serializable,
+            force_key_copy,
+            allow_duplicate_keys>(key_types, num_entries, payload_sizes, handles, storage_manager);
+/*        return new SimpleScalarSeparateChainingHashTable<
+            ValueT,
+            resizable,
+            serializable,
+            force_key_copy,
+            allow_duplicate_keys>(key_types, num_entries, storage_manager);*/
+      default: {
+        LOG(FATAL) << "Unrecognized HashTableImplType in HashTableFactory::createResizable()\n";
+      }
+    }
+  }
+
+  /**
+   * @brief Create a new fixed-sized HashTable, with the type selected by
+   *        hash_table_type. Other parameters are forwarded to the HashTables's
+   *        constructor.
+   *
+   * @param hash_table_type The specific HashTable implementation that should
+   *        be used.
+   * @param key_types A vector of one or more types (>1 indicates a composite
+   *        key). Forwarded as-is to the HashTable's constructor.
+   * @param hash_table_memory A pointer to memory to use for the HashTable.
+   *        Forwarded as-is to the HashTable's constructor.
+   * @param hash_table_memory_size The size of hash_table_memory in bytes.
+   *        Forwarded as-is to the HashTable's constructor.
+   * @param new_hash_table If true, the HashTable is being constructed for the
+   *        first time and hash_table_memory will be cleared. If false, reload
+   *        a pre-existing HashTable. Forwarded as-is to the HashTable's
+   *        constructor.
+   * @param hash_table_memory_zeroed If new_hash_table is true, setting this to
+   *        true means that the HashTable will assume that hash_table_memory
+   *        has already been zeroed-out (any newly-allocated block or blob
+   *        memory from StorageManager is zeroed-out). If false, the HashTable
+   *        will explicitly zero-fill its memory as neccessary. This parameter
+   *        has no effect when new_hash_table is false. Forwarded as-is to the
+   *        HashTable's constructor.
+   * @return A new (or reloaded) fixed-size HashTable.
+   **/
+  static FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>*
+      CreateFixedSize(const HashTableImplType hash_table_type,
+                      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) {
+    DCHECK(!resizable);
+
+    switch (hash_table_type) {
+      case HashTableImplType::kSeparateChaining:
+        return new SeparateChainingHashTable<
+            int,
+            resizable,
+            serializable,
+            force_key_copy,
+            allow_duplicate_keys>(key_types,
+                                  hash_table_memory,
+                                  hash_table_memory_size,
+                                  new_hash_table,
+                                  hash_table_memory_zeroed);
+      case HashTableImplType::kLinearOpenAddressing:
+/*        return new LinearOpenAddressingHashTable<
+            ValueT,
+            resizable,
+            serializable,
+            force_key_copy,
+            allow_duplicate_keys>(key_types,
+                                  hash_table_memory,
+                                  hash_table_memory_size,
+                                  new_hash_table,
+                                  hash_table_memory_zeroed);*/
+      case HashTableImplType::kSimpleScalarSeparateChaining:
+/*        return new SimpleScalarSeparateChainingHashTable<
+            ValueT,
+            resizable,
+            serializable,
+            force_key_copy,
+            allow_duplicate_keys>(key_types,
+                                  hash_table_memory,
+                                  hash_table_memory_size,
+                                  new_hash_table,
+                                  hash_table_memory_zeroed);*/
+      default: {
+        LOG(FATAL) << "Unrecognized HashTableImplType\n";
+      }
+    }
+  }
+
+  /**
+   * @brief Check whether a serialization::HashTable describing a resizable
+   *        HashTable is fully-formed and all parts are valid.
+   *
+   * @param proto A serialized Protocol Buffer description of a HashTable,
+   *        originally generated by the optimizer.
+   * @return Whether proto is fully-formed and valid.
+   **/
+  static bool ProtoIsValid(const serialization::HashTable &proto) {
+    if (!proto.IsInitialized() ||
+        !serialization::HashTableImplType_IsValid(
+            proto.hash_table_impl_type())) {
+      return false;
+    }
+
+    for (int i = 0; i < proto.key_types_size(); ++i) {
+      if (!TypeFactory::ProtoIsValid(proto.key_types(i))) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * @brief Create a new resizable HashTable according to a protobuf
+   *        description.
+   *
+   * @param proto A protobuf description of a resizable HashTable.
+   * @param storage_manager The StorageManager to use (a StorageBlob will be
+   *        allocated to hold the HashTable's contents).
+   * @param bloom_filters A vector of pointers to bloom filters that may be used
+   *        during hash table construction in build/probe phase.
+   * @return A new resizable HashTable with parameters specified by proto.
+   **/
+  static FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>*
+      CreateResizableFromProto(const serialization::HashTable &proto,
+                               StorageManager *storage_manager,
+                               const std::vector<std::unique_ptr<BloomFilter>> &bloom_filters) {
+    DCHECK(ProtoIsValid(proto))
+        << "Attempted to create HashTable from invalid proto description:\n"
+        << proto.DebugString();
+
+    std::vector<const Type*> key_types;
+    for (int i = 0; i < proto.key_types_size(); ++i) {
+      key_types.emplace_back(&TypeFactory::ReconstructFromProto(proto.key_types(i)));
+    }
+
+    auto hash_table = CreateResizable(HashTableImplTypeFromProto(proto.hash_table_impl_type()),
+                                      key_types,
+                                      proto.estimated_num_entries(),
+                                      storage_manager);
+
+    // TODO(ssaurabh): These lazy initializations can be moved from here and pushed to the
+    //                 individual implementations of the hash table constructors.
+
+    // Check if there are any build side bloom filter defined on the hash table.
+    if (proto.build_side_bloom_filter_id_size() > 0) {
+      hash_table->enableBuildSideBloomFilter();
+      hash_table->setBuildSideBloomFilter(bloom_filters[proto.build_side_bloom_filter_id(0)].get());
+    }
+
+    // Check if there are any probe side bloom filters defined on the hash table.
+    if (proto.probe_side_bloom_filters_size() > 0) {
+      hash_table->enableProbeSideBloomFilter();
+      // Add as many probe bloom filters as defined by the proto.
+      for (int j = 0; j < proto.probe_side_bloom_filters_size(); ++j) {
+        // Add the pointer to the probe bloom filter within the list of probe bloom filters to use.
+        const auto probe_side_bloom_filter = proto.probe_side_bloom_filters(j);
+        hash_table->addProbeSideBloomFilter(bloom_filters[probe_side_bloom_filter.probe_side_bloom_filter_id()].get());
+
+        // Add the attribute ids corresponding to this probe bloom filter.
+        std::vector<attribute_id> probe_attribute_ids;
+        for (int k = 0; k < probe_side_bloom_filter.probe_side_attr_ids_size(); ++k) {
+          const attribute_id probe_attribute_id = probe_side_bloom_filter.probe_side_attr_ids(k);
+          probe_attribute_ids.push_back(probe_attribute_id);
+        }
+        hash_table->addProbeSideAttributeIds(std::move(probe_attribute_ids));
+      }
+    }
+
+    return hash_table;
+  }
+
+ private:
+  // Class is all-static and should not be instantiated.
+  FastHashTableFactory();
+
+  DISALLOW_COPY_AND_ASSIGN(FastHashTableFactory);
+};
+
+/**
+ * @brief Convenient alias that provides a HashTableFactory whose only template
+ *        parameter is the aggregate state type.
+ **/
+using AggregationStateFastHashTableFactory
+    = FastHashTableFactory<true, false, true, false>;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_HASH_TABLE_FACTORY_HPP_


[28/28] incubator-quickstep git commit: Parallel work order generation support.

Posted by hb...@apache.org.
Parallel work order generation support.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/bc524a97
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/bc524a97
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/bc524a97

Branch: refs/heads/partitioned-aggregation
Commit: bc524a97375edbfd748945c69cf9b2928c4944a1
Parents: 3b5b8a9
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Fri Aug 19 10:35:09 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:57:06 2016 -0500

----------------------------------------------------------------------
 .../FinalizeAggregationOperator.cpp             | 35 ++++++++++++++++----
 .../FinalizeAggregationOperator.hpp             | 10 ++++--
 2 files changed, 36 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bc524a97/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 7e337de..55d1357 100644
--- a/relational_operators/FinalizeAggregationOperator.cpp
+++ b/relational_operators/FinalizeAggregationOperator.cpp
@@ -41,12 +41,29 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
 
   if (blocking_dependencies_met_ && !started_) {
     started_ = true;
-    container->addNormalWorkOrder(
-        new FinalizeAggregationWorkOrder(
-            query_id_,
-            query_context->getAggregationState(aggr_state_index_),
-            query_context->getInsertDestination(output_destination_index_)),
-        op_index_);
+    DCHECK(query_context->getAggregationState(aggr_state_index_) != nullptr);
+    if (query_context->getAggregationState(aggr_state_index_)->isAggregatePartitioned()) {
+      // The same AggregationState is shared across all the WorkOrders.
+      for (std::size_t part_id = 0;
+           part_id < query_context->getAggregationState(aggr_state_index_)
+                         ->getNumPartitions();
+           ++part_id) {
+        container->addNormalWorkOrder(
+            new FinalizeAggregationWorkOrder(
+                query_id_,
+                query_context->getAggregationState(aggr_state_index_),
+                query_context->getInsertDestination(output_destination_index_),
+                static_cast<int>(part_id)),
+            op_index_);
+      }
+    } else {
+      container->addNormalWorkOrder(
+          new FinalizeAggregationWorkOrder(
+              query_id_,
+              query_context->getAggregationState(aggr_state_index_),
+              query_context->getInsertDestination(output_destination_index_)),
+          op_index_);
+    }
   }
   return started_;
 }
@@ -70,7 +87,11 @@ bool FinalizeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer
 
 
 void FinalizeAggregationWorkOrder::execute() {
-  state_->finalizeAggregate(output_destination_);
+  if (state_->isAggregatePartitioned()) {
+    state_->finalizeAggregatePartitioned(part_id_, output_destination_);
+  } else {
+    state_->finalizeAggregate(output_destination_);
+  }
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bc524a97/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index 0aeac2a..7517d58 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -119,13 +119,18 @@ class FinalizeAggregationWorkOrder : public WorkOrder {
    * @param state The AggregationState to use.
    * @param output_destination The InsertDestination to insert aggregation
    *        results.
+   * @param part_id The partition ID for which the Finalize aggregation work
+   *        order is issued. Ignore this field if the aggregation is not
+   *        partitioned.
    */
   FinalizeAggregationWorkOrder(const std::size_t query_id,
                                AggregationOperationState *state,
-                               InsertDestination *output_destination)
+                               InsertDestination *output_destination,
+                               int part_id = -1)
       : WorkOrder(query_id),
         state_(DCHECK_NOTNULL(state)),
-        output_destination_(DCHECK_NOTNULL(output_destination)) {}
+        output_destination_(DCHECK_NOTNULL(output_destination)),
+        part_id_(part_id) {}
 
   ~FinalizeAggregationWorkOrder() override {}
 
@@ -134,6 +139,7 @@ class FinalizeAggregationWorkOrder : public WorkOrder {
  private:
   AggregationOperationState *state_;
   InsertDestination *output_destination_;
+  const int part_id_;
 
   DISALLOW_COPY_AND_ASSIGN(FinalizeAggregationWorkOrder);
 };


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

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/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 <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.");
 
-  // 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 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 +187,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).
@@ -237,8 +211,8 @@ class FastHashTable : public HashTableBase<resizable,
    *         exhausted.
    **/
 
-  HashTablePutResult putCompositeKeyFast(const std::vector<TypedValue> &key,
-                                     const uint8_t *value_ptr);
+  HashTablePutResult putCompositeKey(const std::vector<TypedValue> &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<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 +395,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 +455,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 +557,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 +582,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 +610,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 +636,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 +704,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 +746,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 +797,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 +823,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 +848,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 +873,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 +898,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 +956,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 +1009,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 +1038,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 +1097,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 +1144,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 +1162,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 +1178,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 +1191,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 +1228,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 +1255,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 +1322,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 +1343,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 +1369,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>::
+    putCompositeKey(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 +1389,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 +1412,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());
-        }
-        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 <bool resizable,
@@ -1540,11 +1533,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 +1549,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 +1629,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 +1651,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 +1675,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 +1695,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 +1709,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
@@ -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 <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 +1758,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 +1774,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 +1861,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 +1965,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 +2013,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 +2042,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 +2096,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 +2150,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 +2161,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 +2196,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 +2211,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 +2259,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 +2295,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 +2345,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 +2364,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 +2379,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 +2403,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, serializa

<TRUNCATED>


[13/28] incubator-quickstep git commit: Single aggregationGroupBy method in StorageBlock.

Posted by hb...@apache.org.
Single aggregationGroupBy method in StorageBlock.

- New methods for separating unary and nullary updation of states.
- Added TODO to move method from HashTableBase class.
- Added doxygen for the AggregationHandle new functions.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/95a46bbe
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/95a46bbe
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/95a46bbe

Branch: refs/heads/partitioned-aggregation
Commit: 95a46bbe6d7684438afde1f4eee1249967e15238
Parents: c123bd4
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Mon Sep 12 16:03:01 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:56:07 2016 -0500

----------------------------------------------------------------------
 catalog/CatalogTypedefs.hpp                     |  2 +
 .../aggregation/AggregationConcreteHandle.cpp   |  7 +-
 expressions/aggregation/AggregationHandle.hpp   | 58 ++++++++++++-
 .../aggregation/AggregationHandleAvg.hpp        |  6 +-
 .../aggregation/AggregationHandleCount.hpp      | 15 ++--
 .../aggregation/AggregationHandleMax.hpp        |  6 +-
 .../aggregation/AggregationHandleMin.hpp        |  6 +-
 .../aggregation/AggregationHandleSum.hpp        |  6 +-
 query_optimizer/ExecutionGenerator.cpp          | 20 ++---
 .../tests/AggregationOperator_unittest.cpp      |  3 +-
 storage/AggregationOperationState.cpp           | 64 +++++++-------
 storage/AggregationOperationState.hpp           |  4 +-
 storage/FastHashTable.hpp                       | 60 +++++++------
 storage/FastHashTableFactory.hpp                | 46 ----------
 storage/HashTableBase.hpp                       | 19 +++-
 storage/StorageBlock.cpp                        | 91 ++------------------
 storage/StorageBlock.hpp                        | 25 ++----
 17 files changed, 182 insertions(+), 256 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/catalog/CatalogTypedefs.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogTypedefs.hpp b/catalog/CatalogTypedefs.hpp
index f7a2d53..70bac84 100644
--- a/catalog/CatalogTypedefs.hpp
+++ b/catalog/CatalogTypedefs.hpp
@@ -49,6 +49,8 @@ constexpr int kInvalidCatalogId = -1;
 // Used to indicate no preference for a NUMA Node ID.
 constexpr numa_node_id kAnyNUMANodeID = -1;
 
+constexpr attribute_id kInvalidAttributeID = -1;
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/expressions/aggregation/AggregationConcreteHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.cpp b/expressions/aggregation/AggregationConcreteHandle.cpp
index ae677d9..e3fb520 100644
--- a/expressions/aggregation/AggregationConcreteHandle.cpp
+++ b/expressions/aggregation/AggregationConcreteHandle.cpp
@@ -56,13 +56,10 @@ void AggregationConcreteHandle::insertValueAccessorIntoDistinctifyHashTable(
   AggregationStateFastHashTable *hash_table =
       static_cast<AggregationStateFastHashTable *>(distinctify_hash_table);
   if (key_ids.size() == 1) {
-    std::vector<std::vector<attribute_id>> args;
-    args.emplace_back(key_ids);
     hash_table->upsertValueAccessorFast(
-        args, accessor, key_ids[0], true /* check_for_null_keys */);
+        key_ids, accessor, key_ids[0], true /* check_for_null_keys */);
   } else {
-    std::vector<std::vector<attribute_id>> empty_args;
-    empty_args.resize(1);
+    std::vector<attribute_id> empty_args {kInvalidAttributeID};
     hash_table->upsertValueAccessorCompositeKeyFast(
         empty_args, accessor, key_ids, true /* check_for_null_keys */);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index d2cee6d..c4663cf 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -130,7 +130,7 @@ class AggregationHandle {
    *        A StorageBlob will be allocated to serve as the HashTable's
    *        in-memory storage.
    * @return A new HashTable instance with the appropriate state type for this
-   *         aggregate as the ValueT.
+   *         aggregate.
    **/
   virtual AggregationStateHashTableBase* createGroupByHashTable(
       const HashTableImplType hash_table_impl,
@@ -297,7 +297,7 @@ class AggregationHandle {
    * in-memory
    *        storage.
    * @return A new HashTable instance with the appropriate state type for this
-   *         aggregate as the ValueT.
+   *         aggregate.
    */
   virtual AggregationStateHashTableBase* createDistinctifyHashTable(
       const HashTableImplType hash_table_impl,
@@ -356,13 +356,63 @@ class AggregationHandle {
       AggregationStateHashTableBase *aggregation_hash_table,
       std::size_t index) const = 0;
 
+  /**
+   * @brief Get the number of bytes to store the aggregation handle's state.
+   **/
   virtual std::size_t getPayloadSize() const { return 1; }
-  virtual void updateState(const std::vector<TypedValue> &arguments,
-                           std::uint8_t *byte_ptr) const {}
+
+  /**
+   * @brief Update the aggregation state for nullary aggregation function e.g.
+   *        COUNT(*).
+   *
+   * @note This function should be overloaded by those aggregation function
+   *       which can perform nullary operations, e.g. COUNT.
+   *
+   * @param byte_ptr The pointer where the aggregation state is stored.
+   **/
+  virtual void updateStateNullary(std::uint8_t *byte_ptr) const {}
+
+  /**
+   * @brief Update the aggregation state for unary aggregation function e.g.
+   *        SUM(a).
+   *
+   * @param argument The argument which will be used to update the state of the
+   *        aggregation function.
+   * @param byte_ptr The pointer where the aggregation state is stored.
+   **/
+  virtual void updateStateUnary(const TypedValue &argument,
+                                std::uint8_t *byte_ptr) const {}
+
+  /**
+   * @brief Merge two aggregation states for this aggregation handle.
+   *
+   * @note This function should be used with the hash table specifically meant
+   *       for aggregations only.
+   *
+   * @param src A pointer to the source aggregation state.
+   * @param dst A pointer to the destination aggregation state.
+   **/
   virtual void mergeStatesFast(const std::uint8_t *src,
                                std::uint8_t *dst) const {}
+
+  /**
+   * @brief Initialize the payload (in the aggregation hash table) for the given
+   *        aggregation handle.
+   *
+   * @param byte_ptr The pointer to the aggregation state in the hash table.
+   **/
   virtual void initPayload(std::uint8_t *byte_ptr) const {}
+
+  /**
+   * @brief Inform the aggregation handle to block (prohibit) updates on the
+   *        aggregation state.
+   **/
   virtual void blockUpdate() {}
+
+  /**
+   * @brief Inform the aggregation handle to allow updates on the
+   *        aggregation state.
+   **/
   virtual void allowUpdate() {}
 
  protected:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index 3e49213..366ba8e 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -141,10 +141,10 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     ++(*count_ptr);
   }
 
-  inline void updateState(const std::vector<TypedValue> &arguments,
-                          std::uint8_t *byte_ptr) const override {
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
     if (!block_update_) {
-      iterateUnaryInlFast(arguments.front(), byte_ptr);
+      iterateUnaryInlFast(argument, byte_ptr);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/expressions/aggregation/AggregationHandleCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index 2c6d717..9b97590 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.hpp
@@ -135,13 +135,16 @@ class AggregationHandleCount : public AggregationConcreteHandle {
     }
   }
 
-  inline void updateState(const std::vector<TypedValue> &arguments,
-                          std::uint8_t *byte_ptr) const override {
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
     if (!block_update_) {
-      if (arguments.size())
-        iterateUnaryInlFast(arguments.front(), byte_ptr);
-      else
-        iterateNullaryInlFast(byte_ptr);
+      iterateUnaryInlFast(argument, byte_ptr);
+    }
+  }
+
+  inline void updateStateNullary(std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateNullaryInlFast(byte_ptr);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index de173c9..6c54b9d 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -112,10 +112,10 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     compareAndUpdateFast(max_ptr, value);
   }
 
-  inline void updateState(const std::vector<TypedValue> &arguments,
-                          std::uint8_t *byte_ptr) const override {
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
     if (!block_update_) {
-      iterateUnaryInlFast(arguments.front(), byte_ptr);
+      iterateUnaryInlFast(argument, byte_ptr);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index 4a0eca4..9baf736 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -114,10 +114,10 @@ class AggregationHandleMin : public AggregationConcreteHandle {
     compareAndUpdateFast(min_ptr, value);
   }
 
-  inline void updateState(const std::vector<TypedValue> &arguments,
-                          std::uint8_t *byte_ptr) const override {
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
     if (!block_update_) {
-      iterateUnaryInlFast(arguments.front(), byte_ptr);
+      iterateUnaryInlFast(argument, byte_ptr);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index 8d719ab..18d45d9 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -133,10 +133,10 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     *null_ptr = false;
   }
 
-  inline void updateState(const std::vector<TypedValue> &arguments,
-                          std::uint8_t *byte_ptr) const override {
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
     if (!block_update_) {
-      iterateUnaryInlFast(arguments.front(), byte_ptr);
+      iterateUnaryInlFast(argument, byte_ptr);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 130134c..968314e 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1371,13 +1371,9 @@ void ExecutionGenerator::convertAggregate(
   }
 
   if (!group_by_types.empty()) {
-    // SimplifyHashTableImplTypeProto() switches the hash table implementation
-    // from SeparateChaining to SimpleScalarSeparateChaining when there is a
-    // single scalar key type with a reversible hash function.
+    // Right now, only SeparateChaining is supported.
     aggr_state_proto->set_hash_table_impl_type(
-        SimplifyHashTableImplTypeProto(
-            HashTableImplTypeProtoFromString(FLAGS_aggregate_hashtable_type),
-            group_by_types));
+        serialization::HashTableImplType::SEPARATE_CHAINING);
   }
 
   for (const E::AliasPtr &named_aggregate_expression : physical_plan->aggregate_expressions()) {
@@ -1404,15 +1400,9 @@ void ExecutionGenerator::convertAggregate(
     if (unnamed_aggregate_expression->is_distinct()) {
       const std::vector<E::ScalarPtr> &arguments = unnamed_aggregate_expression->getArguments();
       DCHECK_GE(arguments.size(), 1u);
-      if (group_by_types.empty() && arguments.size() == 1) {
-        aggr_state_proto->add_distinctify_hash_table_impl_types(
-            SimplifyHashTableImplTypeProto(
-                HashTableImplTypeProtoFromString(FLAGS_aggregate_hashtable_type),
-                {&arguments[0]->getValueType()}));
-      } else {
-        aggr_state_proto->add_distinctify_hash_table_impl_types(
-            HashTableImplTypeProtoFromString(FLAGS_aggregate_hashtable_type));
-      }
+      // Right now only SeparateChaining implementation is supported.
+      aggr_state_proto->add_distinctify_hash_table_impl_types(
+          serialization::HashTableImplType::SEPARATE_CHAINING);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index 0138362..6881dea 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -363,8 +363,9 @@ class AggregationOperatorTest : public ::testing::Test {
     aggr_state_proto->set_estimated_num_entries(estimated_entries);
 
     // Also need to set the HashTable implementation for GROUP BY.
+    // Right now, only SeparateChaining is supported.
     aggr_state_proto->set_hash_table_impl_type(
-        serialization::HashTableImplType::LINEAR_OPEN_ADDRESSING);
+        serialization::HashTableImplType::SEPARATE_CHAINING);
 
     // Create Operators.
     op_.reset(new AggregationOperator(0, *table_, true, aggr_state_index));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index c5f59f9..073b813 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -94,13 +94,12 @@ AggregationOperationState::AggregationOperationState(
     handles_.emplace_back(new AggregationHandleDistinct());
     arguments_.push_back({});
     is_distinct_.emplace_back(false);
-    group_by_hashtable_pools_.emplace_back(
-        std::unique_ptr<HashTablePool>(new HashTablePool(estimated_num_entries,
-                                                         hash_table_impl_type,
-                                                         group_by_types,
-                                                         {1},
-                                                         handles_,
-                                                         storage_manager)));
+    group_by_hashtable_pool_.reset(new HashTablePool(estimated_num_entries,
+                                                     hash_table_impl_type,
+                                                     group_by_types,
+                                                     {1},
+                                                     handles_,
+                                                     storage_manager));
   } else {
     // Set up each individual aggregate in this operation.
     std::vector<const AggregateFunction *>::const_iterator agg_func_it =
@@ -196,13 +195,12 @@ AggregationOperationState::AggregationOperationState(
     if (!group_by_handles.empty()) {
       // Aggregation with GROUP BY: create a HashTable pool for per-group
       // states.
-      group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
-          new HashTablePool(estimated_num_entries,
-                            hash_table_impl_type,
-                            group_by_types,
-                            payload_sizes,
-                            group_by_handles,
-                            storage_manager)));
+      group_by_hashtable_pool_.reset(new HashTablePool(estimated_num_entries,
+                                                       hash_table_impl_type,
+                                                       group_by_types,
+                                                       payload_sizes,
+                                                       group_by_handles,
+                                                       storage_manager));
     }
   }
 }
@@ -444,17 +442,17 @@ void AggregationOperationState::aggregateBlockHashTable(
   // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
   // directly into the (threadsafe) shared global HashTable for this
   // aggregate.
-  DCHECK(group_by_hashtable_pools_[0] != nullptr);
+  DCHECK(group_by_hashtable_pool_ != nullptr);
   AggregationStateHashTableBase *agg_hash_table =
-      group_by_hashtable_pools_[0]->getHashTableFast();
+      group_by_hashtable_pool_->getHashTableFast();
   DCHECK(agg_hash_table != nullptr);
-  block->aggregateGroupByFast(arguments_,
-                              group_by_list_,
-                              predicate_.get(),
-                              agg_hash_table,
-                              &reuse_matches,
-                              &reuse_group_by_vectors);
-  group_by_hashtable_pools_[0]->returnHashTable(agg_hash_table);
+  block->aggregateGroupBy(arguments_,
+                          group_by_list_,
+                          predicate_.get(),
+                          agg_hash_table,
+                          &reuse_matches,
+                          &reuse_group_by_vectors);
+  group_by_hashtable_pool_->returnHashTable(agg_hash_table);
 }
 
 void AggregationOperationState::finalizeSingleState(
@@ -497,7 +495,7 @@ void AggregationOperationState::finalizeHashTable(
   // TODO(harshad) - Find heuristics for faster merge, even in a single thread.
   // e.g. Keep merging entries from smaller hash tables to larger.
 
-  auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
+  auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
   if (hash_tables->size() > 1) {
     for (int hash_table_index = 0;
          hash_table_index < static_cast<int>(hash_tables->size() - 1);
@@ -512,17 +510,17 @@ void AggregationOperationState::finalizeHashTable(
   std::vector<std::unique_ptr<ColumnVector>> final_values;
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
     if (is_distinct_[agg_idx]) {
-      DCHECK(group_by_hashtable_pools_[0] != nullptr);
-      auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
+      DCHECK(group_by_hashtable_pool_ != nullptr);
+      auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
       DCHECK(hash_tables != nullptr);
       if (hash_tables->empty()) {
         // We may have a case where hash_tables is empty, e.g. no input blocks.
         // However for aggregateOnDistinctifyHashTableForGroupBy to work
         // correctly, we should create an empty group by hash table.
         AggregationStateHashTableBase *new_hash_table =
-            group_by_hashtable_pools_[0]->getHashTableFast();
-        group_by_hashtable_pools_[0]->returnHashTable(new_hash_table);
-        hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
+            group_by_hashtable_pool_->getHashTableFast();
+        group_by_hashtable_pool_->returnHashTable(new_hash_table);
+        hash_tables = group_by_hashtable_pool_->getAllHashTables();
       }
       DCHECK(hash_tables->back() != nullptr);
       AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
@@ -532,16 +530,16 @@ void AggregationOperationState::finalizeHashTable(
           *distinctify_hashtables_[agg_idx], agg_hash_table, agg_idx);
     }
 
-    auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
+    auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
     DCHECK(hash_tables != nullptr);
     if (hash_tables->empty()) {
       // We may have a case where hash_tables is empty, e.g. no input blocks.
       // However for aggregateOnDistinctifyHashTableForGroupBy to work
       // correctly, we should create an empty group by hash table.
       AggregationStateHashTableBase *new_hash_table =
-          group_by_hashtable_pools_[0]->getHashTable();
-      group_by_hashtable_pools_[0]->returnHashTable(new_hash_table);
-      hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
+          group_by_hashtable_pool_->getHashTable();
+      group_by_hashtable_pool_->returnHashTable(new_hash_table);
+      hash_tables = group_by_hashtable_pool_->getAllHashTables();
     }
     AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
     DCHECK(agg_hash_table != nullptr);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 7956bc6..cbbfc22 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -221,8 +221,8 @@ class AggregationOperationState {
   std::vector<std::unique_ptr<AggregationStateHashTableBase>>
       group_by_hashtables_;
 
-  // A vector of group by hash table pools, one for each group by clause.
-  std::vector<std::unique_ptr<HashTablePool>> group_by_hashtable_pools_;
+  // A vector of group by hash table pools.
+  std::unique_ptr<HashTablePool> group_by_hashtable_pool_;
 
   StorageManager *storage_manager_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
index f1e8d1a..4a95cd9 100644
--- a/storage/FastHashTable.hpp
+++ b/storage/FastHashTable.hpp
@@ -456,7 +456,7 @@ class FastHashTable : public HashTableBase<resizable,
    *         not be inserted).
    **/
   bool upsertValueAccessorFast(
-      const std::vector<std::vector<attribute_id>> &argument_ids,
+      const std::vector<attribute_id> &argument_ids,
       ValueAccessor *accessor,
       const attribute_id key_attr_id,
       const bool check_for_null_keys);
@@ -509,7 +509,7 @@ class FastHashTable : public HashTableBase<resizable,
    *         not be inserted).
    **/
   bool upsertValueAccessorCompositeKeyFast(
-      const std::vector<std::vector<attribute_id>> &argument,
+      const std::vector<attribute_id> &argument,
       ValueAccessor *accessor,
       const std::vector<attribute_id> &key_attr_ids,
       const bool check_for_null_keys) override;
@@ -1866,13 +1866,12 @@ bool FastHashTable<resizable,
                    force_key_copy,
                    allow_duplicate_keys>::
     upsertValueAccessorFast(
-        const std::vector<std::vector<attribute_id>> &argument_ids,
+        const 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)
@@ -1898,13 +1897,14 @@ bool FastHashTable<resizable,
                 } 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()));
+                    if (argument_ids[k] != kInvalidAttributeID) {
+                      handles_[k]->updateStateUnary(
+                          accessor->getTypedValue(argument_ids[k]),
+                          value + payload_offsets_[k]);
+                    } else {
+                      handles_[k]->updateStateNullary(value +
+                                                      payload_offsets_[k]);
                     }
-                    handles_[k]->updateState(local,
-                                             value + payload_offsets_[k]);
                   }
                 }
               }
@@ -1929,12 +1929,14 @@ bool FastHashTable<resizable,
             } 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()));
+                if (argument_ids[k] != kInvalidAttributeID) {
+                  handles_[k]->updateStateUnary(
+                      accessor->getTypedValue(argument_ids[k]),
+                      value + payload_offsets_[k]);
+                } else {
+                  handles_[k]->updateStateNullary(value +
+                                                  payload_offsets_[k]);
                 }
-                handles_[k]->updateState(local, value + payload_offsets_[k]);
               }
             }
           }
@@ -1953,7 +1955,7 @@ bool FastHashTable<resizable,
                    force_key_copy,
                    allow_duplicate_keys>::
     upsertValueAccessorCompositeKeyFast(
-        const std::vector<std::vector<attribute_id>> &argument_ids,
+        const std::vector<attribute_id> &argument_ids,
         ValueAccessor *accessor,
         const std::vector<attribute_id> &key_attr_ids,
         const bool check_for_null_keys) {
@@ -1961,7 +1963,6 @@ bool FastHashTable<resizable,
   std::size_t variable_size;
   std::vector<TypedValue> key_vector;
   key_vector.resize(key_attr_ids.size());
-  std::vector<TypedValue> local;
   return InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
@@ -1989,13 +1990,14 @@ bool FastHashTable<resizable,
                 } 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()));
+                    if (argument_ids[k] != kInvalidAttributeID) {
+                      handles_[k]->updateStateUnary(
+                          accessor->getTypedValue(argument_ids[k]),
+                          value + payload_offsets_[k]);
+                    } else {
+                      handles_[k]->updateStateNullary(value +
+                                                      payload_offsets_[k]);
                     }
-                    handles_[k]->updateState(local,
-                                             value + payload_offsets_[k]);
                   }
                 }
               }
@@ -2022,12 +2024,14 @@ bool FastHashTable<resizable,
             } 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()));
+                if (argument_ids[k] != kInvalidAttributeID) {
+                  handles_[k]->updateStateUnary(
+                      accessor->getTypedValue(argument_ids[k]),
+                      value + payload_offsets_[k]);
+                } else {
+                  handles_[k]->updateStateNullary(value +
+                                                  payload_offsets_[k]);
                 }
-                handles_[k]->updateState(local, value + payload_offsets_[k]);
               }
             }
           }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/storage/FastHashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTableFactory.hpp b/storage/FastHashTableFactory.hpp
index 6ad3212..dc4f893 100644
--- a/storage/FastHashTableFactory.hpp
+++ b/storage/FastHashTableFactory.hpp
@@ -90,30 +90,6 @@ class FastHashTableFactory {
             serializable,
             force_key_copy,
             allow_duplicate_keys>(key_types, num_entries, payload_sizes, handles, storage_manager);
-      case HashTableImplType::kLinearOpenAddressing:
-/*        return new LinearOpenAddressingHashTable<
-            ValueT,
-            resizable,
-            serializable,
-            force_key_copy,
-            allow_duplicate_keys>(key_types, num_entries, storage_manager);*/
-        return new FastSeparateChainingHashTable<
-            resizable,
-            serializable,
-            force_key_copy,
-            allow_duplicate_keys>(key_types, num_entries, payload_sizes, handles, storage_manager);
-      case HashTableImplType::kSimpleScalarSeparateChaining:
-        return new FastSeparateChainingHashTable<
-            resizable,
-            serializable,
-            force_key_copy,
-            allow_duplicate_keys>(key_types, num_entries, payload_sizes, handles, storage_manager);
-/*        return new SimpleScalarSeparateChainingHashTable<
-            ValueT,
-            resizable,
-            serializable,
-            force_key_copy,
-            allow_duplicate_keys>(key_types, num_entries, storage_manager);*/
       default: {
         LOG(FATAL) << "Unrecognized HashTableImplType in HashTableFactory::createResizable()\n";
       }
@@ -167,28 +143,6 @@ class FastHashTableFactory {
                                   hash_table_memory_size,
                                   new_hash_table,
                                   hash_table_memory_zeroed);
-      case HashTableImplType::kLinearOpenAddressing:
-/*        return new LinearOpenAddressingHashTable<
-            ValueT,
-            resizable,
-            serializable,
-            force_key_copy,
-            allow_duplicate_keys>(key_types,
-                                  hash_table_memory,
-                                  hash_table_memory_size,
-                                  new_hash_table,
-                                  hash_table_memory_zeroed);*/
-      case HashTableImplType::kSimpleScalarSeparateChaining:
-/*        return new SimpleScalarSeparateChainingHashTable<
-            ValueT,
-            resizable,
-            serializable,
-            force_key_copy,
-            allow_duplicate_keys>(key_types,
-                                  hash_table_memory,
-                                  hash_table_memory_size,
-                                  new_hash_table,
-                                  hash_table_memory_zeroed);*/
       default: {
         LOG(FATAL) << "Unrecognized HashTableImplType\n";
       }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index b908d6f..cd0a141 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -74,8 +74,25 @@ class HashTableBase {
  public:
   virtual ~HashTableBase() {}
 
+  /**
+   * TODO(harshad) We should get rid of this function from here. We are
+   * postponing it because of the amount of work to be done is significant.
+   * The steps are as follows:
+   * 1. Replace AggregationStateHashTableBase occurence in HashTablePool to
+   * the FastHashTable implementation (i.e. an implementation specialized for
+   * aggregation).
+   * 2. Remove createGroupByHashTable from the AggregationHandle* classes.
+   * 3. Replace AggregationStateHashTableBase occurences in AggregationHandle*
+   * clases to the FastHashTable implementation (i.e. an implementation
+   * specialized for aggregation).
+   * 4. Move this method to the FastHashTable class from here, so that it can
+   * be called from the AggregationHandle* classes.
+   *
+   * Optionally, we can also remove the AggregationStateHashTableBase
+   * specialization from this file.
+   **/
   virtual bool upsertValueAccessorCompositeKeyFast(
-      const std::vector<std::vector<attribute_id>> &argument,
+      const std::vector<attribute_id> &argument,
       ValueAccessor *accessor,
       const std::vector<attribute_id> &key_attr_ids,
       const bool check_for_null_keys) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 8ff18b5..ec5990f 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -415,87 +415,6 @@ AggregationState* StorageBlock::aggregate(
 }
 
 void StorageBlock::aggregateGroupBy(
-    const AggregationHandle &handle,
-    const std::vector<std::unique_ptr<const Scalar>> &arguments,
-    const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const Predicate *predicate,
-    AggregationStateHashTableBase *hash_table,
-    std::unique_ptr<TupleIdSequence> *reuse_matches,
-    std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
-  DCHECK_GT(group_by.size(), 0u)
-      << "Called aggregateGroupBy() with zero GROUP BY expressions";
-
-  SubBlocksReference sub_blocks_ref(*tuple_store_,
-                                    indices_,
-                                    indices_consistent_);
-
-  // IDs of 'arguments' as attributes in the ValueAccessor we create below.
-  std::vector<attribute_id> argument_ids;
-
-  // IDs of GROUP BY key element(s) in the ValueAccessor we create below.
-  std::vector<attribute_id> key_ids;
-
-  // An intermediate ValueAccessor that stores the materialized 'arguments' for
-  // this aggregate, as well as the GROUP BY expression values.
-  ColumnVectorsValueAccessor temp_result;
-  {
-    std::unique_ptr<ValueAccessor> accessor;
-    if (predicate) {
-      if (!*reuse_matches) {
-        // If there is a filter predicate that hasn't already been evaluated,
-        // evaluate it now and save the results for other aggregates on this
-        // same block.
-        reuse_matches->reset(getMatchesForPredicate(predicate));
-      }
-
-      // Create a filtered ValueAccessor that only iterates over predicate
-      // matches.
-      accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));
-    } else {
-      // Create a ValueAccessor that iterates over all tuples in this block
-      accessor.reset(tuple_store_->createValueAccessor());
-    }
-
-    attribute_id attr_id = 0;
-
-    // First, put GROUP BY keys into 'temp_result'.
-    if (reuse_group_by_vectors->empty()) {
-      // Compute GROUP BY values from group_by Scalars, and store them in
-      // reuse_group_by_vectors for reuse by other aggregates on this same
-      // block.
-      reuse_group_by_vectors->reserve(group_by.size());
-      for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
-        reuse_group_by_vectors->emplace_back(
-            group_by_element->getAllValues(accessor.get(), &sub_blocks_ref));
-        temp_result.addColumn(reuse_group_by_vectors->back().get(), false);
-        key_ids.push_back(attr_id++);
-      }
-    } else {
-      // Reuse precomputed GROUP BY values from reuse_group_by_vectors.
-      DCHECK_EQ(group_by.size(), reuse_group_by_vectors->size())
-          << "Wrong number of reuse_group_by_vectors";
-      for (const std::unique_ptr<ColumnVector> &reuse_cv : *reuse_group_by_vectors) {
-        temp_result.addColumn(reuse_cv.get(), false);
-        key_ids.push_back(attr_id++);
-      }
-    }
-
-    // Compute argument vectors and add them to 'temp_result'.
-    for (const std::unique_ptr<const Scalar> &argument : arguments) {
-      temp_result.addColumn(argument->getAllValues(accessor.get(), &sub_blocks_ref));
-      argument_ids.push_back(attr_id++);
-    }
-  }
-
-  // Actually do aggregation into '*hash_table'.
-  handle.aggregateValueAccessorIntoHashTable(&temp_result,
-                                             argument_ids,
-                                             key_ids,
-                                             hash_table);
-}
-
-
-void StorageBlock::aggregateGroupByFast(
     const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
     const std::vector<std::unique_ptr<const Scalar>> &group_by,
     const Predicate *predicate,
@@ -510,8 +429,7 @@ void StorageBlock::aggregateGroupByFast(
                                     indices_consistent_);
 
   // IDs of 'arguments' as attributes in the ValueAccessor we create below.
-  std::vector<attribute_id> arg_ids;
-  std::vector<std::vector<attribute_id>> argument_ids;
+  std::vector<attribute_id> argument_ids;
 
   // IDs of GROUP BY key element(s) in the ValueAccessor we create below.
   std::vector<attribute_id> key_ids;
@@ -563,12 +481,13 @@ void StorageBlock::aggregateGroupByFast(
 
     // Compute argument vectors and add them to 'temp_result'.
     for (const std::vector<std::unique_ptr<const Scalar>> &argument : arguments) {
-        arg_ids.clear();
         for (const std::unique_ptr<const Scalar> &args : argument) {
           temp_result.addColumn(args->getAllValues(accessor.get(), &sub_blocks_ref));
-          arg_ids.push_back(attr_id++);
+          argument_ids.push_back(attr_id++);
+        }
+        if (argument.empty()) {
+          argument_ids.push_back(kInvalidAttributeID);
         }
-        argument_ids.push_back(arg_ids);
      }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/95a46bbe/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 8b59a3c..398008e 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -459,23 +459,14 @@ class StorageBlock : public StorageBlockBase {
    * attributes as std::vector<attribute_id> (like in selectSimple()) for fast
    * path when there are no expressions specified in the query.
    */
-  void aggregateGroupBy(const AggregationHandle &handle,
-                        const std::vector<std::unique_ptr<const Scalar>> &arguments,
-                        const std::vector<std::unique_ptr<const Scalar>> &group_by,
-                        const Predicate *predicate,
-                        AggregationStateHashTableBase *hash_table,
-                        std::unique_ptr<TupleIdSequence> *reuse_matches,
-                        std::vector<std::unique_ptr<ColumnVector>>
-                            *reuse_group_by_vectors) const;
-
-
-  void aggregateGroupByFast(const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
-                        const std::vector<std::unique_ptr<const Scalar>> &group_by,
-                        const Predicate *predicate,
-                        AggregationStateHashTableBase *hash_table,
-                        std::unique_ptr<TupleIdSequence> *reuse_matches,
-                        std::vector<std::unique_ptr<ColumnVector>>
-                            *reuse_group_by_vectors) const;
+  void aggregateGroupBy(
+      const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
+      const std::vector<std::unique_ptr<const Scalar>> &group_by,
+      const Predicate *predicate,
+      AggregationStateHashTableBase *hash_table,
+      std::unique_ptr<TupleIdSequence> *reuse_matches,
+      std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
+
   /**
    * @brief Inserts the GROUP BY expressions and aggregation arguments together
    *        as keys into the distinctify hash table.



[26/28] incubator-quickstep git commit: Minor change in DCHECK.

Posted by hb...@apache.org.
Minor change in DCHECK.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/3b5b8a90
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/3b5b8a90
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/3b5b8a90

Branch: refs/heads/partitioned-aggregation
Commit: 3b5b8a904d8d36a49436f47a967d4c5020b81171
Parents: 157ce78
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Thu Aug 18 22:14:09 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:57:06 2016 -0500

----------------------------------------------------------------------
 storage/StorageBlock.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3b5b8a90/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index bb43630..5ba2a6e 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -1377,7 +1377,7 @@ void StorageBlock::aggregateGroupByPartitioned(
     std::unique_ptr<TupleIdSequence> *reuse_matches,
     std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors,
     PartitionedHashTablePool *hashtable_pool) const {
-  DCHECK_EQ(group_by.size(), 0u)
+  DCHECK(!group_by.empty())
       << "Called aggregateGroupByPartitioned() with zero GROUP BY expressions";
 
   SubBlocksReference sub_blocks_ref(*tuple_store_,


[10/28] incubator-quickstep git commit: Fixed signed-unsigned comparison failure. Minor code cleanup.

Posted by hb...@apache.org.
Fixed signed-unsigned comparison failure. Minor code cleanup.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/935d5d07
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/935d5d07
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/935d5d07

Branch: refs/heads/partitioned-aggregation
Commit: 935d5d0784b094e184ab75f11063ea8b1955c5c2
Parents: c510d24
Author: rathijit <ra...@node-2.aggregation.quickstep-pg0.wisc.cloudlab.us>
Authored: Sun Aug 14 16:14:36 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:56:07 2016 -0500

----------------------------------------------------------------------
 storage/AggregationOperationState.cpp | 36 ++----------------------------
 storage/FastHashTable.hpp             | 20 +++++------------
 2 files changed, 8 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/935d5d07/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 833b707..90b8fcc 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -94,13 +94,6 @@ AggregationOperationState::AggregationOperationState(
     handles_.emplace_back(new AggregationHandleDistinct());
     arguments_.push_back({});
     is_distinct_.emplace_back(false);
-
- /*   group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
-        new HashTablePool(estimated_num_entries,
-                          hash_table_impl_type,
-                          group_by_types,
-                          handles_.back().get(),
-                          storage_manager)));*/
     group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
         new HashTablePool(estimated_num_entries,
                           hash_table_impl_type,
@@ -136,19 +129,12 @@ AggregationOperationState::AggregationOperationState(
       handles_.emplace_back((*agg_func_it)->createHandle(argument_types));
 
       if (!group_by_list_.empty()) {
-        // Aggregation with GROUP BY: create a HashTable pool for per-group states.
- /*       group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
-            new HashTablePool(estimated_num_entries,
-                              hash_table_impl_type,
-                              group_by_types,
-                              handles_.back().get(),
-                              storage_manager)));*/
+        // Aggregation with GROUP BY: combined payload is partially updated in the presence of DISTINCT.
          if (*is_distinct_it) {
             handles_.back()->BlockUpdate();
          }
          group_by_handles.emplace_back(handles_.back());
          payload_sizes.emplace_back(group_by_handles.back()->getPayloadSize());
-
       } else {
         // Aggregation without GROUP BY: create a single global state.
         single_states_.emplace_back(handles_.back()->createInitialState());
@@ -183,23 +169,13 @@ AggregationOperationState::AggregationOperationState(
         // the number of entries in the distinctify hash table. We may estimate
         // for each distinct aggregation an estimated_num_distinct_keys value during
         // query optimization, if it worths.
- /*       distinctify_hashtables_.emplace_back(
-            handles_.back()->createDistinctifyHashTable(
-                *distinctify_hash_table_impl_types_it,
-                key_types,
-                estimated_num_entries,
-                storage_manager));*/
-
-        std::vector<AggregationHandle *> local;
-        // local.emplace_back(handles_.back());
-        local.clear();
         distinctify_hashtables_.emplace_back(
         AggregationStateFastHashTableFactory::CreateResizable(
                 *distinctify_hash_table_impl_types_it,
                 key_types,
                 estimated_num_entries,
                 {0},
-                local,
+                {},
                 storage_manager));
         ++distinctify_hash_table_impl_types_it;
       } else {
@@ -455,13 +431,6 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
   DCHECK(group_by_hashtable_pools_[0] != nullptr);
   AggregationStateHashTableBase *agg_hash_table = group_by_hashtable_pools_[0]->getHashTableFast();
   DCHECK(agg_hash_table != nullptr);
- /*     block->aggregateGroupBy(*handles_[agg_idx],
-                              arguments_[agg_idx],
-                              group_by_list_,
-                              predicate_.get(),
-                              agg_hash_table,
-                              &reuse_matches,
-                              &reuse_group_by_vectors);*/
   block->aggregateGroupByFast(arguments_,
                               group_by_list_,
                               predicate_.get(),
@@ -507,7 +476,6 @@ void AggregationOperationState::finalizeHashTable(InsertDestination *output_dest
 
   // TODO(harshad) - Find heuristics for faster merge, even in a single thread.
   // e.g. Keep merging entries from smaller hash tables to larger.
-//  auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
 
   auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/935d5d07/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
index e7887ab..8d8d82b 100644
--- a/storage/FastHashTable.hpp
+++ b/storage/FastHashTable.hpp
@@ -1901,7 +1901,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
         uint8_t *value = upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
         if (value != nullptr) {
             SpinMutex lock(value);
-            for (int k = 0; k < handles_.size(); ++k) {
+            for (unsigned int k = 0; k < handles_.size(); ++k) {
                 handles_[k]->mergeStatesFast(source_state + payload_offsets_[k], value + payload_offsets_[k]);
             }
           return true;
@@ -1915,7 +1915,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
       return false;
     } else {
       SpinMutex lock(value);
-      for (int k = 0; k < handles_.size(); ++k) {
+      for (unsigned int k = 0; k < handles_.size(); ++k) {
           handles_[k]->mergeStatesFast(source_state + payload_offsets_[k], value + payload_offsets_[k]);
       }
       return true;
@@ -2018,10 +2018,8 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
               break;
             } else {
               SpinMutex lock(value);
-              for (int k = 0; k < handles_.size(); ++k) {
+              for (unsigned int k = 0; k < handles_.size(); ++k) {
                   local.clear();
-//                std::for_each(argument_ids[k].begin(),argument_ids[k].end(),[&](attribute_id id)
-//                {local.emplace_back(accessor->getTypedValue(id));});
                   if (argument_ids[k].size()) {
                     local.emplace_back(accessor->getTypedValue(argument_ids[k].front()));
                   }
@@ -2047,10 +2045,8 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
           return false;
         } else {
           SpinMutex lock(value);
-          for (int k = 0; k < handles_.size(); ++k) {
+          for (unsigned int k = 0; k < handles_.size(); ++k) {
               local.clear();
-//            std::for_each(argument_ids[k].begin(),argument_ids[k].end(),[&](attribute_id id)
-//            {local.emplace_back(accessor->getTypedValue(id));});
               if (argument_ids[k].size()) {
                  local.emplace_back(accessor->getTypedValue(argument_ids[k].front()));
               }
@@ -2175,10 +2171,8 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
               break;
             } else {
               SpinMutex lock(value);
-              for (int k = 0; k < handles_.size(); ++k) {
+              for (unsigned int k = 0; k < handles_.size(); ++k) {
                   local.clear();
-//                std::for_each(argument_ids[k].begin(),argument_ids[k].end(),[&](attribute_id id)
-//                {local.emplace_back(accessor->getTypedValue(id));});
                   if (argument_ids[k].size()) {
                       local.emplace_back(accessor->getTypedValue(argument_ids[k].front()));
                   }
@@ -2208,10 +2202,8 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
           return false;
         } else {
           SpinMutex lock(value);
-          for (int k = 0; k < handles_.size(); ++k) {
+          for (unsigned int k = 0; k < handles_.size(); ++k) {
               local.clear();
-//            std::for_each(argument_ids[k].begin(),argument_ids[k].end(),[&](attribute_id id)
-//            {local.emplace_back(accessor->getTypedValue(id));});
               if (argument_ids[k].size()) {
                  local.emplace_back(accessor->getTypedValue(argument_ids[k].front()));
               }


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

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp b/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
index 0e35151..1d1c084 100644
--- a/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
@@ -28,6 +28,8 @@
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "expressions/aggregation/AggregationHandleSum.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
+#include "storage/AggregationOperationState.hpp"
+#include "storage/FastHashTableFactory.hpp"
 #include "storage/StorageManager.hpp"
 #include "types/CharType.hpp"
 #include "types/DatetimeIntervalType.hpp"
@@ -52,51 +54,56 @@
 
 namespace quickstep {
 
-class AggregationHandleSumTest : public::testing::Test {
+class AggregationHandleSumTest : public ::testing::Test {
  protected:
   static const int kNumSamples = 1000;
 
   // Helper method that calls AggregationHandleSum::iterateUnaryInl() to
   // aggregate 'value' into '*state'.
   void iterateHandle(AggregationState *state, const TypedValue &value) {
-    static_cast<const AggregationHandleSum&>(*aggregation_handle_sum_).iterateUnaryInl(
-        static_cast<AggregationStateSum*>(state),
-        value);
+    static_cast<const AggregationHandleSum &>(*aggregation_handle_sum_)
+        .iterateUnaryInl(static_cast<AggregationStateSum *>(state), value);
   }
 
   void initializeHandle(const Type &type) {
     aggregation_handle_sum_.reset(
-        AggregateFunctionFactory::Get(AggregationID::kSum).createHandle(
-            std::vector<const Type*>(1, &type)));
+        AggregateFunctionFactory::Get(AggregationID::kSum)
+            .createHandle(std::vector<const Type *>(1, &type)));
     aggregation_handle_sum_state_.reset(
         aggregation_handle_sum_->createInitialState());
   }
 
   static bool ApplyToTypesTest(TypeID typeID) {
-    const Type &type = (typeID == kChar || typeID == kVarChar) ?
-        TypeFactory::GetType(typeID, static_cast<std::size_t>(10)) :
-        TypeFactory::GetType(typeID);
+    const Type &type =
+        (typeID == kChar || typeID == kVarChar)
+            ? TypeFactory::GetType(typeID, static_cast<std::size_t>(10))
+            : TypeFactory::GetType(typeID);
 
-    return AggregateFunctionFactory::Get(AggregationID::kSum).canApplyToTypes(
-        std::vector<const Type*>(1, &type));
+    return AggregateFunctionFactory::Get(AggregationID::kSum)
+        .canApplyToTypes(std::vector<const Type *>(1, &type));
   }
 
   static bool ResultTypeForArgumentTypeTest(TypeID input_type_id,
                                             TypeID output_type_id) {
-    const Type *result_type
-        = AggregateFunctionFactory::Get(AggregationID::kSum).resultTypeForArgumentTypes(
-            std::vector<const Type*>(1, &TypeFactory::GetType(input_type_id)));
+    const Type *result_type =
+        AggregateFunctionFactory::Get(AggregationID::kSum)
+            .resultTypeForArgumentTypes(std::vector<const Type *>(
+                1, &TypeFactory::GetType(input_type_id)));
     return (result_type->getTypeID() == output_type_id);
   }
 
   template <typename CppType>
-  static void CheckSumValue(
-      CppType expected,
-      const AggregationHandle &target,
-      const AggregationState &state) {
+  static void CheckSumValue(CppType expected,
+                            const AggregationHandle &target,
+                            const AggregationState &state) {
     EXPECT_EQ(expected, target.finalize(state).getLiteral<CppType>());
   }
 
+  template <typename CppType>
+  static void CheckSumValue(CppType expected, const TypedValue &value) {
+    EXPECT_EQ(expected, value.getLiteral<CppType>());
+  }
+
   // Static templated method to set a meaningful to data types.
   template <typename CppType>
   static void SetDataType(int value, CppType *data) {
@@ -108,7 +115,9 @@ class AggregationHandleSumTest : public::testing::Test {
     const GenericType &type = GenericType::Instance(true);
 
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_sum_->finalize(*aggregation_handle_sum_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_sum_->finalize(*aggregation_handle_sum_state_)
+            .isNull());
 
     typename GenericType::cpptype val;
     typename PrecisionType::cpptype sum;
@@ -119,13 +128,14 @@ class AggregationHandleSumTest : public::testing::Test {
       if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
         SetDataType(i - 10, &val);
       } else {
-        SetDataType(static_cast<float>(i - 10)/10, &val);
+        SetDataType(static_cast<float>(i - 10) / 10, &val);
       }
       iterateHandle(aggregation_handle_sum_state_.get(), type.makeValue(&val));
       sum += val;
     }
     iterateHandle(aggregation_handle_sum_state_.get(), type.makeNullValue());
-    CheckSumValue<typename PrecisionType::cpptype>(sum, *aggregation_handle_sum_, *aggregation_handle_sum_state_);
+    CheckSumValue<typename PrecisionType::cpptype>(
+        sum, *aggregation_handle_sum_, *aggregation_handle_sum_state_);
 
     // Test mergeStates().
     std::unique_ptr<AggregationState> merge_state(
@@ -138,7 +148,7 @@ class AggregationHandleSumTest : public::testing::Test {
       if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
         SetDataType(i - 10, &val);
       } else {
-        SetDataType(static_cast<float>(i - 10)/10, &val);
+        SetDataType(static_cast<float>(i - 10) / 10, &val);
       }
       iterateHandle(merge_state.get(), type.makeValue(&val));
       sum += val;
@@ -146,13 +156,11 @@ class AggregationHandleSumTest : public::testing::Test {
     aggregation_handle_sum_->mergeStates(*merge_state,
                                          aggregation_handle_sum_state_.get());
     CheckSumValue<typename PrecisionType::cpptype>(
-        sum,
-        *aggregation_handle_sum_,
-        *aggregation_handle_sum_state_);
+        sum, *aggregation_handle_sum_, *aggregation_handle_sum_state_);
   }
 
   template <typename GenericType, typename Output>
-  ColumnVector *createColumnVectorGeneric(const Type &type, Output *sum) {
+  ColumnVector* createColumnVectorGeneric(const Type &type, Output *sum) {
     NativeColumnVector *column = new NativeColumnVector(type, kNumSamples + 3);
 
     typename GenericType::cpptype val;
@@ -163,12 +171,12 @@ class AggregationHandleSumTest : public::testing::Test {
       if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
         SetDataType(i - 10, &val);
       } else {
-        SetDataType(static_cast<float>(i - 10)/10, &val);
+        SetDataType(static_cast<float>(i - 10) / 10, &val);
       }
       column->appendTypedValue(type.makeValue(&val));
       *sum += val;
       // One NULL in the middle.
-      if (i == kNumSamples/2) {
+      if (i == kNumSamples / 2) {
         column->appendTypedValue(type.makeNullValue());
       }
     }
@@ -182,12 +190,15 @@ class AggregationHandleSumTest : public::testing::Test {
     const GenericType &type = GenericType::Instance(true);
 
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_sum_->finalize(*aggregation_handle_sum_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_sum_->finalize(*aggregation_handle_sum_state_)
+            .isNull());
 
     typename PrecisionType::cpptype sum;
     std::vector<std::unique_ptr<ColumnVector>> column_vectors;
     column_vectors.emplace_back(
-        createColumnVectorGeneric<GenericType, typename PrecisionType::cpptype>(type, &sum));
+        createColumnVectorGeneric<GenericType, typename PrecisionType::cpptype>(
+            type, &sum));
 
     std::unique_ptr<AggregationState> cv_state(
         aggregation_handle_sum_->accumulateColumnVectors(column_vectors));
@@ -195,15 +206,12 @@ class AggregationHandleSumTest : public::testing::Test {
     // Test the state generated directly by accumulateColumnVectors(), and also
     // test after merging back.
     CheckSumValue<typename PrecisionType::cpptype>(
-        sum,
-        *aggregation_handle_sum_,
-        *cv_state);
+        sum, *aggregation_handle_sum_, *cv_state);
 
-    aggregation_handle_sum_->mergeStates(*cv_state, aggregation_handle_sum_state_.get());
+    aggregation_handle_sum_->mergeStates(*cv_state,
+                                         aggregation_handle_sum_state_.get());
     CheckSumValue<typename PrecisionType::cpptype>(
-        sum,
-        *aggregation_handle_sum_,
-        *aggregation_handle_sum_state_);
+        sum, *aggregation_handle_sum_, *aggregation_handle_sum_state_);
   }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -212,29 +220,30 @@ class AggregationHandleSumTest : public::testing::Test {
     const GenericType &type = GenericType::Instance(true);
 
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_sum_->finalize(*aggregation_handle_sum_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_sum_->finalize(*aggregation_handle_sum_state_)
+            .isNull());
 
     typename PrecisionType::cpptype sum;
-    std::unique_ptr<ColumnVectorsValueAccessor> accessor(new ColumnVectorsValueAccessor());
+    std::unique_ptr<ColumnVectorsValueAccessor> accessor(
+        new ColumnVectorsValueAccessor());
     accessor->addColumn(
-        createColumnVectorGeneric<GenericType, typename PrecisionType::cpptype>(type, &sum));
+        createColumnVectorGeneric<GenericType, typename PrecisionType::cpptype>(
+            type, &sum));
 
     std::unique_ptr<AggregationState> va_state(
-        aggregation_handle_sum_->accumulateValueAccessor(accessor.get(),
-                                                         std::vector<attribute_id>(1, 0)));
+        aggregation_handle_sum_->accumulateValueAccessor(
+            accessor.get(), std::vector<attribute_id>(1, 0)));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
     CheckSumValue<typename PrecisionType::cpptype>(
-        sum,
-        *aggregation_handle_sum_,
-        *va_state);
+        sum, *aggregation_handle_sum_, *va_state);
 
-    aggregation_handle_sum_->mergeStates(*va_state, aggregation_handle_sum_state_.get());
+    aggregation_handle_sum_->mergeStates(*va_state,
+                                         aggregation_handle_sum_state_.get());
     CheckSumValue<typename PrecisionType::cpptype>(
-        sum,
-        *aggregation_handle_sum_,
-        *aggregation_handle_sum_state_);
+        sum, *aggregation_handle_sum_, *aggregation_handle_sum_state_);
   }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
@@ -247,9 +256,7 @@ const int AggregationHandleSumTest::kNumSamples;
 
 template <>
 void AggregationHandleSumTest::CheckSumValue<float>(
-    float val,
-    const AggregationHandle &handle,
-    const AggregationState &state) {
+    float val, const AggregationHandle &handle, const AggregationState &state) {
   EXPECT_FLOAT_EQ(val, handle.finalize(state).getLiteral<float>());
 }
 
@@ -262,12 +269,14 @@ void AggregationHandleSumTest::CheckSumValue<double>(
 }
 
 template <>
-void AggregationHandleSumTest::SetDataType<DatetimeIntervalLit>(int value, DatetimeIntervalLit *data) {
+void AggregationHandleSumTest::SetDataType<DatetimeIntervalLit>(
+    int value, DatetimeIntervalLit *data) {
   data->interval_ticks = value;
 }
 
 template <>
-void AggregationHandleSumTest::SetDataType<YearMonthIntervalLit>(int value, YearMonthIntervalLit *data) {
+void AggregationHandleSumTest::SetDataType<YearMonthIntervalLit>(
+    int value, YearMonthIntervalLit *data) {
   data->months = value;
 }
 
@@ -314,11 +323,13 @@ TEST_F(AggregationHandleSumTest, DoubleTypeColumnVectorTest) {
 }
 
 TEST_F(AggregationHandleSumTest, DatetimeIntervalTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<DatetimeIntervalType, DatetimeIntervalType>();
+  checkAggregationSumGenericColumnVector<DatetimeIntervalType,
+                                         DatetimeIntervalType>();
 }
 
 TEST_F(AggregationHandleSumTest, YearMonthIntervalTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<YearMonthIntervalType, YearMonthIntervalType>();
+  checkAggregationSumGenericColumnVector<YearMonthIntervalType,
+                                         YearMonthIntervalType>();
 }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -339,11 +350,13 @@ TEST_F(AggregationHandleSumTest, DoubleTypeValueAccessorTest) {
 }
 
 TEST_F(AggregationHandleSumTest, DatetimeIntervalTypeValueAccessorTest) {
-  checkAggregationSumGenericValueAccessor<DatetimeIntervalType, DatetimeIntervalType>();
+  checkAggregationSumGenericValueAccessor<DatetimeIntervalType,
+                                          DatetimeIntervalType>();
 }
 
 TEST_F(AggregationHandleSumTest, YearMonthIntervalTypeValueAccessorTest) {
-  checkAggregationSumGenericValueAccessor<YearMonthIntervalType, YearMonthIntervalType>();
+  checkAggregationSumGenericValueAccessor<YearMonthIntervalType,
+                                          YearMonthIntervalType>();
 }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
@@ -373,38 +386,53 @@ TEST_F(AggregationHandleSumDeathTest, WrongTypeTest) {
   float float_val = 0;
 
   // Passes.
-  iterateHandle(aggregation_handle_sum_state_.get(), int_non_null_type.makeValue(&int_val));
+  iterateHandle(aggregation_handle_sum_state_.get(),
+                int_non_null_type.makeValue(&int_val));
 
-  EXPECT_DEATH(iterateHandle(aggregation_handle_sum_state_.get(), long_type.makeValue(&long_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_sum_state_.get(), double_type.makeValue(&double_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_sum_state_.get(), float_type.makeValue(&float_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_sum_state_.get(), char_type.makeValue("asdf", 5)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_sum_state_.get(), varchar_type.makeValue("asdf", 5)), "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_sum_state_.get(),
+                             long_type.makeValue(&long_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_sum_state_.get(),
+                             double_type.makeValue(&double_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_sum_state_.get(),
+                             float_type.makeValue(&float_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_sum_state_.get(),
+                             char_type.makeValue("asdf", 5)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_sum_state_.get(),
+                             varchar_type.makeValue("asdf", 5)),
+               "");
 
   // Test mergeStates() with incorrectly typed handles.
   std::unique_ptr<AggregationHandle> aggregation_handle_sum_double(
-      AggregateFunctionFactory::Get(AggregationID::kSum).createHandle(
-          std::vector<const Type*>(1, &double_type)));
+      AggregateFunctionFactory::Get(AggregationID::kSum)
+          .createHandle(std::vector<const Type *>(1, &double_type)));
   std::unique_ptr<AggregationState> aggregation_state_sum_merge_double(
       aggregation_handle_sum_double->createInitialState());
-  static_cast<const AggregationHandleSum&>(*aggregation_handle_sum_double).iterateUnaryInl(
-      static_cast<AggregationStateSum*>(aggregation_state_sum_merge_double.get()),
-      double_type.makeValue(&double_val));
-  EXPECT_DEATH(aggregation_handle_sum_->mergeStates(*aggregation_state_sum_merge_double,
-                                                    aggregation_handle_sum_state_.get()),
-               "");
+  static_cast<const AggregationHandleSum &>(*aggregation_handle_sum_double)
+      .iterateUnaryInl(static_cast<AggregationStateSum *>(
+                           aggregation_state_sum_merge_double.get()),
+                       double_type.makeValue(&double_val));
+  EXPECT_DEATH(
+      aggregation_handle_sum_->mergeStates(*aggregation_state_sum_merge_double,
+                                           aggregation_handle_sum_state_.get()),
+      "");
 
   std::unique_ptr<AggregationHandle> aggregation_handle_sum_float(
-      AggregateFunctionFactory::Get(AggregationID::kSum).createHandle(
-          std::vector<const Type*>(1, &float_type)));
+      AggregateFunctionFactory::Get(AggregationID::kSum)
+          .createHandle(std::vector<const Type *>(1, &float_type)));
   std::unique_ptr<AggregationState> aggregation_state_sum_merge_float(
       aggregation_handle_sum_float->createInitialState());
-  static_cast<const AggregationHandleSum&>(*aggregation_handle_sum_float).iterateUnaryInl(
-      static_cast<AggregationStateSum*>(aggregation_state_sum_merge_float.get()),
-      float_type.makeValue(&float_val));
-  EXPECT_DEATH(aggregation_handle_sum_->mergeStates(*aggregation_state_sum_merge_float,
-                                                    aggregation_handle_sum_state_.get()),
-               "");
+  static_cast<const AggregationHandleSum &>(*aggregation_handle_sum_float)
+      .iterateUnaryInl(static_cast<AggregationStateSum *>(
+                           aggregation_state_sum_merge_float.get()),
+                       float_type.makeValue(&float_val));
+  EXPECT_DEATH(
+      aggregation_handle_sum_->mergeStates(*aggregation_state_sum_merge_float,
+                                           aggregation_handle_sum_state_.get()),
+      "");
 }
 #endif
 
@@ -425,8 +453,10 @@ TEST_F(AggregationHandleSumTest, ResultTypeForArgumentTypeTest) {
   EXPECT_TRUE(ResultTypeForArgumentTypeTest(kLong, kLong));
   EXPECT_TRUE(ResultTypeForArgumentTypeTest(kFloat, kDouble));
   EXPECT_TRUE(ResultTypeForArgumentTypeTest(kDouble, kDouble));
-  EXPECT_TRUE(ResultTypeForArgumentTypeTest(kDatetimeInterval, kDatetimeInterval));
-  EXPECT_TRUE(ResultTypeForArgumentTypeTest(kYearMonthInterval, kYearMonthInterval));
+  EXPECT_TRUE(
+      ResultTypeForArgumentTypeTest(kDatetimeInterval, kDatetimeInterval));
+  EXPECT_TRUE(
+      ResultTypeForArgumentTypeTest(kYearMonthInterval, kYearMonthInterval));
 }
 
 TEST_F(AggregationHandleSumTest, GroupByTableMergeTest) {
@@ -434,25 +464,28 @@ TEST_F(AggregationHandleSumTest, GroupByTableMergeTest) {
   initializeHandle(long_non_null_type);
   storage_manager_.reset(new StorageManager("./test_sum_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      aggregation_handle_sum_->createGroupByHashTable(
-          HashTableImplType::kSimpleScalarSeparateChaining,
+      AggregationStateFastHashTableFactory::CreateResizable(
+          HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
+          {aggregation_handle_sum_.get()->getPayloadSize()},
+          {aggregation_handle_sum_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      aggregation_handle_sum_->createGroupByHashTable(
-          HashTableImplType::kSimpleScalarSeparateChaining,
+      AggregationStateFastHashTableFactory::CreateResizable(
+          HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
+          {aggregation_handle_sum_.get()->getPayloadSize()},
+          {aggregation_handle_sum_.get()},
           storage_manager_.get()));
 
-  AggregationStateHashTable<AggregationStateSum> *destination_hash_table_derived =
-      static_cast<AggregationStateHashTable<AggregationStateSum> *>(
+  AggregationStateFastHashTable *destination_hash_table_derived =
+      static_cast<AggregationStateFastHashTable *>(
           destination_hash_table.get());
 
-  AggregationStateHashTable<AggregationStateSum> *source_hash_table_derived =
-      static_cast<AggregationStateHashTable<AggregationStateSum> *>(
-          source_hash_table.get());
+  AggregationStateFastHashTable *source_hash_table_derived =
+      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
 
   AggregationHandleSum *aggregation_handle_sum_derived =
       static_cast<AggregationHandleSum *>(aggregation_handle_sum_.get());
@@ -471,7 +504,8 @@ TEST_F(AggregationHandleSumTest, GroupByTableMergeTest) {
   const std::int64_t common_key_destination_sum = 4000;
   TypedValue common_key_destination_sum_val(common_key_destination_sum);
 
-  const std::int64_t merged_common_key = common_key_source_sum + common_key_destination_sum;
+  const std::int64_t merged_common_key =
+      common_key_source_sum + common_key_destination_sum;
   TypedValue common_key_merged_val(merged_common_key);
 
   const std::int64_t exclusive_key_source_sum = 100;
@@ -496,59 +530,82 @@ TEST_F(AggregationHandleSumTest, GroupByTableMergeTest) {
   // Create sum value states for keys.
   aggregation_handle_sum_derived->iterateUnaryInl(common_key_source_state.get(),
                                                   common_key_source_sum_val);
-  std::int64_t actual_val = aggregation_handle_sum_->finalize(*common_key_source_state)
-                       .getLiteral<std::int64_t>();
+  std::int64_t actual_val =
+      aggregation_handle_sum_->finalize(*common_key_source_state)
+          .getLiteral<std::int64_t>();
   EXPECT_EQ(common_key_source_sum_val.getLiteral<std::int64_t>(), actual_val);
 
   aggregation_handle_sum_derived->iterateUnaryInl(
       common_key_destination_state.get(), common_key_destination_sum_val);
   actual_val = aggregation_handle_sum_->finalize(*common_key_destination_state)
                    .getLiteral<std::int64_t>();
-  EXPECT_EQ(common_key_destination_sum_val.getLiteral<std::int64_t>(), actual_val);
+  EXPECT_EQ(common_key_destination_sum_val.getLiteral<std::int64_t>(),
+            actual_val);
 
   aggregation_handle_sum_derived->iterateUnaryInl(
       exclusive_key_destination_state.get(), exclusive_key_destination_sum_val);
   actual_val =
       aggregation_handle_sum_->finalize(*exclusive_key_destination_state)
           .getLiteral<std::int64_t>();
-  EXPECT_EQ(exclusive_key_destination_sum_val.getLiteral<std::int64_t>(), actual_val);
+  EXPECT_EQ(exclusive_key_destination_sum_val.getLiteral<std::int64_t>(),
+            actual_val);
 
   aggregation_handle_sum_derived->iterateUnaryInl(
       exclusive_key_source_state.get(), exclusive_key_source_sum_val);
   actual_val = aggregation_handle_sum_->finalize(*exclusive_key_source_state)
                    .getLiteral<std::int64_t>();
-  EXPECT_EQ(exclusive_key_source_sum_val.getLiteral<std::int64_t>(), actual_val);
+  EXPECT_EQ(exclusive_key_source_sum_val.getLiteral<std::int64_t>(),
+            actual_val);
 
   // Add the key-state pairs to the hash tables.
-  source_hash_table_derived->putCompositeKey(common_key,
-                                             *common_key_source_state);
-  destination_hash_table_derived->putCompositeKey(
-      common_key, *common_key_destination_state);
-  source_hash_table_derived->putCompositeKey(exclusive_source_key,
-                                             *exclusive_key_source_state);
-  destination_hash_table_derived->putCompositeKey(
-      exclusive_destination_key, *exclusive_key_destination_state);
+  unsigned char buffer[100];
+  buffer[0] = '\0';
+  memcpy(buffer + 1,
+         common_key_source_state.get()->getPayloadAddress(),
+         aggregation_handle_sum_.get()->getPayloadSize());
+  source_hash_table_derived->putCompositeKey(common_key, buffer);
+
+  memcpy(buffer + 1,
+         common_key_destination_state.get()->getPayloadAddress(),
+         aggregation_handle_sum_.get()->getPayloadSize());
+  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+
+  memcpy(buffer + 1,
+         exclusive_key_source_state.get()->getPayloadAddress(),
+         aggregation_handle_sum_.get()->getPayloadSize());
+  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+
+  memcpy(buffer + 1,
+         exclusive_key_destination_state.get()->getPayloadAddress(),
+         aggregation_handle_sum_.get()->getPayloadSize());
+  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
+                                                      buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  aggregation_handle_sum_->mergeGroupByHashTables(*source_hash_table,
-                                                  destination_hash_table.get());
+  AggregationOperationState::mergeGroupByHashTables(
+      source_hash_table.get(), destination_hash_table.get());
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckSumValue<std::int64_t>(
       common_key_merged_val.getLiteral<std::int64_t>(),
-      *aggregation_handle_sum_derived,
-      *(destination_hash_table_derived->getSingleCompositeKey(common_key)));
-  CheckSumValue<std::int64_t>(exclusive_key_destination_sum_val.getLiteral<std::int64_t>(),
-                     *aggregation_handle_sum_derived,
-                     *(destination_hash_table_derived->getSingleCompositeKey(
-                         exclusive_destination_key)));
-  CheckSumValue<std::int64_t>(exclusive_key_source_sum_val.getLiteral<std::int64_t>(),
-                     *aggregation_handle_sum_derived,
-                     *(source_hash_table_derived->getSingleCompositeKey(
-                         exclusive_source_key)));
+      aggregation_handle_sum_derived->finalizeHashTableEntryFast(
+          destination_hash_table_derived->getSingleCompositeKey(common_key) +
+          1));
+  CheckSumValue<std::int64_t>(
+      exclusive_key_destination_sum_val.getLiteral<std::int64_t>(),
+      aggregation_handle_sum_derived->finalizeHashTableEntryFast(
+          destination_hash_table_derived->getSingleCompositeKey(
+              exclusive_destination_key) +
+          1));
+  CheckSumValue<std::int64_t>(
+      exclusive_key_source_sum_val.getLiteral<std::int64_t>(),
+      aggregation_handle_sum_derived->finalizeHashTableEntryFast(
+          source_hash_table_derived->getSingleCompositeKey(
+              exclusive_source_key) +
+          1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 05d0636..c5f59f9 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -59,7 +59,7 @@ namespace quickstep {
 
 AggregationOperationState::AggregationOperationState(
     const CatalogRelationSchema &input_relation,
-    const std::vector<const AggregateFunction*> &aggregate_functions,
+    const std::vector<const AggregateFunction *> &aggregate_functions,
     std::vector<std::vector<std::unique_ptr<const Scalar>>> &&arguments,
     std::vector<bool> &&is_distinct,
     std::vector<std::unique_ptr<const Scalar>> &&group_by,
@@ -78,7 +78,7 @@ AggregationOperationState::AggregationOperationState(
   DCHECK(aggregate_functions.size() == arguments_.size());
 
   // Get the types of GROUP BY expressions for creating HashTables below.
-  std::vector<const Type*> group_by_types;
+  std::vector<const Type *> group_by_types;
   for (const std::unique_ptr<const Scalar> &group_by_element : group_by_list_) {
     group_by_types.emplace_back(&group_by_element->getType());
   }
@@ -94,27 +94,29 @@ AggregationOperationState::AggregationOperationState(
     handles_.emplace_back(new AggregationHandleDistinct());
     arguments_.push_back({});
     is_distinct_.emplace_back(false);
-    group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
-        new HashTablePool(estimated_num_entries,
-                          hash_table_impl_type,
-                          group_by_types,
-                          {1},
-                          handles_,
-                          storage_manager)));
+    group_by_hashtable_pools_.emplace_back(
+        std::unique_ptr<HashTablePool>(new HashTablePool(estimated_num_entries,
+                                                         hash_table_impl_type,
+                                                         group_by_types,
+                                                         {1},
+                                                         handles_,
+                                                         storage_manager)));
   } else {
     // Set up each individual aggregate in this operation.
-    std::vector<const AggregateFunction*>::const_iterator agg_func_it
-        = aggregate_functions.begin();
-    std::vector<std::vector<std::unique_ptr<const Scalar>>>::const_iterator args_it
-        = arguments_.begin();
+    std::vector<const AggregateFunction *>::const_iterator agg_func_it =
+        aggregate_functions.begin();
+    std::vector<std::vector<std::unique_ptr<const Scalar>>>::const_iterator
+        args_it = arguments_.begin();
     std::vector<bool>::const_iterator is_distinct_it = is_distinct_.begin();
-    std::vector<HashTableImplType>::const_iterator distinctify_hash_table_impl_types_it
-        = distinctify_hash_table_impl_types.begin();
+    std::vector<HashTableImplType>::const_iterator
+        distinctify_hash_table_impl_types_it =
+            distinctify_hash_table_impl_types.begin();
     std::vector<std::size_t> payload_sizes;
-    for (; agg_func_it != aggregate_functions.end(); ++agg_func_it, ++args_it, ++is_distinct_it) {
+    for (; agg_func_it != aggregate_functions.end();
+         ++agg_func_it, ++args_it, ++is_distinct_it) {
       // Get the Types of this aggregate's arguments so that we can create an
       // AggregationHandle.
-      std::vector<const Type*> argument_types;
+      std::vector<const Type *> argument_types;
       for (const std::unique_ptr<const Scalar> &argument : *args_it) {
         argument_types.emplace_back(&argument->getType());
       }
@@ -129,12 +131,13 @@ AggregationOperationState::AggregationOperationState(
       handles_.emplace_back((*agg_func_it)->createHandle(argument_types));
 
       if (!group_by_list_.empty()) {
-        // Aggregation with GROUP BY: combined payload is partially updated in the presence of DISTINCT.
-         if (*is_distinct_it) {
-            handles_.back()->BlockUpdate();
-         }
-         group_by_handles.emplace_back(handles_.back());
-         payload_sizes.emplace_back(group_by_handles.back()->getPayloadSize());
+        // Aggregation with GROUP BY: combined payload is partially updated in
+        // the presence of DISTINCT.
+        if (*is_distinct_it) {
+          handles_.back()->blockUpdate();
+        }
+        group_by_handles.emplace_back(handles_.back());
+        payload_sizes.emplace_back(group_by_handles.back()->getPayloadSize());
       } else {
         // Aggregation without GROUP BY: create a single global state.
         single_states_.emplace_back(handles_.back()->createInitialState());
@@ -146,31 +149,38 @@ AggregationOperationState::AggregationOperationState(
         std::vector<attribute_id> local_arguments_as_attributes;
         local_arguments_as_attributes.reserve(args_it->size());
         for (const std::unique_ptr<const Scalar> &argument : *args_it) {
-          const attribute_id argument_id = argument->getAttributeIdForValueAccessor();
+          const attribute_id argument_id =
+              argument->getAttributeIdForValueAccessor();
           if (argument_id == -1) {
             local_arguments_as_attributes.clear();
             break;
           } else {
-            DCHECK_EQ(input_relation_.getID(), argument->getRelationIdForValueAccessor());
+            DCHECK_EQ(input_relation_.getID(),
+                      argument->getRelationIdForValueAccessor());
             local_arguments_as_attributes.push_back(argument_id);
           }
         }
 
-        arguments_as_attributes_.emplace_back(std::move(local_arguments_as_attributes));
+        arguments_as_attributes_.emplace_back(
+            std::move(local_arguments_as_attributes));
 #endif
       }
 
-      // Initialize the corresponding distinctify hash table if this is a DISTINCT
+      // Initialize the corresponding distinctify hash table if this is a
+      // DISTINCT
       // aggregation.
       if (*is_distinct_it) {
-        std::vector<const Type*> key_types(group_by_types);
-        key_types.insert(key_types.end(), argument_types.begin(), argument_types.end());
-        // TODO(jianqiao): estimated_num_entries is quite inaccurate for estimating
+        std::vector<const Type *> key_types(group_by_types);
+        key_types.insert(
+            key_types.end(), argument_types.begin(), argument_types.end());
+        // TODO(jianqiao): estimated_num_entries is quite inaccurate for
+        // estimating
         // the number of entries in the distinctify hash table. We may estimate
-        // for each distinct aggregation an estimated_num_distinct_keys value during
+        // for each distinct aggregation an estimated_num_distinct_keys value
+        // during
         // query optimization, if it worths.
         distinctify_hashtables_.emplace_back(
-        AggregationStateFastHashTableFactory::CreateResizable(
+            AggregationStateFastHashTableFactory::CreateResizable(
                 *distinctify_hash_table_impl_types_it,
                 key_types,
                 estimated_num_entries,
@@ -184,16 +194,17 @@ AggregationOperationState::AggregationOperationState(
     }
 
     if (!group_by_handles.empty()) {
-      // Aggregation with GROUP BY: create a HashTable pool for per-group states.
+      // Aggregation with GROUP BY: create a HashTable pool for per-group
+      // states.
       group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
-            new HashTablePool(estimated_num_entries,
-                              hash_table_impl_type,
-                              group_by_types,
-                              payload_sizes,
-                              group_by_handles,
-                              storage_manager)));
-      }
+          new HashTablePool(estimated_num_entries,
+                            hash_table_impl_type,
+                            group_by_types,
+                            payload_sizes,
+                            group_by_handles,
+                            storage_manager)));
     }
+  }
 }
 
 AggregationOperationState* AggregationOperationState::ReconstructFromProto(
@@ -203,7 +214,7 @@ AggregationOperationState* AggregationOperationState::ReconstructFromProto(
   DCHECK(ProtoIsValid(proto, database));
 
   // Rebuild contructor arguments from their representation in 'proto'.
-  std::vector<const AggregateFunction*> aggregate_functions;
+  std::vector<const AggregateFunction *> aggregate_functions;
   std::vector<std::vector<std::unique_ptr<const Scalar>>> arguments;
   std::vector<bool> is_distinct;
   std::vector<HashTableImplType> distinctify_hash_table_impl_types;
@@ -216,62 +227,63 @@ AggregationOperationState* AggregationOperationState::ReconstructFromProto(
 
     arguments.emplace_back();
     arguments.back().reserve(agg_proto.argument_size());
-    for (int argument_idx = 0; argument_idx < agg_proto.argument_size(); ++argument_idx) {
+    for (int argument_idx = 0; argument_idx < agg_proto.argument_size();
+         ++argument_idx) {
       arguments.back().emplace_back(ScalarFactory::ReconstructFromProto(
-          agg_proto.argument(argument_idx),
-          database));
+          agg_proto.argument(argument_idx), database));
     }
 
     is_distinct.emplace_back(agg_proto.is_distinct());
 
     if (agg_proto.is_distinct()) {
       distinctify_hash_table_impl_types.emplace_back(
-          HashTableImplTypeFromProto(
-              proto.distinctify_hash_table_impl_types(distinctify_hash_table_impl_type_index)));
+          HashTableImplTypeFromProto(proto.distinctify_hash_table_impl_types(
+              distinctify_hash_table_impl_type_index)));
       ++distinctify_hash_table_impl_type_index;
     }
   }
 
   std::vector<std::unique_ptr<const Scalar>> group_by_expressions;
-  for (int group_by_idx = 0;
-       group_by_idx < proto.group_by_expressions_size();
+  for (int group_by_idx = 0; group_by_idx < proto.group_by_expressions_size();
        ++group_by_idx) {
     group_by_expressions.emplace_back(ScalarFactory::ReconstructFromProto(
-        proto.group_by_expressions(group_by_idx),
-        database));
+        proto.group_by_expressions(group_by_idx), database));
   }
 
   unique_ptr<Predicate> predicate;
   if (proto.has_predicate()) {
     predicate.reset(
-        PredicateFactory::ReconstructFromProto(proto.predicate(),
-                                               database));
+        PredicateFactory::ReconstructFromProto(proto.predicate(), database));
   }
 
-  return new AggregationOperationState(database.getRelationSchemaById(proto.relation_id()),
-                                       aggregate_functions,
-                                       std::move(arguments),
-                                       std::move(is_distinct),
-                                       std::move(group_by_expressions),
-                                       predicate.release(),
-                                       proto.estimated_num_entries(),
-                                       HashTableImplTypeFromProto(proto.hash_table_impl_type()),
-                                       distinctify_hash_table_impl_types,
-                                       storage_manager);
+  return new AggregationOperationState(
+      database.getRelationSchemaById(proto.relation_id()),
+      aggregate_functions,
+      std::move(arguments),
+      std::move(is_distinct),
+      std::move(group_by_expressions),
+      predicate.release(),
+      proto.estimated_num_entries(),
+      HashTableImplTypeFromProto(proto.hash_table_impl_type()),
+      distinctify_hash_table_impl_types,
+      storage_manager);
 }
 
-bool AggregationOperationState::ProtoIsValid(const serialization::AggregationOperationState &proto,
-                                             const CatalogDatabaseLite &database) {
+bool AggregationOperationState::ProtoIsValid(
+    const serialization::AggregationOperationState &proto,
+    const CatalogDatabaseLite &database) {
   if (!proto.IsInitialized() ||
       !database.hasRelationWithId(proto.relation_id()) ||
       (proto.aggregates_size() < 0)) {
     return false;
   }
 
-  std::size_t num_distinctify_hash_tables = proto.distinctify_hash_table_impl_types_size();
+  std::size_t num_distinctify_hash_tables =
+      proto.distinctify_hash_table_impl_types_size();
   std::size_t distinctify_hash_table_impl_type_index = 0;
   for (int i = 0; i < proto.aggregates_size(); ++i) {
-    if (!AggregateFunctionFactory::ProtoIsValid(proto.aggregates(i).function())) {
+    if (!AggregateFunctionFactory::ProtoIsValid(
+            proto.aggregates(i).function())) {
       return false;
     }
 
@@ -282,16 +294,18 @@ bool AggregationOperationState::ProtoIsValid(const serialization::AggregationOpe
     for (int argument_idx = 0;
          argument_idx < proto.aggregates(i).argument_size();
          ++argument_idx) {
-      if (!ScalarFactory::ProtoIsValid(proto.aggregates(i).argument(argument_idx),
-                                       database)) {
+      if (!ScalarFactory::ProtoIsValid(
+              proto.aggregates(i).argument(argument_idx), database)) {
         return false;
       }
     }
 
     if (proto.aggregates(i).is_distinct()) {
-      if (distinctify_hash_table_impl_type_index >= num_distinctify_hash_tables ||
+      if (distinctify_hash_table_impl_type_index >=
+              num_distinctify_hash_tables ||
           !serialization::HashTableImplType_IsValid(
-              proto.distinctify_hash_table_impl_types(distinctify_hash_table_impl_type_index))) {
+              proto.distinctify_hash_table_impl_types(
+                  distinctify_hash_table_impl_type_index))) {
         return false;
       }
     }
@@ -304,8 +318,9 @@ bool AggregationOperationState::ProtoIsValid(const serialization::AggregationOpe
   }
 
   if (proto.group_by_expressions_size() > 0) {
-    if (!proto.has_hash_table_impl_type()
-        || !serialization::HashTableImplType_IsValid(proto.hash_table_impl_type())) {
+    if (!proto.has_hash_table_impl_type() ||
+        !serialization::HashTableImplType_IsValid(
+            proto.hash_table_impl_type())) {
       return false;
     }
   }
@@ -327,7 +342,8 @@ void AggregationOperationState::aggregateBlock(const block_id input_block) {
   }
 }
 
-void AggregationOperationState::finalizeAggregate(InsertDestination *output_destination) {
+void AggregationOperationState::finalizeAggregate(
+    InsertDestination *output_destination) {
   if (group_by_list_.empty()) {
     finalizeSingleState(output_destination);
   } else {
@@ -346,19 +362,19 @@ void AggregationOperationState::mergeSingleState(
   }
 }
 
-void AggregationOperationState::aggregateBlockSingleState(const block_id input_block) {
+void AggregationOperationState::aggregateBlockSingleState(
+    const block_id input_block) {
   // Aggregate per-block state for each aggregate.
   std::vector<std::unique_ptr<AggregationState>> local_state;
 
-  BlockReference block(storage_manager_->getBlock(input_block, input_relation_));
+  BlockReference block(
+      storage_manager_->getBlock(input_block, input_relation_));
 
   // If there is a filter predicate, 'reuse_matches' holds the set of matching
   // tuples so that it can be reused across multiple aggregates (i.e. we only
   // pay the cost of evaluating the predicate once).
   std::unique_ptr<TupleIdSequence> reuse_matches;
-  for (std::size_t agg_idx = 0;
-       agg_idx < handles_.size();
-       ++agg_idx) {
+  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
     const std::vector<attribute_id> *local_arguments_as_attributes = nullptr;
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
     // If all arguments are attributes of the input relation, elide a copy.
@@ -381,12 +397,11 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
       local_state.emplace_back(nullptr);
     } else {
       // Call StorageBlock::aggregate() to actually do the aggregation.
-      local_state.emplace_back(
-          block->aggregate(*handles_[agg_idx],
-                           arguments_[agg_idx],
-                           local_arguments_as_attributes,
-                           predicate_.get(),
-                           &reuse_matches));
+      local_state.emplace_back(block->aggregate(*handles_[agg_idx],
+                                                arguments_[agg_idx],
+                                                local_arguments_as_attributes,
+                                                predicate_.get(),
+                                                &reuse_matches));
     }
   }
 
@@ -394,8 +409,10 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
   mergeSingleState(local_state);
 }
 
-void AggregationOperationState::aggregateBlockHashTable(const block_id input_block) {
-  BlockReference block(storage_manager_->getBlock(input_block, input_relation_));
+void AggregationOperationState::aggregateBlockHashTable(
+    const block_id input_block) {
+  BlockReference block(
+      storage_manager_->getBlock(input_block, input_relation_));
 
   // If there is a filter predicate, 'reuse_matches' holds the set of matching
   // tuples so that it can be reused across multiple aggregates (i.e. we only
@@ -407,11 +424,10 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
   // GROUP BY expressions once).
   std::vector<std::unique_ptr<ColumnVector>> reuse_group_by_vectors;
 
-  for (std::size_t agg_idx = 0;
-       agg_idx < handles_.size();
-       ++agg_idx) {
+  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
     if (is_distinct_[agg_idx]) {
-      // Call StorageBlock::aggregateDistinct() to insert the GROUP BY expression
+      // Call StorageBlock::aggregateDistinct() to insert the GROUP BY
+      // expression
       // values and the aggregation arguments together as keys directly into the
       // (threadsafe) shared global distinctify HashTable for this aggregate.
       block->aggregateDistinct(*handles_[agg_idx],
@@ -429,7 +445,8 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
   // directly into the (threadsafe) shared global HashTable for this
   // aggregate.
   DCHECK(group_by_hashtable_pools_[0] != nullptr);
-  AggregationStateHashTableBase *agg_hash_table = group_by_hashtable_pools_[0]->getHashTableFast();
+  AggregationStateHashTableBase *agg_hash_table =
+      group_by_hashtable_pools_[0]->getHashTableFast();
   DCHECK(agg_hash_table != nullptr);
   block->aggregateGroupByFast(arguments_,
                               group_by_list_,
@@ -440,32 +457,35 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
   group_by_hashtable_pools_[0]->returnHashTable(agg_hash_table);
 }
 
-void AggregationOperationState::finalizeSingleState(InsertDestination *output_destination) {
+void AggregationOperationState::finalizeSingleState(
+    InsertDestination *output_destination) {
   // Simply build up a Tuple from the finalized values for each aggregate and
   // insert it in '*output_destination'.
   std::vector<TypedValue> attribute_values;
 
-  for (std::size_t agg_idx = 0;
-       agg_idx < handles_.size();
-       ++agg_idx) {
+  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
     if (is_distinct_[agg_idx]) {
       single_states_[agg_idx].reset(
-          handles_[agg_idx]->aggregateOnDistinctifyHashTableForSingle(*distinctify_hashtables_[agg_idx]));
+          handles_[agg_idx]->aggregateOnDistinctifyHashTableForSingle(
+              *distinctify_hashtables_[agg_idx]));
     }
 
-    attribute_values.emplace_back(handles_[agg_idx]->finalize(*single_states_[agg_idx]));
+    attribute_values.emplace_back(
+        handles_[agg_idx]->finalize(*single_states_[agg_idx]));
   }
 
   output_destination->insertTuple(Tuple(std::move(attribute_values)));
 }
 
-void AggregationOperationState::mergeGroupByHashTables(AggregationStateHashTableBase *src,
-                                                       AggregationStateHashTableBase *dst) {
-    HashTableMergerFast merger(dst);
-    (static_cast<FastHashTable<true, false, true, false> *>(src))->forEachCompositeKeyFast(&merger);
+void AggregationOperationState::mergeGroupByHashTables(
+    AggregationStateHashTableBase *src, AggregationStateHashTableBase *dst) {
+  HashTableMergerFast merger(dst);
+  (static_cast<FastHashTable<true, false, true, false> *>(src))
+      ->forEachCompositeKeyFast(&merger);
 }
 
-void AggregationOperationState::finalizeHashTable(InsertDestination *output_destination) {
+void AggregationOperationState::finalizeHashTable(
+    InsertDestination *output_destination) {
   // Each element of 'group_by_keys' is a vector of values for a particular
   // group (which is also the prefix of the finalized Tuple for that group).
   std::vector<std::vector<TypedValue>> group_by_keys;
@@ -483,17 +503,14 @@ void AggregationOperationState::finalizeHashTable(InsertDestination *output_dest
          hash_table_index < static_cast<int>(hash_tables->size() - 1);
          ++hash_table_index) {
       // Merge each hash table to the last hash table.
-      mergeGroupByHashTables(
-          (*hash_tables)[hash_table_index].get(),
-          hash_tables->back().get());
+      mergeGroupByHashTables((*hash_tables)[hash_table_index].get(),
+                             hash_tables->back().get());
     }
   }
 
   // Collect per-aggregate finalized values.
   std::vector<std::unique_ptr<ColumnVector>> final_values;
-  for (std::size_t agg_idx = 0;
-       agg_idx < handles_.size();
-       ++agg_idx) {
+  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
     if (is_distinct_[agg_idx]) {
       DCHECK(group_by_hashtable_pools_[0] != nullptr);
       auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
@@ -502,18 +519,17 @@ void AggregationOperationState::finalizeHashTable(InsertDestination *output_dest
         // We may have a case where hash_tables is empty, e.g. no input blocks.
         // However for aggregateOnDistinctifyHashTableForGroupBy to work
         // correctly, we should create an empty group by hash table.
-        AggregationStateHashTableBase *new_hash_table = group_by_hashtable_pools_[0]->getHashTableFast();
+        AggregationStateHashTableBase *new_hash_table =
+            group_by_hashtable_pools_[0]->getHashTableFast();
         group_by_hashtable_pools_[0]->returnHashTable(new_hash_table);
         hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
       }
       DCHECK(hash_tables->back() != nullptr);
       AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
       DCHECK(agg_hash_table != nullptr);
-      handles_[agg_idx]->AllowUpdate();
+      handles_[agg_idx]->allowUpdate();
       handles_[agg_idx]->aggregateOnDistinctifyHashTableForGroupBy(
-          *distinctify_hashtables_[agg_idx],
-          agg_hash_table,
-          agg_idx);
+          *distinctify_hashtables_[agg_idx], agg_hash_table, agg_idx);
     }
 
     auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
@@ -522,16 +538,15 @@ void AggregationOperationState::finalizeHashTable(InsertDestination *output_dest
       // We may have a case where hash_tables is empty, e.g. no input blocks.
       // However for aggregateOnDistinctifyHashTableForGroupBy to work
       // correctly, we should create an empty group by hash table.
-      AggregationStateHashTableBase *new_hash_table = group_by_hashtable_pools_[0]->getHashTable();
+      AggregationStateHashTableBase *new_hash_table =
+          group_by_hashtable_pools_[0]->getHashTable();
       group_by_hashtable_pools_[0]->returnHashTable(new_hash_table);
       hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
     }
     AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
     DCHECK(agg_hash_table != nullptr);
-    ColumnVector* agg_result_col =
-        handles_[agg_idx]->finalizeHashTable(*agg_hash_table,
-                                             &group_by_keys,
-                                              agg_idx);
+    ColumnVector *agg_result_col = handles_[agg_idx]->finalizeHashTable(
+        *agg_hash_table, &group_by_keys, agg_idx);
     if (agg_result_col != nullptr) {
       final_values.emplace_back(agg_result_col);
     }
@@ -549,16 +564,20 @@ void AggregationOperationState::finalizeHashTable(InsertDestination *output_dest
   for (const std::unique_ptr<const Scalar> &group_by_element : group_by_list_) {
     const Type &group_by_type = group_by_element->getType();
     if (NativeColumnVector::UsableForType(group_by_type)) {
-      NativeColumnVector *element_cv = new NativeColumnVector(group_by_type, group_by_keys.size());
+      NativeColumnVector *element_cv =
+          new NativeColumnVector(group_by_type, group_by_keys.size());
       group_by_cvs.emplace_back(element_cv);
       for (std::vector<TypedValue> &group_key : group_by_keys) {
-        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
+        element_cv->appendTypedValue(
+            std::move(group_key[group_by_element_idx]));
       }
     } else {
-      IndirectColumnVector *element_cv = new IndirectColumnVector(group_by_type, group_by_keys.size());
+      IndirectColumnVector *element_cv =
+          new IndirectColumnVector(group_by_type, group_by_keys.size());
       group_by_cvs.emplace_back(element_cv);
       for (std::vector<TypedValue> &group_key : group_by_keys) {
-        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
+        element_cv->appendTypedValue(
+            std::move(group_key[group_by_element_idx]));
       }
     }
     ++group_by_element_idx;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index d408c22..7956bc6 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -102,16 +102,17 @@ class AggregationOperationState {
    *        tables. Single aggregation state (when GROUP BY list is not
    *        specified) is not allocated using memory from storage manager.
    */
-  AggregationOperationState(const CatalogRelationSchema &input_relation,
-                            const std::vector<const AggregateFunction*> &aggregate_functions,
-                            std::vector<std::vector<std::unique_ptr<const Scalar>>> &&arguments,
-                            std::vector<bool> &&is_distinct,
-                            std::vector<std::unique_ptr<const Scalar>> &&group_by,
-                            const Predicate *predicate,
-                            const std::size_t estimated_num_entries,
-                            const HashTableImplType hash_table_impl_type,
-                            const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
-                            StorageManager *storage_manager);
+  AggregationOperationState(
+      const CatalogRelationSchema &input_relation,
+      const std::vector<const AggregateFunction *> &aggregate_functions,
+      std::vector<std::vector<std::unique_ptr<const Scalar>>> &&arguments,
+      std::vector<bool> &&is_distinct,
+      std::vector<std::unique_ptr<const Scalar>> &&group_by,
+      const Predicate *predicate,
+      const std::size_t estimated_num_entries,
+      const HashTableImplType hash_table_impl_type,
+      const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
+      StorageManager *storage_manager);
 
   ~AggregationOperationState() {}
 
@@ -143,8 +144,9 @@ class AggregationOperationState {
    *        in.
    * @return Whether proto is fully-formed and valid.
    **/
-  static bool ProtoIsValid(const serialization::AggregationOperationState &proto,
-                           const CatalogDatabaseLite &database);
+  static bool ProtoIsValid(
+      const serialization::AggregationOperationState &proto,
+      const CatalogDatabaseLite &database);
 
   /**
    * @brief Compute aggregates on the tuples of the given storage block,
@@ -165,12 +167,16 @@ class AggregationOperationState {
    **/
   void finalizeAggregate(InsertDestination *output_destination);
 
+  static void mergeGroupByHashTables(AggregationStateHashTableBase *src,
+                                     AggregationStateHashTableBase *dst);
+
   int dflag;
 
  private:
   // Merge locally (per storage block) aggregated states with global aggregation
   // states.
-  void mergeSingleState(const std::vector<std::unique_ptr<AggregationState>> &local_state);
+  void mergeSingleState(
+      const std::vector<std::unique_ptr<AggregationState>> &local_state);
 
   // Aggregate on input block.
   void aggregateBlockSingleState(const block_id input_block);
@@ -187,7 +193,7 @@ class AggregationOperationState {
 
   // Each individual aggregate in this operation has an AggregationHandle and
   // some number of Scalar arguments.
-//  std::vector<std::unique_ptr<AggregationHandle>> handles_;
+  //  std::vector<std::unique_ptr<AggregationHandle>> handles_;
   std::vector<AggregationHandle *> handles_;
   std::vector<std::vector<std::unique_ptr<const Scalar>>> arguments_;
 
@@ -196,7 +202,8 @@ class AggregationOperationState {
   std::vector<bool> is_distinct_;
 
   // Hash table for obtaining distinct (i.e. unique) arguments.
-  std::vector<std::unique_ptr<AggregationStateHashTableBase>> distinctify_hashtables_;
+  std::vector<std::unique_ptr<AggregationStateHashTableBase>>
+      distinctify_hashtables_;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   // If all an aggregate's argument expressions are simply attributes in
@@ -211,15 +218,14 @@ class AggregationOperationState {
   //
   // TODO(shoban): We should ideally store the aggregation state together in one
   // hash table to prevent multiple lookups.
-  std::vector<std::unique_ptr<AggregationStateHashTableBase>> group_by_hashtables_;
+  std::vector<std::unique_ptr<AggregationStateHashTableBase>>
+      group_by_hashtables_;
 
   // A vector of group by hash table pools, one for each group by clause.
   std::vector<std::unique_ptr<HashTablePool>> group_by_hashtable_pools_;
 
   StorageManager *storage_manager_;
 
-  void mergeGroupByHashTables(AggregationStateHashTableBase *src, AggregationStateHashTableBase *dst);
-
   DISALLOW_COPY_AND_ASSIGN(AggregationOperationState);
 };
 


[27/28] incubator-quickstep git commit: Changes due to rebasing.

Posted by hb...@apache.org.
Changes due to rebasing.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/402c99a4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/402c99a4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/402c99a4

Branch: refs/heads/partitioned-aggregation
Commit: 402c99a4639c37bd055b7e4c1ca3e341245f0fb5
Parents: 1db46d6
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Tue Sep 6 15:14:16 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:57:06 2016 -0500

----------------------------------------------------------------------
 query_execution/QueryContext.hpp      | 11 ---------
 relational_operators/CMakeLists.txt   |  1 -
 storage/AggregationOperationState.cpp | 36 ++++++++++++++++--------------
 storage/AggregationOperationState.hpp |  3 +++
 storage/CMakeLists.txt                |  1 +
 storage/StorageBlock.cpp              |  8 ++++---
 6 files changed, 28 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/402c99a4/query_execution/QueryContext.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.hpp b/query_execution/QueryContext.hpp
index f6c08ba..a69f607 100644
--- a/query_execution/QueryContext.hpp
+++ b/query_execution/QueryContext.hpp
@@ -195,17 +195,6 @@ class QueryContext {
   }
 
   /**
-   * @brief Destroy the given aggregation state.
-   *
-   * @param id The ID of the AggregationOperationState to destroy.
-   **/
-  inline void destroyAggregationState(const aggregation_state_id id) {
-    DCHECK_LT(id, aggregation_states_.size());
-    DCHECK(aggregation_states_[id]);
-    aggregation_states_[id].reset(nullptr);
-  }
-
-  /**
    * @brief Whether the given BloomFilter id is valid.
    *
    * @param id The BloomFilter id.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/402c99a4/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 2632995..a87e370 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -44,7 +44,6 @@ add_library(quickstep_relationaloperators_DestroyAggregationStateOperator
             DestroyAggregationStateOperator.cpp 
             DestroyAggregationStateOperator.hpp)
 add_library(quickstep_relationaloperators_DeleteOperator DeleteOperator.cpp DeleteOperator.hpp)
-add_library(quickstep_relationaloperators_DestroyAggregationStateOperator DestroyAggregationStateOperator.cpp DestroyAggregationStateOperator.hpp)
 add_library(quickstep_relationaloperators_DestroyHashOperator DestroyHashOperator.cpp DestroyHashOperator.hpp)
 add_library(quickstep_relationaloperators_DropTableOperator DropTableOperator.cpp DropTableOperator.hpp)
 add_library(quickstep_relationaloperators_FinalizeAggregationOperator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/402c99a4/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 66c4a83..2e88e70 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -68,7 +68,10 @@ AggregationOperationState::AggregationOperationState(
     const HashTableImplType hash_table_impl_type,
     const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
     StorageManager *storage_manager)
-    : is_aggregate_partitioned_(estimated_num_entries > kPartitionedAggregateThreshold && !group_by.empty()),
+    : is_aggregate_partitioned_(
+          estimated_num_entries > kPartitionedAggregateThreshold &&
+          !group_by.empty() && !aggregate_functions.empty()),
+      // Use partitioned aggregation only for non-distinct group by.
       input_relation_(input_relation),
       predicate_(predicate),
       group_by_list_(std::move(group_by)),
@@ -197,13 +200,12 @@ AggregationOperationState::AggregationOperationState(
     if (!group_by_handles.empty()) {
       // Aggregation with GROUP BY: create a HashTable pool for per-group states.
       if (!is_aggregate_partitioned_) {
-        group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
-              new HashTablePool(estimated_num_entries,
-                                hash_table_impl_type,
-                                group_by_types,
-                                payload_sizes,
-                                group_by_handles,
-                                storage_manager)));
+        group_by_hashtable_pool_.reset(new HashTablePool(estimated_num_entries,
+                                                         hash_table_impl_type,
+                                                         group_by_types,
+                                                         payload_sizes,
+                                                         group_by_handles,
+                                                         storage_manager));
         }
       else {
         partitioned_group_by_hashtable_pool_.reset(
@@ -457,16 +459,16 @@ void AggregationOperationState::aggregateBlockHashTable(
     // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
     // directly into the (threadsafe) shared global HashTable for this
     // aggregate.
-    DCHECK(group_by_hashtable_pools_[0] != nullptr);
-    AggregationStateHashTableBase *agg_hash_table = group_by_hashtable_pools_[0]->getHashTableFast();
+    DCHECK(group_by_hashtable_pool_ != nullptr);
+    AggregationStateHashTableBase *agg_hash_table = group_by_hashtable_pool_->getHashTableFast();
     DCHECK(agg_hash_table != nullptr);
-    block->aggregateGroupByFast(arguments_,
-                                group_by_list_,
-                                predicate_.get(),
-                                agg_hash_table,
-                                &reuse_matches,
-                                &reuse_group_by_vectors);
-    group_by_hashtable_pools_[0]->returnHashTable(agg_hash_table);
+    block->aggregateGroupBy(arguments_,
+                            group_by_list_,
+                            predicate_.get(),
+                            agg_hash_table,
+                            &reuse_matches,
+                            &reuse_group_by_vectors);
+    group_by_hashtable_pool_->returnHashTable(agg_hash_table);
   } else {
     block->aggregateGroupByPartitioned(
         arguments_,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/402c99a4/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index ab82c2e..ca63055 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -205,6 +205,9 @@ class AggregationOperationState {
     }
   }
 
+  static void mergeGroupByHashTables(AggregationStateHashTableBase *src,
+                                     AggregationStateHashTableBase *dst);
+
   int dflag;
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/402c99a4/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 3d6e8d4..81b4dae 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -981,6 +981,7 @@ target_link_libraries(quickstep_storage_StorageBlock
                       quickstep_storage_IndexSubBlock
                       quickstep_storage_InsertDestinationInterface
                       quickstep_storage_PackedRowStoreTupleStorageSubBlock
+                      quickstep_storage_PartitionedHashTablePool
                       quickstep_storage_SMAIndexSubBlock
                       quickstep_storage_SplitRowStoreTupleStorageSubBlock
                       quickstep_storage_StorageBlockBase

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/402c99a4/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 5ba2a6e..94c46a8 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -1386,7 +1386,7 @@ void StorageBlock::aggregateGroupByPartitioned(
 
   // IDs of 'arguments' as attributes in the ValueAccessor we create below.
   std::vector<attribute_id> arg_ids;
-  std::vector<std::vector<attribute_id>> argument_ids;
+  std::vector<attribute_id> argument_ids;
 
   // IDs of GROUP BY key element(s) in the ValueAccessor we create below.
   std::vector<attribute_id> key_ids;
@@ -1440,9 +1440,11 @@ void StorageBlock::aggregateGroupByPartitioned(
     arg_ids.clear();
     for (const std::unique_ptr<const Scalar> &args : argument) {
       temp_result.addColumn(args->getAllValues(accessor.get(), &sub_blocks_ref));
-      arg_ids.push_back(attr_id++);
+      argument_ids.push_back(attr_id++);
+    }
+    if (argument.empty()) {
+      argument_ids.push_back(kInvalidAttributeID);
     }
-    argument_ids.push_back(arg_ids);
   }
 
   // Compute the partitions for the tuple formed by group by values.


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

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index 4e0c72b..4a0eca4 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -28,8 +28,8 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/HashTableBase.hpp"
 #include "storage/FastHashTable.hpp"
+#include "storage/HashTableBase.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -56,19 +56,18 @@ class AggregationStateMin : public AggregationState {
   /**
    * @brief Copy constructor (ignores mutex).
    */
-  AggregationStateMin(const AggregationStateMin &orig)
-      : min_(orig.min_) {
-  }
+  AggregationStateMin(const AggregationStateMin &orig) : min_(orig.min_) {}
 
   /**
    * @brief Destructor.
    */
   ~AggregationStateMin() override {}
 
-  size_t getPayloadSize() const {
-     return sizeof(TypedValue);
-  }
+  std::size_t getPayloadSize() const { return sizeof(TypedValue); }
 
+  const std::uint8_t *getPayloadAddress() const {
+    return reinterpret_cast<const uint8_t *>(&min_);
+  }
 
  private:
   friend class AggregationHandleMin;
@@ -76,9 +75,7 @@ class AggregationStateMin : public AggregationState {
   explicit AggregationStateMin(const Type &type)
       : min_(type.getNullableVersion().makeNullValue()) {}
 
-  explicit AggregationStateMin(TypedValue &&value)
-      : min_(std::move(value)) {
-  }
+  explicit AggregationStateMin(TypedValue &&value) : min_(std::move(value)) {}
 
   TypedValue min_;
   SpinMutex mutex_;
@@ -89,8 +86,7 @@ class AggregationStateMin : public AggregationState {
  **/
 class AggregationHandleMin : public AggregationConcreteHandle {
  public:
-  ~AggregationHandleMin() override {
-  }
+  ~AggregationHandleMin() override {}
 
   AggregationState* createInitialState() const override {
     return new AggregationStateMin(type_);
@@ -98,45 +94,46 @@ class AggregationHandleMin : public AggregationConcreteHandle {
 
   AggregationStateHashTableBase* createGroupByHashTable(
       const HashTableImplType hash_table_impl,
-      const std::vector<const Type*> &group_by_types,
+      const std::vector<const Type *> &group_by_types,
       const std::size_t estimated_num_groups,
       StorageManager *storage_manager) const override;
 
   /**
    * @brief Iterate with min aggregation state.
    */
-  inline void iterateUnaryInl(AggregationStateMin *state, const TypedValue &value) const {
+  inline void iterateUnaryInl(AggregationStateMin *state,
+                              const TypedValue &value) const {
     DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
     compareAndUpdate(state, value);
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) const {
-      DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
-      TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
-      compareAndUpdateFast(min_ptr, value);
+  inline void iterateUnaryInlFast(const TypedValue &value,
+                                  std::uint8_t *byte_ptr) const {
+    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
+    compareAndUpdateFast(min_ptr, value);
   }
 
-  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const override {
-    if (block_update) return;
-    iterateUnaryInlFast(arguments.front(), byte_ptr);
+  inline void updateState(const std::vector<TypedValue> &arguments,
+                          std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateUnaryInlFast(arguments.front(), byte_ptr);
+    }
   }
 
-  void BlockUpdate() override {
-      block_update = true;
-  }
+  void blockUpdate() override { block_update_ = true; }
 
-  void AllowUpdate() override {
-      block_update = false;
-  }
+  void allowUpdate() override { block_update_ = false; }
 
-  void initPayload(uint8_t *byte_ptr) const override {
+  void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
     TypedValue t1 = (type_.getNullableVersion().makeNullValue());
     *min_ptr = t1;
   }
 
   AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
+      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
+      const override;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   AggregationState* accumulateValueAccessor(
@@ -153,18 +150,20 @@ class AggregationHandleMin : public AggregationConcreteHandle {
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override;
 
-  void mergeStatesFast(const uint8_t *source,
-                   uint8_t *destination) const override;
+  void mergeStatesFast(const std::uint8_t *source,
+                       std::uint8_t *destination) const override;
 
   TypedValue finalize(const AggregationState &state) const override {
-    return static_cast<const AggregationStateMin&>(state).min_;
+    return static_cast<const AggregationStateMin &>(state).min_;
   }
 
-  inline TypedValue finalizeHashTableEntry(const AggregationState &state) const {
-    return static_cast<const AggregationStateMin&>(state).min_;
+  inline TypedValue finalizeHashTableEntry(
+      const AggregationState &state) const {
+    return static_cast<const AggregationStateMin &>(state).min_;
   }
 
-  inline TypedValue finalizeHashTableEntryFast(const std::uint8_t *byte_ptr) const {
+  inline TypedValue finalizeHashTableEntryFast(
+      const std::uint8_t *byte_ptr) const {
     const TypedValue *min_ptr = reinterpret_cast<const TypedValue *>(byte_ptr);
     return TypedValue(*min_ptr);
   }
@@ -175,24 +174,25 @@ class AggregationHandleMin : public AggregationConcreteHandle {
       int index) const override;
 
   /**
-   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
+   * @brief Implementation of
+   * AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
    *        for MIN aggregation.
    */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table) const override;
+      const AggregationStateHashTableBase &distinctify_hash_table)
+      const override;
 
   /**
-   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
+   * @brief Implementation of
+   * AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
    *        for MIN aggregation.
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
       AggregationStateHashTableBase *aggregation_hash_table,
-      int index) const override;
+      std::size_t index) const override;
 
-  size_t getPayloadSize() const override {
-      return sizeof(TypedValue);
-  }
+  std::size_t getPayloadSize() const override { return sizeof(TypedValue); }
 
  private:
   friend class AggregateFunctionMin;
@@ -205,23 +205,28 @@ class AggregationHandleMin : public AggregationConcreteHandle {
   explicit AggregationHandleMin(const Type &type);
 
   /**
-   * @brief compare the value with min_ and update it if the value is smaller than
+   * @brief compare the value with min_ and update it if the value is smaller
+   *than
    *        current minimum. NULLs are ignored.
    *
    * @param value A TypedValue to compare.
    **/
-  inline void compareAndUpdate(AggregationStateMin *state, const TypedValue &value) const {
+  inline void compareAndUpdate(AggregationStateMin *state,
+                               const TypedValue &value) const {
     if (value.isNull()) return;
 
     SpinMutexLock lock(state->mutex_);
-    if (state->min_.isNull() || fast_comparator_->compareTypedValues(value, state->min_)) {
+    if (state->min_.isNull() ||
+        fast_comparator_->compareTypedValues(value, state->min_)) {
       state->min_ = value;
     }
   }
 
-  inline void compareAndUpdateFast(TypedValue *min_ptr, const TypedValue &value) const {
+  inline void compareAndUpdateFast(TypedValue *min_ptr,
+                                   const TypedValue &value) const {
     if (value.isNull()) return;
-    if (min_ptr->isNull() || fast_comparator_->compareTypedValues(value, *min_ptr)) {
+    if (min_ptr->isNull() ||
+        fast_comparator_->compareTypedValues(value, *min_ptr)) {
       *min_ptr = value;
     }
   }
@@ -229,7 +234,7 @@ class AggregationHandleMin : public AggregationConcreteHandle {
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 
-  bool block_update;
+  bool block_update_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMin);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 1b0bbcd..642d88d 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -43,7 +43,7 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleSum::AggregationHandleSum(const Type &type)
-    : argument_type_(type), block_update(false) {
+    : argument_type_(type), block_update_(false) {
   // We sum Int as Long and Float as Double so that we have more headroom when
   // adding many values.
   TypeID type_precision_id;
@@ -66,11 +66,13 @@ AggregationHandleSum::AggregationHandleSum(const Type &type)
 
   // Make operators to do arithmetic:
   // Add operator for summing argument values.
-  fast_operator_.reset(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-                       .makeUncheckedBinaryOperatorForTypes(sum_type, argument_type_));
+  fast_operator_.reset(
+      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
+          .makeUncheckedBinaryOperatorForTypes(sum_type, argument_type_));
   // Add operator for merging states.
-  merge_operator_.reset(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-                        .makeUncheckedBinaryOperatorForTypes(sum_type, sum_type));
+  merge_operator_.reset(
+      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
+          .makeUncheckedBinaryOperatorForTypes(sum_type, sum_type));
 
   // Result is nullable, because SUM() over 0 values (or all NULL values) is
   // NULL.
@@ -79,14 +81,11 @@ AggregationHandleSum::AggregationHandleSum(const Type &type)
 
 AggregationStateHashTableBase* AggregationHandleSum::createGroupByHashTable(
     const HashTableImplType hash_table_impl,
-    const std::vector<const Type*> &group_by_types,
+    const std::vector<const Type *> &group_by_types,
     const std::size_t estimated_num_groups,
     StorageManager *storage_manager) const {
   return AggregationStateHashTableFactory<AggregationStateSum>::CreateResizable(
-      hash_table_impl,
-      group_by_types,
-      estimated_num_groups,
-      storage_manager);
+      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
 }
 
 AggregationState* AggregationHandleSum::accumulateColumnVectors(
@@ -95,9 +94,7 @@ AggregationState* AggregationHandleSum::accumulateColumnVectors(
       << "Got wrong number of ColumnVectors for SUM: " << column_vectors.size();
   std::size_t num_tuples = 0;
   TypedValue cv_sum = fast_operator_->accumulateColumnVector(
-      blank_state_.sum_,
-      *column_vectors.front(),
-      &num_tuples);
+      blank_state_.sum_, *column_vectors.front(), &num_tuples);
   return new AggregationStateSum(std::move(cv_sum), num_tuples == 0);
 }
 
@@ -110,10 +107,7 @@ AggregationState* AggregationHandleSum::accumulateValueAccessor(
 
   std::size_t num_tuples = 0;
   TypedValue va_sum = fast_operator_->accumulateValueAccessor(
-      blank_state_.sum_,
-      accessor,
-      accessor_ids.front(),
-      &num_tuples);
+      blank_state_.sum_, accessor, accessor_ids.front(), &num_tuples);
   return new AggregationStateSum(std::move(va_sum), num_tuples == 0);
 }
 #endif
@@ -127,31 +121,37 @@ void AggregationHandleSum::aggregateValueAccessorIntoHashTable(
       << "Got wrong number of arguments for SUM: " << argument_ids.size();
 }
 
-void AggregationHandleSum::mergeStates(
-    const AggregationState &source,
-    AggregationState *destination) const {
-  const AggregationStateSum &sum_source = static_cast<const AggregationStateSum&>(source);
-  AggregationStateSum *sum_destination = static_cast<AggregationStateSum*>(destination);
+void AggregationHandleSum::mergeStates(const AggregationState &source,
+                                       AggregationState *destination) const {
+  const AggregationStateSum &sum_source =
+      static_cast<const AggregationStateSum &>(source);
+  AggregationStateSum *sum_destination =
+      static_cast<AggregationStateSum *>(destination);
 
   SpinMutexLock lock(sum_destination->mutex_);
-  sum_destination->sum_ = merge_operator_->applyToTypedValues(sum_destination->sum_,
-                                                              sum_source.sum_);
+  sum_destination->sum_ = merge_operator_->applyToTypedValues(
+      sum_destination->sum_, sum_source.sum_);
   sum_destination->null_ = sum_destination->null_ && sum_source.null_;
 }
 
-void AggregationHandleSum::mergeStatesFast(
-    const uint8_t *source,
-    uint8_t *destination) const {
-    const TypedValue *src_sum_ptr = reinterpret_cast<const TypedValue *>(source+blank_state_.sum_offset);
-    const bool *src_null_ptr = reinterpret_cast<const bool *>(source+blank_state_.null_offset);
-    TypedValue *dst_sum_ptr = reinterpret_cast<TypedValue *>(destination+blank_state_.sum_offset);
-    bool *dst_null_ptr = reinterpret_cast<bool *>(destination+blank_state_.null_offset);
-    *dst_sum_ptr = merge_operator_->applyToTypedValues(*dst_sum_ptr, *src_sum_ptr);
-    *dst_null_ptr = (*dst_null_ptr) && (*src_null_ptr);
+void AggregationHandleSum::mergeStatesFast(const std::uint8_t *source,
+                                           std::uint8_t *destination) const {
+  const TypedValue *src_sum_ptr =
+      reinterpret_cast<const TypedValue *>(source + blank_state_.sum_offset_);
+  const bool *src_null_ptr =
+      reinterpret_cast<const bool *>(source + blank_state_.null_offset_);
+  TypedValue *dst_sum_ptr =
+      reinterpret_cast<TypedValue *>(destination + blank_state_.sum_offset_);
+  bool *dst_null_ptr =
+      reinterpret_cast<bool *>(destination + blank_state_.null_offset_);
+  *dst_sum_ptr =
+      merge_operator_->applyToTypedValues(*dst_sum_ptr, *src_sum_ptr);
+  *dst_null_ptr = (*dst_null_ptr) && (*src_null_ptr);
 }
 
 TypedValue AggregationHandleSum::finalize(const AggregationState &state) const {
-  const AggregationStateSum &agg_state = static_cast<const AggregationStateSum&>(state);
+  const AggregationStateSum &agg_state =
+      static_cast<const AggregationStateSum &>(state);
   if (agg_state.null_) {
     // SUM() over no values is NULL.
     return result_type_->makeNullValue();
@@ -165,31 +165,26 @@ ColumnVector* AggregationHandleSum::finalizeHashTable(
     std::vector<std::vector<TypedValue>> *group_by_keys,
     int index) const {
   return finalizeHashTableHelperFast<AggregationHandleSum,
-                                 AggregationStateFastHashTable>(
-      *result_type_,
-      hash_table,
-      group_by_keys,
-      index);
+                                     AggregationStateFastHashTable>(
+      *result_type_, hash_table, group_by_keys, index);
 }
 
-AggregationState* AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle(
+AggregationState*
+AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
   return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
       AggregationHandleSum,
-      AggregationStateSum>(
-          distinctify_hash_table);
+      AggregationStateSum>(distinctify_hash_table);
 }
 
 void AggregationHandleSum::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
     AggregationStateHashTableBase *aggregation_hash_table,
-    int index) const {
+    std::size_t index) const {
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleSum,
       AggregationStateFastHashTable>(
-          distinctify_hash_table,
-          aggregation_hash_table,
-          index);
+      distinctify_hash_table, aggregation_hash_table, index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index 3e1de48..8d719ab 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -28,8 +28,8 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/HashTableBase.hpp"
 #include "storage/FastHashTable.hpp"
+#include "storage/HashTableBase.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -59,27 +59,31 @@ class AggregationStateSum : public AggregationState {
   AggregationStateSum(const AggregationStateSum &orig)
       : sum_(orig.sum_),
         null_(orig.null_),
-        sum_offset(orig.sum_offset),
-        null_offset(orig.null_offset) {
+        sum_offset_(orig.sum_offset_),
+        null_offset_(orig.null_offset_) {}
+
+  std::size_t getPayloadSize() const {
+    std::size_t p1 = reinterpret_cast<std::size_t>(&sum_);
+    std::size_t p2 = reinterpret_cast<std::size_t>(&mutex_);
+    return (p2 - p1);
+  }
+
+  const std::uint8_t* getPayloadAddress() const {
+    return reinterpret_cast<const uint8_t *>(&sum_);
   }
 
  private:
   friend class AggregationHandleSum;
 
   AggregationStateSum()
-      : sum_(0), null_(true), sum_offset(0),
-        null_offset(reinterpret_cast<uint8_t *>(&null_)-reinterpret_cast<uint8_t *>(&sum_)) {
-  }
+      : sum_(0),
+        null_(true),
+        sum_offset_(0),
+        null_offset_(reinterpret_cast<std::uint8_t *>(&null_) -
+                     reinterpret_cast<std::uint8_t *>(&sum_)) {}
 
   AggregationStateSum(TypedValue &&sum, const bool is_null)
-      : sum_(std::move(sum)), null_(is_null) {
-  }
-
-  size_t getPayloadSize() const {
-     size_t p1 = reinterpret_cast<size_t>(&sum_);
-     size_t p2 = reinterpret_cast<size_t>(&mutex_);
-     return (p2-p1);
-  }
+      : sum_(std::move(sum)), null_(is_null) {}
 
   // TODO(shoban): We might want to specialize sum_ to use atomics for int types
   // similar to in AggregationStateCount.
@@ -87,17 +91,15 @@ class AggregationStateSum : public AggregationState {
   bool null_;
   SpinMutex mutex_;
 
-  int sum_offset, null_offset;
+  int sum_offset_, null_offset_;
 };
 
-
 /**
  * @brief An aggregationhandle for sum.
  **/
 class AggregationHandleSum : public AggregationConcreteHandle {
  public:
-  ~AggregationHandleSum() override {
-  }
+  ~AggregationHandleSum() override {}
 
   AggregationState* createInitialState() const override {
     return new AggregationStateSum(blank_state_);
@@ -105,11 +107,12 @@ class AggregationHandleSum : public AggregationConcreteHandle {
 
   AggregationStateHashTableBase* createGroupByHashTable(
       const HashTableImplType hash_table_impl,
-      const std::vector<const Type*> &group_by_types,
+      const std::vector<const Type *> &group_by_types,
       const std::size_t estimated_num_groups,
       StorageManager *storage_manager) const override;
 
-  inline void iterateUnaryInl(AggregationStateSum *state, const TypedValue &value) const {
+  inline void iterateUnaryInl(AggregationStateSum *state,
+                              const TypedValue &value) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
     if (value.isNull()) return;
 
@@ -118,37 +121,41 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     state->null_ = false;
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) const {
+  inline void iterateUnaryInlFast(const TypedValue &value,
+                                  std::uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
     if (value.isNull()) return;
-    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
-    bool *null_ptr = reinterpret_cast<bool *>(byte_ptr + blank_state_.null_offset);
+    TypedValue *sum_ptr =
+        reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset_);
+    bool *null_ptr =
+        reinterpret_cast<bool *>(byte_ptr + blank_state_.null_offset_);
     *sum_ptr = fast_operator_->applyToTypedValues(*sum_ptr, value);
     *null_ptr = false;
   }
 
-  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) const override {
-     if (block_update) return;
-     iterateUnaryInlFast(arguments.front(), byte_ptr);
+  inline void updateState(const std::vector<TypedValue> &arguments,
+                          std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateUnaryInlFast(arguments.front(), byte_ptr);
+    }
   }
 
-  void BlockUpdate() override {
-      block_update = true;
-  }
+  void blockUpdate() override { block_update_ = true; }
 
-  void AllowUpdate() override {
-      block_update = false;
-  }
+  void allowUpdate() override { block_update_ = false; }
 
-  void initPayload(uint8_t *byte_ptr) const override {
-    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
-    bool *null_ptr = reinterpret_cast<bool *>(byte_ptr + blank_state_.null_offset);
+  void initPayload(std::uint8_t *byte_ptr) const override {
+    TypedValue *sum_ptr =
+        reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset_);
+    bool *null_ptr =
+        reinterpret_cast<bool *>(byte_ptr + blank_state_.null_offset_);
     *sum_ptr = blank_state_.sum_;
     *null_ptr = true;
   }
 
   AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
+      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
+      const override;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   AggregationState* accumulateValueAccessor(
@@ -165,18 +172,21 @@ class AggregationHandleSum : public AggregationConcreteHandle {
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override;
 
-  void mergeStatesFast(const uint8_t *source,
-                   uint8_t *destination) const override;
+  void mergeStatesFast(const std::uint8_t *source,
+                       std::uint8_t *destination) const override;
 
   TypedValue finalize(const AggregationState &state) const override;
 
-  inline TypedValue finalizeHashTableEntry(const AggregationState &state) const {
-    return static_cast<const AggregationStateSum&>(state).sum_;
+  inline TypedValue finalizeHashTableEntry(
+      const AggregationState &state) const {
+    return static_cast<const AggregationStateSum &>(state).sum_;
   }
 
-  inline TypedValue finalizeHashTableEntryFast(const uint8_t *byte_ptr) const {
-    uint8_t *value_ptr = const_cast<uint8_t*>(byte_ptr);
-    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(value_ptr + blank_state_.sum_offset);
+  inline TypedValue finalizeHashTableEntryFast(
+      const std::uint8_t *byte_ptr) const {
+    std::uint8_t *value_ptr = const_cast<std::uint8_t *>(byte_ptr);
+    TypedValue *sum_ptr =
+        reinterpret_cast<TypedValue *>(value_ptr + blank_state_.sum_offset_);
     return *sum_ptr;
   }
 
@@ -186,23 +196,26 @@ class AggregationHandleSum : public AggregationConcreteHandle {
       int index) const override;
 
   /**
-   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
+   * @brief Implementation of
+   * AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
    *        for SUM aggregation.
    */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table) const override;
+      const AggregationStateHashTableBase &distinctify_hash_table)
+      const override;
 
   /**
-   * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
+   * @brief Implementation of
+   * AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
    *        for SUM aggregation.
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
       AggregationStateHashTableBase *aggregation_hash_table,
-      int index) const override;
+      std::size_t index) const override;
 
-  size_t getPayloadSize() const override {
-      return blank_state_.getPayloadSize();
+  std::size_t getPayloadSize() const override {
+    return blank_state_.getPayloadSize();
   }
 
  private:
@@ -221,7 +234,7 @@ class AggregationHandleSum : public AggregationConcreteHandle {
   std::unique_ptr<UncheckedBinaryOperator> fast_operator_;
   std::unique_ptr<UncheckedBinaryOperator> merge_operator_;
 
-  bool block_update;
+  bool block_update_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleSum);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index 30f9784..e9503f7 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -280,45 +280,46 @@ target_link_libraries(quickstep_expressions_aggregation
 # Tests:
 
 # Unified executable to ammortize cost of linking.
-# add_executable(AggregationHandle_tests
-#               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleAvg_unittest.cpp"
-#               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleCount_unittest.cpp"
-#               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleMax_unittest.cpp"
-#               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleMin_unittest.cpp"
-#               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleSum_unittest.cpp")
-# target_link_libraries(AggregationHandle_tests
-#                      gtest
-#                      gtest_main
-#                      quickstep_catalog_CatalogTypedefs
-#                      quickstep_expressions_aggregation_AggregateFunction
-#                      quickstep_expressions_aggregation_AggregateFunctionFactory
-#                      quickstep_expressions_aggregation_AggregationHandle
-#                      quickstep_expressions_aggregation_AggregationHandleAvg
-#                      quickstep_expressions_aggregation_AggregationHandleCount
-#                      quickstep_expressions_aggregation_AggregationHandleMax
-#                      quickstep_expressions_aggregation_AggregationHandleMin
-#                      quickstep_expressions_aggregation_AggregationHandleSum
-#                      quickstep_expressions_aggregation_AggregationID
-#                      quickstep_storage_HashTableBase
-#                      quickstep_storage_StorageManager
-#                      quickstep_types_CharType
-#                      quickstep_types_DateOperatorOverloads
-#                      quickstep_types_DatetimeIntervalType
-#                      quickstep_types_DatetimeType
-#                      quickstep_types_DoubleType
-#                      quickstep_types_FloatType
-#                      quickstep_types_IntType
-#                      quickstep_types_IntervalLit
-#                      quickstep_types_LongType
-#                      quickstep_types_Type
-#                      quickstep_types_TypeFactory
-#                      quickstep_types_TypeID
-#                      quickstep_types_TypedValue
-#                      quickstep_types_VarCharType
-#                      quickstep_types_YearMonthIntervalType
-#                      quickstep_types_containers_ColumnVector
-#                      quickstep_types_containers_ColumnVectorsValueAccessor
-#                      quickstep_types_operations_comparisons_Comparison
-#                      quickstep_types_operations_comparisons_ComparisonFactory
-#                      quickstep_types_operations_comparisons_ComparisonID)
-#add_test(AggregationHandle_tests AggregationHandle_tests)
+add_executable(AggregationHandle_tests
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleAvg_unittest.cpp"
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleCount_unittest.cpp"
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleMax_unittest.cpp"
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleMin_unittest.cpp"
+               "${CMAKE_CURRENT_SOURCE_DIR}/tests/AggregationHandleSum_unittest.cpp")
+target_link_libraries(AggregationHandle_tests
+                      gtest
+                      gtest_main
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregateFunction
+                      quickstep_expressions_aggregation_AggregateFunctionFactory
+                      quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_expressions_aggregation_AggregationHandleAvg
+                      quickstep_expressions_aggregation_AggregationHandleCount
+                      quickstep_expressions_aggregation_AggregationHandleMax
+                      quickstep_expressions_aggregation_AggregationHandleMin
+                      quickstep_expressions_aggregation_AggregationHandleSum
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_AggregationOperationState
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_StorageManager
+                      quickstep_types_CharType
+                      quickstep_types_DateOperatorOverloads
+                      quickstep_types_DatetimeIntervalType
+                      quickstep_types_DatetimeType
+                      quickstep_types_DoubleType
+                      quickstep_types_FloatType
+                      quickstep_types_IntType
+                      quickstep_types_IntervalLit
+                      quickstep_types_LongType
+                      quickstep_types_Type
+                      quickstep_types_TypeFactory
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_types_VarCharType
+                      quickstep_types_YearMonthIntervalType
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_types_operations_comparisons_Comparison
+                      quickstep_types_operations_comparisons_ComparisonFactory
+                      quickstep_types_operations_comparisons_ComparisonID)
+add_test(AggregationHandle_tests AggregationHandle_tests)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp b/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
index afc02ec..79d4448 100644
--- a/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
@@ -28,6 +28,8 @@
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "expressions/aggregation/AggregationHandleAvg.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
+#include "storage/AggregationOperationState.hpp"
+#include "storage/FastHashTableFactory.hpp"
 #include "storage/StorageManager.hpp"
 #include "types/CharType.hpp"
 #include "types/DateOperatorOverloads.hpp"
@@ -53,51 +55,56 @@
 
 namespace quickstep {
 
-class AggregationHandleAvgTest : public::testing::Test {
+class AggregationHandleAvgTest : public ::testing::Test {
  protected:
   static const int kNumSamples = 100;
 
   // Helper method that calls AggregationHandleAvg::iterateUnaryInl() to
   // aggregate 'value' into '*state'.
   void iterateHandle(AggregationState *state, const TypedValue &value) {
-    static_cast<const AggregationHandleAvg&>(*aggregation_handle_avg_).iterateUnaryInl(
-        static_cast<AggregationStateAvg*>(state),
-        value);
+    static_cast<const AggregationHandleAvg &>(*aggregation_handle_avg_)
+        .iterateUnaryInl(static_cast<AggregationStateAvg *>(state), value);
   }
 
   void initializeHandle(const Type &type) {
     aggregation_handle_avg_.reset(
-        AggregateFunctionFactory::Get(AggregationID::kAvg).createHandle(
-            std::vector<const Type*>(1, &type)));
+        AggregateFunctionFactory::Get(AggregationID::kAvg)
+            .createHandle(std::vector<const Type *>(1, &type)));
     aggregation_handle_avg_state_.reset(
         aggregation_handle_avg_->createInitialState());
   }
 
   static bool ApplyToTypesTest(TypeID typeID) {
-    const Type &type = (typeID == kChar || typeID == kVarChar) ?
-        TypeFactory::GetType(typeID, static_cast<std::size_t>(10)) :
-        TypeFactory::GetType(typeID);
+    const Type &type =
+        (typeID == kChar || typeID == kVarChar)
+            ? TypeFactory::GetType(typeID, static_cast<std::size_t>(10))
+            : TypeFactory::GetType(typeID);
 
-    return AggregateFunctionFactory::Get(AggregationID::kAvg).canApplyToTypes(
-        std::vector<const Type*>(1, &type));
+    return AggregateFunctionFactory::Get(AggregationID::kAvg)
+        .canApplyToTypes(std::vector<const Type *>(1, &type));
   }
 
   static bool ResultTypeForArgumentTypeTest(TypeID input_type_id,
                                             TypeID output_type_id) {
-    const Type *result_type
-        = AggregateFunctionFactory::Get(AggregationID::kAvg).resultTypeForArgumentTypes(
-            std::vector<const Type*>(1, &TypeFactory::GetType(input_type_id)));
+    const Type *result_type =
+        AggregateFunctionFactory::Get(AggregationID::kAvg)
+            .resultTypeForArgumentTypes(std::vector<const Type *>(
+                1, &TypeFactory::GetType(input_type_id)));
     return (result_type->getTypeID() == output_type_id);
   }
 
   template <typename CppType>
-  static void CheckAvgValue(
-      CppType expected,
-      const AggregationHandle &handle,
-      const AggregationState &state) {
+  static void CheckAvgValue(CppType expected,
+                            const AggregationHandle &handle,
+                            const AggregationState &state) {
     EXPECT_EQ(expected, handle.finalize(state).getLiteral<CppType>());
   }
 
+  template <typename CppType>
+  static void CheckAvgValue(CppType expected, const TypedValue &value) {
+    EXPECT_EQ(expected, value.getLiteral<CppType>());
+  }
+
   // Static templated method for set a meaningful value to data types.
   template <typename CppType>
   static void SetDataType(int value, CppType *data) {
@@ -108,7 +115,9 @@ class AggregationHandleAvgTest : public::testing::Test {
   void checkAggregationAvgGeneric() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_avg_->finalize(*aggregation_handle_avg_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_avg_->finalize(*aggregation_handle_avg_state_)
+            .isNull());
 
     typename GenericType::cpptype val;
     typename GenericType::cpptype sum;
@@ -119,15 +128,16 @@ class AggregationHandleAvgTest : public::testing::Test {
       if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
         SetDataType(i - 10, &val);
       } else {
-        SetDataType(static_cast<float>(i - 10)/10, &val);
+        SetDataType(static_cast<float>(i - 10) / 10, &val);
       }
       iterateHandle(aggregation_handle_avg_state_.get(), type.makeValue(&val));
       sum += val;
     }
     iterateHandle(aggregation_handle_avg_state_.get(), type.makeNullValue());
-    CheckAvgValue<typename OutputType::cpptype>(static_cast<typename OutputType::cpptype>(sum) / kNumSamples,
-                                                *aggregation_handle_avg_,
-                                                *aggregation_handle_avg_state_);
+    CheckAvgValue<typename OutputType::cpptype>(
+        static_cast<typename OutputType::cpptype>(sum) / kNumSamples,
+        *aggregation_handle_avg_,
+        *aggregation_handle_avg_state_);
 
     // Test mergeStates().
     std::unique_ptr<AggregationState> merge_state(
@@ -140,7 +150,7 @@ class AggregationHandleAvgTest : public::testing::Test {
       if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
         SetDataType(i - 10, &val);
       } else {
-        SetDataType(static_cast<float>(i - 10)/10, &val);
+        SetDataType(static_cast<float>(i - 10) / 10, &val);
       }
       iterateHandle(merge_state.get(), type.makeValue(&val));
       sum += val;
@@ -155,7 +165,8 @@ class AggregationHandleAvgTest : public::testing::Test {
   }
 
   template <typename GenericType>
-  ColumnVector *createColumnVectorGeneric(const Type &type, typename GenericType::cpptype *sum) {
+  ColumnVector* createColumnVectorGeneric(const Type &type,
+                                          typename GenericType::cpptype *sum) {
     NativeColumnVector *column = new NativeColumnVector(type, kNumSamples + 3);
 
     typename GenericType::cpptype val;
@@ -166,12 +177,12 @@ class AggregationHandleAvgTest : public::testing::Test {
       if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
         SetDataType(i - 10, &val);
       } else {
-        SetDataType(static_cast<float>(i - 10)/10, &val);
+        SetDataType(static_cast<float>(i - 10) / 10, &val);
       }
       column->appendTypedValue(type.makeValue(&val));
       *sum += val;
       // One NULL in the middle.
-      if (i == kNumSamples/2) {
+      if (i == kNumSamples / 2) {
         column->appendTypedValue(type.makeNullValue());
       }
     }
@@ -184,12 +195,15 @@ class AggregationHandleAvgTest : public::testing::Test {
   void checkAggregationAvgGenericColumnVector() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_avg_->finalize(*aggregation_handle_avg_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_avg_->finalize(*aggregation_handle_avg_state_)
+            .isNull());
 
     typename GenericType::cpptype sum;
     SetDataType(0, &sum);
     std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(createColumnVectorGeneric<GenericType>(type, &sum));
+    column_vectors.emplace_back(
+        createColumnVectorGeneric<GenericType>(type, &sum));
 
     std::unique_ptr<AggregationState> cv_state(
         aggregation_handle_avg_->accumulateColumnVectors(column_vectors));
@@ -201,7 +215,8 @@ class AggregationHandleAvgTest : public::testing::Test {
         *aggregation_handle_avg_,
         *cv_state);
 
-    aggregation_handle_avg_->mergeStates(*cv_state, aggregation_handle_avg_state_.get());
+    aggregation_handle_avg_->mergeStates(*cv_state,
+                                         aggregation_handle_avg_state_.get());
     CheckAvgValue<typename OutputType::cpptype>(
         static_cast<typename OutputType::cpptype>(sum) / kNumSamples,
         *aggregation_handle_avg_,
@@ -213,16 +228,19 @@ class AggregationHandleAvgTest : public::testing::Test {
   void checkAggregationAvgGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_avg_->finalize(*aggregation_handle_avg_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_avg_->finalize(*aggregation_handle_avg_state_)
+            .isNull());
 
     typename GenericType::cpptype sum;
     SetDataType(0, &sum);
-    std::unique_ptr<ColumnVectorsValueAccessor> accessor(new ColumnVectorsValueAccessor());
+    std::unique_ptr<ColumnVectorsValueAccessor> accessor(
+        new ColumnVectorsValueAccessor());
     accessor->addColumn(createColumnVectorGeneric<GenericType>(type, &sum));
 
     std::unique_ptr<AggregationState> va_state(
-        aggregation_handle_avg_->accumulateValueAccessor(accessor.get(),
-                                                         std::vector<attribute_id>(1, 0)));
+        aggregation_handle_avg_->accumulateValueAccessor(
+            accessor.get(), std::vector<attribute_id>(1, 0)));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -231,7 +249,8 @@ class AggregationHandleAvgTest : public::testing::Test {
         *aggregation_handle_avg_,
         *va_state);
 
-    aggregation_handle_avg_->mergeStates(*va_state, aggregation_handle_avg_state_.get());
+    aggregation_handle_avg_->mergeStates(*va_state,
+                                         aggregation_handle_avg_state_.get());
     CheckAvgValue<typename OutputType::cpptype>(
         static_cast<typename OutputType::cpptype>(sum) / kNumSamples,
         *aggregation_handle_avg_,
@@ -255,12 +274,14 @@ void AggregationHandleAvgTest::CheckAvgValue<double>(
 }
 
 template <>
-void AggregationHandleAvgTest::SetDataType<DatetimeIntervalLit>(int value, DatetimeIntervalLit *data) {
+void AggregationHandleAvgTest::SetDataType<DatetimeIntervalLit>(
+    int value, DatetimeIntervalLit *data) {
   data->interval_ticks = value;
 }
 
 template <>
-void AggregationHandleAvgTest::SetDataType<YearMonthIntervalLit>(int value, YearMonthIntervalLit *data) {
+void AggregationHandleAvgTest::SetDataType<YearMonthIntervalLit>(
+    int value, YearMonthIntervalLit *data) {
   data->months = value;
 }
 
@@ -307,11 +328,13 @@ TEST_F(AggregationHandleAvgTest, DoubleTypeColumnVectorTest) {
 }
 
 TEST_F(AggregationHandleAvgTest, DatetimeIntervalTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<DatetimeIntervalType, DatetimeIntervalType>();
+  checkAggregationAvgGenericColumnVector<DatetimeIntervalType,
+                                         DatetimeIntervalType>();
 }
 
 TEST_F(AggregationHandleAvgTest, YearMonthIntervalTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<YearMonthIntervalType, YearMonthIntervalType>();
+  checkAggregationAvgGenericColumnVector<YearMonthIntervalType,
+                                         YearMonthIntervalType>();
 }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -332,11 +355,13 @@ TEST_F(AggregationHandleAvgTest, DoubleTypeValueAccessorTest) {
 }
 
 TEST_F(AggregationHandleAvgTest, DatetimeIntervalTypeValueAccessorTest) {
-  checkAggregationAvgGenericValueAccessor<DatetimeIntervalType, DatetimeIntervalType>();
+  checkAggregationAvgGenericValueAccessor<DatetimeIntervalType,
+                                          DatetimeIntervalType>();
 }
 
 TEST_F(AggregationHandleAvgTest, YearMonthIntervalTypeValueAccessorTest) {
-  checkAggregationAvgGenericValueAccessor<YearMonthIntervalType, YearMonthIntervalType>();
+  checkAggregationAvgGenericValueAccessor<YearMonthIntervalType,
+                                          YearMonthIntervalType>();
 }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
@@ -365,38 +390,53 @@ TEST_F(AggregationHandleAvgDeathTest, WrongTypeTest) {
   double double_val = 0;
   float float_val = 0;
 
-  iterateHandle(aggregation_handle_avg_state_.get(), int_non_null_type.makeValue(&int_val));
+  iterateHandle(aggregation_handle_avg_state_.get(),
+                int_non_null_type.makeValue(&int_val));
 
-  EXPECT_DEATH(iterateHandle(aggregation_handle_avg_state_.get(), long_type.makeValue(&long_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_avg_state_.get(), double_type.makeValue(&double_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_avg_state_.get(), float_type.makeValue(&float_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_avg_state_.get(), char_type.makeValue("asdf", 5)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_avg_state_.get(), varchar_type.makeValue("asdf", 5)), "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_avg_state_.get(),
+                             long_type.makeValue(&long_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_avg_state_.get(),
+                             double_type.makeValue(&double_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_avg_state_.get(),
+                             float_type.makeValue(&float_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_avg_state_.get(),
+                             char_type.makeValue("asdf", 5)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_avg_state_.get(),
+                             varchar_type.makeValue("asdf", 5)),
+               "");
 
   // Test mergeStates() with incorrectly typed handles.
   std::unique_ptr<AggregationHandle> aggregation_handle_avg_double(
-      AggregateFunctionFactory::Get(AggregationID::kAvg).createHandle(
-          std::vector<const Type*>(1, &double_type)));
+      AggregateFunctionFactory::Get(AggregationID::kAvg)
+          .createHandle(std::vector<const Type *>(1, &double_type)));
   std::unique_ptr<AggregationState> aggregation_state_avg_merge_double(
       aggregation_handle_avg_double->createInitialState());
-  static_cast<const AggregationHandleAvg&>(*aggregation_handle_avg_double).iterateUnaryInl(
-      static_cast<AggregationStateAvg*>(aggregation_state_avg_merge_double.get()),
-      double_type.makeValue(&double_val));
-  EXPECT_DEATH(aggregation_handle_avg_->mergeStates(*aggregation_state_avg_merge_double,
-                                                    aggregation_handle_avg_state_.get()),
-               "");
+  static_cast<const AggregationHandleAvg &>(*aggregation_handle_avg_double)
+      .iterateUnaryInl(static_cast<AggregationStateAvg *>(
+                           aggregation_state_avg_merge_double.get()),
+                       double_type.makeValue(&double_val));
+  EXPECT_DEATH(
+      aggregation_handle_avg_->mergeStates(*aggregation_state_avg_merge_double,
+                                           aggregation_handle_avg_state_.get()),
+      "");
 
   std::unique_ptr<AggregationHandle> aggregation_handle_avg_float(
-      AggregateFunctionFactory::Get(AggregationID::kAvg).createHandle(
-          std::vector<const Type*>(1, &float_type)));
+      AggregateFunctionFactory::Get(AggregationID::kAvg)
+          .createHandle(std::vector<const Type *>(1, &float_type)));
   std::unique_ptr<AggregationState> aggregation_state_avg_merge_float(
       aggregation_handle_avg_float->createInitialState());
-  static_cast<const AggregationHandleAvg&>(*aggregation_handle_avg_float).iterateUnaryInl(
-      static_cast<AggregationStateAvg*>(aggregation_state_avg_merge_float.get()),
-      float_type.makeValue(&float_val));
-  EXPECT_DEATH(aggregation_handle_avg_->mergeStates(*aggregation_state_avg_merge_float,
-                                                    aggregation_handle_avg_state_.get()),
-               "");
+  static_cast<const AggregationHandleAvg &>(*aggregation_handle_avg_float)
+      .iterateUnaryInl(static_cast<AggregationStateAvg *>(
+                           aggregation_state_avg_merge_float.get()),
+                       float_type.makeValue(&float_val));
+  EXPECT_DEATH(
+      aggregation_handle_avg_->mergeStates(*aggregation_state_avg_merge_float,
+                                           aggregation_handle_avg_state_.get()),
+      "");
 }
 #endif
 
@@ -417,8 +457,10 @@ TEST_F(AggregationHandleAvgTest, ResultTypeForArgumentTypeTest) {
   EXPECT_TRUE(ResultTypeForArgumentTypeTest(kLong, kDouble));
   EXPECT_TRUE(ResultTypeForArgumentTypeTest(kFloat, kDouble));
   EXPECT_TRUE(ResultTypeForArgumentTypeTest(kDouble, kDouble));
-  EXPECT_TRUE(ResultTypeForArgumentTypeTest(kDatetimeInterval, kDatetimeInterval));
-  EXPECT_TRUE(ResultTypeForArgumentTypeTest(kYearMonthInterval, kYearMonthInterval));
+  EXPECT_TRUE(
+      ResultTypeForArgumentTypeTest(kDatetimeInterval, kDatetimeInterval));
+  EXPECT_TRUE(
+      ResultTypeForArgumentTypeTest(kYearMonthInterval, kYearMonthInterval));
 }
 
 TEST_F(AggregationHandleAvgTest, GroupByTableMergeTestAvg) {
@@ -426,25 +468,28 @@ TEST_F(AggregationHandleAvgTest, GroupByTableMergeTestAvg) {
   initializeHandle(long_non_null_type);
   storage_manager_.reset(new StorageManager("./test_avg_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      aggregation_handle_avg_->createGroupByHashTable(
-          HashTableImplType::kSimpleScalarSeparateChaining,
+      AggregationStateFastHashTableFactory::CreateResizable(
+          HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
+          {aggregation_handle_avg_.get()->getPayloadSize()},
+          {aggregation_handle_avg_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      aggregation_handle_avg_->createGroupByHashTable(
-          HashTableImplType::kSimpleScalarSeparateChaining,
+      AggregationStateFastHashTableFactory::CreateResizable(
+          HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
+          {aggregation_handle_avg_.get()->getPayloadSize()},
+          {aggregation_handle_avg_.get()},
           storage_manager_.get()));
 
-  AggregationStateHashTable<AggregationStateAvg> *destination_hash_table_derived =
-      static_cast<AggregationStateHashTable<AggregationStateAvg> *>(
+  AggregationStateFastHashTable *destination_hash_table_derived =
+      static_cast<AggregationStateFastHashTable *>(
           destination_hash_table.get());
 
-  AggregationStateHashTable<AggregationStateAvg> *source_hash_table_derived =
-      static_cast<AggregationStateHashTable<AggregationStateAvg> *>(
-          source_hash_table.get());
+  AggregationStateFastHashTable *source_hash_table_derived =
+      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
 
   AggregationHandleAvg *aggregation_handle_avg_derived =
       static_cast<AggregationHandleAvg *>(aggregation_handle_avg_.get());
@@ -496,36 +541,56 @@ TEST_F(AggregationHandleAvgTest, GroupByTableMergeTestAvg) {
       exclusive_key_source_state.get(), exclusive_key_source_avg_val);
 
   // Add the key-state pairs to the hash tables.
-  source_hash_table_derived->putCompositeKey(common_key,
-                                             *common_key_source_state);
-  destination_hash_table_derived->putCompositeKey(
-      common_key, *common_key_destination_state);
-  source_hash_table_derived->putCompositeKey(exclusive_source_key,
-                                             *exclusive_key_source_state);
-  destination_hash_table_derived->putCompositeKey(
-      exclusive_destination_key, *exclusive_key_destination_state);
+  unsigned char buffer[100];
+  buffer[0] = '\0';
+  memcpy(buffer + 1,
+         common_key_source_state.get()->getPayloadAddress(),
+         aggregation_handle_avg_.get()->getPayloadSize());
+  source_hash_table_derived->putCompositeKey(common_key, buffer);
+
+  memcpy(buffer + 1,
+         common_key_destination_state.get()->getPayloadAddress(),
+         aggregation_handle_avg_.get()->getPayloadSize());
+  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+
+  memcpy(buffer + 1,
+         exclusive_key_source_state.get()->getPayloadAddress(),
+         aggregation_handle_avg_.get()->getPayloadSize());
+  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+
+  memcpy(buffer + 1,
+         exclusive_key_destination_state.get()->getPayloadAddress(),
+         aggregation_handle_avg_.get()->getPayloadSize());
+  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
+                                                      buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  aggregation_handle_avg_->mergeGroupByHashTables(*source_hash_table,
-                                                  destination_hash_table.get());
+  AggregationOperationState::mergeGroupByHashTables(
+      source_hash_table.get(), destination_hash_table.get());
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckAvgValue<double>(
       (common_key_destination_avg_val.getLiteral<std::int64_t>() +
-          common_key_source_avg_val.getLiteral<std::int64_t>()) / static_cast<double>(2),
-      *aggregation_handle_avg_derived,
-      *(destination_hash_table_derived->getSingleCompositeKey(common_key)));
-  CheckAvgValue<double>(exclusive_key_destination_avg_val.getLiteral<std::int64_t>(),
-                  *aggregation_handle_avg_derived,
-                  *(destination_hash_table_derived->getSingleCompositeKey(
-                      exclusive_destination_key)));
-  CheckAvgValue<double>(exclusive_key_source_avg_val.getLiteral<std::int64_t>(),
-                  *aggregation_handle_avg_derived,
-                  *(source_hash_table_derived->getSingleCompositeKey(
-                      exclusive_source_key)));
+       common_key_source_avg_val.getLiteral<std::int64_t>()) /
+          static_cast<double>(2),
+      aggregation_handle_avg_derived->finalizeHashTableEntryFast(
+          destination_hash_table_derived->getSingleCompositeKey(common_key) +
+          1));
+  CheckAvgValue<double>(
+      exclusive_key_destination_avg_val.getLiteral<std::int64_t>(),
+      aggregation_handle_avg_derived->finalizeHashTableEntryFast(
+          destination_hash_table_derived->getSingleCompositeKey(
+              exclusive_destination_key) +
+          1));
+  CheckAvgValue<double>(
+      exclusive_key_source_avg_val.getLiteral<std::int64_t>(),
+      aggregation_handle_avg_derived->finalizeHashTableEntryFast(
+          source_hash_table_derived->getSingleCompositeKey(
+              exclusive_source_key) +
+          1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp b/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
index 6565a41..78bd249 100644
--- a/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
@@ -29,6 +29,8 @@
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "expressions/aggregation/AggregationHandleCount.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
+#include "storage/AggregationOperationState.hpp"
+#include "storage/FastHashTableFactory.hpp"
 #include "storage/StorageManager.hpp"
 #include "types/CharType.hpp"
 #include "types/DoubleType.hpp"
@@ -50,85 +52,94 @@
 
 namespace quickstep {
 
-class AggregationHandleCountTest : public::testing::Test {
+class AggregationHandleCountTest : public ::testing::Test {
  protected:
   const Type &dummy_type = TypeFactory::GetType(kInt);
 
   void iterateHandleNullary(AggregationState *state) {
-    static_cast<const AggregationHandleCount<true, false>&>(
-        *aggregation_handle_count_).iterateNullaryInl(
-            static_cast<AggregationStateCount*>(state));
+    static_cast<const AggregationHandleCount<true, false> &>(
+        *aggregation_handle_count_)
+        .iterateNullaryInl(static_cast<AggregationStateCount *>(state));
   }
 
   // Helper method that calls AggregationHandleCount::iterateUnaryInl() to
   // aggregate 'value' into '*state'.
   void iterateHandle(AggregationState *state, const TypedValue &value) {
-    static_cast<const AggregationHandleCount<false, true>&>(
-        *aggregation_handle_count_).iterateUnaryInl(
-            static_cast<AggregationStateCount*>(state),
-            value);
+    static_cast<const AggregationHandleCount<false, true> &>(
+        *aggregation_handle_count_)
+        .iterateUnaryInl(static_cast<AggregationStateCount *>(state), value);
   }
 
   void initializeHandle(const Type *argument_type) {
     if (argument_type == nullptr) {
       aggregation_handle_count_.reset(
-          AggregateFunctionFactory::Get(AggregationID::kCount).createHandle(
-              std::vector<const Type*>()));
+          AggregateFunctionFactory::Get(AggregationID::kCount)
+              .createHandle(std::vector<const Type *>()));
     } else {
       aggregation_handle_count_.reset(
-          AggregateFunctionFactory::Get(AggregationID::kCount).createHandle(
-              std::vector<const Type*>(1, argument_type)));
+          AggregateFunctionFactory::Get(AggregationID::kCount)
+              .createHandle(std::vector<const Type *>(1, argument_type)));
     }
     aggregation_handle_count_state_.reset(
         aggregation_handle_count_->createInitialState());
   }
 
   static bool ApplyToTypesTest(TypeID typeID) {
-    const Type &type = (typeID == kChar || typeID == kVarChar) ?
-        TypeFactory::GetType(typeID, static_cast<std::size_t>(10)) :
-        TypeFactory::GetType(typeID);
+    const Type &type =
+        (typeID == kChar || typeID == kVarChar)
+            ? TypeFactory::GetType(typeID, static_cast<std::size_t>(10))
+            : TypeFactory::GetType(typeID);
 
-    return AggregateFunctionFactory::Get(AggregationID::kCount).canApplyToTypes(
-        std::vector<const Type*>(1, &type));
+    return AggregateFunctionFactory::Get(AggregationID::kCount)
+        .canApplyToTypes(std::vector<const Type *>(1, &type));
   }
 
   static bool ResultTypeForArgumentTypeTest(TypeID input_type_id,
                                             TypeID output_type_id) {
-    const Type *result_type
-        = AggregateFunctionFactory::Get(AggregationID::kCount).resultTypeForArgumentTypes(
-            std::vector<const Type*>(1, &TypeFactory::GetType(input_type_id)));
+    const Type *result_type =
+        AggregateFunctionFactory::Get(AggregationID::kCount)
+            .resultTypeForArgumentTypes(std::vector<const Type *>(
+                1, &TypeFactory::GetType(input_type_id)));
     return (result_type->getTypeID() == output_type_id);
   }
 
-  static void CheckCountValue(
-      std::int64_t expected,
-      const AggregationHandle &handle,
-      const AggregationState &state) {
+  static void CheckCountValue(std::int64_t expected,
+                              const AggregationHandle &handle,
+                              const AggregationState &state) {
     EXPECT_EQ(expected, handle.finalize(state).getLiteral<std::int64_t>());
   }
 
+  static void CheckCountValue(std::int64_t expected, const TypedValue &value) {
+    EXPECT_EQ(expected, value.getLiteral<std::int64_t>());
+  }
+
   void checkAggregationCountNullary(int test_count) {
     initializeHandle(nullptr);
-    CheckCountValue(0, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    CheckCountValue(
+        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
 
     for (int i = 0; i < test_count; ++i) {
       iterateHandleNullary(aggregation_handle_count_state_.get());
     }
-    CheckCountValue(test_count, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    CheckCountValue(test_count,
+                    *aggregation_handle_count_,
+                    *aggregation_handle_count_state_);
 
     // Test mergeStates.
     std::unique_ptr<AggregationState> merge_state(
         aggregation_handle_count_->createInitialState());
-    aggregation_handle_count_->mergeStates(*merge_state,
-                                           aggregation_handle_count_state_.get());
+    aggregation_handle_count_->mergeStates(
+        *merge_state, aggregation_handle_count_state_.get());
 
     for (int i = 0; i < test_count; ++i) {
       iterateHandleNullary(merge_state.get());
     }
 
-    aggregation_handle_count_->mergeStates(*merge_state,
-                                           aggregation_handle_count_state_.get());
-    CheckCountValue(2 * test_count, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    aggregation_handle_count_->mergeStates(
+        *merge_state, aggregation_handle_count_state_.get());
+    CheckCountValue(2 * test_count,
+                    *aggregation_handle_count_,
+                    *aggregation_handle_count_state_);
   }
 
   void checkAggregationCountNullaryAccumulate(int test_count) {
@@ -139,12 +150,10 @@ class AggregationHandleCountTest : public::testing::Test {
 
     // Test the state generated directly by accumulateNullary(), and also test
     // after merging back.
-    CheckCountValue(test_count,
-                    *aggregation_handle_count_,
-                    *accumulated_state);
+    CheckCountValue(test_count, *aggregation_handle_count_, *accumulated_state);
 
-    aggregation_handle_count_->mergeStates(*accumulated_state,
-                                           aggregation_handle_count_state_.get());
+    aggregation_handle_count_->mergeStates(
+        *accumulated_state, aggregation_handle_count_state_.get());
     CheckCountValue(test_count,
                     *aggregation_handle_count_,
                     *aggregation_handle_count_state_);
@@ -154,24 +163,27 @@ class AggregationHandleCountTest : public::testing::Test {
   void checkAggregationCountNumeric(int test_count) {
     const NumericType &type = NumericType::Instance(true);
     initializeHandle(&type);
-    CheckCountValue(0, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    CheckCountValue(
+        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
 
     typename NumericType::cpptype val = 0;
     int count = 0;
 
     iterateHandle(aggregation_handle_count_state_.get(), type.makeNullValue());
     for (int i = 0; i < test_count; ++i) {
-      iterateHandle(aggregation_handle_count_state_.get(), type.makeValue(&val));
+      iterateHandle(aggregation_handle_count_state_.get(),
+                    type.makeValue(&val));
       ++count;
     }
     iterateHandle(aggregation_handle_count_state_.get(), type.makeNullValue());
-    CheckCountValue(count, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    CheckCountValue(
+        count, *aggregation_handle_count_, *aggregation_handle_count_state_);
 
     // Test mergeStates.
     std::unique_ptr<AggregationState> merge_state(
         aggregation_handle_count_->createInitialState());
-    aggregation_handle_count_->mergeStates(*merge_state,
-                                           aggregation_handle_count_state_.get());
+    aggregation_handle_count_->mergeStates(
+        *merge_state, aggregation_handle_count_state_.get());
 
     iterateHandle(merge_state.get(), type.makeNullValue());
     for (int i = 0; i < test_count; ++i) {
@@ -180,13 +192,14 @@ class AggregationHandleCountTest : public::testing::Test {
     }
     iterateHandle(merge_state.get(), type.makeNullValue());
 
-    aggregation_handle_count_->mergeStates(*merge_state,
-                                           aggregation_handle_count_state_.get());
-    CheckCountValue(count, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    aggregation_handle_count_->mergeStates(
+        *merge_state, aggregation_handle_count_state_.get());
+    CheckCountValue(
+        count, *aggregation_handle_count_, *aggregation_handle_count_state_);
   }
 
   template <typename NumericType>
-  ColumnVector *createColumnVectorNumeric(const Type &type, int test_count) {
+  ColumnVector* createColumnVectorNumeric(const Type &type, int test_count) {
     NativeColumnVector *column = new NativeColumnVector(type, test_count + 3);
 
     typename NumericType::cpptype val = 0;
@@ -194,7 +207,7 @@ class AggregationHandleCountTest : public::testing::Test {
     for (int i = 0; i < test_count; ++i) {
       column->appendTypedValue(type.makeValue(&val));
       // One NULL in the middle.
-      if (i == test_count/2) {
+      if (i == test_count / 2) {
         column->appendTypedValue(type.makeNullValue());
       }
     }
@@ -206,21 +219,22 @@ class AggregationHandleCountTest : public::testing::Test {
   void checkAggregationCountNumericColumnVector(int test_count) {
     const NumericType &type = NumericType::Instance(true);
     initializeHandle(&type);
-    CheckCountValue(0, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    CheckCountValue(
+        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
 
     std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(createColumnVectorNumeric<NumericType>(type, test_count));
+    column_vectors.emplace_back(
+        createColumnVectorNumeric<NumericType>(type, test_count));
 
     std::unique_ptr<AggregationState> cv_state(
         aggregation_handle_count_->accumulateColumnVectors(column_vectors));
 
     // Test the state generated directly by accumulateColumnVectors(), and also
     // test after merging back.
-    CheckCountValue(test_count,
-                    *aggregation_handle_count_,
-                    *cv_state);
+    CheckCountValue(test_count, *aggregation_handle_count_, *cv_state);
 
-    aggregation_handle_count_->mergeStates(*cv_state, aggregation_handle_count_state_.get());
+    aggregation_handle_count_->mergeStates(
+        *cv_state, aggregation_handle_count_state_.get());
     CheckCountValue(test_count,
                     *aggregation_handle_count_,
                     *aggregation_handle_count_state_);
@@ -231,22 +245,24 @@ class AggregationHandleCountTest : public::testing::Test {
   void checkAggregationCountNumericValueAccessor(int test_count) {
     const NumericType &type = NumericType::Instance(true);
     initializeHandle(&type);
-    CheckCountValue(0, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    CheckCountValue(
+        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
 
-    std::unique_ptr<ColumnVectorsValueAccessor> accessor(new ColumnVectorsValueAccessor());
-    accessor->addColumn(createColumnVectorNumeric<NumericType>(type, test_count));
+    std::unique_ptr<ColumnVectorsValueAccessor> accessor(
+        new ColumnVectorsValueAccessor());
+    accessor->addColumn(
+        createColumnVectorNumeric<NumericType>(type, test_count));
 
     std::unique_ptr<AggregationState> va_state(
-        aggregation_handle_count_->accumulateValueAccessor(accessor.get(),
-                                                           std::vector<attribute_id>(1, 0)));
+        aggregation_handle_count_->accumulateValueAccessor(
+            accessor.get(), std::vector<attribute_id>(1, 0)));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
-    CheckCountValue(test_count,
-                    *aggregation_handle_count_,
-                    *va_state);
+    CheckCountValue(test_count, *aggregation_handle_count_, *va_state);
 
-    aggregation_handle_count_->mergeStates(*va_state, aggregation_handle_count_state_.get());
+    aggregation_handle_count_->mergeStates(
+        *va_state, aggregation_handle_count_state_.get());
     CheckCountValue(test_count,
                     *aggregation_handle_count_,
                     *aggregation_handle_count_state_);
@@ -257,7 +273,8 @@ class AggregationHandleCountTest : public::testing::Test {
   void checkAggregationCountString(int test_count) {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(&type);
-    CheckCountValue(0, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    CheckCountValue(
+        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
 
     std::string string_literal = "test_str";
     int count = 0;
@@ -269,7 +286,8 @@ class AggregationHandleCountTest : public::testing::Test {
       ++count;
     }
     iterateHandle(aggregation_handle_count_state_.get(), type.makeNullValue());
-    CheckCountValue(count, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    CheckCountValue(
+        count, *aggregation_handle_count_, *aggregation_handle_count_state_);
 
     // Test mergeStates().
     std::unique_ptr<AggregationState> merge_state(
@@ -277,18 +295,20 @@ class AggregationHandleCountTest : public::testing::Test {
 
     iterateHandle(merge_state.get(), type.makeNullValue());
     for (int i = 0; i < test_count; ++i) {
-      iterateHandle(merge_state.get(), type.makeValue(string_literal.c_str(), 10));
+      iterateHandle(merge_state.get(),
+                    type.makeValue(string_literal.c_str(), 10));
       ++count;
     }
     iterateHandle(merge_state.get(), type.makeNullValue());
 
-    aggregation_handle_count_->mergeStates(*merge_state,
-                                           aggregation_handle_count_state_.get());
-    CheckCountValue(count, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    aggregation_handle_count_->mergeStates(
+        *merge_state, aggregation_handle_count_state_.get());
+    CheckCountValue(
+        count, *aggregation_handle_count_, *aggregation_handle_count_state_);
   }
 
   template <typename ColumnVectorType>
-  ColumnVector *createColumnVectorString(const Type &type, int test_count) {
+  ColumnVector* createColumnVectorString(const Type &type, int test_count) {
     ColumnVectorType *column = new ColumnVectorType(type, test_count + 3);
 
     std::string string_literal = "test_str";
@@ -296,7 +316,7 @@ class AggregationHandleCountTest : public::testing::Test {
     for (int i = 0; i < test_count; ++i) {
       column->appendTypedValue(type.makeValue(string_literal.c_str(), 10));
       // One NULL in the middle.
-      if (i == test_count/2) {
+      if (i == test_count / 2) {
         column->appendTypedValue(type.makeNullValue());
       }
     }
@@ -309,21 +329,22 @@ class AggregationHandleCountTest : public::testing::Test {
   void checkAggregationCountStringColumnVector(int test_count) {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(&type);
-    CheckCountValue(0, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    CheckCountValue(
+        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
 
     std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(createColumnVectorString<ColumnVectorType>(type, test_count));
+    column_vectors.emplace_back(
+        createColumnVectorString<ColumnVectorType>(type, test_count));
 
     std::unique_ptr<AggregationState> cv_state(
         aggregation_handle_count_->accumulateColumnVectors(column_vectors));
 
     // Test the state generated directly by accumulateColumnVectors(), and also
     // test after merging back.
-    CheckCountValue(test_count,
-                    *aggregation_handle_count_,
-                    *cv_state);
+    CheckCountValue(test_count, *aggregation_handle_count_, *cv_state);
 
-    aggregation_handle_count_->mergeStates(*cv_state, aggregation_handle_count_state_.get());
+    aggregation_handle_count_->mergeStates(
+        *cv_state, aggregation_handle_count_state_.get());
     CheckCountValue(test_count,
                     *aggregation_handle_count_,
                     *aggregation_handle_count_state_);
@@ -334,22 +355,24 @@ class AggregationHandleCountTest : public::testing::Test {
   void checkAggregationCountStringValueAccessor(int test_count) {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(&type);
-    CheckCountValue(0, *aggregation_handle_count_, *aggregation_handle_count_state_);
+    CheckCountValue(
+        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
 
-    std::unique_ptr<ColumnVectorsValueAccessor> accessor(new ColumnVectorsValueAccessor());
-    accessor->addColumn(createColumnVectorString<ColumnVectorType>(type, test_count));
+    std::unique_ptr<ColumnVectorsValueAccessor> accessor(
+        new ColumnVectorsValueAccessor());
+    accessor->addColumn(
+        createColumnVectorString<ColumnVectorType>(type, test_count));
 
     std::unique_ptr<AggregationState> va_state(
-        aggregation_handle_count_->accumulateValueAccessor(accessor.get(),
-                                                           std::vector<attribute_id>(1, 0)));
+        aggregation_handle_count_->accumulateValueAccessor(
+            accessor.get(), std::vector<attribute_id>(1, 0)));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
-    CheckCountValue(test_count,
-                    *aggregation_handle_count_,
-                    *va_state);
+    CheckCountValue(test_count, *aggregation_handle_count_, *va_state);
 
-    aggregation_handle_count_->mergeStates(*va_state, aggregation_handle_count_state_.get());
+    aggregation_handle_count_->mergeStates(
+        *va_state, aggregation_handle_count_state_.get());
     CheckCountValue(test_count,
                     *aggregation_handle_count_,
                     *aggregation_handle_count_state_);
@@ -364,13 +387,12 @@ class AggregationHandleCountTest : public::testing::Test {
 typedef AggregationHandleCountTest AggregationHandleCountDeathTest;
 
 TEST_F(AggregationHandleCountTest, CountStarTest) {
-  checkAggregationCountNullary(0),
-  checkAggregationCountNullary(10000);
+  checkAggregationCountNullary(0), checkAggregationCountNullary(10000);
 }
 
 TEST_F(AggregationHandleCountTest, CountStarAccumulateTest) {
   checkAggregationCountNullaryAccumulate(0),
-  checkAggregationCountNullaryAccumulate(10000);
+      checkAggregationCountNullaryAccumulate(10000);
 }
 
 TEST_F(AggregationHandleCountTest, IntTypeTest) {
@@ -430,7 +452,8 @@ TEST_F(AggregationHandleCountTest, CharTypeColumnVectorTest) {
 
 TEST_F(AggregationHandleCountTest, VarCharTypeColumnVectorTest) {
   checkAggregationCountStringColumnVector<VarCharType, IndirectColumnVector>(0);
-  checkAggregationCountStringColumnVector<VarCharType, IndirectColumnVector>(10000);
+  checkAggregationCountStringColumnVector<VarCharType, IndirectColumnVector>(
+      10000);
 }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -460,8 +483,10 @@ TEST_F(AggregationHandleCountTest, CharTypeValueAccessorTest) {
 }
 
 TEST_F(AggregationHandleCountTest, VarCharTypeValueAccessorTest) {
-  checkAggregationCountStringValueAccessor<VarCharType, IndirectColumnVector>(0);
-  checkAggregationCountStringValueAccessor<VarCharType, IndirectColumnVector>(10000);
+  checkAggregationCountStringValueAccessor<VarCharType, IndirectColumnVector>(
+      0);
+  checkAggregationCountStringValueAccessor<VarCharType, IndirectColumnVector>(
+      10000);
 }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
@@ -486,25 +511,28 @@ TEST_F(AggregationHandleCountTest, GroupByTableMergeTestCount) {
   initializeHandle(&long_non_null_type);
   storage_manager_.reset(new StorageManager("./test_count_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      aggregation_handle_count_->createGroupByHashTable(
-          HashTableImplType::kSimpleScalarSeparateChaining,
+      AggregationStateFastHashTableFactory::CreateResizable(
+          HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
+          {aggregation_handle_count_.get()->getPayloadSize()},
+          {aggregation_handle_count_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      aggregation_handle_count_->createGroupByHashTable(
-          HashTableImplType::kSimpleScalarSeparateChaining,
+      AggregationStateFastHashTableFactory::CreateResizable(
+          HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
+          {aggregation_handle_count_.get()->getPayloadSize()},
+          {aggregation_handle_count_.get()},
           storage_manager_.get()));
 
-  AggregationStateHashTable<AggregationStateCount> *destination_hash_table_derived =
-      static_cast<AggregationStateHashTable<AggregationStateCount> *>(
+  AggregationStateFastHashTable *destination_hash_table_derived =
+      static_cast<AggregationStateFastHashTable *>(
           destination_hash_table.get());
 
-  AggregationStateHashTable<AggregationStateCount> *source_hash_table_derived =
-      static_cast<AggregationStateHashTable<AggregationStateCount> *>(
-          source_hash_table.get());
+  AggregationStateFastHashTable *source_hash_table_derived =
+      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
 
   // TODO(harshad) - Use TemplateUtil::CreateBoolInstantiatedInstance to
   // generate all the combinations of the bool template arguments and test them.
@@ -530,7 +558,8 @@ TEST_F(AggregationHandleCountTest, GroupByTableMergeTestCount) {
   TypedValue exclusive_key_source_count_val(exclusive_key_source_count);
 
   const std::int64_t exclusive_key_destination_count = 1;
-  TypedValue exclusive_key_destination_count_val(exclusive_key_destination_count);
+  TypedValue exclusive_key_destination_count_val(
+      exclusive_key_destination_count);
 
   std::unique_ptr<AggregationStateCount> common_key_source_state(
       static_cast<AggregationStateCount *>(
@@ -546,62 +575,86 @@ TEST_F(AggregationHandleCountTest, GroupByTableMergeTestCount) {
           aggregation_handle_count_->createInitialState()));
 
   // Create count value states for keys.
-  aggregation_handle_count_derived->iterateUnaryInl(common_key_source_state.get(),
-                                                  common_key_source_count_val);
-  std::int64_t actual_val = aggregation_handle_count_->finalize(*common_key_source_state)
-                       .getLiteral<std::int64_t>();
+  aggregation_handle_count_derived->iterateUnaryInl(
+      common_key_source_state.get(), common_key_source_count_val);
+  std::int64_t actual_val =
+      aggregation_handle_count_->finalize(*common_key_source_state)
+          .getLiteral<std::int64_t>();
   EXPECT_EQ(common_key_source_count_val.getLiteral<std::int64_t>(), actual_val);
 
   aggregation_handle_count_derived->iterateUnaryInl(
       common_key_destination_state.get(), common_key_destination_count_val);
-  actual_val = aggregation_handle_count_->finalize(*common_key_destination_state)
-                   .getLiteral<std::int64_t>();
-  EXPECT_EQ(common_key_destination_count_val.getLiteral<std::int64_t>(), actual_val);
+  actual_val =
+      aggregation_handle_count_->finalize(*common_key_destination_state)
+          .getLiteral<std::int64_t>();
+  EXPECT_EQ(common_key_destination_count_val.getLiteral<std::int64_t>(),
+            actual_val);
 
   aggregation_handle_count_derived->iterateUnaryInl(
-      exclusive_key_destination_state.get(), exclusive_key_destination_count_val);
+      exclusive_key_destination_state.get(),
+      exclusive_key_destination_count_val);
   actual_val =
       aggregation_handle_count_->finalize(*exclusive_key_destination_state)
           .getLiteral<std::int64_t>();
-  EXPECT_EQ(exclusive_key_destination_count_val.getLiteral<std::int64_t>(), actual_val);
+  EXPECT_EQ(exclusive_key_destination_count_val.getLiteral<std::int64_t>(),
+            actual_val);
 
   aggregation_handle_count_derived->iterateUnaryInl(
       exclusive_key_source_state.get(), exclusive_key_source_count_val);
   actual_val = aggregation_handle_count_->finalize(*exclusive_key_source_state)
                    .getLiteral<std::int64_t>();
-  EXPECT_EQ(exclusive_key_source_count_val.getLiteral<std::int64_t>(), actual_val);
+  EXPECT_EQ(exclusive_key_source_count_val.getLiteral<std::int64_t>(),
+            actual_val);
 
   // Add the key-state pairs to the hash tables.
-  source_hash_table_derived->putCompositeKey(common_key,
-                                             *common_key_source_state);
-  destination_hash_table_derived->putCompositeKey(
-      common_key, *common_key_destination_state);
-  source_hash_table_derived->putCompositeKey(exclusive_source_key,
-                                             *exclusive_key_source_state);
-  destination_hash_table_derived->putCompositeKey(
-      exclusive_destination_key, *exclusive_key_destination_state);
+  unsigned char buffer[100];
+  buffer[0] = '\0';
+  memcpy(buffer + 1,
+         common_key_source_state.get()->getPayloadAddress(),
+         aggregation_handle_count_.get()->getPayloadSize());
+  source_hash_table_derived->putCompositeKey(common_key, buffer);
+
+  memcpy(buffer + 1,
+         common_key_destination_state.get()->getPayloadAddress(),
+         aggregation_handle_count_.get()->getPayloadSize());
+  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+
+  memcpy(buffer + 1,
+         exclusive_key_source_state.get()->getPayloadAddress(),
+         aggregation_handle_count_.get()->getPayloadSize());
+  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+
+  memcpy(buffer + 1,
+         exclusive_key_destination_state.get()->getPayloadAddress(),
+         aggregation_handle_count_.get()->getPayloadSize());
+  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
+                                                      buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  aggregation_handle_count_->mergeGroupByHashTables(*source_hash_table,
-                                                  destination_hash_table.get());
+  AggregationOperationState::mergeGroupByHashTables(
+      source_hash_table.get(), destination_hash_table.get());
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckCountValue(
       common_key_destination_count_val.getLiteral<std::int64_t>() +
           common_key_source_count_val.getLiteral<std::int64_t>(),
-      *aggregation_handle_count_derived,
-      *(destination_hash_table_derived->getSingleCompositeKey(common_key)));
-  CheckCountValue(exclusive_key_destination_count_val.getLiteral<std::int64_t>(),
-                  *aggregation_handle_count_derived,
-                  *(destination_hash_table_derived->getSingleCompositeKey(
-                      exclusive_destination_key)));
+      aggregation_handle_count_derived->finalizeHashTableEntryFast(
+          destination_hash_table_derived->getSingleCompositeKey(common_key) +
+          1));
+  CheckCountValue(
+      exclusive_key_destination_count_val.getLiteral<std::int64_t>(),
+      aggregation_handle_count_derived->finalizeHashTableEntryFast(
+          destination_hash_table_derived->getSingleCompositeKey(
+              exclusive_destination_key) +
+          1));
   CheckCountValue(exclusive_key_source_count_val.getLiteral<std::int64_t>(),
-                  *aggregation_handle_count_derived,
-                  *(source_hash_table_derived->getSingleCompositeKey(
-                      exclusive_source_key)));
+                  aggregation_handle_count_derived->finalizeHashTableEntryFast(
+                      source_hash_table_derived->getSingleCompositeKey(
+                          exclusive_source_key) +
+                      1));
 }
 
 }  // namespace quickstep


[06/28] incubator-quickstep git commit: Initial commit for QUICKSTEP-28 and QUICKSTEP-29. Code refactoring and cleanup, some more optimizations are pending.

Posted by hb...@apache.org.
Initial commit for QUICKSTEP-28 and QUICKSTEP-29. Code refactoring and cleanup, some more optimizations are pending.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/d0756e7e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/d0756e7e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/d0756e7e

Branch: refs/heads/partitioned-aggregation
Commit: d0756e7e0c31f65ee8de3cadaaf2d3f037d913b5
Parents: 43c7a42
Author: rathijit <ra...@node-2.hashtable.quickstep-pg0.wisc.cloudlab.us>
Authored: Mon Jul 4 02:44:48 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:56:06 2016 -0500

----------------------------------------------------------------------
 .../aggregation/AggregationConcreteHandle.cpp   |   29 +-
 .../aggregation/AggregationConcreteHandle.hpp   |  223 ++
 expressions/aggregation/AggregationHandle.hpp   |    8 +-
 .../aggregation/AggregationHandleAvg.cpp        |   40 +-
 .../aggregation/AggregationHandleAvg.hpp        |   62 +-
 .../aggregation/AggregationHandleCount.cpp      |   38 +-
 .../aggregation/AggregationHandleCount.hpp      |   50 +-
 .../aggregation/AggregationHandleDistinct.cpp   |    2 +-
 .../aggregation/AggregationHandleDistinct.hpp   |    2 +-
 .../aggregation/AggregationHandleMax.cpp        |   29 +-
 .../aggregation/AggregationHandleMax.hpp        |   39 +-
 .../aggregation/AggregationHandleMin.cpp        |   30 +-
 .../aggregation/AggregationHandleMin.hpp        |   44 +-
 .../aggregation/AggregationHandleSum.cpp        |   31 +-
 .../aggregation/AggregationHandleSum.hpp        |   52 +-
 expressions/aggregation/CMakeLists.txt          |    7 +
 storage/AggregationOperationState.cpp           |   95 +-
 storage/AggregationOperationState.hpp           |    7 +-
 storage/CMakeLists.txt                          |   58 +
 storage/FastHashTable.hpp                       | 2640 ++++++++++++++++++
 storage/FastHashTableFactory.hpp                |  300 ++
 storage/FastSeparateChainingHashTable.hpp       | 1761 ++++++++++++
 storage/HashTableBase.hpp                       |    2 +-
 storage/HashTablePool.hpp                       |   42 +
 storage/StorageBlock.cpp                        |   88 +-
 storage/StorageBlock.hpp                        |    8 +
 threading/SpinMutex.hpp                         |    2 +
 27 files changed, 5587 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationConcreteHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.cpp b/expressions/aggregation/AggregationConcreteHandle.cpp
index 719920f..1efe010 100644
--- a/expressions/aggregation/AggregationConcreteHandle.cpp
+++ b/expressions/aggregation/AggregationConcreteHandle.cpp
@@ -24,6 +24,7 @@
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "storage/HashTable.hpp"
+#include "storage/FastHashTable.hpp"
 #include "storage/HashTableFactory.hpp"
 
 namespace quickstep {
@@ -51,22 +52,24 @@ void AggregationConcreteHandle::insertValueAccessorIntoDistinctifyHashTable(
     AggregationStateHashTableBase *distinctify_hash_table) const {
   // If the key-value pair is already there, we don't need to update the value,
   // which should always be "true". I.e. the value is just a placeholder.
-  const auto noop_upserter = [](const auto &accessor, const bool *value) -> void {};
+//  const auto noop_upserter = [](const auto &accessor, const bool *value) -> void {};
 
-  AggregationStateHashTable<bool> *hash_table =
-      static_cast<AggregationStateHashTable<bool>*>(distinctify_hash_table);
+  AggregationStateFastHashTable *hash_table =
+      static_cast<AggregationStateFastHashTable *>(distinctify_hash_table);
   if (key_ids.size() == 1) {
-    hash_table->upsertValueAccessor(accessor,
-                                    key_ids[0],
-                                    true /* check_for_null_keys */,
-                                    true /* initial_value */,
-                                    &noop_upserter);
+// TODO(rathijit): fix
+//    hash_table->upsertValueAccessor(accessor,
+//                                    key_ids[0],
+//                                    true /* check_for_null_keys */,
+//                                    true /* initial_value */,
+//                                    &noop_upserter);
   } else {
-    hash_table->upsertValueAccessorCompositeKey(accessor,
-                                                key_ids,
-                                                true /* check_for_null_keys */,
-                                                true /* initial_value */,
-                                                &noop_upserter);
+    std::vector<std::vector<attribute_id>> empty_args;
+    empty_args.resize(1);
+    hash_table->upsertValueAccessorCompositeKeyFast(empty_args,
+                                                    accessor,
+                                                    key_ids,
+                                                    true /* check_for_null_keys */);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationConcreteHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.hpp b/expressions/aggregation/AggregationConcreteHandle.hpp
index c5ca061..d332ec9 100644
--- a/expressions/aggregation/AggregationConcreteHandle.hpp
+++ b/expressions/aggregation/AggregationConcreteHandle.hpp
@@ -31,6 +31,7 @@
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
 #include "utility/Macros.hpp"
+#include "threading/SpinMutex.hpp"
 
 #include "glog/logging.h"
 
@@ -79,6 +80,37 @@ class HashTableStateUpserter {
   DISALLOW_COPY_AND_ASSIGN(HashTableStateUpserter);
 };
 
+template <typename HandleT>
+class HashTableStateUpserterFast {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param handle The aggregation handle being used.
+   * @param source_state The aggregation state in the source aggregation hash
+   *        table. The corresponding state (for the same key) in the destination
+   *        hash table will be upserted.
+   **/
+  HashTableStateUpserterFast(const HandleT &handle, const uint8_t *source_state)
+      : handle_(handle), source_state_(source_state) {}
+
+  /**
+   * @brief The operator for the functor required for the upsert.
+   *
+   * @param destination_state The aggregation state in the aggregation hash
+   *        table that is being upserted.
+   **/
+  void operator()(uint8_t *destination_state) {
+    handle_.mergeStatesFast(source_state_, destination_state);
+  }
+
+ private:
+  const HandleT &handle_;
+  const uint8_t *source_state_;
+
+  DISALLOW_COPY_AND_ASSIGN(HashTableStateUpserterFast);
+};
+
 /**
  * @brief A class to support the functor for merging group by hash tables.
  **/
@@ -129,6 +161,53 @@ class HashTableMerger {
   DISALLOW_COPY_AND_ASSIGN(HashTableMerger);
 };
 
+template <typename HandleT, typename HashTableT>
+class HashTableMergerFast {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param handle The Aggregation handle being used.
+   * @param destination_hash_table The destination hash table to which other
+   *        hash tables will be merged.
+   **/
+  HashTableMergerFast(const HandleT &handle,
+                  AggregationStateHashTableBase *destination_hash_table)
+      : handle_(handle),
+        destination_hash_table_(
+            static_cast<HashTableT *>(destination_hash_table)) {}
+
+  /**
+   * @brief The operator for the functor.
+   *
+   * @param group_by_key The group by key being merged.
+   * @param source_state The aggregation state for the given key in the source
+   *        aggregation hash table.
+   **/
+  inline void operator()(const std::vector<TypedValue> &group_by_key,
+                         const uint8_t *source_state) {
+    const uint8_t *original_state =
+        destination_hash_table_->getSingleCompositeKey(group_by_key);
+    if (original_state != nullptr) {
+      HashTableStateUpserterFast<HandleT> upserter(
+          handle_, source_state);
+      // The CHECK is required as upsertCompositeKey can return false if the
+      // hash table runs out of space during the upsert process. The ideal
+      // solution will be to retry again if the upsert fails.
+      CHECK(destination_hash_table_->upsertCompositeKeyFast(
+          group_by_key, original_state, &upserter));
+    } else {
+      destination_hash_table_->putCompositeKeyFast(group_by_key, source_state);
+    }
+  }
+
+ private:
+  const HandleT &handle_;
+  HashTableT *destination_hash_table_;
+
+  DISALLOW_COPY_AND_ASSIGN(HashTableMergerFast);
+};
+
 /**
  * @brief The helper intermediate subclass of AggregationHandle that provides
  *        virtual method implementations as well as helper methods that are
@@ -208,11 +287,26 @@ class AggregationConcreteHandle : public AggregationHandle {
 
   template <typename HandleT,
             typename HashTableT>
+  void aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
+      const AggregationStateHashTableBase &distinctify_hash_table,
+      AggregationStateHashTableBase *hash_table) const;
+
+
+  template <typename HandleT,
+            typename HashTableT>
   ColumnVector* finalizeHashTableHelper(
       const Type &result_type,
       const AggregationStateHashTableBase &hash_table,
       std::vector<std::vector<TypedValue>> *group_by_keys) const;
 
+  template <typename HandleT,
+            typename HashTableT>
+  ColumnVector* finalizeHashTableHelperFast(
+      const Type &result_type,
+      const AggregationStateHashTableBase &hash_table,
+      std::vector<std::vector<TypedValue>> *group_by_keys,
+      int index) const;
+
   template <typename HandleT, typename HashTableT>
   inline TypedValue finalizeGroupInHashTable(
       const AggregationStateHashTableBase &hash_table,
@@ -224,11 +318,29 @@ class AggregationConcreteHandle : public AggregationHandle {
     return static_cast<const HandleT*>(this)->finalizeHashTableEntry(*group_state);
   }
 
+  template <typename HandleT, typename HashTableT>
+  inline TypedValue finalizeGroupInHashTableFast(
+      const AggregationStateHashTableBase &hash_table,
+      const std::vector<TypedValue> &group_key,
+      int index) const {
+    const std::uint8_t *group_state
+        = static_cast<const HashTableT&>(hash_table).getSingleCompositeKey(group_key, index);
+    DCHECK(group_state != nullptr)
+        << "Could not find entry for specified group_key in HashTable";
+    return static_cast<const HandleT*>(this)->finalizeHashTableEntryFast(group_state);
+  }
+
   template <typename HandleT, typename StateT, typename HashTableT>
   void mergeGroupByHashTablesHelper(
       const AggregationStateHashTableBase &source_hash_table,
       AggregationStateHashTableBase *destination_hash_table) const;
 
+  template <typename HandleT, typename HashTableT>
+  void mergeGroupByHashTablesHelperFast(
+      const AggregationStateHashTableBase &source_hash_table,
+      AggregationStateHashTableBase *destination_hash_table) const;
+
+
  private:
   DISALLOW_COPY_AND_ASSIGN(AggregationConcreteHandle);
 };
@@ -302,6 +414,12 @@ class HashTableAggregateFinalizer {
     output_column_vector_->appendTypedValue(handle_.finalizeHashTableEntry(group_state));
   }
 
+  inline void operator()(const std::vector<TypedValue> &group_by_key,
+                         const unsigned char *byte_ptr) {
+    group_by_keys_->emplace_back(group_by_key);
+    output_column_vector_->appendTypedValue(handle_.finalizeHashTableEntryFast(byte_ptr));
+  }
+
  private:
   const HandleT &handle_;
   std::vector<std::vector<TypedValue>> *group_by_keys_;
@@ -414,6 +532,42 @@ void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHe
 
 template <typename HandleT,
           typename HashTableT>
+void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
+    const AggregationStateHashTableBase &distinctify_hash_table,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  const HandleT& handle = static_cast<const HandleT&>(*this);
+  HashTableT *target_hash_table = static_cast<HashTableT*>(aggregation_hash_table);
+
+  // A lambda function which will be called on each key-value pair from the
+  // distinctify hash table.
+  const auto aggregate_functor = [&handle, &target_hash_table](
+      std::vector<TypedValue> &key,
+      const bool &dumb_placeholder) {
+    // For each (composite) key vector in the distinctify hash table with size N.
+    // The first N-1 entries are GROUP BY columns and the last entry is the argument
+    // to be aggregated on.
+    const TypedValue argument(std::move(key.back()));
+    key.pop_back();
+
+    // An upserter as lambda function for aggregating the argument into its
+    // GROUP BY group's entry inside aggregation_hash_table.
+    const auto upserter = [&handle, &argument](std::uint8_t *state) {
+      handle.iterateUnaryInlFast(argument, state+sizeof(SpinMutex));
+    };
+
+    target_hash_table->upsertCompositeKeyFast(key, nullptr, &upserter);
+  };
+
+  const HashTableT &source_hash_table =
+      static_cast<const HashTableT&>(distinctify_hash_table);
+  // Invoke the lambda function "aggregate_functor" on each composite key vector
+  // from the distinctify hash table.
+  source_hash_table.forEachCompositeKeyFast(&aggregate_functor);
+}
+
+
+template <typename HandleT,
+          typename HashTableT>
 ColumnVector* AggregationConcreteHandle::finalizeHashTableHelper(
     const Type &result_type,
     const AggregationStateHashTableBase &hash_table,
@@ -463,6 +617,59 @@ ColumnVector* AggregationConcreteHandle::finalizeHashTableHelper(
 }
 
 template <typename HandleT,
+          typename HashTableT>
+ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
+    const Type &result_type,
+    const AggregationStateHashTableBase &hash_table,
+    std::vector<std::vector<TypedValue>> *group_by_keys,
+    int index) const {
+  const HandleT &handle = static_cast<const HandleT&>(*this);
+  const HashTableT &hash_table_concrete = static_cast<const HashTableT&>(hash_table);
+
+  if (group_by_keys->empty()) {
+    if (NativeColumnVector::UsableForType(result_type)) {
+      NativeColumnVector *result = new NativeColumnVector(result_type,
+                                                          hash_table_concrete.numEntries());
+      HashTableAggregateFinalizer<HandleT, NativeColumnVector> finalizer(
+          handle,
+          group_by_keys,
+          result);
+      hash_table_concrete.forEachCompositeKeyFast(&finalizer, index);
+      return result;
+    } else {
+      IndirectColumnVector *result = new IndirectColumnVector(result_type,
+                                                              hash_table_concrete.numEntries());
+      HashTableAggregateFinalizer<HandleT, IndirectColumnVector> finalizer(
+          handle,
+          group_by_keys,
+          result);
+      hash_table_concrete.forEachCompositeKeyFast(&finalizer, index);
+      return result;
+    }
+  } else {
+    if (NativeColumnVector::UsableForType(result_type)) {
+      NativeColumnVector *result = new NativeColumnVector(result_type,
+                                                          group_by_keys->size());
+      for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
+        result->appendTypedValue(finalizeGroupInHashTableFast<HandleT, HashTableT>(hash_table,
+                                                                                   group_by_key,
+                                                                                   index));
+      }
+      return result;
+    } else {
+      IndirectColumnVector *result = new IndirectColumnVector(result_type,
+                                                              hash_table_concrete.numEntries());
+      for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
+        result->appendTypedValue(finalizeGroupInHashTableFast<HandleT, HashTableT>(hash_table,
+                                                                                   group_by_key,
+                                                                                   index));
+      }
+      return result;
+    }
+  }
+}
+
+template <typename HandleT,
           typename StateT,
           typename HashTableT>
 void AggregationConcreteHandle::mergeGroupByHashTablesHelper(
@@ -478,6 +685,22 @@ void AggregationConcreteHandle::mergeGroupByHashTablesHelper(
   source_hash_table_concrete.forEachCompositeKey(&merger);
 }
 
+template <typename HandleT,
+          typename HashTableT>
+void AggregationConcreteHandle::mergeGroupByHashTablesHelperFast(
+    const AggregationStateHashTableBase &source_hash_table,
+    AggregationStateHashTableBase *destination_hash_table) const {
+  const HandleT &handle = static_cast<const HandleT &>(*this);
+  const HashTableT &source_hash_table_concrete =
+      static_cast<const HashTableT &>(source_hash_table);
+
+  HashTableMergerFast<HandleT, HashTableT> merger(handle,
+                                              destination_hash_table);
+
+  source_hash_table_concrete.forEachCompositeKeyFast(&merger);
+}
+
+
 }  // namespace quickstep
 
 #endif  // QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index 3d6e872..92cd6a7 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -265,7 +265,7 @@ class AggregationHandle {
    **/
   virtual ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys) const = 0;
+      std::vector<std::vector<TypedValue>> *group_by_keys, int index) const = 0;
 
   /**
    * @brief Create a new HashTable for the distinctify step for DISTINCT aggregation.
@@ -362,6 +362,12 @@ class AggregationHandle {
       const AggregationStateHashTableBase &source_hash_table,
       AggregationStateHashTableBase *destination_hash_table) const = 0;
 
+  virtual size_t getPayloadSize() const {return 8;}
+  virtual void setPayloadOffset(std::size_t) {}
+  virtual void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) {}
+  virtual void mergeStatesFast(const uint8_t *src, uint8_t *dst) const {}
+  virtual void initPayload(uint8_t *byte_ptr) {}
+
  protected:
   AggregationHandle() {
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index 4bd43d6..f38c628 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.cpp
@@ -137,8 +137,7 @@ void AggregationHandleAvg::aggregateValueAccessorIntoHashTable(
     AggregationStateHashTableBase *hash_table) const {
   DCHECK_EQ(1u, argument_ids.size())
       << "Got wrong number of arguments for AVG: " << argument_ids.size();
-
-  aggregateValueAccessorIntoHashTableUnaryHelper<
+/*  aggregateValueAccessorIntoHashTableUnaryHelper<
       AggregationHandleAvg,
       AggregationStateAvg,
       AggregationStateHashTable<AggregationStateAvg>>(
@@ -146,7 +145,14 @@ void AggregationHandleAvg::aggregateValueAccessorIntoHashTable(
           argument_ids.front(),
           group_by_key_ids,
           blank_state_,
-          hash_table);
+          hash_table); */
+
+/*     static_cast<AggregationStateFastHashTable *>(hash_table)->upsertValueAccessorCompositeKeyFast(
+      argument_ids.front(),
+      accessor,
+      group_by_key_ids,
+      true,
+      const_cast<AggregationHandleAvg *>(this));*/
 }
 
 void AggregationHandleAvg::mergeStates(
@@ -161,6 +167,19 @@ void AggregationHandleAvg::mergeStates(
                                                                   avg_source.sum_);
 }
 
+void AggregationHandleAvg::mergeStatesFast(
+    const uint8_t *source,
+    uint8_t *destination) const {
+    const TypedValue *src_sum_ptr = reinterpret_cast<const TypedValue *>(source + blank_state_.sum_offset);
+    const std::int64_t *src_count_ptr = reinterpret_cast<const std::int64_t *>(source + blank_state_.count_offset);
+    TypedValue *dst_sum_ptr = reinterpret_cast<TypedValue *>(destination+blank_state_.sum_offset);
+    std::int64_t *dst_count_ptr = reinterpret_cast<std::int64_t *>(destination + blank_state_.count_offset);
+    (*dst_count_ptr) += (*src_count_ptr);
+    *dst_sum_ptr = merge_add_operator_->applyToTypedValues(*dst_sum_ptr, *src_sum_ptr);
+}
+
+
+
 TypedValue AggregationHandleAvg::finalize(const AggregationState &state) const {
   const AggregationStateAvg &agg_state = static_cast<const AggregationStateAvg&>(state);
   if (agg_state.count_ == 0) {
@@ -175,12 +194,14 @@ TypedValue AggregationHandleAvg::finalize(const AggregationState &state) const {
 
 ColumnVector* AggregationHandleAvg::finalizeHashTable(
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys) const {
-  return finalizeHashTableHelper<AggregationHandleAvg,
-                                 AggregationStateHashTable<AggregationStateAvg>>(
+    std::vector<std::vector<TypedValue>> *group_by_keys,
+    int index) const {
+  return finalizeHashTableHelperFast<AggregationHandleAvg,
+                                 AggregationStateFastHashTable>(
       *result_type_,
       hash_table,
-      group_by_keys);
+      group_by_keys,
+      index);
 }
 
 AggregationState* AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
@@ -206,9 +227,8 @@ void AggregationHandleAvg::aggregateOnDistinctifyHashTableForGroupBy(
 void AggregationHandleAvg::mergeGroupByHashTables(
     const AggregationStateHashTableBase &source_hash_table,
     AggregationStateHashTableBase *destination_hash_table) const {
-  mergeGroupByHashTablesHelper<AggregationHandleAvg,
-                               AggregationStateAvg,
-                               AggregationStateHashTable<AggregationStateAvg>>(
+  mergeGroupByHashTablesHelperFast<AggregationHandleAvg,
+                               AggregationStateFastHashTable>(
       source_hash_table, destination_hash_table);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index 31997b1..e187d22 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -29,6 +29,7 @@
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/FastHashTable.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -57,7 +58,10 @@ class AggregationStateAvg : public AggregationState {
    */
   AggregationStateAvg(const AggregationStateAvg &orig)
       : sum_(orig.sum_),
-        count_(orig.count_) {
+        count_(orig.count_),
+        sum_offset(orig.sum_offset),
+        count_offset(orig.count_offset),
+        mutex_offset(orig.mutex_offset) {
   }
 
   /**
@@ -65,11 +69,19 @@ class AggregationStateAvg : public AggregationState {
    */
   ~AggregationStateAvg() override {}
 
+  size_t getPayloadSize() const {
+     size_t p1 = reinterpret_cast<size_t>(&sum_);
+     size_t p2 = reinterpret_cast<size_t>(&mutex_);
+     return (p2-p1);
+  }
+
  private:
   friend class AggregationHandleAvg;
 
   AggregationStateAvg()
-      : sum_(0), count_(0) {
+      : sum_(0), count_(0), sum_offset(0),
+        count_offset(reinterpret_cast<uint8_t *>(&count_)-reinterpret_cast<uint8_t *>(&sum_)),
+        mutex_offset(reinterpret_cast<uint8_t *>(&mutex_)-reinterpret_cast<uint8_t *>(&sum_)) {
   }
 
   // TODO(shoban): We might want to specialize sum_ and count_ to use atomics
@@ -77,6 +89,8 @@ class AggregationStateAvg : public AggregationState {
   TypedValue sum_;
   std::int64_t count_;
   SpinMutex mutex_;
+
+  int sum_offset, count_offset, mutex_offset;
 };
 
 /**
@@ -109,6 +123,26 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     ++state->count_;
   }
 
+  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) {
+    DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
+    if (value.isNull()) return;
+    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
+    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr + blank_state_.count_offset);
+    *sum_ptr = fast_add_operator_->applyToTypedValues(*sum_ptr, value);
+    ++(*count_ptr);
+  }
+
+  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+     iterateUnaryInlFast(arguments.front(), byte_ptr);
+  }
+
+  void initPayload(uint8_t *byte_ptr) override {
+    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
+    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr + blank_state_.count_offset);
+    *sum_ptr = blank_state_.sum_;
+    *count_ptr = blank_state_.count_;
+  }
+
   AggregationState* accumulateColumnVectors(
       const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
 
@@ -127,6 +161,9 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override;
 
+  void mergeStatesFast(const uint8_t *source,
+                   uint8_t *destination) const override;
+
   TypedValue finalize(const AggregationState &state) const override;
 
   inline TypedValue finalizeHashTableEntry(const AggregationState &state) const {
@@ -139,9 +176,24 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
                                                 TypedValue(static_cast<double>(agg_state.count_)));
   }
 
+  inline TypedValue finalizeHashTableEntryFast(const uint8_t *byte_ptr) const {
+//    const AggregationStateAvg &agg_state = static_cast<const AggregationStateAvg&>(state);
+    // TODO(chasseur): Could improve performance further if we made a special
+    // version of finalizeHashTable() that collects all the sums into one
+    // ColumnVector and all the counts into another and then applies
+    // '*divide_operator_' to them in bulk.
+
+    uint8_t *value_ptr = const_cast<uint8_t*>(byte_ptr);
+    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(value_ptr + blank_state_.sum_offset);
+    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(value_ptr + blank_state_.count_offset);
+    return divide_operator_->applyToTypedValues(*sum_ptr,
+                                                TypedValue(static_cast<double>(*count_ptr)));
+  }
+
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
+      std::vector<std::vector<TypedValue>> *group_by_keys,
+      int index) const override;
 
   /**
    * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
@@ -162,6 +214,10 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
       const AggregationStateHashTableBase &source_hash_table,
       AggregationStateHashTableBase *destination_hash_table) const override;
 
+  size_t getPayloadSize() const override {
+      return blank_state_.getPayloadSize();
+  }
+
  private:
   friend class AggregateFunctionAvg;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.cpp b/expressions/aggregation/AggregationHandleCount.cpp
index dfcf131..f1eadf1 100644
--- a/expressions/aggregation/AggregationHandleCount.cpp
+++ b/expressions/aggregation/AggregationHandleCount.cpp
@@ -135,18 +135,18 @@ template <bool count_star, bool nullable_type>
   if (count_star) {
     DCHECK_EQ(0u, argument_ids.size())
         << "Got wrong number of arguments for COUNT(*): " << argument_ids.size();
-    aggregateValueAccessorIntoHashTableNullaryHelper<
+/*    aggregateValueAccessorIntoHashTableNullaryHelper<
         AggregationHandleCount<count_star, nullable_type>,
         AggregationStateCount,
         AggregationStateHashTable<AggregationStateCount>>(
             accessor,
             group_by_key_ids,
             AggregationStateCount(),
-            hash_table);
+            hash_table);*/
   } else {
     DCHECK_EQ(1u, argument_ids.size())
         << "Got wrong number of arguments for COUNT: " << argument_ids.size();
-    aggregateValueAccessorIntoHashTableUnaryHelper<
+/*    aggregateValueAccessorIntoHashTableUnaryHelper<
         AggregationHandleCount<count_star, nullable_type>,
         AggregationStateCount,
         AggregationStateHashTable<AggregationStateCount>>(
@@ -154,7 +154,7 @@ template <bool count_star, bool nullable_type>
             argument_ids.front(),
             group_by_key_ids,
             AggregationStateCount(),
-            hash_table);
+            hash_table); */
   }
 }
 
@@ -170,14 +170,25 @@ template <bool count_star, bool nullable_type>
 }
 
 template <bool count_star, bool nullable_type>
+void AggregationHandleCount<count_star, nullable_type>::mergeStatesFast(
+    const uint8_t *source,
+    uint8_t *destination) const {
+    const std::int64_t *src_count_ptr = reinterpret_cast<const std::int64_t *>(source);
+    std::int64_t *dst_count_ptr = reinterpret_cast<std::int64_t *>(destination);
+    (*dst_count_ptr) += (*src_count_ptr);
+}
+
+template <bool count_star, bool nullable_type>
     ColumnVector* AggregationHandleCount<count_star, nullable_type>::finalizeHashTable(
         const AggregationStateHashTableBase &hash_table,
-        std::vector<std::vector<TypedValue>> *group_by_keys) const {
-  return finalizeHashTableHelper<AggregationHandleCount<count_star, nullable_type>,
-                                 AggregationStateHashTable<AggregationStateCount>>(
+        std::vector<std::vector<TypedValue>> *group_by_keys,
+        int index) const {
+  return finalizeHashTableHelperFast<AggregationHandleCount<count_star, nullable_type>,
+                                 AggregationStateFastHashTable>(
       TypeFactory::GetType(kLong),
       hash_table,
-      group_by_keys);
+      group_by_keys,
+      index);
 }
 
 template <bool count_star, bool nullable_type>
@@ -197,12 +208,10 @@ void AggregationHandleCount<count_star, nullable_type>
         const AggregationStateHashTableBase &distinctify_hash_table,
         AggregationStateHashTableBase *aggregation_hash_table) const {
   DCHECK_EQ(count_star, false);
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleCount<count_star, nullable_type>,
-      AggregationStateCount,
-      AggregationStateHashTable<AggregationStateCount>>(
+      AggregationStateFastHashTable>(
           distinctify_hash_table,
-          AggregationStateCount(),
           aggregation_hash_table);
 }
 
@@ -210,10 +219,9 @@ template <bool count_star, bool nullable_type>
 void AggregationHandleCount<count_star, nullable_type>::mergeGroupByHashTables(
     const AggregationStateHashTableBase &source_hash_table,
     AggregationStateHashTableBase *destination_hash_table) const {
-  mergeGroupByHashTablesHelper<
+  mergeGroupByHashTablesHelperFast<
       AggregationHandleCount,
-      AggregationStateCount,
-      AggregationStateHashTable<AggregationStateCount>>(source_hash_table,
+      AggregationStateFastHashTable>(source_hash_table,
                                                         destination_hash_table);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index 1cd5bda..ed21c41 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.hpp
@@ -30,6 +30,7 @@
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/FastHashTable.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
@@ -62,6 +63,10 @@ class AggregationStateCount : public AggregationState {
    */
   ~AggregationStateCount() override {}
 
+  size_t getPayloadSize() const {
+     return sizeof(count_);
+  }
+
  private:
   friend class AggregationHandleCount<false, false>;
   friend class AggregationHandleCount<false, true>;
@@ -108,6 +113,11 @@ class AggregationHandleCount : public AggregationConcreteHandle {
     state->count_.fetch_add(1, std::memory_order_relaxed);
   }
 
+  inline void iterateNullaryInlFast(uint8_t *byte_ptr) {
+      std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
+      (*count_ptr)++;
+  }
+
   /**
    * @brief Iterate with count aggregation state.
    */
@@ -117,6 +127,25 @@ class AggregationHandleCount : public AggregationConcreteHandle {
     }
   }
 
+  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) const {
+    if ((!nullable_type) || (!value.isNull())) {
+      std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
+      (*count_ptr)++;
+    }
+  }
+
+  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+     if (arguments.size())
+         iterateUnaryInlFast(arguments.front(), byte_ptr);
+     else
+         iterateNullaryInlFast(byte_ptr);
+  }
+
+  void initPayload(uint8_t *byte_ptr) override {
+    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
+    *count_ptr = 0;
+  }
+
   AggregationState* accumulateNullary(const std::size_t num_tuples) const override {
     return new AggregationStateCount(num_tuples);
   }
@@ -139,6 +168,9 @@ class AggregationHandleCount : public AggregationConcreteHandle {
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override;
 
+  void mergeStatesFast(const uint8_t *source,
+                   uint8_t *destination) const override;
+
   TypedValue finalize(const AggregationState &state) const override {
     return TypedValue(static_cast<const AggregationStateCount&>(state).count_.load(std::memory_order_relaxed));
   }
@@ -147,9 +179,21 @@ class AggregationHandleCount : public AggregationConcreteHandle {
     return TypedValue(static_cast<const AggregationStateCount&>(state).count_.load(std::memory_order_relaxed));
   }
 
+  inline TypedValue finalizeHashTableEntryFast(const uint8_t *byte_ptr) const {
+//    const AggregationStateAvg &agg_state = static_cast<const AggregationStateAvg&>(state);
+    // TODO(chasseur): Could improve performance further if we made a special
+    // version of finalizeHashTable() that collects all the sums into one
+    // ColumnVector and all the counts into another and then applies
+    // '*divide_operator_' to them in bulk.
+
+    const std::int64_t *count_ptr = reinterpret_cast<const std::int64_t *>(byte_ptr);
+    return TypedValue(*count_ptr);
+  }
+
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
+      std::vector<std::vector<TypedValue>> *group_by_keys,
+      int index) const override;
 
   /**
    * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
@@ -170,6 +214,10 @@ class AggregationHandleCount : public AggregationConcreteHandle {
       const AggregationStateHashTableBase &source_hash_table,
       AggregationStateHashTableBase *destination_hash_table) const override;
 
+  size_t getPayloadSize() const override {
+      return sizeof(std::int64_t);
+  }
+
  private:
   friend class AggregateFunctionCount;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleDistinct.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.cpp b/expressions/aggregation/AggregationHandleDistinct.cpp
index 68fcd4c..2b9391a 100644
--- a/expressions/aggregation/AggregationHandleDistinct.cpp
+++ b/expressions/aggregation/AggregationHandleDistinct.cpp
@@ -65,7 +65,7 @@ void AggregationHandleDistinct::aggregateValueAccessorIntoHashTable(
 
 ColumnVector* AggregationHandleDistinct::finalizeHashTable(
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys) const {
+    std::vector<std::vector<TypedValue>> *group_by_keys, int index) const {
   DCHECK(group_by_keys->empty());
 
   const auto keys_retriever = [&group_by_keys](std::vector<TypedValue> &group_by_key,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleDistinct.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.hpp b/expressions/aggregation/AggregationHandleDistinct.hpp
index 8524fcc..0a3acb3 100644
--- a/expressions/aggregation/AggregationHandleDistinct.hpp
+++ b/expressions/aggregation/AggregationHandleDistinct.hpp
@@ -109,7 +109,7 @@ class AggregationHandleDistinct : public AggregationConcreteHandle {
 
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
+      std::vector<std::vector<TypedValue>> *group_by_keys, int index) const override;
 
   void mergeGroupByHashTables(
       const AggregationStateHashTableBase &source_hash_table,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.cpp b/expressions/aggregation/AggregationHandleMax.cpp
index 435f5f2..2080a03 100644
--- a/expressions/aggregation/AggregationHandleMax.cpp
+++ b/expressions/aggregation/AggregationHandleMax.cpp
@@ -89,7 +89,7 @@ void AggregationHandleMax::aggregateValueAccessorIntoHashTable(
   DCHECK_EQ(1u, argument_ids.size())
       << "Got wrong number of arguments for MAX: " << argument_ids.size();
 
-  aggregateValueAccessorIntoHashTableUnaryHelper<
+/*  aggregateValueAccessorIntoHashTableUnaryHelper<
       AggregationHandleMax,
       AggregationStateMax,
       AggregationStateHashTable<AggregationStateMax>>(
@@ -97,7 +97,7 @@ void AggregationHandleMax::aggregateValueAccessorIntoHashTable(
           argument_ids.front(),
           group_by_key_ids,
           AggregationStateMax(type_),
-          hash_table);
+          hash_table);*/
 }
 
 void AggregationHandleMax::mergeStates(
@@ -111,14 +111,26 @@ void AggregationHandleMax::mergeStates(
   }
 }
 
+void AggregationHandleMax::mergeStatesFast(
+    const std::uint8_t *source,
+    std::uint8_t *destination) const {
+    const TypedValue *src_max_ptr = reinterpret_cast<const TypedValue *>(source);
+    TypedValue *dst_max_ptr = reinterpret_cast<TypedValue *>(destination);
+    if (!(src_max_ptr->isNull())) {
+      compareAndUpdateFast(dst_max_ptr, *src_max_ptr);
+  }
+}
+
 ColumnVector* AggregationHandleMax::finalizeHashTable(
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys) const {
-  return finalizeHashTableHelper<AggregationHandleMax,
-                                 AggregationStateHashTable<AggregationStateMax>>(
+    std::vector<std::vector<TypedValue>> *group_by_keys,
+    int index) const {
+  return finalizeHashTableHelperFast<AggregationHandleMax,
+                                 AggregationStateFastHashTable>(
       type_.getNullableVersion(),
       hash_table,
-      group_by_keys);
+      group_by_keys,
+      index);
 }
 
 AggregationState* AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
@@ -144,9 +156,8 @@ void AggregationHandleMax::aggregateOnDistinctifyHashTableForGroupBy(
 void AggregationHandleMax::mergeGroupByHashTables(
     const AggregationStateHashTableBase &source_hash_table,
     AggregationStateHashTableBase *destination_hash_table) const {
-  mergeGroupByHashTablesHelper<AggregationHandleMax,
-                               AggregationStateMax,
-                               AggregationStateHashTable<AggregationStateMax>>(
+  mergeGroupByHashTablesHelperFast<AggregationHandleMax,
+                               AggregationStateFastHashTable>(
       source_hash_table, destination_hash_table);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index 7e38473..3c06fc4 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -29,6 +29,7 @@
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/FastHashTable.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -105,6 +106,22 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     compareAndUpdate(static_cast<AggregationStateMax*>(state), value);
   }
 
+  inline void iterateUnaryInlFast(const TypedValue &value, std::uint8_t *byte_ptr) const {
+    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
+    compareAndUpdateFast(max_ptr, value);
+  }
+
+  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+    iterateUnaryInlFast(arguments.front(), byte_ptr);
+  }
+
+  void initPayload(uint8_t *byte_ptr) override {
+    TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
+    TypedValue t1 = (type_.getNullableVersion().makeNullValue());
+    *max_ptr = t1;
+  }
+
   AggregationState* accumulateColumnVectors(
       const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
 
@@ -123,6 +140,9 @@ class AggregationHandleMax : public AggregationConcreteHandle {
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override;
 
+  void mergeStatesFast(const std::uint8_t *source,
+                   std::uint8_t *destination) const override;
+
   TypedValue finalize(const AggregationState &state) const override {
     return TypedValue(static_cast<const AggregationStateMax&>(state).max_);
   }
@@ -131,9 +151,15 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     return TypedValue(static_cast<const AggregationStateMax&>(state).max_);
   }
 
+  inline TypedValue finalizeHashTableEntryFast(const std::uint8_t *byte_ptr) const {
+    const TypedValue *max_ptr = reinterpret_cast<const TypedValue *>(byte_ptr);
+    return TypedValue(*max_ptr);
+  }
+
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
+      std::vector<std::vector<TypedValue>> *group_by_keys,
+      int index) const override;
 
   /**
    * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
@@ -155,6 +181,10 @@ class AggregationHandleMax : public AggregationConcreteHandle {
       const AggregationStateHashTableBase &source_hash_table,
       AggregationStateHashTableBase *destination_hash_table) const override;
 
+  size_t getPayloadSize() const override {
+      return sizeof(TypedValue);
+  }
+
  private:
   friend class AggregateFunctionMax;
 
@@ -181,6 +211,13 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     }
   }
 
+  inline void compareAndUpdateFast(TypedValue *max_ptr, const TypedValue &value) const {
+    if (value.isNull()) return;
+    if (max_ptr->isNull() || fast_comparator_->compareTypedValues(value, *max_ptr)) {
+      *max_ptr = value;
+    }
+  }
+
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.cpp b/expressions/aggregation/AggregationHandleMin.cpp
index e860d8d..9d5be72 100644
--- a/expressions/aggregation/AggregationHandleMin.cpp
+++ b/expressions/aggregation/AggregationHandleMin.cpp
@@ -89,7 +89,7 @@ void AggregationHandleMin::aggregateValueAccessorIntoHashTable(
   DCHECK_EQ(1u, argument_ids.size())
       << "Got wrong number of arguments for MIN: " << argument_ids.size();
 
-  aggregateValueAccessorIntoHashTableUnaryHelper<
+/*  aggregateValueAccessorIntoHashTableUnaryHelper<
       AggregationHandleMin,
       AggregationStateMin,
       AggregationStateHashTable<AggregationStateMin>>(
@@ -97,7 +97,7 @@ void AggregationHandleMin::aggregateValueAccessorIntoHashTable(
           argument_ids.front(),
           group_by_key_ids,
           AggregationStateMin(type_),
-          hash_table);
+          hash_table);*/
 }
 
 void AggregationHandleMin::mergeStates(
@@ -111,14 +111,27 @@ void AggregationHandleMin::mergeStates(
   }
 }
 
+void AggregationHandleMin::mergeStatesFast(
+    const std::uint8_t *source,
+    std::uint8_t *destination) const {
+    const TypedValue *src_min_ptr = reinterpret_cast<const TypedValue *>(source);
+    TypedValue *dst_min_ptr = reinterpret_cast<TypedValue *>(destination);
+
+    if (!(src_min_ptr->isNull())) {
+      compareAndUpdateFast(dst_min_ptr, *src_min_ptr);
+    }
+}
+
 ColumnVector* AggregationHandleMin::finalizeHashTable(
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys) const {
-  return finalizeHashTableHelper<AggregationHandleMin,
-                                 AggregationStateHashTable<AggregationStateMin>>(
+    std::vector<std::vector<TypedValue>> *group_by_keys,
+    int index) const {
+  return finalizeHashTableHelperFast<AggregationHandleMin,
+                                 AggregationStateFastHashTable>(
       type_.getNonNullableVersion(),
       hash_table,
-      group_by_keys);
+      group_by_keys,
+      index);
 }
 
 AggregationState* AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
@@ -144,9 +157,8 @@ void AggregationHandleMin::aggregateOnDistinctifyHashTableForGroupBy(
 void AggregationHandleMin::mergeGroupByHashTables(
     const AggregationStateHashTableBase &source_hash_table,
     AggregationStateHashTableBase *destination_hash_table) const {
-  mergeGroupByHashTablesHelper<AggregationHandleMin,
-                               AggregationStateMin,
-                               AggregationStateHashTable<AggregationStateMin>>(
+  mergeGroupByHashTablesHelperFast<AggregationHandleMin,
+                               AggregationStateFastHashTable>(
       source_hash_table, destination_hash_table);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index 924698c..6329cd7 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -29,6 +29,7 @@
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/FastHashTable.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -64,6 +65,11 @@ class AggregationStateMin : public AggregationState {
    */
   ~AggregationStateMin() override {}
 
+  size_t getPayloadSize() const {
+     return sizeof(TypedValue);
+  }
+
+
  private:
   friend class AggregationHandleMin;
 
@@ -104,6 +110,22 @@ class AggregationHandleMin : public AggregationConcreteHandle {
     compareAndUpdate(state, value);
   }
 
+  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) {
+      DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+      TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
+      compareAndUpdateFast(min_ptr, value);
+  }
+
+  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+    iterateUnaryInlFast(arguments.front(), byte_ptr);
+  }
+
+  void initPayload(uint8_t *byte_ptr) override {
+    TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
+    TypedValue t1 = (type_.getNullableVersion().makeNullValue());
+    *min_ptr = t1;
+  }
+
   AggregationState* accumulateColumnVectors(
       const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
 
@@ -122,6 +144,9 @@ class AggregationHandleMin : public AggregationConcreteHandle {
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override;
 
+  void mergeStatesFast(const uint8_t *source,
+                   uint8_t *destination) const override;
+
   TypedValue finalize(const AggregationState &state) const override {
     return static_cast<const AggregationStateMin&>(state).min_;
   }
@@ -130,9 +155,15 @@ class AggregationHandleMin : public AggregationConcreteHandle {
     return static_cast<const AggregationStateMin&>(state).min_;
   }
 
+  inline TypedValue finalizeHashTableEntryFast(const std::uint8_t *byte_ptr) const {
+    const TypedValue *min_ptr = reinterpret_cast<const TypedValue *>(byte_ptr);
+    return TypedValue(*min_ptr);
+  }
+
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
+      std::vector<std::vector<TypedValue>> *group_by_keys,
+      int index) const override;
 
   /**
    * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
@@ -153,6 +184,10 @@ class AggregationHandleMin : public AggregationConcreteHandle {
       const AggregationStateHashTableBase &source_hash_table,
       AggregationStateHashTableBase *destination_hash_table) const override;
 
+  size_t getPayloadSize() const override {
+      return sizeof(TypedValue);
+  }
+
  private:
   friend class AggregateFunctionMin;
 
@@ -178,6 +213,13 @@ class AggregationHandleMin : public AggregationConcreteHandle {
     }
   }
 
+  inline void compareAndUpdateFast(TypedValue *min_ptr, const TypedValue &value) const {
+    if (value.isNull()) return;
+    if (min_ptr->isNull() || fast_comparator_->compareTypedValues(value, *min_ptr)) {
+      *min_ptr = value;
+    }
+  }
+
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index b5036a8..7a16605 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -93,7 +93,6 @@ AggregationState* AggregationHandleSum::accumulateColumnVectors(
     const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
   DCHECK_EQ(1u, column_vectors.size())
       << "Got wrong number of ColumnVectors for SUM: " << column_vectors.size();
-
   std::size_t num_tuples = 0;
   TypedValue cv_sum = fast_operator_->accumulateColumnVector(
       blank_state_.sum_,
@@ -127,7 +126,7 @@ void AggregationHandleSum::aggregateValueAccessorIntoHashTable(
   DCHECK_EQ(1u, argument_ids.size())
       << "Got wrong number of arguments for SUM: " << argument_ids.size();
 
-  aggregateValueAccessorIntoHashTableUnaryHelper<
+/*  aggregateValueAccessorIntoHashTableUnaryHelper<
       AggregationHandleSum,
       AggregationStateSum,
       AggregationStateHashTable<AggregationStateSum>>(
@@ -135,7 +134,7 @@ void AggregationHandleSum::aggregateValueAccessorIntoHashTable(
           argument_ids.front(),
           group_by_key_ids,
           blank_state_,
-          hash_table);
+          hash_table);*/
 }
 
 void AggregationHandleSum::mergeStates(
@@ -150,6 +149,17 @@ void AggregationHandleSum::mergeStates(
   sum_destination->null_ = sum_destination->null_ && sum_source.null_;
 }
 
+void AggregationHandleSum::mergeStatesFast(
+    const uint8_t *source,
+    uint8_t *destination) const {
+    const TypedValue *src_sum_ptr = reinterpret_cast<const TypedValue *>(source+blank_state_.sum_offset);
+    const bool *src_null_ptr = reinterpret_cast<const bool *>(source+blank_state_.null_offset);
+    TypedValue *dst_sum_ptr = reinterpret_cast<TypedValue *>(destination+blank_state_.sum_offset);
+    bool *dst_null_ptr = reinterpret_cast<bool *>(destination+blank_state_.null_offset);
+    *dst_sum_ptr = merge_operator_->applyToTypedValues(*dst_sum_ptr, *src_sum_ptr);
+    *dst_null_ptr = (*dst_null_ptr) && (*src_null_ptr);
+}
+
 TypedValue AggregationHandleSum::finalize(const AggregationState &state) const {
   const AggregationStateSum &agg_state = static_cast<const AggregationStateSum&>(state);
   if (agg_state.null_) {
@@ -162,12 +172,14 @@ TypedValue AggregationHandleSum::finalize(const AggregationState &state) const {
 
 ColumnVector* AggregationHandleSum::finalizeHashTable(
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys) const {
-  return finalizeHashTableHelper<AggregationHandleSum,
-                                 AggregationStateHashTable<AggregationStateSum>>(
+    std::vector<std::vector<TypedValue>> *group_by_keys,
+    int index) const {
+  return finalizeHashTableHelperFast<AggregationHandleSum,
+                                 AggregationStateFastHashTable>(
       *result_type_,
       hash_table,
-      group_by_keys);
+      group_by_keys,
+      index);
 }
 
 AggregationState* AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle(
@@ -193,9 +205,8 @@ void AggregationHandleSum::aggregateOnDistinctifyHashTableForGroupBy(
 void AggregationHandleSum::mergeGroupByHashTables(
     const AggregationStateHashTableBase &source_hash_table,
     AggregationStateHashTableBase *destination_hash_table) const {
-  mergeGroupByHashTablesHelper<AggregationHandleSum,
-                               AggregationStateSum,
-                               AggregationStateHashTable<AggregationStateSum>>(
+  mergeGroupByHashTablesHelperFast<AggregationHandleSum,
+                               AggregationStateFastHashTable>(
       source_hash_table, destination_hash_table);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index 3382646..79f8331 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -29,6 +29,7 @@
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/FastHashTable.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -57,27 +58,39 @@ class AggregationStateSum : public AggregationState {
    */
   AggregationStateSum(const AggregationStateSum &orig)
       : sum_(orig.sum_),
-        null_(orig.null_) {
+        null_(orig.null_),
+        sum_offset(orig.sum_offset),
+        null_offset(orig.null_offset) {
   }
 
  private:
   friend class AggregationHandleSum;
 
   AggregationStateSum()
-      : sum_(0), null_(true) {
+      : sum_(0), null_(true), sum_offset(0),
+        null_offset(reinterpret_cast<uint8_t *>(&null_)-reinterpret_cast<uint8_t *>(&sum_)) {
   }
 
   AggregationStateSum(TypedValue &&sum, const bool is_null)
       : sum_(std::move(sum)), null_(is_null) {
   }
 
+  size_t getPayloadSize() const {
+     size_t p1 = reinterpret_cast<size_t>(&sum_);
+     size_t p2 = reinterpret_cast<size_t>(&mutex_);
+     return (p2-p1);
+  }
+
   // TODO(shoban): We might want to specialize sum_ to use atomics for int types
   // similar to in AggregationStateCount.
   TypedValue sum_;
   bool null_;
   SpinMutex mutex_;
+
+  int sum_offset, null_offset;
 };
 
+
 /**
  * @brief An aggregationhandle for sum.
  **/
@@ -105,6 +118,26 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     state->null_ = false;
   }
 
+  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) {
+    DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
+    if (value.isNull()) return;
+    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
+    bool *null_ptr = reinterpret_cast<bool *>(byte_ptr + blank_state_.null_offset);
+    *sum_ptr = fast_operator_->applyToTypedValues(*sum_ptr, value);
+    *null_ptr = false;
+  }
+
+  inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+     iterateUnaryInlFast(arguments.front(), byte_ptr);
+  }
+
+  void initPayload(uint8_t *byte_ptr) override {
+    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
+    bool *null_ptr = reinterpret_cast<bool *>(byte_ptr + blank_state_.null_offset);
+    *sum_ptr = blank_state_.sum_;
+    *null_ptr = true;
+  }
+
   AggregationState* accumulateColumnVectors(
       const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
 
@@ -123,15 +156,24 @@ class AggregationHandleSum : public AggregationConcreteHandle {
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override;
 
+  void mergeStatesFast(const uint8_t *source,
+                   uint8_t *destination) const override;
+
   TypedValue finalize(const AggregationState &state) const override;
 
   inline TypedValue finalizeHashTableEntry(const AggregationState &state) const {
     return static_cast<const AggregationStateSum&>(state).sum_;
   }
 
+  inline TypedValue finalizeHashTableEntryFast(const uint8_t *byte_ptr) const {
+    uint8_t *value_ptr = const_cast<uint8_t*>(byte_ptr);
+    TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(value_ptr + blank_state_.sum_offset);
+    return *sum_ptr;
+  }
+
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
+      std::vector<std::vector<TypedValue>> *group_by_keys, int index) const override;
 
   /**
    * @brief Implementation of AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
@@ -152,6 +194,10 @@ class AggregationHandleSum : public AggregationConcreteHandle {
       const AggregationStateHashTableBase &source_hash_table,
       AggregationStateHashTableBase *destination_hash_table) const override;
 
+  size_t getPayloadSize() const override {
+      return blank_state_.getPayloadSize();
+  }
+
  private:
   friend class AggregateFunctionSum;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index 888d95c..98222df 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -146,9 +146,11 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationConcreteHandl
                       glog
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
+                      quickstep_threading_SpinMutex
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_utility_Macros)
@@ -163,6 +165,7 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleAvg
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
@@ -180,6 +183,7 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleCount
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
@@ -204,6 +208,7 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMax
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
@@ -220,6 +225,7 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMin
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
@@ -236,6 +242,7 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleSum
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 3f6e23a..7d6d179 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -83,6 +83,9 @@ AggregationOperationState::AggregationOperationState(
     group_by_types.emplace_back(&group_by_element->getType());
   }
 
+  std::vector<AggregationHandle *> group_by_handles;
+  group_by_handles.clear();
+
   if (aggregate_functions.size() == 0) {
     // If there is no aggregation function, then it is a distinctify operation
     // on the group-by expressions.
@@ -92,11 +95,17 @@ AggregationOperationState::AggregationOperationState(
     arguments_.push_back({});
     is_distinct_.emplace_back(false);
 
-    group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
+ /*   group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
         new HashTablePool(estimated_num_entries,
                           hash_table_impl_type,
                           group_by_types,
                           handles_.back().get(),
+                          storage_manager)));*/
+    group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
+        new HashTablePool(estimated_num_entries,
+                          hash_table_impl_type,
+                          group_by_types,
+                          handles_.back(),
                           storage_manager)));
   } else {
     // Set up each individual aggregate in this operation.
@@ -107,6 +116,7 @@ AggregationOperationState::AggregationOperationState(
     std::vector<bool>::const_iterator is_distinct_it = is_distinct_.begin();
     std::vector<HashTableImplType>::const_iterator distinctify_hash_table_impl_types_it
         = distinctify_hash_table_impl_types.begin();
+    std::vector<std::size_t> payload_sizes;
     for (; agg_func_it != aggregate_functions.end(); ++agg_func_it, ++args_it, ++is_distinct_it) {
       // Get the Types of this aggregate's arguments so that we can create an
       // AggregationHandle.
@@ -126,12 +136,15 @@ AggregationOperationState::AggregationOperationState(
 
       if (!group_by_list_.empty()) {
         // Aggregation with GROUP BY: create a HashTable pool for per-group states.
-        group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
+ /*       group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
             new HashTablePool(estimated_num_entries,
                               hash_table_impl_type,
                               group_by_types,
                               handles_.back().get(),
-                              storage_manager)));
+                              storage_manager)));*/
+         group_by_handles.emplace_back(handles_.back());
+         payload_sizes.emplace_back(handles_.back()->getPayloadSize());
+
       } else {
         // Aggregation without GROUP BY: create a single global state.
         single_states_.emplace_back(handles_.back()->createInitialState());
@@ -166,17 +179,40 @@ AggregationOperationState::AggregationOperationState(
         // the number of entries in the distinctify hash table. We may estimate
         // for each distinct aggregation an estimated_num_distinct_keys value during
         // query optimization, if it worths.
-        distinctify_hashtables_.emplace_back(
+ /*       distinctify_hashtables_.emplace_back(
             handles_.back()->createDistinctifyHashTable(
                 *distinctify_hash_table_impl_types_it,
                 key_types,
                 estimated_num_entries,
+                storage_manager));*/
+
+std::vector<AggregationHandle *> local;
+local.emplace_back(handles_.back());
+        distinctify_hashtables_.emplace_back(
+AggregationStateFastHashTableFactory::CreateResizable(
+                *distinctify_hash_table_impl_types_it,
+                key_types,
+                estimated_num_entries,
+                {0},
+                local,
                 storage_manager));
+
         ++distinctify_hash_table_impl_types_it;
       } else {
         distinctify_hashtables_.emplace_back(nullptr);
       }
     }
+
+      if (!group_by_handles.empty()) {
+        // Aggregation with GROUP BY: create a HashTable pool for per-group states.
+        group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
+            new HashTablePool(estimated_num_entries,
+                              hash_table_impl_type,
+                              group_by_types,
+                              payload_sizes,
+                              group_by_handles,
+                              storage_manager)));
+      }
   }
 }
 
@@ -410,17 +446,24 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
       // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
       // directly into the (threadsafe) shared global HashTable for this
       // aggregate.
-      DCHECK(group_by_hashtable_pools_[agg_idx] != nullptr);
-      AggregationStateHashTableBase *agg_hash_table = group_by_hashtable_pools_[agg_idx]->getHashTable();
+      DCHECK(group_by_hashtable_pools_[0] != nullptr);
+      AggregationStateHashTableBase *agg_hash_table = group_by_hashtable_pools_[0]->getHashTableFast();
       DCHECK(agg_hash_table != nullptr);
-      block->aggregateGroupBy(*handles_[agg_idx],
+ /*     block->aggregateGroupBy(*handles_[agg_idx],
                               arguments_[agg_idx],
                               group_by_list_,
                               predicate_.get(),
                               agg_hash_table,
                               &reuse_matches,
+                              &reuse_group_by_vectors);*/
+      block->aggregateGroupByFast(arguments_,
+                              group_by_list_,
+                              predicate_.get(),
+                              agg_hash_table,
+                              &reuse_matches,
                               &reuse_group_by_vectors);
-      group_by_hashtable_pools_[agg_idx]->returnHashTable(agg_hash_table);
+      group_by_hashtable_pools_[0]->returnHashTable(agg_hash_table);
+      break;
     }
   }
 }
@@ -444,6 +487,12 @@ void AggregationOperationState::finalizeSingleState(InsertDestination *output_de
   output_destination->insertTuple(Tuple(std::move(attribute_values)));
 }
 
+void AggregationOperationState::mergeGroupByHashTables(AggregationStateHashTableBase *src,
+                                                       AggregationStateHashTableBase *dst) {
+    HashTableMergerNewFast merger(dst);
+    (static_cast<FastHashTable<true, false, true, false> *>(src))->forEachCompositeKeyFast(&merger);
+}
+
 void AggregationOperationState::finalizeHashTable(InsertDestination *output_destination) {
   // Each element of 'group_by_keys' is a vector of values for a particular
   // group (which is also the prefix of the finalized Tuple for that group).
@@ -455,18 +504,21 @@ void AggregationOperationState::finalizeHashTable(InsertDestination *output_dest
 
   // TODO(harshad) - Find heuristics for faster merge, even in a single thread.
   // e.g. Keep merging entries from smaller hash tables to larger.
+//  auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
+
+  auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    auto *hash_tables = group_by_hashtable_pools_[agg_idx]->getAllHashTables();
     if (hash_tables->size() > 1) {
       for (int hash_table_index = 0;
            hash_table_index < static_cast<int>(hash_tables->size() - 1);
            ++hash_table_index) {
         // Merge each hash table to the last hash table.
-        handles_[agg_idx]->mergeGroupByHashTables(
-            (*(*hash_tables)[hash_table_index]),
+        mergeGroupByHashTables(
+            (*hash_tables)[hash_table_index].get(),
             hash_tables->back().get());
       }
     }
+    break;
   }
 
   // Collect per-aggregate finalized values.
@@ -475,16 +527,16 @@ void AggregationOperationState::finalizeHashTable(InsertDestination *output_dest
        agg_idx < handles_.size();
        ++agg_idx) {
     if (is_distinct_[agg_idx]) {
-      DCHECK(group_by_hashtable_pools_[agg_idx] != nullptr);
-      auto *hash_tables = group_by_hashtable_pools_[agg_idx]->getAllHashTables();
+      DCHECK(group_by_hashtable_pools_[0] != nullptr);
+      auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
       DCHECK(hash_tables != nullptr);
       if (hash_tables->empty()) {
         // We may have a case where hash_tables is empty, e.g. no input blocks.
         // However for aggregateOnDistinctifyHashTableForGroupBy to work
         // correctly, we should create an empty group by hash table.
-        AggregationStateHashTableBase *new_hash_table = group_by_hashtable_pools_[agg_idx]->getHashTable();
-        group_by_hashtable_pools_[agg_idx]->returnHashTable(new_hash_table);
-        hash_tables = group_by_hashtable_pools_[agg_idx]->getAllHashTables();
+        AggregationStateHashTableBase *new_hash_table = group_by_hashtable_pools_[0]->getHashTableFast();
+        group_by_hashtable_pools_[0]->returnHashTable(new_hash_table);
+        hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
       }
       DCHECK(hash_tables->back() != nullptr);
       AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
@@ -494,21 +546,22 @@ void AggregationOperationState::finalizeHashTable(InsertDestination *output_dest
           agg_hash_table);
     }
 
-    auto *hash_tables = group_by_hashtable_pools_[agg_idx]->getAllHashTables();
+    auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
     DCHECK(hash_tables != nullptr);
     if (hash_tables->empty()) {
       // We may have a case where hash_tables is empty, e.g. no input blocks.
       // However for aggregateOnDistinctifyHashTableForGroupBy to work
       // correctly, we should create an empty group by hash table.
-      AggregationStateHashTableBase *new_hash_table = group_by_hashtable_pools_[agg_idx]->getHashTable();
-      group_by_hashtable_pools_[agg_idx]->returnHashTable(new_hash_table);
-      hash_tables = group_by_hashtable_pools_[agg_idx]->getAllHashTables();
+      AggregationStateHashTableBase *new_hash_table = group_by_hashtable_pools_[0]->getHashTable();
+      group_by_hashtable_pools_[0]->returnHashTable(new_hash_table);
+      hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();
     }
     AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
     DCHECK(agg_hash_table != nullptr);
     ColumnVector* agg_result_col =
         handles_[agg_idx]->finalizeHashTable(*agg_hash_table,
-                                             &group_by_keys);
+                                             &group_by_keys,
+                                              agg_idx);
     if (agg_result_col != nullptr) {
       final_values.emplace_back(agg_result_col);
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index ecd116b..d408c22 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -165,6 +165,8 @@ class AggregationOperationState {
    **/
   void finalizeAggregate(InsertDestination *output_destination);
 
+  int dflag;
+
  private:
   // Merge locally (per storage block) aggregated states with global aggregation
   // states.
@@ -185,7 +187,8 @@ class AggregationOperationState {
 
   // Each individual aggregate in this operation has an AggregationHandle and
   // some number of Scalar arguments.
-  std::vector<std::unique_ptr<AggregationHandle>> handles_;
+//  std::vector<std::unique_ptr<AggregationHandle>> handles_;
+  std::vector<AggregationHandle *> handles_;
   std::vector<std::vector<std::unique_ptr<const Scalar>>> arguments_;
 
   // For each aggregate, whether DISTINCT should be applied to the aggregate's
@@ -215,6 +218,8 @@ class AggregationOperationState {
 
   StorageManager *storage_manager_;
 
+  void mergeGroupByHashTables(AggregationStateHashTableBase *src, AggregationStateHashTableBase *dst);
+
   DISALLOW_COPY_AND_ASSIGN(AggregationOperationState);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 65a7975..b6f2ef9 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -198,6 +198,9 @@ if (ENABLE_DISTRIBUTED)
 endif()
 
 add_library(quickstep_storage_EvictionPolicy EvictionPolicy.cpp EvictionPolicy.hpp)
+add_library(quickstep_storage_FastHashTable ../empty_src.cpp FastHashTable.hpp)
+add_library(quickstep_storage_FastHashTableFactory ../empty_src.cpp FastHashTableFactory.hpp)
+add_library(quickstep_storage_FastSeparateChainingHashTable ../empty_src.cpp FastSeparateChainingHashTable.hpp)
 add_library(quickstep_storage_FileManager ../empty_src.cpp FileManager.hpp)
 if (QUICKSTEP_HAVE_FILE_MANAGER_HDFS)
   add_library(quickstep_storage_FileManagerHdfs FileManagerHdfs.cpp FileManagerHdfs.hpp)
@@ -626,6 +629,55 @@ target_link_libraries(quickstep_storage_EvictionPolicy
                       quickstep_threading_SpinMutex
                       quickstep_threading_SpinSharedMutex
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_storage_FastHashTable
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationHandleAvg
+                      quickstep_storage_HashTable
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_StorageBlob
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_StorageManager
+                      quickstep_storage_TupleReference
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_threading_SpinMutex
+                      quickstep_threading_SpinSharedMutex
+                      quickstep_types_Type
+                      quickstep_types_TypedValue
+                      quickstep_utility_BloomFilter
+                      quickstep_utility_HashPair
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_storage_FastHashTableFactory
+                      glog
+                      quickstep_storage_FastHashTable
+                      quickstep_storage_FastSeparateChainingHashTable
+                      quickstep_storage_HashTable
+                      quickstep_storage_HashTable_proto
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableFactory
+                      quickstep_storage_LinearOpenAddressingHashTable
+                      quickstep_storage_SeparateChainingHashTable
+                      quickstep_storage_SimpleScalarSeparateChainingHashTable
+                      quickstep_storage_TupleReference
+                      quickstep_types_TypeFactory
+                      quickstep_utility_BloomFilter
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_storage_FastSeparateChainingHashTable
+                      quickstep_storage_FastHashTable
+                      quickstep_storage_HashTable
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableKeyManager
+                      quickstep_storage_StorageBlob
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_StorageManager
+                      quickstep_threading_SpinSharedMutex
+                      quickstep_types_Type
+                      quickstep_types_TypedValue
+                      quickstep_utility_Alignment
+                      quickstep_utility_Macros
+                      quickstep_utility_PrimeNumber)
 target_link_libraries(quickstep_storage_FileManager
                       quickstep_storage_StorageBlockInfo
                       quickstep_utility_Macros
@@ -711,6 +763,8 @@ target_link_libraries(quickstep_storage_HashTableKeyManager
 target_link_libraries(quickstep_storage_HashTablePool
                       glog
                       quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_FastHashTable
+                      quickstep_storage_FastHashTableFactory
                       quickstep_storage_HashTableBase
                       quickstep_threading_SpinMutex
                       quickstep_utility_Macros
@@ -915,6 +969,7 @@ target_link_libraries(quickstep_storage_StorageBlock
                       quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
                       quickstep_storage_CompressedPackedRowStoreTupleStorageSubBlock
                       quickstep_storage_CountedReference
+                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_IndexSubBlock
                       quickstep_storage_InsertDestinationInterface
@@ -1098,6 +1153,9 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_EvictionPolicy
                       quickstep_storage_FileManager
                       quickstep_storage_FileManagerLocal
+                      quickstep_storage_FastHashTable
+                      quickstep_storage_FastHashTableFactory
+                      quickstep_storage_FastSeparateChainingHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTable_proto
                       quickstep_storage_HashTableBase


[02/28] incubator-quickstep git commit: Initial commit for QUICKSTEP-28 and QUICKSTEP-29. Code refactoring and cleanup, some more optimizations are pending.

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 97b4773..8b59a3c 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -468,6 +468,14 @@ class StorageBlock : public StorageBlockBase {
                         std::vector<std::unique_ptr<ColumnVector>>
                             *reuse_group_by_vectors) const;
 
+
+  void aggregateGroupByFast(const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
+                        const std::vector<std::unique_ptr<const Scalar>> &group_by,
+                        const Predicate *predicate,
+                        AggregationStateHashTableBase *hash_table,
+                        std::unique_ptr<TupleIdSequence> *reuse_matches,
+                        std::vector<std::unique_ptr<ColumnVector>>
+                            *reuse_group_by_vectors) const;
   /**
    * @brief Inserts the GROUP BY expressions and aggregation arguments together
    *        as keys into the distinctify hash table.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/threading/SpinMutex.hpp
----------------------------------------------------------------------
diff --git a/threading/SpinMutex.hpp b/threading/SpinMutex.hpp
index 5ed1405..106ef13 100644
--- a/threading/SpinMutex.hpp
+++ b/threading/SpinMutex.hpp
@@ -44,6 +44,8 @@ class SpinMutex {
   SpinMutex() : locked_(false) {
   }
 
+  explicit SpinMutex(uint8_t *ptr): locked_(*ptr) {}
+
   /**
    * @note This call does NOT yield when contended. SpinMutex is intended
    *       mainly for cases where locks are held briefly and it is better to


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

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp b/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
index b7cf02a..026bd1d 100644
--- a/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
@@ -31,6 +31,8 @@
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "expressions/aggregation/AggregationHandleMax.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
+#include "storage/AggregationOperationState.hpp"
+#include "storage/FastHashTableFactory.hpp"
 #include "storage/HashTableBase.hpp"
 #include "storage/StorageManager.hpp"
 #include "types/CharType.hpp"
@@ -70,54 +72,59 @@ class AggregationHandleMaxTest : public ::testing::Test {
   // Helper method that calls AggregationHandleMax::iterateUnaryInl() to
   // aggregate 'value' into '*state'.
   void iterateHandle(AggregationState *state, const TypedValue &value) {
-    static_cast<const AggregationHandleMax&>(*aggregation_handle_max_).iterateUnaryInl(
-        static_cast<AggregationStateMax*>(state),
-        value);
+    static_cast<const AggregationHandleMax &>(*aggregation_handle_max_)
+        .iterateUnaryInl(static_cast<AggregationStateMax *>(state), value);
   }
 
   void initializeHandle(const Type &type) {
     aggregation_handle_max_.reset(
-        AggregateFunctionFactory::Get(AggregationID::kMax).createHandle(
-            std::vector<const Type*>(1, &type)));
+        AggregateFunctionFactory::Get(AggregationID::kMax)
+            .createHandle(std::vector<const Type *>(1, &type)));
     aggregation_handle_max_state_.reset(
         aggregation_handle_max_->createInitialState());
   }
 
   static bool ApplyToTypesTest(TypeID typeID) {
-    const Type &type = (typeID == kChar || typeID == kVarChar) ?
-        TypeFactory::GetType(typeID, static_cast<std::size_t>(10)) :
-        TypeFactory::GetType(typeID);
+    const Type &type =
+        (typeID == kChar || typeID == kVarChar)
+            ? TypeFactory::GetType(typeID, static_cast<std::size_t>(10))
+            : TypeFactory::GetType(typeID);
 
-    return AggregateFunctionFactory::Get(AggregationID::kMax).canApplyToTypes(
-        std::vector<const Type*>(1, &type));
+    return AggregateFunctionFactory::Get(AggregationID::kMax)
+        .canApplyToTypes(std::vector<const Type *>(1, &type));
   }
 
   static bool ResultTypeForArgumentTypeTest(TypeID input_type_id,
                                             TypeID output_type_id) {
-    const Type *result_type
-        = AggregateFunctionFactory::Get(AggregationID::kMax).resultTypeForArgumentTypes(
-            std::vector<const Type*>(1, &TypeFactory::GetType(input_type_id)));
+    const Type *result_type =
+        AggregateFunctionFactory::Get(AggregationID::kMax)
+            .resultTypeForArgumentTypes(std::vector<const Type *>(
+                1, &TypeFactory::GetType(input_type_id)));
     return (result_type->getTypeID() == output_type_id);
   }
 
   template <typename CppType>
-  static void CheckMaxValue(
-      CppType expected,
-      const AggregationHandle &handle,
-      const AggregationState &state) {
+  static void CheckMaxValue(CppType expected,
+                            const AggregationHandle &handle,
+                            const AggregationState &state) {
     EXPECT_EQ(expected, handle.finalize(state).getLiteral<CppType>());
   }
 
-  static void CheckMaxString(
-      const std::string &expected,
-      const AggregationHandle &handle,
-      const AggregationState &state) {
+  template <typename CppType>
+  static void CheckMaxValue(CppType expected, const TypedValue &value) {
+    EXPECT_EQ(expected, value.getLiteral<CppType>());
+  }
+
+  static void CheckMaxString(const std::string &expected,
+                             const AggregationHandle &handle,
+                             const AggregationState &state) {
     TypedValue value = handle.finalize(state);
 
     ASSERT_EQ(expected.length(), value.getAsciiStringLength());
-    EXPECT_EQ(0, std::strncmp(expected.c_str(),
-                              static_cast<const char*>(value.getDataPtr()),
-                              value.getAsciiStringLength()));
+    EXPECT_EQ(0,
+              std::strncmp(expected.c_str(),
+                           static_cast<const char *>(value.getDataPtr()),
+                           value.getAsciiStringLength()));
   }
 
   // Static templated method to initialize data types.
@@ -130,7 +137,9 @@ class AggregationHandleMaxTest : public ::testing::Test {
   void checkAggregationMaxGeneric() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_max_->finalize(*aggregation_handle_max_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_max_->finalize(*aggregation_handle_max_state_)
+            .isNull());
 
     typename GenericType::cpptype val;
     typename GenericType::cpptype max;
@@ -142,16 +151,18 @@ class AggregationHandleMaxTest : public ::testing::Test {
         if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
           SetDataType(i * kNumSamples + j - 10, &val);
         } else {
-          SetDataType(static_cast<float>(i * kNumSamples + j - 10)/10, &val);
+          SetDataType(static_cast<float>(i * kNumSamples + j - 10) / 10, &val);
         }
-        iterateHandle(aggregation_handle_max_state_.get(), type.makeValue(&val));
+        iterateHandle(aggregation_handle_max_state_.get(),
+                      type.makeValue(&val));
         if (max < val) {
           max = val;
         }
       }
     }
     iterateHandle(aggregation_handle_max_state_.get(), type.makeNullValue());
-    CheckMaxValue<typename GenericType::cpptype>(max, *aggregation_handle_max_, *aggregation_handle_max_state_);
+    CheckMaxValue<typename GenericType::cpptype>(
+        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
 
     // Test mergeStates().
     std::unique_ptr<AggregationState> merge_state(
@@ -165,7 +176,7 @@ class AggregationHandleMaxTest : public ::testing::Test {
         if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
           SetDataType(i * kNumSamples + j - 20, &val);
         } else {
-          SetDataType(static_cast<float>(i * kNumSamples + j - 20)/10, &val);
+          SetDataType(static_cast<float>(i * kNumSamples + j - 20) / 10, &val);
         }
         iterateHandle(merge_state.get(), type.makeValue(&val));
         if (max < val) {
@@ -176,14 +187,14 @@ class AggregationHandleMaxTest : public ::testing::Test {
     aggregation_handle_max_->mergeStates(*merge_state,
                                          aggregation_handle_max_state_.get());
     CheckMaxValue<typename GenericType::cpptype>(
-        max,
-        *aggregation_handle_max_,
-        *aggregation_handle_max_state_);
+        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
   }
 
   template <typename GenericType>
-  ColumnVector *createColumnVectorGeneric(const Type &type, typename GenericType::cpptype *max) {
-    NativeColumnVector *column = new NativeColumnVector(type, kIterations * kNumSamples + 3);
+  ColumnVector* createColumnVectorGeneric(const Type &type,
+                                          typename GenericType::cpptype *max) {
+    NativeColumnVector *column =
+        new NativeColumnVector(type, kIterations * kNumSamples + 3);
 
     typename GenericType::cpptype val;
     SetDataType(0, max);
@@ -194,7 +205,7 @@ class AggregationHandleMaxTest : public ::testing::Test {
         if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
           SetDataType(i * kNumSamples + j - 10, &val);
         } else {
-          SetDataType(static_cast<float>(i * kNumSamples + j - 10)/10, &val);
+          SetDataType(static_cast<float>(i * kNumSamples + j - 10) / 10, &val);
         }
         column->appendTypedValue(type.makeValue(&val));
         if (*max < val) {
@@ -202,7 +213,7 @@ class AggregationHandleMaxTest : public ::testing::Test {
         }
       }
       // One NULL in the middle.
-      if (i == kIterations/2) {
+      if (i == kIterations / 2) {
         column->appendTypedValue(type.makeNullValue());
       }
     }
@@ -215,11 +226,14 @@ class AggregationHandleMaxTest : public ::testing::Test {
   void checkAggregationMaxGenericColumnVector() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_max_->finalize(*aggregation_handle_max_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_max_->finalize(*aggregation_handle_max_state_)
+            .isNull());
 
     typename GenericType::cpptype max;
     std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(createColumnVectorGeneric<GenericType>(type, &max));
+    column_vectors.emplace_back(
+        createColumnVectorGeneric<GenericType>(type, &max));
 
     std::unique_ptr<AggregationState> cv_state(
         aggregation_handle_max_->accumulateColumnVectors(column_vectors));
@@ -227,15 +241,12 @@ class AggregationHandleMaxTest : public ::testing::Test {
     // Test the state generated directly by accumulateColumnVectors(), and also
     // test after merging back.
     CheckMaxValue<typename GenericType::cpptype>(
-        max,
-        *aggregation_handle_max_,
-        *cv_state);
+        max, *aggregation_handle_max_, *cv_state);
 
-    aggregation_handle_max_->mergeStates(*cv_state, aggregation_handle_max_state_.get());
+    aggregation_handle_max_->mergeStates(*cv_state,
+                                         aggregation_handle_max_state_.get());
     CheckMaxValue<typename GenericType::cpptype>(
-        max,
-        *aggregation_handle_max_,
-        *aggregation_handle_max_state_);
+        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
   }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -243,29 +254,29 @@ class AggregationHandleMaxTest : public ::testing::Test {
   void checkAggregationMaxGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_max_->finalize(*aggregation_handle_max_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_max_->finalize(*aggregation_handle_max_state_)
+            .isNull());
 
-    std::unique_ptr<ColumnVectorsValueAccessor> accessor(new ColumnVectorsValueAccessor());
+    std::unique_ptr<ColumnVectorsValueAccessor> accessor(
+        new ColumnVectorsValueAccessor());
 
     typename GenericType::cpptype max;
     accessor->addColumn(createColumnVectorGeneric<GenericType>(type, &max));
 
     std::unique_ptr<AggregationState> va_state(
-        aggregation_handle_max_->accumulateValueAccessor(accessor.get(),
-                                                         std::vector<attribute_id>(1, 0)));
+        aggregation_handle_max_->accumulateValueAccessor(
+            accessor.get(), std::vector<attribute_id>(1, 0)));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
     CheckMaxValue<typename GenericType::cpptype>(
-        max,
-        *aggregation_handle_max_,
-        *va_state);
+        max, *aggregation_handle_max_, *va_state);
 
-    aggregation_handle_max_->mergeStates(*va_state, aggregation_handle_max_state_.get());
+    aggregation_handle_max_->mergeStates(*va_state,
+                                         aggregation_handle_max_state_.get());
     CheckMaxValue<typename GenericType::cpptype>(
-        max,
-        *aggregation_handle_max_,
-        *aggregation_handle_max_state_);
+        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
   }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
@@ -273,11 +284,14 @@ class AggregationHandleMaxTest : public ::testing::Test {
   void checkAggregationMaxString() {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_max_->finalize(*aggregation_handle_max_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_max_->finalize(*aggregation_handle_max_state_)
+            .isNull());
 
     std::unique_ptr<UncheckedComparator> fast_comparator_;
-    fast_comparator_.reset(ComparisonFactory::GetComparison(ComparisonID::kGreater)
-                           .makeUncheckedComparatorForTypes(type, type));
+    fast_comparator_.reset(
+        ComparisonFactory::GetComparison(ComparisonID::kGreater)
+            .makeUncheckedComparatorForTypes(type, type));
     std::string string_literal;
     std::string max = "";
     int val;
@@ -291,15 +305,17 @@ class AggregationHandleMaxTest : public ::testing::Test {
 
         iterateHandle(
             aggregation_handle_max_state_.get(),
-            type.makeValue(string_literal.c_str(),
-                           string_literal.length() + 1).ensureNotReference());
-        if (fast_comparator_->compareDataPtrs(string_literal.c_str(), max.c_str())) {
+            type.makeValue(string_literal.c_str(), string_literal.length() + 1)
+                .ensureNotReference());
+        if (fast_comparator_->compareDataPtrs(string_literal.c_str(),
+                                              max.c_str())) {
           max = string_literal;
         }
       }
     }
     iterateHandle(aggregation_handle_max_state_.get(), type.makeNullValue());
-    CheckMaxString(max, *aggregation_handle_max_, *aggregation_handle_max_state_);
+    CheckMaxString(
+        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
 
     // Test mergeStates().
     std::unique_ptr<AggregationState> merge_state(
@@ -317,24 +333,28 @@ class AggregationHandleMaxTest : public ::testing::Test {
 
         iterateHandle(
             merge_state.get(),
-            type.makeValue(string_literal.c_str(),
-                           string_literal.length() + 1).ensureNotReference());
-        if (fast_comparator_->compareDataPtrs(string_literal.c_str(), max.c_str())) {
+            type.makeValue(string_literal.c_str(), string_literal.length() + 1)
+                .ensureNotReference());
+        if (fast_comparator_->compareDataPtrs(string_literal.c_str(),
+                                              max.c_str())) {
           max = string_literal;
         }
       }
     }
     aggregation_handle_max_->mergeStates(*merge_state,
                                          aggregation_handle_max_state_.get());
-    CheckMaxString(max, *aggregation_handle_max_, *aggregation_handle_max_state_);
+    CheckMaxString(
+        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
   }
 
   template <typename ColumnVectorType>
-  ColumnVector *createColumnVectorString(const Type &type, std::string *max) {
-    ColumnVectorType *column = new ColumnVectorType(type, kIterations * kNumSamples + 3);
+  ColumnVector* createColumnVectorString(const Type &type, std::string *max) {
+    ColumnVectorType *column =
+        new ColumnVectorType(type, kIterations * kNumSamples + 3);
     std::unique_ptr<UncheckedComparator> fast_comparator_;
-    fast_comparator_.reset(ComparisonFactory::GetComparison(ComparisonID::kGreater)
-                           .makeUncheckedComparatorForTypes(type, type));
+    fast_comparator_.reset(
+        ComparisonFactory::GetComparison(ComparisonID::kGreater)
+            .makeUncheckedComparatorForTypes(type, type));
     std::string string_literal;
     *max = "";
     int val;
@@ -346,14 +366,16 @@ class AggregationHandleMaxTest : public ::testing::Test {
         oss << "max" << val;
         string_literal = oss.str();
 
-        column->appendTypedValue(type.makeValue(string_literal.c_str(), string_literal.length() + 1)
-            .ensureNotReference());
-        if (fast_comparator_->compareDataPtrs(string_literal.c_str(), max->c_str())) {
+        column->appendTypedValue(
+            type.makeValue(string_literal.c_str(), string_literal.length() + 1)
+                .ensureNotReference());
+        if (fast_comparator_->compareDataPtrs(string_literal.c_str(),
+                                              max->c_str())) {
           *max = string_literal;
         }
       }
       // One NULL in the middle.
-      if (i == kIterations/2) {
+      if (i == kIterations / 2) {
         column->appendTypedValue(type.makeNullValue());
       }
     }
@@ -366,25 +388,26 @@ class AggregationHandleMaxTest : public ::testing::Test {
   void checkAggregationMaxStringColumnVector() {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_max_->finalize(*aggregation_handle_max_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_max_->finalize(*aggregation_handle_max_state_)
+            .isNull());
 
     std::string max;
     std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(createColumnVectorString<ColumnVectorType>(type, &max));
+    column_vectors.emplace_back(
+        createColumnVectorString<ColumnVectorType>(type, &max));
 
     std::unique_ptr<AggregationState> cv_state(
         aggregation_handle_max_->accumulateColumnVectors(column_vectors));
 
     // Test the state generated directly by accumulateColumnVectors(), and also
     // test after merging back.
-    CheckMaxString(max,
-                   *aggregation_handle_max_,
-                   *cv_state);
-
-    aggregation_handle_max_->mergeStates(*cv_state, aggregation_handle_max_state_.get());
-    CheckMaxString(max,
-                   *aggregation_handle_max_,
-                   *aggregation_handle_max_state_);
+    CheckMaxString(max, *aggregation_handle_max_, *cv_state);
+
+    aggregation_handle_max_->mergeStates(*cv_state,
+                                         aggregation_handle_max_state_.get());
+    CheckMaxString(
+        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
   }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -392,26 +415,27 @@ class AggregationHandleMaxTest : public ::testing::Test {
   void checkAggregationMaxStringValueAccessor() {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_max_->finalize(*aggregation_handle_max_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_max_->finalize(*aggregation_handle_max_state_)
+            .isNull());
 
     std::string max;
-    std::unique_ptr<ColumnVectorsValueAccessor> accessor(new ColumnVectorsValueAccessor());
+    std::unique_ptr<ColumnVectorsValueAccessor> accessor(
+        new ColumnVectorsValueAccessor());
     accessor->addColumn(createColumnVectorString<ColumnVectorType>(type, &max));
 
     std::unique_ptr<AggregationState> va_state(
-        aggregation_handle_max_->accumulateValueAccessor(accessor.get(),
-                                                         std::vector<attribute_id>(1, 0)));
+        aggregation_handle_max_->accumulateValueAccessor(
+            accessor.get(), std::vector<attribute_id>(1, 0)));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
-    CheckMaxString(max,
-                   *aggregation_handle_max_,
-                   *va_state);
-
-    aggregation_handle_max_->mergeStates(*va_state, aggregation_handle_max_state_.get());
-    CheckMaxString(max,
-                   *aggregation_handle_max_,
-                   *aggregation_handle_max_state_);
+    CheckMaxString(max, *aggregation_handle_max_, *va_state);
+
+    aggregation_handle_max_->mergeStates(*va_state,
+                                         aggregation_handle_max_state_.get());
+    CheckMaxString(
+        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
   }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
@@ -422,9 +446,7 @@ class AggregationHandleMaxTest : public ::testing::Test {
 
 template <>
 void AggregationHandleMaxTest::CheckMaxValue<float>(
-    float val,
-    const AggregationHandle &handle,
-    const AggregationState &state) {
+    float val, const AggregationHandle &handle, const AggregationState &state) {
   EXPECT_FLOAT_EQ(val, handle.finalize(state).getLiteral<float>());
 }
 
@@ -437,17 +459,20 @@ void AggregationHandleMaxTest::CheckMaxValue<double>(
 }
 
 template <>
-void AggregationHandleMaxTest::SetDataType<DatetimeLit>(int value, DatetimeLit *data) {
+void AggregationHandleMaxTest::SetDataType<DatetimeLit>(int value,
+                                                        DatetimeLit *data) {
   data->ticks = value;
 }
 
 template <>
-void AggregationHandleMaxTest::SetDataType<DatetimeIntervalLit>(int value, DatetimeIntervalLit *data) {
+void AggregationHandleMaxTest::SetDataType<DatetimeIntervalLit>(
+    int value, DatetimeIntervalLit *data) {
   data->interval_ticks = value;
 }
 
 template <>
-void AggregationHandleMaxTest::SetDataType<YearMonthIntervalLit>(int value, YearMonthIntervalLit *data) {
+void AggregationHandleMaxTest::SetDataType<YearMonthIntervalLit>(
+    int value, YearMonthIntervalLit *data) {
   data->months = value;
 }
 
@@ -579,50 +604,67 @@ TEST_F(AggregationHandleMaxDeathTest, WrongTypeTest) {
   float float_val = 0;
 
   // Passes.
-  iterateHandle(aggregation_handle_max_state_.get(), int_non_null_type.makeValue(&int_val));
+  iterateHandle(aggregation_handle_max_state_.get(),
+                int_non_null_type.makeValue(&int_val));
 
-  EXPECT_DEATH(iterateHandle(aggregation_handle_max_state_.get(), long_type.makeValue(&long_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_max_state_.get(), double_type.makeValue(&double_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_max_state_.get(), float_type.makeValue(&float_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_max_state_.get(), char_type.makeValue("asdf", 5)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_max_state_.get(), varchar_type.makeValue("asdf", 5)), "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_max_state_.get(),
+                             long_type.makeValue(&long_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_max_state_.get(),
+                             double_type.makeValue(&double_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_max_state_.get(),
+                             float_type.makeValue(&float_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_max_state_.get(),
+                             char_type.makeValue("asdf", 5)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_max_state_.get(),
+                             varchar_type.makeValue("asdf", 5)),
+               "");
 
   // Test mergeStates() with incorrectly typed handles.
   std::unique_ptr<AggregationHandle> aggregation_handle_max_long(
-      AggregateFunctionFactory::Get(AggregationID::kMax).createHandle(
-          std::vector<const Type*>(1, &long_type)));
+      AggregateFunctionFactory::Get(AggregationID::kMax)
+          .createHandle(std::vector<const Type *>(1, &long_type)));
   std::unique_ptr<AggregationState> aggregation_state_max_merge_long(
       aggregation_handle_max_long->createInitialState());
-  static_cast<const AggregationHandleMax&>(*aggregation_handle_max_long).iterateUnaryInl(
-      static_cast<AggregationStateMax*>(aggregation_state_max_merge_long.get()),
-      long_type.makeValue(&long_val));
-  EXPECT_DEATH(aggregation_handle_max_->mergeStates(*aggregation_state_max_merge_long,
-                                                    aggregation_handle_max_state_.get()),
-               "");
+  static_cast<const AggregationHandleMax &>(*aggregation_handle_max_long)
+      .iterateUnaryInl(static_cast<AggregationStateMax *>(
+                           aggregation_state_max_merge_long.get()),
+                       long_type.makeValue(&long_val));
+  EXPECT_DEATH(
+      aggregation_handle_max_->mergeStates(*aggregation_state_max_merge_long,
+                                           aggregation_handle_max_state_.get()),
+      "");
 
   std::unique_ptr<AggregationHandle> aggregation_handle_max_double(
-      AggregateFunctionFactory::Get(AggregationID::kMax).createHandle(
-          std::vector<const Type*>(1, &double_type)));
+      AggregateFunctionFactory::Get(AggregationID::kMax)
+          .createHandle(std::vector<const Type *>(1, &double_type)));
   std::unique_ptr<AggregationState> aggregation_state_max_merge_double(
       aggregation_handle_max_double->createInitialState());
-  static_cast<const AggregationHandleMax&>(*aggregation_handle_max_double).iterateUnaryInl(
-      static_cast<AggregationStateMax*>(aggregation_state_max_merge_double.get()),
-      double_type.makeValue(&double_val));
-  EXPECT_DEATH(aggregation_handle_max_->mergeStates(*aggregation_state_max_merge_double,
-                                                    aggregation_handle_max_state_.get()),
-               "");
+  static_cast<const AggregationHandleMax &>(*aggregation_handle_max_double)
+      .iterateUnaryInl(static_cast<AggregationStateMax *>(
+                           aggregation_state_max_merge_double.get()),
+                       double_type.makeValue(&double_val));
+  EXPECT_DEATH(
+      aggregation_handle_max_->mergeStates(*aggregation_state_max_merge_double,
+                                           aggregation_handle_max_state_.get()),
+      "");
 
   std::unique_ptr<AggregationHandle> aggregation_handle_max_float(
-      AggregateFunctionFactory::Get(AggregationID::kMax).createHandle(
-          std::vector<const Type*>(1, &float_type)));
+      AggregateFunctionFactory::Get(AggregationID::kMax)
+          .createHandle(std::vector<const Type *>(1, &float_type)));
   std::unique_ptr<AggregationState> aggregation_state_max_merge_float(
       aggregation_handle_max_float->createInitialState());
-  static_cast<const AggregationHandleMax&>(*aggregation_handle_max_float).iterateUnaryInl(
-      static_cast<AggregationStateMax*>(aggregation_state_max_merge_float.get()),
-      float_type.makeValue(&float_val));
-  EXPECT_DEATH(aggregation_handle_max_->mergeStates(*aggregation_state_max_merge_float,
-                                                    aggregation_handle_max_state_.get()),
-               "");
+  static_cast<const AggregationHandleMax &>(*aggregation_handle_max_float)
+      .iterateUnaryInl(static_cast<AggregationStateMax *>(
+                           aggregation_state_max_merge_float.get()),
+                       float_type.makeValue(&float_val));
+  EXPECT_DEATH(
+      aggregation_handle_max_->mergeStates(*aggregation_state_max_merge_float,
+                                           aggregation_handle_max_state_.get()),
+      "");
 }
 #endif
 
@@ -647,25 +689,28 @@ TEST_F(AggregationHandleMaxTest, GroupByTableMergeTest) {
   initializeHandle(int_non_null_type);
   storage_manager_.reset(new StorageManager("./test_max_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      aggregation_handle_max_->createGroupByHashTable(
-          HashTableImplType::kSimpleScalarSeparateChaining,
+      AggregationStateFastHashTableFactory::CreateResizable(
+          HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
+          {aggregation_handle_max_.get()->getPayloadSize()},
+          {aggregation_handle_max_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      aggregation_handle_max_->createGroupByHashTable(
-          HashTableImplType::kSimpleScalarSeparateChaining,
+      AggregationStateFastHashTableFactory::CreateResizable(
+          HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
+          {aggregation_handle_max_.get()->getPayloadSize()},
+          {aggregation_handle_max_.get()},
           storage_manager_.get()));
 
-  AggregationStateHashTable<AggregationStateMax> *destination_hash_table_derived =
-      static_cast<AggregationStateHashTable<AggregationStateMax> *>(
+  AggregationStateFastHashTable *destination_hash_table_derived =
+      static_cast<AggregationStateFastHashTable *>(
           destination_hash_table.get());
 
-  AggregationStateHashTable<AggregationStateMax> *source_hash_table_derived =
-      static_cast<AggregationStateHashTable<AggregationStateMax> *>(
-          source_hash_table.get());
+  AggregationStateFastHashTable *source_hash_table_derived =
+      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
 
   AggregationHandleMax *aggregation_handle_max_derived =
       static_cast<AggregationHandleMax *>(aggregation_handle_max_.get());
@@ -730,35 +775,52 @@ TEST_F(AggregationHandleMaxTest, GroupByTableMergeTest) {
   EXPECT_EQ(exclusive_key_source_max_val.getLiteral<int>(), actual_val);
 
   // Add the key-state pairs to the hash tables.
-  source_hash_table_derived->putCompositeKey(common_key,
-                                             *common_key_source_state);
-  destination_hash_table_derived->putCompositeKey(
-      common_key, *common_key_destination_state);
-  source_hash_table_derived->putCompositeKey(exclusive_source_key,
-                                             *exclusive_key_source_state);
-  destination_hash_table_derived->putCompositeKey(
-      exclusive_destination_key, *exclusive_key_destination_state);
+  unsigned char buffer[100];
+  buffer[0] = '\0';
+  memcpy(buffer + 1,
+         common_key_source_state.get()->getPayloadAddress(),
+         aggregation_handle_max_.get()->getPayloadSize());
+  source_hash_table_derived->putCompositeKey(common_key, buffer);
+
+  memcpy(buffer + 1,
+         common_key_destination_state.get()->getPayloadAddress(),
+         aggregation_handle_max_.get()->getPayloadSize());
+  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+
+  memcpy(buffer + 1,
+         exclusive_key_source_state.get()->getPayloadAddress(),
+         aggregation_handle_max_.get()->getPayloadSize());
+  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+
+  memcpy(buffer + 1,
+         exclusive_key_destination_state.get()->getPayloadAddress(),
+         aggregation_handle_max_.get()->getPayloadSize());
+  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
+                                                      buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  aggregation_handle_max_->mergeGroupByHashTables(*source_hash_table,
-                                                  destination_hash_table.get());
+  AggregationOperationState::mergeGroupByHashTables(
+      source_hash_table.get(), destination_hash_table.get());
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckMaxValue<int>(
       common_key_destination_max_val.getLiteral<int>(),
-      *aggregation_handle_max_derived,
-      *(destination_hash_table_derived->getSingleCompositeKey(common_key)));
+      aggregation_handle_max_derived->finalizeHashTableEntryFast(
+          destination_hash_table_derived->getSingleCompositeKey(common_key) +
+          1));
   CheckMaxValue<int>(exclusive_key_destination_max_val.getLiteral<int>(),
-                     *aggregation_handle_max_derived,
-                     *(destination_hash_table_derived->getSingleCompositeKey(
-                         exclusive_destination_key)));
+                     aggregation_handle_max_derived->finalizeHashTableEntryFast(
+                         destination_hash_table_derived->getSingleCompositeKey(
+                             exclusive_destination_key) +
+                         1));
   CheckMaxValue<int>(exclusive_key_source_max_val.getLiteral<int>(),
-                     *aggregation_handle_max_derived,
-                     *(source_hash_table_derived->getSingleCompositeKey(
-                         exclusive_source_key)));
+                     aggregation_handle_max_derived->finalizeHashTableEntryFast(
+                         source_hash_table_derived->getSingleCompositeKey(
+                             exclusive_source_key) +
+                         1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c123bd49/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp b/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
index 85c3bf3..6e6d188 100644
--- a/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
@@ -31,6 +31,8 @@
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "expressions/aggregation/AggregationHandleMin.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
+#include "storage/AggregationOperationState.hpp"
+#include "storage/FastHashTableFactory.hpp"
 #include "storage/StorageManager.hpp"
 #include "types/CharType.hpp"
 #include "types/DatetimeIntervalType.hpp"
@@ -69,54 +71,59 @@ class AggregationHandleMinTest : public ::testing::Test {
   // Helper method that calls AggregationHandleMin::iterateUnaryInl() to
   // aggregate 'value' into '*state'.
   void iterateHandle(AggregationState *state, const TypedValue &value) {
-    static_cast<const AggregationHandleMin&>(*aggregation_handle_min_).iterateUnaryInl(
-        static_cast<AggregationStateMin*>(state),
-        value);
+    static_cast<const AggregationHandleMin &>(*aggregation_handle_min_)
+        .iterateUnaryInl(static_cast<AggregationStateMin *>(state), value);
   }
 
   void initializeHandle(const Type &type) {
     aggregation_handle_min_.reset(
-        AggregateFunctionFactory::Get(AggregationID::kMin).createHandle(
-            std::vector<const Type*>(1, &type)));
+        AggregateFunctionFactory::Get(AggregationID::kMin)
+            .createHandle(std::vector<const Type *>(1, &type)));
     aggregation_handle_min_state_.reset(
         aggregation_handle_min_->createInitialState());
   }
 
   static bool ApplyToTypesTest(TypeID typeID) {
-    const Type &type = (typeID == kChar || typeID == kVarChar) ?
-        TypeFactory::GetType(typeID, static_cast<std::size_t>(10)) :
-        TypeFactory::GetType(typeID);
+    const Type &type =
+        (typeID == kChar || typeID == kVarChar)
+            ? TypeFactory::GetType(typeID, static_cast<std::size_t>(10))
+            : TypeFactory::GetType(typeID);
 
-    return AggregateFunctionFactory::Get(AggregationID::kMin).canApplyToTypes(
-        std::vector<const Type*>(1, &type));
+    return AggregateFunctionFactory::Get(AggregationID::kMin)
+        .canApplyToTypes(std::vector<const Type *>(1, &type));
   }
 
   static bool ResultTypeForArgumentTypeTest(TypeID input_type_id,
                                             TypeID output_type_id) {
-    const Type *result_type
-        = AggregateFunctionFactory::Get(AggregationID::kMin).resultTypeForArgumentTypes(
-            std::vector<const Type*>(1, &TypeFactory::GetType(input_type_id)));
+    const Type *result_type =
+        AggregateFunctionFactory::Get(AggregationID::kMin)
+            .resultTypeForArgumentTypes(std::vector<const Type *>(
+                1, &TypeFactory::GetType(input_type_id)));
     return (result_type->getTypeID() == output_type_id);
   }
 
   template <typename CppType>
-  static void CheckMinValue(
-      CppType expected,
-      const AggregationHandle &handle,
-      const AggregationState &state) {
+  static void CheckMinValue(CppType expected,
+                            const AggregationHandle &handle,
+                            const AggregationState &state) {
     EXPECT_EQ(expected, handle.finalize(state).getLiteral<CppType>());
   }
 
-  static void CheckMinString(
-      const std::string &expected,
-      const AggregationHandle &handle,
-      const AggregationState &state) {
+  template <typename CppType>
+  static void CheckMinValue(CppType expected, const TypedValue &value) {
+    EXPECT_EQ(expected, value.getLiteral<CppType>());
+  }
+
+  static void CheckMinString(const std::string &expected,
+                             const AggregationHandle &handle,
+                             const AggregationState &state) {
     TypedValue value = handle.finalize(state);
 
     ASSERT_EQ(expected.length(), value.getAsciiStringLength());
-    EXPECT_EQ(0, std::strncmp(expected.c_str(),
-                              static_cast <const char *>(value.getDataPtr()),
-                              value.getAsciiStringLength()));
+    EXPECT_EQ(0,
+              std::strncmp(expected.c_str(),
+                           static_cast<const char *>(value.getDataPtr()),
+                           value.getAsciiStringLength()));
   }
 
   // Static templated method to initialize data types.
@@ -129,7 +136,9 @@ class AggregationHandleMinTest : public ::testing::Test {
   void checkAggregationMinGeneric() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_min_->finalize(*aggregation_handle_min_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_min_->finalize(*aggregation_handle_min_state_)
+            .isNull());
 
     typename GenericType::cpptype val;
     typename GenericType::cpptype min;
@@ -141,16 +150,18 @@ class AggregationHandleMinTest : public ::testing::Test {
         if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
           SetDataType(i * kNumSamples + j - 10, &val);
         } else {
-          SetDataType(static_cast<float>(i * kNumSamples + j - 10)/10, &val);
+          SetDataType(static_cast<float>(i * kNumSamples + j - 10) / 10, &val);
         }
-        iterateHandle(aggregation_handle_min_state_.get(), type.makeValue(&val));
+        iterateHandle(aggregation_handle_min_state_.get(),
+                      type.makeValue(&val));
         if (min > val) {
           min = val;
         }
       }
     }
     iterateHandle(aggregation_handle_min_state_.get(), type.makeNullValue());
-    CheckMinValue<typename GenericType::cpptype>(min, *aggregation_handle_min_, *aggregation_handle_min_state_);
+    CheckMinValue<typename GenericType::cpptype>(
+        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
 
     // Test mergeStates().
     std::unique_ptr<AggregationState> merge_state(
@@ -164,7 +175,7 @@ class AggregationHandleMinTest : public ::testing::Test {
         if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
           SetDataType(i * kNumSamples + j - 20, &val);
         } else {
-          SetDataType(static_cast<float>(i * kNumSamples + j - 20)/10, &val);
+          SetDataType(static_cast<float>(i * kNumSamples + j - 20) / 10, &val);
         }
         iterateHandle(merge_state.get(), type.makeValue(&val));
         if (min > val) {
@@ -175,14 +186,14 @@ class AggregationHandleMinTest : public ::testing::Test {
     aggregation_handle_min_->mergeStates(*merge_state,
                                          aggregation_handle_min_state_.get());
     CheckMinValue<typename GenericType::cpptype>(
-        min,
-        *aggregation_handle_min_,
-        *aggregation_handle_min_state_);
+        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
   }
 
   template <typename GenericType>
-  ColumnVector *createColumnVectorGeneric(const Type &type, typename GenericType::cpptype *min) {
-    NativeColumnVector *column = new NativeColumnVector(type, kIterations * kNumSamples + 3);
+  ColumnVector* createColumnVectorGeneric(const Type &type,
+                                          typename GenericType::cpptype *min) {
+    NativeColumnVector *column =
+        new NativeColumnVector(type, kIterations * kNumSamples + 3);
 
     typename GenericType::cpptype val;
     SetDataType(1000, min);
@@ -193,7 +204,7 @@ class AggregationHandleMinTest : public ::testing::Test {
         if (type.getTypeID() == kInt || type.getTypeID() == kLong) {
           SetDataType(i * kNumSamples + j - 10, &val);
         } else {
-          SetDataType(static_cast<float>(i * kNumSamples + j - 10)/10, &val);
+          SetDataType(static_cast<float>(i * kNumSamples + j - 10) / 10, &val);
         }
         column->appendTypedValue(type.makeValue(&val));
         if (*min > val) {
@@ -201,7 +212,7 @@ class AggregationHandleMinTest : public ::testing::Test {
         }
       }
       // One NULL in the middle.
-      if (i == kIterations/2) {
+      if (i == kIterations / 2) {
         column->appendTypedValue(type.makeNullValue());
       }
     }
@@ -214,11 +225,14 @@ class AggregationHandleMinTest : public ::testing::Test {
   void checkAggregationMinGenericColumnVector() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_min_->finalize(*aggregation_handle_min_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_min_->finalize(*aggregation_handle_min_state_)
+            .isNull());
 
     typename GenericType::cpptype min;
     std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(createColumnVectorGeneric<GenericType>(type, &min));
+    column_vectors.emplace_back(
+        createColumnVectorGeneric<GenericType>(type, &min));
 
     std::unique_ptr<AggregationState> cv_state(
         aggregation_handle_min_->accumulateColumnVectors(column_vectors));
@@ -226,15 +240,12 @@ class AggregationHandleMinTest : public ::testing::Test {
     // Test the state generated directly by accumulateColumnVectors(), and also
     // test after merging back.
     CheckMinValue<typename GenericType::cpptype>(
-        min,
-        *aggregation_handle_min_,
-        *cv_state);
+        min, *aggregation_handle_min_, *cv_state);
 
-    aggregation_handle_min_->mergeStates(*cv_state, aggregation_handle_min_state_.get());
+    aggregation_handle_min_->mergeStates(*cv_state,
+                                         aggregation_handle_min_state_.get());
     CheckMinValue<typename GenericType::cpptype>(
-        min,
-        *aggregation_handle_min_,
-        *aggregation_handle_min_state_);
+        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
   }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -242,29 +253,29 @@ class AggregationHandleMinTest : public ::testing::Test {
   void checkAggregationMinGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_min_->finalize(*aggregation_handle_min_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_min_->finalize(*aggregation_handle_min_state_)
+            .isNull());
 
-    std::unique_ptr<ColumnVectorsValueAccessor> accessor(new ColumnVectorsValueAccessor());
+    std::unique_ptr<ColumnVectorsValueAccessor> accessor(
+        new ColumnVectorsValueAccessor());
 
     typename GenericType::cpptype min;
     accessor->addColumn(createColumnVectorGeneric<GenericType>(type, &min));
 
     std::unique_ptr<AggregationState> va_state(
-        aggregation_handle_min_->accumulateValueAccessor(accessor.get(),
-                                                         std::vector<attribute_id>(1, 0)));
+        aggregation_handle_min_->accumulateValueAccessor(
+            accessor.get(), std::vector<attribute_id>(1, 0)));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
     CheckMinValue<typename GenericType::cpptype>(
-        min,
-        *aggregation_handle_min_,
-        *va_state);
+        min, *aggregation_handle_min_, *va_state);
 
-    aggregation_handle_min_->mergeStates(*va_state, aggregation_handle_min_state_.get());
+    aggregation_handle_min_->mergeStates(*va_state,
+                                         aggregation_handle_min_state_.get());
     CheckMinValue<typename GenericType::cpptype>(
-        min,
-        *aggregation_handle_min_,
-        *aggregation_handle_min_state_);
+        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
   }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
@@ -272,11 +283,13 @@ class AggregationHandleMinTest : public ::testing::Test {
   void checkAggregationMinString() {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_min_->finalize(*aggregation_handle_min_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_min_->finalize(*aggregation_handle_min_state_)
+            .isNull());
 
     std::unique_ptr<UncheckedComparator> fast_comparator_;
     fast_comparator_.reset(ComparisonFactory::GetComparison(ComparisonID::kLess)
-                           .makeUncheckedComparatorForTypes(type, type));
+                               .makeUncheckedComparatorForTypes(type, type));
     std::string string_literal;
     std::string min = "z";
     int val;
@@ -288,15 +301,19 @@ class AggregationHandleMinTest : public ::testing::Test {
         oss << "test" << val;
         string_literal = oss.str();
 
-        iterateHandle(aggregation_handle_min_state_.get(), type.makeValue(string_literal.c_str(),
-                                                        string_literal.length() + 1).ensureNotReference());
-        if (fast_comparator_->compareDataPtrs(string_literal.c_str(), min.c_str())) {
+        iterateHandle(
+            aggregation_handle_min_state_.get(),
+            type.makeValue(string_literal.c_str(), string_literal.length() + 1)
+                .ensureNotReference());
+        if (fast_comparator_->compareDataPtrs(string_literal.c_str(),
+                                              min.c_str())) {
           min = string_literal;
         }
       }
     }
     iterateHandle(aggregation_handle_min_state_.get(), type.makeNullValue());
-    CheckMinString(min, *aggregation_handle_min_, *aggregation_handle_min_state_);
+    CheckMinString(
+        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
 
     // Test mergeStates().
     std::unique_ptr<AggregationState> merge_state(
@@ -314,24 +331,27 @@ class AggregationHandleMinTest : public ::testing::Test {
 
         iterateHandle(
             merge_state.get(),
-            type.makeValue(string_literal.c_str(),
-                           string_literal.length() + 1).ensureNotReference());
-        if (fast_comparator_->compareDataPtrs(string_literal.c_str(), min.c_str())) {
+            type.makeValue(string_literal.c_str(), string_literal.length() + 1)
+                .ensureNotReference());
+        if (fast_comparator_->compareDataPtrs(string_literal.c_str(),
+                                              min.c_str())) {
           min = string_literal;
         }
       }
     }
     aggregation_handle_min_->mergeStates(*merge_state,
                                          aggregation_handle_min_state_.get());
-    CheckMinString(min, *aggregation_handle_min_, *aggregation_handle_min_state_);
+    CheckMinString(
+        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
   }
 
   template <typename ColumnVectorType>
-  ColumnVector *createColumnVectorString(const Type &type, std::string *min) {
-    ColumnVectorType *column = new ColumnVectorType(type, kIterations * kNumSamples + 3);
+  ColumnVector* createColumnVectorString(const Type &type, std::string *min) {
+    ColumnVectorType *column =
+        new ColumnVectorType(type, kIterations * kNumSamples + 3);
     std::unique_ptr<UncheckedComparator> fast_comparator_;
     fast_comparator_.reset(ComparisonFactory::GetComparison(ComparisonID::kLess)
-                           .makeUncheckedComparatorForTypes(type, type));
+                               .makeUncheckedComparatorForTypes(type, type));
     std::string string_literal;
     *min = "z";
     int val;
@@ -343,14 +363,16 @@ class AggregationHandleMinTest : public ::testing::Test {
         oss << "test" << val;
         string_literal = oss.str();
 
-        column->appendTypedValue(type.makeValue(string_literal.c_str(), string_literal.length() + 1)
-            .ensureNotReference());
-        if (fast_comparator_->compareDataPtrs(string_literal.c_str(), min->c_str())) {
+        column->appendTypedValue(
+            type.makeValue(string_literal.c_str(), string_literal.length() + 1)
+                .ensureNotReference());
+        if (fast_comparator_->compareDataPtrs(string_literal.c_str(),
+                                              min->c_str())) {
           *min = string_literal;
         }
       }
       // One NULL in the middle.
-      if (i == kIterations/2) {
+      if (i == kIterations / 2) {
         column->appendTypedValue(type.makeNullValue());
       }
     }
@@ -363,25 +385,26 @@ class AggregationHandleMinTest : public ::testing::Test {
   void checkAggregationMinStringColumnVector() {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_min_->finalize(*aggregation_handle_min_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_min_->finalize(*aggregation_handle_min_state_)
+            .isNull());
 
     std::string min;
     std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(createColumnVectorString<ColumnVectorType>(type, &min));
+    column_vectors.emplace_back(
+        createColumnVectorString<ColumnVectorType>(type, &min));
 
     std::unique_ptr<AggregationState> cv_state(
         aggregation_handle_min_->accumulateColumnVectors(column_vectors));
 
     // Test the state generated directly by accumulateColumnVectors(), and also
     // test after merging back.
-    CheckMinString(min,
-                   *aggregation_handle_min_,
-                   *cv_state);
-
-    aggregation_handle_min_->mergeStates(*cv_state, aggregation_handle_min_state_.get());
-    CheckMinString(min,
-                   *aggregation_handle_min_,
-                   *aggregation_handle_min_state_);
+    CheckMinString(min, *aggregation_handle_min_, *cv_state);
+
+    aggregation_handle_min_->mergeStates(*cv_state,
+                                         aggregation_handle_min_state_.get());
+    CheckMinString(
+        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
   }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -389,26 +412,27 @@ class AggregationHandleMinTest : public ::testing::Test {
   void checkAggregationMinStringValueAccessor() {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(type);
-    EXPECT_TRUE(aggregation_handle_min_->finalize(*aggregation_handle_min_state_).isNull());
+    EXPECT_TRUE(
+        aggregation_handle_min_->finalize(*aggregation_handle_min_state_)
+            .isNull());
 
     std::string min;
-    std::unique_ptr<ColumnVectorsValueAccessor> accessor(new ColumnVectorsValueAccessor());
+    std::unique_ptr<ColumnVectorsValueAccessor> accessor(
+        new ColumnVectorsValueAccessor());
     accessor->addColumn(createColumnVectorString<ColumnVectorType>(type, &min));
 
     std::unique_ptr<AggregationState> va_state(
-        aggregation_handle_min_->accumulateValueAccessor(accessor.get(),
-                                                         std::vector<attribute_id>(1, 0)));
+        aggregation_handle_min_->accumulateValueAccessor(
+            accessor.get(), std::vector<attribute_id>(1, 0)));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
-    CheckMinString(min,
-                   *aggregation_handle_min_,
-                   *va_state);
-
-    aggregation_handle_min_->mergeStates(*va_state, aggregation_handle_min_state_.get());
-    CheckMinString(min,
-                   *aggregation_handle_min_,
-                   *aggregation_handle_min_state_);
+    CheckMinString(min, *aggregation_handle_min_, *va_state);
+
+    aggregation_handle_min_->mergeStates(*va_state,
+                                         aggregation_handle_min_state_.get());
+    CheckMinString(
+        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
   }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
@@ -419,9 +443,7 @@ class AggregationHandleMinTest : public ::testing::Test {
 
 template <>
 void AggregationHandleMinTest::CheckMinValue<float>(
-    float val,
-    const AggregationHandle &handle,
-    const AggregationState &state) {
+    float val, const AggregationHandle &handle, const AggregationState &state) {
   EXPECT_FLOAT_EQ(val, handle.finalize(state).getLiteral<float>());
 }
 
@@ -434,17 +456,20 @@ void AggregationHandleMinTest::CheckMinValue<double>(
 }
 
 template <>
-void AggregationHandleMinTest::SetDataType<DatetimeLit>(int value, DatetimeLit *data) {
+void AggregationHandleMinTest::SetDataType<DatetimeLit>(int value,
+                                                        DatetimeLit *data) {
   data->ticks = value;
 }
 
 template <>
-void AggregationHandleMinTest::SetDataType<DatetimeIntervalLit>(int value, DatetimeIntervalLit *data) {
+void AggregationHandleMinTest::SetDataType<DatetimeIntervalLit>(
+    int value, DatetimeIntervalLit *data) {
   data->interval_ticks = value;
 }
 
 template <>
-void AggregationHandleMinTest::SetDataType<YearMonthIntervalLit>(int value, YearMonthIntervalLit *data) {
+void AggregationHandleMinTest::SetDataType<YearMonthIntervalLit>(
+    int value, YearMonthIntervalLit *data) {
   data->months = value;
 }
 
@@ -575,50 +600,67 @@ TEST_F(AggregationHandleMinDeathTest, WrongTypeTest) {
   double double_val = 0;
   float float_val = 0;
 
-  iterateHandle(aggregation_handle_min_state_.get(), int_non_null_type.makeValue(&int_val));
+  iterateHandle(aggregation_handle_min_state_.get(),
+                int_non_null_type.makeValue(&int_val));
 
-  EXPECT_DEATH(iterateHandle(aggregation_handle_min_state_.get(), long_type.makeValue(&long_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_min_state_.get(), double_type.makeValue(&double_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_min_state_.get(), float_type.makeValue(&float_val)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_min_state_.get(), char_type.makeValue("asdf", 5)), "");
-  EXPECT_DEATH(iterateHandle(aggregation_handle_min_state_.get(), varchar_type.makeValue("asdf", 5)), "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_min_state_.get(),
+                             long_type.makeValue(&long_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_min_state_.get(),
+                             double_type.makeValue(&double_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_min_state_.get(),
+                             float_type.makeValue(&float_val)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_min_state_.get(),
+                             char_type.makeValue("asdf", 5)),
+               "");
+  EXPECT_DEATH(iterateHandle(aggregation_handle_min_state_.get(),
+                             varchar_type.makeValue("asdf", 5)),
+               "");
 
   // Test mergeStates() with incorrectly typed handles.
   std::unique_ptr<AggregationHandle> aggregation_handle_min_long(
-      AggregateFunctionFactory::Get(AggregationID::kMin).createHandle(
-          std::vector<const Type*>(1, &long_type)));
+      AggregateFunctionFactory::Get(AggregationID::kMin)
+          .createHandle(std::vector<const Type *>(1, &long_type)));
   std::unique_ptr<AggregationState> aggregation_state_min_merge_long(
       aggregation_handle_min_long->createInitialState());
-  static_cast<const AggregationHandleMin&>(*aggregation_handle_min_long).iterateUnaryInl(
-      static_cast<AggregationStateMin*>(aggregation_state_min_merge_long.get()),
-      long_type.makeValue(&long_val));
-  EXPECT_DEATH(aggregation_handle_min_->mergeStates(*aggregation_state_min_merge_long,
-                                                    aggregation_handle_min_state_.get()),
-               "");
+  static_cast<const AggregationHandleMin &>(*aggregation_handle_min_long)
+      .iterateUnaryInl(static_cast<AggregationStateMin *>(
+                           aggregation_state_min_merge_long.get()),
+                       long_type.makeValue(&long_val));
+  EXPECT_DEATH(
+      aggregation_handle_min_->mergeStates(*aggregation_state_min_merge_long,
+                                           aggregation_handle_min_state_.get()),
+      "");
 
   std::unique_ptr<AggregationHandle> aggregation_handle_min_double(
-      AggregateFunctionFactory::Get(AggregationID::kMin).createHandle(
-          std::vector<const Type*>(1, &double_type)));
+      AggregateFunctionFactory::Get(AggregationID::kMin)
+          .createHandle(std::vector<const Type *>(1, &double_type)));
   std::unique_ptr<AggregationState> aggregation_state_min_merge_double(
       aggregation_handle_min_double->createInitialState());
-  static_cast<const AggregationHandleMin&>(*aggregation_handle_min_double).iterateUnaryInl(
-      static_cast<AggregationStateMin*>(aggregation_state_min_merge_double.get()),
-      double_type.makeValue(&double_val));
-  EXPECT_DEATH(aggregation_handle_min_->mergeStates(*aggregation_state_min_merge_double,
-                                                    aggregation_handle_min_state_.get()),
-               "");
+  static_cast<const AggregationHandleMin &>(*aggregation_handle_min_double)
+      .iterateUnaryInl(static_cast<AggregationStateMin *>(
+                           aggregation_state_min_merge_double.get()),
+                       double_type.makeValue(&double_val));
+  EXPECT_DEATH(
+      aggregation_handle_min_->mergeStates(*aggregation_state_min_merge_double,
+                                           aggregation_handle_min_state_.get()),
+      "");
 
   std::unique_ptr<AggregationHandle> aggregation_handle_min_float(
-      AggregateFunctionFactory::Get(AggregationID::kMin).createHandle(
-          std::vector<const Type*>(1, &float_type)));
+      AggregateFunctionFactory::Get(AggregationID::kMin)
+          .createHandle(std::vector<const Type *>(1, &float_type)));
   std::unique_ptr<AggregationState> aggregation_state_min_merge_float(
       aggregation_handle_min_float->createInitialState());
-  static_cast<const AggregationHandleMin&>(*aggregation_handle_min_float).iterateUnaryInl(
-      static_cast<AggregationStateMin*>(aggregation_state_min_merge_float.get()),
-      float_type.makeValue(&float_val));
-  EXPECT_DEATH(aggregation_handle_min_->mergeStates(*aggregation_state_min_merge_float,
-                                                    aggregation_handle_min_state_.get()),
-               "");
+  static_cast<const AggregationHandleMin &>(*aggregation_handle_min_float)
+      .iterateUnaryInl(static_cast<AggregationStateMin *>(
+                           aggregation_state_min_merge_float.get()),
+                       float_type.makeValue(&float_val));
+  EXPECT_DEATH(
+      aggregation_handle_min_->mergeStates(*aggregation_state_min_merge_float,
+                                           aggregation_handle_min_state_.get()),
+      "");
 }
 #endif
 
@@ -643,25 +685,28 @@ TEST_F(AggregationHandleMinTest, GroupByTableMergeTest) {
   initializeHandle(int_non_null_type);
   storage_manager_.reset(new StorageManager("./test_min_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      aggregation_handle_min_->createGroupByHashTable(
-          HashTableImplType::kSimpleScalarSeparateChaining,
+      AggregationStateFastHashTableFactory::CreateResizable(
+          HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
+          {aggregation_handle_min_.get()->getPayloadSize()},
+          {aggregation_handle_min_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      aggregation_handle_min_->createGroupByHashTable(
-          HashTableImplType::kSimpleScalarSeparateChaining,
+      AggregationStateFastHashTableFactory::CreateResizable(
+          HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
+          {aggregation_handle_min_.get()->getPayloadSize()},
+          {aggregation_handle_min_.get()},
           storage_manager_.get()));
 
-  AggregationStateHashTable<AggregationStateMin> *destination_hash_table_derived =
-      static_cast<AggregationStateHashTable<AggregationStateMin> *>(
+  AggregationStateFastHashTable *destination_hash_table_derived =
+      static_cast<AggregationStateFastHashTable *>(
           destination_hash_table.get());
 
-  AggregationStateHashTable<AggregationStateMin> *source_hash_table_derived =
-      static_cast<AggregationStateHashTable<AggregationStateMin> *>(
-          source_hash_table.get());
+  AggregationStateFastHashTable *source_hash_table_derived =
+      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
 
   AggregationHandleMin *aggregation_handle_min_derived =
       static_cast<AggregationHandleMin *>(aggregation_handle_min_.get());
@@ -726,35 +771,52 @@ TEST_F(AggregationHandleMinTest, GroupByTableMergeTest) {
   EXPECT_EQ(exclusive_key_source_min_val.getLiteral<int>(), actual_val);
 
   // Add the key-state pairs to the hash tables.
-  source_hash_table_derived->putCompositeKey(common_key,
-                                             *common_key_source_state);
-  destination_hash_table_derived->putCompositeKey(
-      common_key, *common_key_destination_state);
-  source_hash_table_derived->putCompositeKey(exclusive_source_key,
-                                             *exclusive_key_source_state);
-  destination_hash_table_derived->putCompositeKey(
-      exclusive_destination_key, *exclusive_key_destination_state);
+  unsigned char buffer[100];
+  buffer[0] = '\0';
+  memcpy(buffer + 1,
+         common_key_source_state.get()->getPayloadAddress(),
+         aggregation_handle_min_.get()->getPayloadSize());
+  source_hash_table_derived->putCompositeKey(common_key, buffer);
+
+  memcpy(buffer + 1,
+         common_key_destination_state.get()->getPayloadAddress(),
+         aggregation_handle_min_.get()->getPayloadSize());
+  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+
+  memcpy(buffer + 1,
+         exclusive_key_source_state.get()->getPayloadAddress(),
+         aggregation_handle_min_.get()->getPayloadSize());
+  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+
+  memcpy(buffer + 1,
+         exclusive_key_destination_state.get()->getPayloadAddress(),
+         aggregation_handle_min_.get()->getPayloadSize());
+  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
+                                                      buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  aggregation_handle_min_->mergeGroupByHashTables(*source_hash_table,
-                                                  destination_hash_table.get());
+  AggregationOperationState::mergeGroupByHashTables(
+      source_hash_table.get(), destination_hash_table.get());
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckMinValue<int>(
       common_key_source_min_val.getLiteral<int>(),
-      *aggregation_handle_min_derived,
-      *(destination_hash_table_derived->getSingleCompositeKey(common_key)));
+      aggregation_handle_min_derived->finalizeHashTableEntryFast(
+          destination_hash_table_derived->getSingleCompositeKey(common_key) +
+          1));
   CheckMinValue<int>(exclusive_key_destination_min_val.getLiteral<int>(),
-                     *aggregation_handle_min_derived,
-                     *(destination_hash_table_derived->getSingleCompositeKey(
-                         exclusive_destination_key)));
+                     aggregation_handle_min_derived->finalizeHashTableEntryFast(
+                         destination_hash_table_derived->getSingleCompositeKey(
+                             exclusive_destination_key) +
+                         1));
   CheckMinValue<int>(exclusive_key_source_min_val.getLiteral<int>(),
-                     *aggregation_handle_min_derived,
-                     *(source_hash_table_derived->getSingleCompositeKey(
-                         exclusive_source_key)));
+                     aggregation_handle_min_derived->finalizeHashTableEntryFast(
+                         source_hash_table_derived->getSingleCompositeKey(
+                             exclusive_source_key) +
+                         1));
 }
 
 }  // namespace quickstep


[05/28] incubator-quickstep git commit: Initial commit for QUICKSTEP-28 and QUICKSTEP-29. Code refactoring and cleanup, some more optimizations are pending.

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d0756e7e/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
new file mode 100644
index 0000000..12e447f
--- /dev/null
+++ b/storage/FastHashTable.hpp
@@ -0,0 +1,2640 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015-2016 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_STORAGE_FAST_HASH_TABLE_HPP_
+#define QUICKSTEP_STORAGE_FAST_HASH_TABLE_HPP_
+
+#include <atomic>
+#include <cstddef>
+#include <cstdlib>
+#include <type_traits>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/TupleReference.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "expressions/aggregation/AggregationHandleAvg.hpp"
+#include "threading/SpinSharedMutex.hpp"
+#include "threading/SpinMutex.hpp"
+#include "types/Type.hpp"
+#include "types/TypedValue.hpp"
+#include "utility/BloomFilter.hpp"
+#include "utility/HashPair.hpp"
+#include "utility/Macros.hpp"
+#include "storage/HashTable.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+/**
+ * @brief Base class for hash table.
+ *
+ * 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
+ *         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
+ *         this implementation has some extra indirection compared to
+ *         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.
+ *
+ * @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
+ *        loaded from disk. If false, some out of band memory may be used (e.g.
+ *        to store variable length keys).
+ * @param force_key_copy If true, inserted keys are always copied into this
+ *        HashTable's memory. If false, pointers to external values may be
+ *        stored instead. force_key_copy should be true if the hash table will
+ *        outlive the external key values which are inserted into it. Note that
+ *        if serializable is true and force_key_copy is false, then relative
+ *        offsets will be used instead of absolute pointers to keys, meaning
+ *        that the pointed-to keys must be serialized and deserialized in
+ *        exactly the same relative byte order (e.g. as part of the same
+ *        StorageBlock), and keys must not change position relative to this
+ *        HashTable (beware TupleStorageSubBlocks that may self-reorganize when
+ *        modified). If serializable and resizable are both true, then
+ *        force_key_copy must also be true.
+ * @param allow_duplicate_keys If true, multiple values can be mapped to the
+ *        same key. If false, one and only one value may be mapped.
+ **/
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+class FastHashTable : public HashTableBase<resizable,
+                                       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;
+  static constexpr bool template_allow_duplicate_keys = allow_duplicate_keys;
+
+  // Some HashTable implementations (notably LinearOpenAddressingHashTable)
+  // use a special hash code to represent an empty bucket, and another special
+  // code to indicate that a bucket is currently being inserted into. For those
+  // HashTables, this is a surrogate hash value for empty buckets. Keys which
+  // actually hash to this value should have their hashes mutated (e.g. by
+  // adding 1). We use zero, since we will often be using memory which is
+  // already zeroed-out and this saves us the trouble of a memset. This has
+  // some downside, as the hash function we use is the identity hash for
+  // integers, and the integer 0 is common in many data sets and must be
+  // adjusted (and will then spuriously collide with 1). Nevertheless, this
+  // expense is outweighed by no longer having to memset large regions of
+  // memory when initializing a HashTable.
+  static constexpr unsigned char kEmptyHashByte = 0x0;
+  static constexpr std::size_t kEmptyHash = 0x0;
+
+  // A surrogate hash value for a bucket which is currently being inserted
+  // into. As with kEmptyHash, keys which actually hash to this value should
+  // have their hashes adjusted.
+  static constexpr std::size_t kPendingHash = ~kEmptyHash;
+
+  /**
+   * @brief Virtual destructor.
+   **/
+  virtual ~FastHashTable() {
+    if (resizable) {
+      if (blob_.valid()) {
+        if (serializable) {
+          DEV_WARNING("Destroying a resizable serializable HashTable's underlying "
+                      "StorageBlob.");
+        }
+        const block_id blob_id = blob_->getID();
+        blob_.release();
+        storage_manager_->deleteBlockOrBlobFile(blob_id);
+      }
+    }
+  }
+
+  /**
+   * @brief Get the ID of the StorageBlob used to store a resizable HashTable.
+   *
+   * @warning This method must not be used for a non-resizable HashTable.
+   *
+   * @return The ID of the StorageBlob used to store this HashTable.
+   **/
+  inline block_id getBlobId() const {
+    DEBUG_ASSERT(resizable);
+    return blob_->getID();
+  }
+
+  /**
+   * @brief Erase all entries in this hash table.
+   *
+   * @warning This method is not guaranteed to be threadsafe.
+   **/
+  virtual void clear() = 0;
+
+  /**
+   * @brief Add a new entry into the hash table.
+   *
+   * @warning The key must not be null.
+   * @warning This method is threadsafe with regard to other calls to put(),
+   *          putCompositeKey(), putValueAccessor(), and
+   *          putValueAccessorCompositeKey(), but should not be used
+   *          simultaneously with upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey().
+   * @note This version is for single scalar keys, see also putCompositeKey().
+   * @note If the hash table is (close to) full and resizable is true, this
+   *       routine might result in rebuilding the entire hash table.
+   *
+   * @param key The key.
+   * @param value The value payload.
+   * @return HashTablePutResult::kOK if an entry was successfully inserted,
+   *         HashTablePutResult::kDuplicateKey if allow_duplicate_keys is false
+   *         and key was a duplicate, or HashTablePutResult::kOutOfSpace if
+   *         resizable is false and storage space for the hash table has been
+   *         exhausted.
+   **/
+  HashTablePutResult put(const TypedValue &key,
+                         const uint8_t &value);
+
+  /**
+   * @brief Add a new entry into the hash table (composite key version).
+   *
+   * @warning No component of the key may be null.
+   * @warning This method is threadsafe with regard to other calls to put(),
+   *          putCompositeKey(), putValueAccessor(), and
+   *          putValueAccessorCompositeKey(), but should not be used
+   *          simultaneously with upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey().
+   * @note This version is for composite keys, see also put().
+   * @note If the hash table is (close to) full and resizable is true, this
+   *       routine might result in rebuilding the entire hash table.
+   *
+   * @param key The components of the key.
+   * @param value The value payload.
+   * @return HashTablePutResult::kOK if an entry was successfully inserted,
+   *         HashTablePutResult::kDuplicateKey if allow_duplicate_keys is false
+   *         and key was a duplicate, or HashTablePutResult::kOutOfSpace if
+   *         resizable is false and storage space for the hash table has been
+   *         exhausted.
+   **/
+  HashTablePutResult putCompositeKey(const std::vector<TypedValue> &key,
+                                     const uint8_t &value);
+
+  HashTablePutResult putCompositeKeyFast(const std::vector<TypedValue> &key,
+                                     const uint8_t *value_ptr);
+
+  /**
+   * @brief Add (multiple) new entries into the hash table from a
+   *        ValueAccessor.
+   *
+   * @warning This method is threadsafe with regard to other calls to put(),
+   *          putCompositeKey(), putValueAccessor(), and
+   *          putValueAccessorCompositeKey(), but should not be used
+   *          simultaneously with upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey().
+   * @note This version is for scalar keys, see also
+   *       putValueAccessorCompositeKey().
+   * @note If the hash table fills up while this call is in progress and
+   *       resizable is true, this might result in rebuilding the entire hash
+   *       table.
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_id The attribute ID of the keys to be read from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        is null before inserting it (null keys are skipped). This must be
+   *        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 a call
+   *        operator that takes const ValueAccessor& as an argument (or better
+   *        yet, a templated call operator which takes a const reference to
+   *        some subclass of ValueAccessor as an argument) and returns either
+   *        a ValueT or a reference to a ValueT. The functor should generate
+   *        the appropriate mapped value for the current tuple the accessor is
+   *        iterating on.
+   * @return HashTablePutResult::kOK if all keys and generated values from
+   *         accessor were successfully inserted.
+   *         HashTablePutResult::kOutOfSpace is returned if this hash-table is
+   *         non-resizable and ran out of space (note that some entries may
+   *         still have been inserted, and accessor's iteration will be left on
+   *         the first tuple which could not be inserted).
+   *         HashTablePutResult::kDuplicateKey is returned if
+   *         allow_duplicate_keys is false and a duplicate key is encountered
+   *         (as with HashTablePutResult::kOutOfSpace, some entries may have
+   *         been inserted, and accessor will be left on the tuple with a
+   *         duplicate key).
+   **/
+  template <typename FunctorT>
+  HashTablePutResult putValueAccessor(ValueAccessor *accessor,
+                                      const attribute_id key_attr_id,
+                                      const bool check_for_null_keys,
+                                      FunctorT *functor);
+
+  /**
+   * @brief Add (multiple) new entries into the hash table from a
+   *        ValueAccessor (composite key version).
+   *
+   * @warning This method is threadsafe with regard to other calls to put(),
+   *          putCompositeKey(), putValueAccessor(), and
+   *          putValueAccessorCompositeKey(), but should not be used
+   *          simultaneously with upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey().
+   * @note This version is for composite keys, see also putValueAccessor().
+   * @note If the hash table fills up while this call is in progress and
+   *       resizable is true, this might result in rebuilding the entire hash
+   *       table.
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_ids The attribute IDs of each key component to be read
+   *        from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        has a null component before inserting it (null keys are skipped).
+   *        This must be 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 a call
+   *        operator that takes const ValueAccessor& as an argument (or better
+   *        yet, a templated call operator which takes a const reference to
+   *        some subclass of ValueAccessor as an argument) and returns either
+   *        a ValueT or a reference to a ValueT. The functor should generate
+   *        the appropriate mapped value for the current tuple the accessor is
+   *        iterating on.
+   * @return HashTablePutResult::kOK if all keys and generated values from
+   *         accessor were successfully inserted.
+   *         HashTablePutResult::kOutOfSpace is returned if this hash-table is
+   *         non-resizable and ran out of space (note that some entries may
+   *         still have been inserted, and accessor's iteration will be left on
+   *         the first tuple which could not be inserted).
+   *         HashTablePutResult::kDuplicateKey is returned if
+   *         allow_duplicate_keys is false and a duplicate key is encountered
+   *         (as with HashTablePutResult::kOutOfSpace, some entries may have
+   *         been inserted, and accessor will be left on the tuple with a
+   *         duplicate key).
+   **/
+  template <typename FunctorT>
+  HashTablePutResult putValueAccessorCompositeKey(
+      ValueAccessor *accessor,
+      const std::vector<attribute_id> &key_attr_ids,
+      const bool check_for_null_keys,
+      FunctorT *functor);
+
+  /**
+   * @brief Apply a functor to the value mapped to a key, first inserting a new
+   *        value if one is not already present.
+   *
+   * @warning The key must not be null.
+   * @warning This method is only usable if allow_duplicate_keys is false.
+   * @warning This method is threadsafe with regard to other calls to upsert(),
+   *          upsertCompositeKey(), upsertValueAccessor(), and
+   *          upsertValueAccessorCompositeKey(), but should not be used
+   *          simultaneously with put(), putCompositeKey(), putValueAccessor(),
+   *          or putValueAccessorCompositeKey().
+   * @warning The ValueT* pointer passed to functor's call operator is only
+   *          guaranteed to be valid for the duration of the call. The functor
+   *          should not store a copy of the pointer and assume that it remains
+   *          valid.
+   * @warning Although this method itself is threadsafe, the ValueT object
+   *          accessed by functor is not guaranteed to be (although it is
+   *          guaranteed that its initial insertion will be atomic). If it is
+   *          possible for multiple threads to call upsert() with the same key
+   *          at the same time, then their access to ValueT should be made
+   *          threadsafe (e.g. with the use of atomic types, mutexes, or some
+   *          other external synchronization).
+   * @note This version is for single scalar keys, see also
+   *       upsertCompositeKey().
+   * @note If the hash table is (close to) full and resizable is true, this
+   *       routine might result in rebuilding the entire hash table.
+   *
+   * @param key The key.
+   * @param initial_value If there was not already a preexisting entry in this
+   *        HashTable for the specified key, then the value will be initialized
+   *        with a copy of initial_value. This parameter is ignored if a value
+   *        is already present for key.
+   * @param functor A pointer to a functor, which should provide a call
+   *        operator which takes ValueT* as an argument. The call operator will
+   *        be invoked once on the value corresponding to key (which may be
+   *        newly inserted and default-constructed).
+   * @return True on success, false if upsert failed because there was not
+   *         enough space to insert a new entry in this HashTable.
+   **/
+  template <typename FunctorT>
+  bool upsert(const TypedValue &key,
+              const uint8_t &initial_value,
+              FunctorT *functor);
+
+  /**
+   * @brief Apply a functor to the value mapped to a key, first inserting a new
+   *        value if one is not already present.
+   *
+   * @warning The key must not be null.
+   * @warning This method is only usable if allow_duplicate_keys is false.
+   * @warning This method is threadsafe with regard to other calls to upsert(),
+   *          upsertCompositeKey(), upsertValueAccessor(), and
+   *          upsertValueAccessorCompositeKey(), but should not be used
+   *          simultaneously with put(), putCompositeKey(), putValueAccessor(),
+   *          or putValueAccessorCompositeKey().
+   * @warning The ValueT* pointer passed to functor's call operator is only
+   *          guaranteed to be valid for the duration of the call. The functor
+   *          should not store a copy of the pointer and assume that it remains
+   *          valid.
+   * @warning Although this method itself is threadsafe, the ValueT object
+   *          accessed by functor is not guaranteed to be (although it is
+   *          guaranteed that its initial insertion will be atomic). If it is
+   *          possible for multiple threads to call upsertCompositeKey() with
+   *          the same key at the same time, then their access to ValueT should
+   *          be made threadsafe (e.g. with the use of atomic types, mutexes,
+   *          or some other external synchronization).
+   * @note This version is for composite keys, see also upsert().
+   * @note If the hash table is (close to) full and resizable is true, this
+   *       routine might result in rebuilding the entire hash table.
+   *
+   * @param key The key.
+   * @param initial_value If there was not already a preexisting entry in this
+   *        HashTable for the specified key, then the value will be initialized
+   *        with a copy of initial_value. This parameter is ignored if a value
+   *        is already present for key.
+   * @param functor A pointer to a functor, which should provide a call
+   *        operator which takes ValueT* as an argument. The call operator will
+   *        be invoked once on the value corresponding to key (which may be
+   *        newly inserted and default-constructed).
+   * @return True on success, false if upsert failed because there was not
+   *         enough space to insert a new entry in this HashTable.
+   **/
+  template <typename FunctorT>
+  bool upsertCompositeKey(const std::vector<TypedValue> &key,
+                          const uint8_t &initial_value,
+                          FunctorT *functor);
+
+
+  template <typename FunctorT>
+  bool upsertCompositeKeyFast(const std::vector<TypedValue> &key,
+                          const uint8_t *init_value_ptr,
+                          FunctorT *functor);
+
+  bool upsertCompositeKeyNewFast(const std::vector<TypedValue> &key,
+                          const uint8_t *init_value_ptr,
+                          const uint8_t *source_state);
+
+  /**
+   * @brief Apply a functor to (multiple) entries in this hash table, with keys
+   *        drawn from a ValueAccessor. New values are first inserted if not
+   *        already present.
+   *
+   * @warning This method is only usable if allow_duplicate_keys is false.
+   * @warning This method is threadsafe with regard to other calls to upsert(),
+   *          upsertCompositeKey(), upsertValueAccessor(), and
+   *          upsertValueAccessorCompositeKey(), but should not be used
+   *          simultaneously with put(), putCompositeKey(), putValueAccessor(),
+   *          or putValueAccessorCompositeKey().
+   * @warning The ValueAccessor reference and ValueT* pointer passed to
+   *          functor's call operator are only guaranteed to be valid for the
+   *          duration of the call. The functor should not store a copy of
+   *          these pointers and assume that they remain valid.
+   * @warning Although this method itself is threadsafe, the ValueT object
+   *          accessed by functor is not guaranteed to be (although it is
+   *          guaranteed that its initial insertion will be atomic). If it is
+   *          possible for multiple threads to call upsertValueAccessor() with
+   *          the same key at the same time, then their access to ValueT should
+   *          be made threadsafe (e.g. with the use of atomic types, mutexes,
+   *          or some other external synchronization).
+   * @note This version is for single scalar keys, see also
+   *       upsertValueAccessorCompositeKey().
+   * @note If the hash table is (close to) full and resizable is true, this
+   *       routine might result in rebuilding the entire hash table.
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_id The attribute ID of the keys to be read from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        is null before upserting it (null keys are skipped). This must be
+   *        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 a call
+   *        operator that takes two 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 ValueT*.
+   *        The call operator will be invoked once for every tuple with a
+   *        non-null key in accessor.
+   * @return True on success, false if upsert failed because there was not
+   *         enough space to insert new entries for all the keys in accessor
+   *         (note that some entries may still have been upserted, and
+   *         accessor's iteration will be left on the first tuple which could
+   *         not be inserted).
+   **/
+  template <typename FunctorT>
+  bool upsertValueAccessor(ValueAccessor *accessor,
+                           const attribute_id key_attr_id,
+                           const bool check_for_null_keys,
+                           const uint8_t &initial_value,
+                           FunctorT *functor);
+
+
+  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
+   *        drawn from a ValueAccessor. New values are first inserted if not
+   *        already present. Composite key version.
+   *
+   * @warning This method is only usable if allow_duplicate_keys is false.
+   * @warning This method is threadsafe with regard to other calls to upsert(),
+   *          upsertCompositeKey(), upsertValueAccessor(), and
+   *          upsertValueAccessorCompositeKey(), but should not be used
+   *          simultaneously with put(), putCompositeKey(), putValueAccessor(),
+   *          or putValueAccessorCompositeKey().
+   * @warning The ValueAccessor reference and ValueT* pointer passed to
+   *          functor's call operator are only guaranteed to be valid for the
+   *          duration of the call. The functor should not store a copy of
+   *          these pointers and assume that they remain valid.
+   * @warning Although this method itself is threadsafe, the ValueT object
+   *          accessed by functor is not guaranteed to be (although it is
+   *          guaranteed that its initial insertion will be atomic). If it is
+   *          possible for multiple threads to call upsertValueAccessor() with
+   *          the same key at the same time, then their access to ValueT should
+   *          be made threadsafe (e.g. with the use of atomic types, mutexes,
+   *          or some other external synchronization).
+   * @note This version is for composite keys, see also upsertValueAccessor().
+   * @note If the hash table is (close to) full and resizable is true, this
+   *       routine might result in rebuilding the entire hash table.
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_ids The attribute IDs of each key component to be read
+   *        from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        is null before upserting it (null keys are skipped). This must be
+   *        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 a call
+   *        operator that takes two 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 ValueT*.
+   *        The call operator will be invoked once for every tuple with a
+   *        non-null key in accessor.
+   * @return True on success, false if upsert failed because there was not
+   *         enough space to insert new entries for all the keys in accessor
+   *         (note that some entries may still have been upserted, and
+   *         accessor's iteration will be left on the first tuple which could
+   *         not be inserted).
+   **/
+  template <typename FunctorT>
+  bool upsertValueAccessorCompositeKey(
+      ValueAccessor *accessor,
+      const std::vector<attribute_id> &key_attr_ids,
+      const bool check_for_null_keys,
+      const uint8_t &initial_value,
+      FunctorT *functor);
+
+  bool upsertValueAccessorCompositeKeyFast(
+      const std::vector<std::vector<attribute_id>> &argument,
+      ValueAccessor *accessor,
+      const std::vector<attribute_id> &key_attr_ids,
+      const bool check_for_null_keys);
+
+  /**
+   * @brief Determine the number of entries (key-value pairs) contained in this
+   *        HashTable.
+   * @note For some HashTable implementations, this is O(1), but for others it
+   *       may be O(n) where n is the number of buckets.
+   *
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @return The number of entries in this HashTable.
+   **/
+  virtual std::size_t numEntries() const = 0;
+
+  /**
+   * @brief Lookup a key against this hash table to find a matching entry.
+   *
+   * @warning Only usable with the hash table that does not allow duplicate
+   *          keys.
+   * @warning The key must not be null.
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   * @note This version is for single scalar keys. See also
+   *       getSingleCompositeKey().
+   *
+   * @param key The key to look up.
+   * @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;
+
+  /**
+   * @brief Lookup a composite key against this hash table to find a matching
+   *        entry.
+   *
+   * @warning Only usable with the hash table that does not allow duplicate
+   *          keys.
+   * @warning The key must not be null.
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   * @note This version is for composite keys. See also getSingle().
+   *
+   * @param key The key to look up.
+   * @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;
+
+  /**
+   * @brief Lookup a key against this hash table to find matching entries.
+   *
+   * @warning The key must not be null.
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   * @note It is more efficient to call getSingle() if the hash table does not
+   *       allow duplicate keys.
+   * @note This version is for single scalar keys. See also
+   *       getAllCompositeKey().
+   *
+   * @param key The key to look up.
+   * @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;
+
+  /**
+   * @brief Lookup a composite key against this hash table to find matching
+   *        entries.
+   *
+   * @warning The key must not be null.
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   * @note It is more efficient to call getSingleCompositeKey() if the hash
+   *       table does not allow duplicate keys.
+   * @note This version is for composite keys. See also getAll().
+   *
+   * @param key The key to look up.
+   * @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;
+
+  /**
+   * @brief Lookup (multiple) keys from a ValueAccessor and apply a functor to
+   *        the matching values.
+   *
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   * @note This version is for single scalar keys. See also
+   *       getAllFromValueAccessorCompositeKey().
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_id The attribute ID of the keys to be read from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        is null before looking it up (null keys are skipped). This must be
+   *        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 a call
+   *        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 functor will be invoked once for each pair of a
+   *        key taken from accessor and matching value.
+   **/
+  template <typename FunctorT>
+  void getAllFromValueAccessor(ValueAccessor *accessor,
+                               const attribute_id key_attr_id,
+                               const bool check_for_null_keys,
+                               FunctorT *functor) const;
+
+  /**
+   * @brief Lookup (multiple) keys from a ValueAccessor, apply a functor to the
+   *        matching values and additionally call a recordMatch() function of
+   *        the functor when the first match for a key is found.
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   * @note This version is for single scalar keys. See also
+   *       getAllFromValueAccessorCompositeKeyWithExtraWorkForFirstMatch().
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_id The attribute ID of the keys to be read from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        is null before looking it up (null keys are skipped). This must be
+   *        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
+   *        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
+   *        key taken from accessor and matching value.
+   *        2) A function hasMatch that takes 1 argument: const ValueAccessor&.
+   *        The function will be called only once for a key from accessor when
+   *        the first match is found.
+   */
+  template <typename FunctorT>
+  void getAllFromValueAccessorWithExtraWorkForFirstMatch(
+      ValueAccessor *accessor,
+      const attribute_id key_attr_id,
+      const bool check_for_null_keys,
+      FunctorT *functor) const;
+
+  /**
+   * @brief Lookup (multiple) keys from a ValueAccessor, apply a functor to the
+   *        matching values and additionally call a recordMatch() function of
+   *        the functor when the first match for a key is found. Composite key
+   *        version.
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_id The attribute ID of the keys to be read from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        is null before looking it up (null keys are skipped). This must be
+   *        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
+   *        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
+   *        key taken from accessor and matching value.
+   *        2) A function hasMatch that takes 1 argument: const ValueAccessor&.
+   *        The function will be called only once for a key from accessor when
+   *        the first match is found.
+   */
+  template <typename FunctorT>
+  void getAllFromValueAccessorCompositeKeyWithExtraWorkForFirstMatch(
+      ValueAccessor *accessor,
+      const std::vector<attribute_id> &key_attr_ids,
+      const bool check_for_null_keys,
+      FunctorT *functor) const;
+
+  /**
+   * @brief Lookup (multiple) keys from a ValueAccessor and apply a functor to
+   *        the matching values. Composite key version.
+   *
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   * @note This version is for composite keys. See also
+   *       getAllFromValueAccessor().
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_ids The attribute IDs of each key component to be read
+   *        from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        has a null component before inserting it (null keys are skipped).
+   *        This must be 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 a call
+   *        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 functor will be invoked once for each pair of a
+   *        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;
+
+  /**
+   * @brief Apply the functor to each key with a match in the hash table.
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_id The attribute ID of the keys to be read from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        is null before looking it up (null keys are skipped). This must be
+   *        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 an operator that
+   *        takes 1 argument: const ValueAccessor&. The operator will be called
+   *        only once for a key from accessor if there is a match.
+   */
+  template <typename FunctorT>
+  void runOverKeysFromValueAccessorIfMatchFound(ValueAccessor *accessor,
+                                                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);
+  }
+
+  /**
+   * @brief Apply the functor to each key with a match in the hash table.
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_id The attribute ID of the keys to be read from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        is null before looking it up (null keys are skipped). This must be
+   *        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 an operator that
+   *        takes 1 argument: const ValueAccessor&. The operator will be called
+   *        only once for a key from accessor if there is a match.
+   */
+  template <typename FunctorT>
+  void runOverKeysFromValueAccessorIfMatchFoundCompositeKey(
+      ValueAccessor *accessor,
+      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);
+  }
+
+  /**
+   * @brief Apply the functor to each key without a match in the hash table.
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_id The attribute ID of the keys to be read from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        is null before looking it up (null keys are skipped). This must be
+   *        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 an operator that
+   *        takes 1 argument: const ValueAccessor&. The operator will be called
+   *        only once for a key from accessor if there is no match.
+   */
+  template <typename FunctorT>
+  void runOverKeysFromValueAccessorIfMatchNotFound(
+      ValueAccessor *accessor,
+      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);
+  }
+
+  /**
+   * @brief Apply the functor to each key without a match in the hash table.
+   *
+   * @param accessor A ValueAccessor which will be used to access keys.
+   *        beginIteration() should be called on accessor before calling this
+   *        method.
+   * @param key_attr_id The attribute ID of the keys to be read from accessor.
+   * @param check_for_null_keys If true, each key will be checked to see if it
+   *        is null before looking it up (null keys are skipped). This must be
+   *        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 an operator that
+   *        takes 1 argument: const ValueAccessor&. The operator will be called
+   *        only once for a key from accessor if there is no match.
+   */
+  template <typename FunctorT>
+  void runOverKeysFromValueAccessorIfMatchNotFoundCompositeKey(
+      ValueAccessor *accessor,
+      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);
+  }
+
+  /**
+   * @brief Apply a functor to each key, value pair in this hash table.
+   *
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   * @note This version is for single scalar keys. See also
+   *       forEachCompositeKey().
+   *
+   * @param functor A pointer to a functor, which should provide a call
+   *        operator which takes 2 arguments: const TypedValue&, const ValueT&.
+   *        The call operator will be invoked once on each key, value pair in
+   *        this hash table (note that if allow_duplicate_keys is true,
+   *        the call may occur multiple times for the same key with different
+   *        values).
+   * @return The number of key-value pairs visited.
+   **/
+  template <typename FunctorT>
+  std::size_t forEach(FunctorT *functor) const;
+
+  /**
+   * @brief Apply a functor to each key, value pair in this hash table.
+   *
+   * @warning This method assumes that no concurrent calls to put(),
+   *          putCompositeKey(), putValueAccessor(),
+   *          putValueAccessorCompositeKey(), upsert(), upsertCompositeKey(),
+   *          upsertValueAccessor(), or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call and as long as the returned pointer may be
+   *          dereferenced). Concurrent calls to getSingle(),
+   *          getSingleCompositeKey(), getAll(), getAllCompositeKey(),
+   *          getAllFromValueAccessor(), getAllFromValueAccessorCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   * @note This version is for composite keys. See also forEach().
+   *
+   * @param functor A pointer to a functor, which should provide a call
+   *        operator which takes 2 arguments: const std::vector<TypedValue>&,
+   *        const ValueT&. The call operator will be invoked once on each key,
+   *        value pair in this hash table (note that if allow_duplicate_keys is
+   *        true, the call may occur multiple times for the same key with
+   *        different values).
+   * @return The number of key-value pairs visited.
+   **/
+  template <typename FunctorT>
+  std::size_t forEachCompositeKey(FunctorT *functor) const;
+
+  template <typename FunctorT>
+  std::size_t forEachCompositeKeyFast(FunctorT *functor) const;
+
+  template <typename FunctorT>
+  std::size_t forEachCompositeKeyFast(FunctorT *functor, int index) const;
+  /**
+   * @brief A call to this function will cause a bloom filter to be built
+   *        during the build phase of this hash table.
+   **/
+  inline void enableBuildSideBloomFilter() {
+    has_build_side_bloom_filter_ = true;
+  }
+
+  /**
+   * @brief A call to this function will cause a set of bloom filters to be
+   *        probed during the probe phase of this hash table.
+   **/
+  inline void enableProbeSideBloomFilter() {
+    has_probe_side_bloom_filter_ = true;
+  }
+
+  /**
+   * @brief This function sets the pointer to the bloom filter to be
+   *        used during the build phase of this hash table.
+   * @warning Should call enable_build_side_bloom_filter() first to enable
+   *          bloom filter usage during build phase.
+   * @note The ownership of the bloom filter lies with the caller.
+   *
+   * @param bloom_filter The pointer to the bloom filter.
+   **/
+  inline void setBuildSideBloomFilter(BloomFilter *bloom_filter) {
+    build_bloom_filter_ = bloom_filter;
+  }
+
+  /**
+   * @brief This function adds a pointer to the list of bloom filters to be
+   *        used during the probe phase of this hash table.
+   * @warning Should call enable_probe_side_bloom_filter() first to enable
+   *          bloom filter usage during probe phase.
+   * @note The ownership of the bloom filter lies with the caller.
+   *
+   * @param bloom_filter The pointer to the bloom filter.
+   **/
+  inline void addProbeSideBloomFilter(const BloomFilter *bloom_filter) {
+    probe_bloom_filters_.emplace_back(bloom_filter);
+  }
+
+  /**
+   * @brief This function adds a vector of attribute ids corresponding to a
+   *        bloom filter used during the probe phase of this hash table.
+   * @warning Should call enable_probe_side_bloom_filter() first to enable
+   *          bloom filter usage during probe phase.
+   *
+   * @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) {
+    probe_attribute_ids_.push_back(probe_attribute_ids);
+  }
+
+ protected:
+  /**
+   * @brief Constructor for new resizable hash table.
+   *
+   * @param key_types A vector of one or more types (>1 indicates a composite
+   *        key).
+   * @param num_entries The estimated number of entries this hash table will
+   *        hold.
+   * @param storage_manager The StorageManager to use (a StorageBlob will be
+   *        allocated to hold this hash table's contents).
+   * @param adjust_hashes If true, the hash of a key should be modified by
+   *        applying AdjustHash() so that it does not collide with one of the
+   *        special values kEmptyHash or kPendingHash. If false, the hash is
+   *        used as-is.
+   * @param use_scalar_literal_hash If true, the key is a single scalar literal
+   *        (non-composite) that it is safe to use the simplified hash function
+   *        TypedValue::getHashScalarLiteral() on. If false, the generic
+   *        TypedValue::getHash() method will be used.
+   * @param preallocate_supported If true, this HashTable overrides
+   *        preallocateForBulkInsert() to allow bulk-allocation of resources
+   *        (i.e. buckets and variable-length key storage) in a single up-front
+   *        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) {
+    DEBUG_ASSERT(resizable);
+    std::size_t running_sum = sizeof(SpinMutex);
+    for (auto size : payload_sizes) {
+        payload_offsets_.emplace_back(running_sum);
+        running_sum+=size;
+    }
+  }
+
+  /**
+   * @brief Constructor for non-resizable hash table.
+   *
+   * @param key_types A vector of one or more types (>1 indicates a composite
+   *        key).
+   * @param hash_table_memory A pointer to memory to use for this hash table.
+   * @param hash_table_memory_size The size of hash_table_memory in bytes.
+   * @param new_hash_table If true, this hash table is being constructed for
+   *        the first time and hash_table_memory will be cleared. If false,
+   *        reload a pre-existing hash table.
+   * @param hash_table_memory_zeroed If new_hash_table is true, setting this to
+   *        true means that this HashTable will assume that hash_table_memory
+   *        has already been zeroed-out (any newly-allocated block or blob
+   *        memory from StorageManager is zeroed-out). If false, this HashTable
+   *        will explicitly zero-fill its memory as neccessary. This parameter
+   *        has no effect when new_hash_table is false.
+   * @param adjust_hashes If true, the hash of a key should be modified by
+   *        applying AdjustHash() so that it does not collide with one of the
+   *        special values kEmptyHash or kPendingHash. If false, the hash is
+   *        used as-is.
+   * @param use_scalar_literal_hash If true, the key is a single scalar literal
+   *        (non-composite) that it is safe to use the simplified hash function
+   *        TypedValue::getHashScalarLiteral() on. If false, the generic
+   *        TypedValue::getHash() method will be used.
+   * @param preallocate_supported If true, this HashTable overrides
+   *        preallocateForBulkInsert() to allow bulk-allocation of resources
+   *        (i.e. buckets and variable-length key storage) in a single up-front
+   *        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)
+      : 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),
+        storage_manager_(nullptr),
+        hash_table_memory_(hash_table_memory),
+        hash_table_memory_size_(hash_table_memory_size) {
+    DEBUG_ASSERT(!resizable);
+  }
+
+  // Adjust 'hash' so that it is not exactly equal to either of the special
+  // values kEmptyHash or kPendingHash.
+  inline constexpr static std::size_t AdjustHash(const std::size_t hash) {
+    return hash + (hash == kEmptyHash) - (hash == kPendingHash);
+  }
+
+  // Set information about which key components are stored inline. This usually
+  // comes from a HashTableKeyManager, and is set by the constructor of a
+  // subclass of HashTable.
+  inline void setKeyInline(const std::vector<bool> *key_inline) {
+    scalar_key_inline_ = key_inline->front();
+    key_inline_ = key_inline;
+  }
+
+  // Generate a hash for a composite key by hashing each component of 'key' and
+  // mixing their bits with CombineHashes().
+  inline std::size_t hashCompositeKey(const std::vector<TypedValue> &key) const;
+
+  // If 'force_key_copy' is true and some part of a composite key is
+  // variable-length, calculate the total number of bytes for variable-length
+  // key components that need to be copied. Otherwise, return 0 to indicate
+  // that no variable-length copy is required.
+  inline std::size_t calculateVariableLengthCompositeKeyCopySize(
+      const std::vector<TypedValue> &key) const;
+
+  // 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 putCompositeKeyInternal(const std::vector<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;
+
+
+  // Helpers for upsert. Both return a pointer to the value corresponding to
+  // 'key'. If this HashTable is resizable, 'resize_shared_mutex_' should be
+  // locked in shared mode while calling and using the returned pointer. May
+  // 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* upsertInternal(const TypedValue &key,
+                                 const std::size_t variable_key_size,
+                                 const uint8_t &initial_value) = 0;
+  virtual uint8_t* upsertInternalFast(const TypedValue &key,
+                                 const std::uint8_t *init_value_ptr,
+                                 const std::size_t variable_key_size) = 0;
+  virtual uint8_t* upsertCompositeKeyInternal(const std::vector<TypedValue> &key,
+                                             const std::size_t variable_key_size,
+                                             const uint8_t &initial_value) = 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;
+
+  // 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
+  // the key of the next entry, '*value' points to the associated value, and
+  // '*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,
+                            std::size_t *entry_num) const = 0;
+  virtual bool getNextEntryCompositeKey(std::vector<TypedValue> *key,
+                                        const uint8_t **value,
+                                        std::size_t *entry_num) const = 0;
+
+  // Helpers for getAllFromValueAccessor. Each return true on success, false if
+  // no more entries exist for the specified key. After a successful call,
+  // '*value' points to the associated value, and '*entry_num' is incremented
+  // to the next (implementation defined) entry to check ('*entry_num' should
+  // initially be set to zero).
+  virtual bool getNextEntryForKey(const TypedValue &key,
+                                  const std::size_t hash_code,
+                                  const 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,
+                                           std::size_t *entry_num) const = 0;
+
+  // Return true if key exists in the hash table.
+  virtual bool hasKey(const TypedValue &key) const = 0;
+  virtual bool hasCompositeKey(const std::vector<TypedValue> &key) const = 0;
+
+  // For a resizable HashTable, grow to accomodate more entries. If
+  // 'extra_buckets' is not zero, it may serve as a "hint" to implementations
+  // that at least the requested number of extra buckets are required when
+  // resizing (mainly used in putValueAccessor() and
+  // putValueAccessorCompositeKey() when 'preallocate_supported_' is true).
+  // Implementations are free to ignore 'extra_buckets'. If
+  // 'extra_variable_storage' is not zero, implementations will attempt to
+  // allocate at least enough additional variable-key storage space to
+  // accomodate the number of bytes specified. 'retry_num' is intended ONLY for
+  // when resize() recursively calls itself and should not be set to nonzero by
+  // any other caller.
+  virtual void resize(const std::size_t extra_buckets,
+                      const std::size_t extra_variable_storage,
+                      const std::size_t retry_num = 0) = 0;
+
+  // In the case where 'allow_duplicate_keys' is true, it is possible to
+  // pre-calculate the number of key-value entries and the amount of
+  // variable-length key storage that will be needed to insert all the
+  // entries from a ValueAccessor in putValueAccessor() or
+  // putValueAccessorCompositeKey() before actually inserting anything. Some
+  // HashTable implemetations (notably SeparateChainingHashTable) can achieve
+  // better performance by ammortizing the cost of allocating certain resources
+  // (buckets and variable-length key storage) in one up-front allocation. This
+  // 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.");
+  }
+
+  // Type(s) of keys.
+  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
+  // HashTableKeyManager and set by calling setKeyInline().
+  bool scalar_key_inline_;
+  const std::vector<bool> *key_inline_;
+
+  // Whether hashes should be adjusted by AdjustHash() before being used.
+  const bool adjust_hashes_;
+  // Whether it is safe to use the simplified TypedValue::getHashScalarLiteral()
+  // method instead of the generic TypedValue::getHash() method.
+  const bool use_scalar_literal_hash_;
+  // Whether preallocateForBulkInsert() is supported by this HashTable.
+  const bool preallocate_supported_;
+
+  const std::vector<AggregationHandle *> handles_;
+  const std::size_t total_payload_size_;
+  std::vector<std::size_t> payload_offsets_;
+
+  // Used only when resizable is true:
+  StorageManager *storage_manager_;
+  MutableBlobReference blob_;
+  // Locked in shared mode for most operations, exclusive mode during resize.
+  // Not locked at all for non-resizable HashTables.
+  alignas(kCacheLineBytes) SpinSharedMutex<true> resize_shared_mutex_;
+
+  // Used only when resizable is false:
+  void *hash_table_memory_;
+  const std::size_t hash_table_memory_size_;
+virtual size_t get_buckets_allocated() const {return 0;}
+
+ private:
+  // Assign '*key_vector' with the attribute values specified by 'key_attr_ids'
+  // at the current position of 'accessor'. If 'check_for_null_keys' is true,
+  // stops and returns true if any of the values is null, otherwise returns
+  // false.
+  template <typename ValueAccessorT>
+  inline static bool GetCompositeKeyFromValueAccessor(
+      const ValueAccessorT &accessor,
+      const std::vector<attribute_id> &key_attr_ids,
+      const bool check_for_null_keys,
+      std::vector<TypedValue> *key_vector) {
+    for (std::vector<attribute_id>::size_type key_idx = 0;
+         key_idx < key_attr_ids.size();
+         ++key_idx) {
+      (*key_vector)[key_idx] = accessor.getTypedValue(key_attr_ids[key_idx]);
+      if (check_for_null_keys && (*key_vector)[key_idx].isNull()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  // If run_if_match_found is true, apply the functor to each key if a match is
+  // found; otherwise, apply the functor if no match is found.
+  template <bool run_if_match_found, typename FunctorT>
+  void runOverKeysFromValueAccessor(ValueAccessor *accessor,
+                                    const attribute_id key_attr_id,
+                                    const bool check_for_null_keys,
+                                    FunctorT *functor) const;
+
+  template <bool run_if_match_found, typename FunctorT>
+  void runOverKeysFromValueAccessorCompositeKey(
+      ValueAccessor *accessor,
+      const std::vector<attribute_id> &key_attr_ids,
+      const bool check_for_null_keys,
+      FunctorT *functor) const;
+
+  // Method containing the actual logic implementing getAllFromValueAccessor().
+  // Has extra template parameters that control behavior to avoid some
+  // inner-loop branching.
+  template <typename FunctorT,
+            bool check_for_null_keys,
+            bool adjust_hashes_template,
+            bool use_scalar_literal_hash_template>
+  void getAllFromValueAccessorImpl(ValueAccessor *accessor,
+                                   const attribute_id key_attr_id,
+                                   FunctorT *functor) const;
+
+  // Data structures used for bloom filter optimized semi-joins.
+  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<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
+ * to blocks that are evicted.
+ **/
+using AggregationStateFastHashTable = FastHashTable<true, false, true, false>;
+
+/** @} */
+
+// ----------------------------------------------------------------------------
+// Implementations of template class methods follow.
+
+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;
+  if (resizable) {
+    HashTablePutResult result = HashTablePutResult::kOutOfSpace;
+    while (result == HashTablePutResult::kOutOfSpace) {
+      {
+        SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
+        result = putInternal(key, variable_size, value, nullptr);
+      }
+      if (result == HashTablePutResult::kOutOfSpace) {
+        resize(0, variable_size);
+      }
+    }
+    return result;
+  } else {
+    return putInternal(key, variable_size, value, nullptr);
+  }
+}
+
+template <bool resizable,
+          bool serializable,
+          bool force_key_copy,
+          bool allow_duplicate_keys>
+HashTablePutResult FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>
+    ::putCompositeKey(const std::vector<TypedValue> &key,
+                      const uint8_t& value) {
+  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 = putCompositeKeyInternal(key, variable_size, value, nullptr);
+      }
+      if (result == HashTablePutResult::kOutOfSpace) {
+        resize(0, variable_size);
+      }
+    }
+    return result;
+  } else {
+    return putCompositeKeyInternal(key, variable_size, value, nullptr);
+  }
+}
+
+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);
+  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);
+      }
+      if (result == HashTablePutResult::kOutOfSpace) {
+        resize(0, variable_size);
+      }
+    }
+    return result;
+  } else {
+    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 result = HashTablePutResult::kOutOfSpace;
+  std::size_t variable_size;
+  HashTablePreallocationState prealloc_state;
+  bool using_prealloc = allow_duplicate_keys && preallocate_supported_;
+  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 (!prealloc_succeeded) {
+            this->resize(total_entries, total_variable_key_size);
+          }
+        }
+      } 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_);
+          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);
+        // 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::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;
+  });
+}
+
+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>
+    ::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;
+  HashTablePreallocationState prealloc_state;
+  bool using_prealloc = allow_duplicate_keys && preallocate_supported_;
+  std::vector<TypedValue> key_vector;
+  key_vector.resize(key_attr_ids.size());
+  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 (!prealloc_succeeded) {
+            this->resize(total_entries, total_variable_key_size);
+          }
+        }
+      } 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_);
+          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,
+                                                   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;
+  });
+}
+
+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>
+    ::upsert(const TypedValue &key,
+             const uint8_t &initial_value,
+             FunctorT *functor) {
+  DEBUG_ASSERT(!allow_duplicate_keys);
+  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 = upsertInternal(key, variable_size, initial_value);
+        if (value != nullptr) {
+          (*functor)(value);
+          return true;
+        }
+      }
+      resize(0, force_key_copy && !scalar_key_inline_ ? key.getDataSize() : 0);
+    }
+  } else {
+    uint8_t *value = upsertInternal(key, variable_size, initial_value);
+    if (value == nullptr) {
+      return false;
+    } else {
+      (*functor)(value);
+      return true;
+    }
+  }
+}
+
+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>
+    ::upsertCompositeKey(const std::vector<TypedValue> &key,
+                         const uint8_t &initial_value,
+                         FunctorT *functor) {
+  DEBUG_ASSERT(!allow_duplicate_keys);
+  const std::size_t variable_size = calculateVariableLengthCompositeKeyCopySize(key);
+  if (resizable) {
+    for (;;) {
+      {
+        SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
+        uint8_t *value = upsertCompositeKeyInternal(key, variable_size, initial_value);
+        if (value 

<TRUNCATED>


[09/28] incubator-quickstep git commit: Fixed 4 failures on unit tests

Posted by hb...@apache.org.
Fixed 4 failures on unit tests


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/1665593e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/1665593e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/1665593e

Branch: refs/heads/partitioned-aggregation
Commit: 1665593ebf4553ea9b99ce7c10eeb2f6e577dcce
Parents: d0756e7
Author: rathijit <ra...@node-2.aggregation.quickstep-pg0.wisc.cloudlab.us>
Authored: Fri Aug 5 06:00:12 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:56:07 2016 -0500

----------------------------------------------------------------------
 .../aggregation/AggregationConcreteHandle.cpp   | 14 +++---
 .../aggregation/AggregationConcreteHandle.hpp   | 41 ++++++++++++++--
 expressions/aggregation/AggregationHandle.hpp   |  6 ++-
 .../aggregation/AggregationHandleAvg.cpp        | 14 +++---
 .../aggregation/AggregationHandleAvg.hpp        | 15 +++++-
 .../aggregation/AggregationHandleCount.cpp      |  7 +--
 .../aggregation/AggregationHandleCount.hpp      | 19 ++++++--
 .../aggregation/AggregationHandleDistinct.cpp   |  2 +-
 .../aggregation/AggregationHandleDistinct.hpp   |  2 +-
 .../aggregation/AggregationHandleMax.cpp        | 14 +++---
 .../aggregation/AggregationHandleMax.hpp        | 13 ++++-
 .../aggregation/AggregationHandleMin.cpp        | 14 +++---
 .../aggregation/AggregationHandleMin.hpp        | 15 +++++-
 .../aggregation/AggregationHandleSum.cpp        | 15 +++---
 .../aggregation/AggregationHandleSum.hpp        | 15 +++++-
 storage/AggregationOperationState.cpp           | 51 +++++++++++---------
 storage/CMakeLists.txt                          |  1 -
 storage/FastHashTable.hpp                       | 41 +++++++++++++++-
 storage/FastSeparateChainingHashTable.hpp       | 16 +++---
 19 files changed, 221 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationConcreteHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.cpp b/expressions/aggregation/AggregationConcreteHandle.cpp
index 1efe010..ac5148b 100644
--- a/expressions/aggregation/AggregationConcreteHandle.cpp
+++ b/expressions/aggregation/AggregationConcreteHandle.cpp
@@ -52,17 +52,17 @@ void AggregationConcreteHandle::insertValueAccessorIntoDistinctifyHashTable(
     AggregationStateHashTableBase *distinctify_hash_table) const {
   // If the key-value pair is already there, we don't need to update the value,
   // which should always be "true". I.e. the value is just a placeholder.
-//  const auto noop_upserter = [](const auto &accessor, const bool *value) -> void {};
+  //  const auto noop_upserter = [](const auto &accessor, const bool *value) -> void {};
 
   AggregationStateFastHashTable *hash_table =
       static_cast<AggregationStateFastHashTable *>(distinctify_hash_table);
   if (key_ids.size() == 1) {
-// TODO(rathijit): fix
-//    hash_table->upsertValueAccessor(accessor,
-//                                    key_ids[0],
-//                                    true /* check_for_null_keys */,
-//                                    true /* initial_value */,
-//                                    &noop_upserter);
+    std::vector<std::vector<attribute_id>> args;
+    args.emplace_back(key_ids);
+    hash_table->upsertValueAccessorFast(args,
+                                    accessor,
+                                    key_ids[0],
+                                    true /* check_for_null_keys */);
   } else {
     std::vector<std::vector<attribute_id>> empty_args;
     empty_args.resize(1);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationConcreteHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.hpp b/expressions/aggregation/AggregationConcreteHandle.hpp
index d332ec9..609937a 100644
--- a/expressions/aggregation/AggregationConcreteHandle.hpp
+++ b/expressions/aggregation/AggregationConcreteHandle.hpp
@@ -27,6 +27,7 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTable.hpp"
+#include "storage/FastHashTable.hpp"
 #include "storage/HashTableBase.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
@@ -278,6 +279,11 @@ class AggregationConcreteHandle : public AggregationHandle {
       const AggregationStateHashTableBase &distinctify_hash_table) const;
 
   template <typename HandleT,
+            typename StateT>
+  StateT* aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
+      const AggregationStateHashTableBase &distinctify_hash_table) const;
+
+  template <typename HandleT,
             typename StateT,
             typename HashTableT>
   void aggregateOnDistinctifyHashTableForGroupByUnaryHelper(
@@ -289,7 +295,7 @@ class AggregationConcreteHandle : public AggregationHandle {
             typename HashTableT>
   void aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *hash_table) const;
+      AggregationStateHashTableBase *hash_table, int index) const;
 
 
   template <typename HandleT,
@@ -494,6 +500,31 @@ StateT* AggregationConcreteHandle::aggregateOnDistinctifyHashTableForSingleUnary
 }
 
 template <typename HandleT,
+          typename StateT>
+StateT* AggregationConcreteHandle::aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
+    const AggregationStateHashTableBase &distinctify_hash_table) const {
+  const HandleT& handle = static_cast<const HandleT&>(*this);
+  StateT *state = static_cast<StateT*>(createInitialState());
+
+  // A lambda function which will be called on each key from the distinctify
+  // hash table.
+  const auto aggregate_functor = [&handle, &state](const TypedValue &key,
+                                                   const std::uint8_t &dumb_placeholder) {
+    // For each (unary) key in the distinctify hash table, aggregate the key
+    // into "state".
+    handle.iterateUnaryInl(state, key);
+  };
+
+  const AggregationStateFastHashTable &hash_table =
+      static_cast<const AggregationStateFastHashTable &>(distinctify_hash_table);
+  // Invoke the lambda function "aggregate_functor" on each key from the distinctify
+  // hash table.
+  hash_table.forEach(&aggregate_functor);
+
+  return state;
+}
+
+template <typename HandleT,
           typename StateT,
           typename HashTableT>
 void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHelper(
@@ -534,13 +565,13 @@ template <typename HandleT,
           typename HashTableT>
 void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table) const {
+    AggregationStateHashTableBase *aggregation_hash_table, int index) const {
   const HandleT& handle = static_cast<const HandleT&>(*this);
   HashTableT *target_hash_table = static_cast<HashTableT*>(aggregation_hash_table);
 
   // A lambda function which will be called on each key-value pair from the
   // distinctify hash table.
-  const auto aggregate_functor = [&handle, &target_hash_table](
+  const auto aggregate_functor = [&handle, &target_hash_table, &index](
       std::vector<TypedValue> &key,
       const bool &dumb_placeholder) {
     // For each (composite) key vector in the distinctify hash table with size N.
@@ -552,10 +583,10 @@ void AggregationConcreteHandle::aggregateOnDistinctifyHashTableForGroupByUnaryHe
     // An upserter as lambda function for aggregating the argument into its
     // GROUP BY group's entry inside aggregation_hash_table.
     const auto upserter = [&handle, &argument](std::uint8_t *state) {
-      handle.iterateUnaryInlFast(argument, state+sizeof(SpinMutex));
+      handle.iterateUnaryInlFast(argument, state);
     };
 
-    target_hash_table->upsertCompositeKeyFast(key, nullptr, &upserter);
+    target_hash_table->upsertCompositeKeyFast(key, nullptr, &upserter, index);
   };
 
   const HashTableT &source_hash_table =

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index 92cd6a7..7c9e544 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -347,7 +347,7 @@ class AggregationHandle {
    */
   virtual void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table) const = 0;
+      AggregationStateHashTableBase *aggregation_hash_table, int index) const = 0;
 
   /**
    * @brief Merge two GROUP BY hash tables in one.
@@ -362,11 +362,13 @@ class AggregationHandle {
       const AggregationStateHashTableBase &source_hash_table,
       AggregationStateHashTableBase *destination_hash_table) const = 0;
 
-  virtual size_t getPayloadSize() const {return 8;}
+  virtual size_t getPayloadSize() const {return 1;}
   virtual void setPayloadOffset(std::size_t) {}
   virtual void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) {}
   virtual void mergeStatesFast(const uint8_t *src, uint8_t *dst) const {}
   virtual void initPayload(uint8_t *byte_ptr) {}
+  virtual void BlockUpdate() {}
+  virtual void AllowUpdate() {}
 
  protected:
   AggregationHandle() {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index f38c628..383a81f 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.cpp
@@ -42,7 +42,7 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleAvg::AggregationHandleAvg(const Type &type)
-    : argument_type_(type) {
+    : argument_type_(type), block_update(false) {
   // We sum Int as Long and Float as Double so that we have more headroom when
   // adding many values.
   TypeID type_precision_id;
@@ -206,7 +206,7 @@ ColumnVector* AggregationHandleAvg::finalizeHashTable(
 
 AggregationState* AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
       AggregationHandleAvg,
       AggregationStateAvg>(
           distinctify_hash_table);
@@ -214,14 +214,12 @@ AggregationState* AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle
 
 void AggregationHandleAvg::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+    AggregationStateHashTableBase *aggregation_hash_table, int index) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleAvg,
-      AggregationStateAvg,
-      AggregationStateHashTable<AggregationStateAvg>>(
+      AggregationStateFastHashTable>(
           distinctify_hash_table,
-          blank_state_,
-          aggregation_hash_table);
+          aggregation_hash_table, index);
 }
 
 void AggregationHandleAvg::mergeGroupByHashTables(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index e187d22..15835e0 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -123,7 +123,7 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     ++state->count_;
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) {
+  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
     if (value.isNull()) return;
     TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
@@ -133,9 +133,18 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
   }
 
   inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+     if (block_update) return;
      iterateUnaryInlFast(arguments.front(), byte_ptr);
   }
 
+  void BlockUpdate() override {
+      block_update = true;
+  }
+
+  void AllowUpdate() override {
+      block_update = false;
+  }
+
   void initPayload(uint8_t *byte_ptr) override {
     TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
     std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr + blank_state_.count_offset);
@@ -208,7 +217,7 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table) const override;
+      AggregationStateHashTableBase *aggregation_hash_table, int index) const override;
 
   void mergeGroupByHashTables(
       const AggregationStateHashTableBase &source_hash_table,
@@ -235,6 +244,8 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
   std::unique_ptr<UncheckedBinaryOperator> merge_add_operator_;
   std::unique_ptr<UncheckedBinaryOperator> divide_operator_;
 
+  bool block_update;
+
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleAvg);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.cpp b/expressions/aggregation/AggregationHandleCount.cpp
index f1eadf1..3a333ef 100644
--- a/expressions/aggregation/AggregationHandleCount.cpp
+++ b/expressions/aggregation/AggregationHandleCount.cpp
@@ -196,7 +196,7 @@ AggregationState* AggregationHandleCount<count_star, nullable_type>
     ::aggregateOnDistinctifyHashTableForSingle(
         const AggregationStateHashTableBase &distinctify_hash_table) const {
   DCHECK_EQ(count_star, false);
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
       AggregationHandleCount<count_star, nullable_type>,
       AggregationStateCount>(
           distinctify_hash_table);
@@ -206,13 +206,14 @@ template <bool count_star, bool nullable_type>
 void AggregationHandleCount<count_star, nullable_type>
     ::aggregateOnDistinctifyHashTableForGroupBy(
         const AggregationStateHashTableBase &distinctify_hash_table,
-        AggregationStateHashTableBase *aggregation_hash_table) const {
+        AggregationStateHashTableBase *aggregation_hash_table, int index) const {
   DCHECK_EQ(count_star, false);
   aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleCount<count_star, nullable_type>,
       AggregationStateFastHashTable>(
           distinctify_hash_table,
-          aggregation_hash_table);
+          aggregation_hash_table,
+          index);
 }
 
 template <bool count_star, bool nullable_type>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index ed21c41..a95cae5 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.hpp
@@ -135,15 +135,24 @@ class AggregationHandleCount : public AggregationConcreteHandle {
   }
 
   inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+     if (block_update) return;
      if (arguments.size())
          iterateUnaryInlFast(arguments.front(), byte_ptr);
      else
          iterateNullaryInlFast(byte_ptr);
   }
 
+  void BlockUpdate() override {
+     block_update = true;
+  }
+
+  void AllowUpdate() override {
+     block_update = false;
+  }
+
   void initPayload(uint8_t *byte_ptr) override {
-    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
-    *count_ptr = 0;
+     std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
+     *count_ptr = 0;
   }
 
   AggregationState* accumulateNullary(const std::size_t num_tuples) const override {
@@ -208,7 +217,7 @@ class AggregationHandleCount : public AggregationConcreteHandle {
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table) const override;
+      AggregationStateHashTableBase *aggregation_hash_table, int index) const override;
 
   void mergeGroupByHashTables(
       const AggregationStateHashTableBase &source_hash_table,
@@ -224,9 +233,11 @@ class AggregationHandleCount : public AggregationConcreteHandle {
   /**
    * @brief Constructor.
    **/
-  AggregationHandleCount() {
+  AggregationHandleCount() : block_update(false) {
   }
 
+  bool block_update;
+
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleCount);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleDistinct.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.cpp b/expressions/aggregation/AggregationHandleDistinct.cpp
index 2b9391a..a5fc095 100644
--- a/expressions/aggregation/AggregationHandleDistinct.cpp
+++ b/expressions/aggregation/AggregationHandleDistinct.cpp
@@ -72,7 +72,7 @@ ColumnVector* AggregationHandleDistinct::finalizeHashTable(
                                                const bool &dumb_placeholder) -> void {
     group_by_keys->emplace_back(std::move(group_by_key));
   };
-  static_cast<const AggregationStateHashTable<bool>&>(hash_table).forEachCompositeKey(&keys_retriever);
+  static_cast<const AggregationStateFastHashTable&>(hash_table).forEachCompositeKey(&keys_retriever);
 
   return nullptr;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleDistinct.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.hpp b/expressions/aggregation/AggregationHandleDistinct.hpp
index 0a3acb3..f6ef0c7 100644
--- a/expressions/aggregation/AggregationHandleDistinct.hpp
+++ b/expressions/aggregation/AggregationHandleDistinct.hpp
@@ -90,7 +90,7 @@ class AggregationHandleDistinct : public AggregationConcreteHandle {
 
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *groupby_hash_table) const override {
+      AggregationStateHashTableBase *groupby_hash_table, int index) const override {
     LOG(FATAL) << "AggregationHandleDistinct does not support "
                << "aggregateOnDistinctifyHashTableForGroupBy().";
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.cpp b/expressions/aggregation/AggregationHandleMax.cpp
index 2080a03..c11fcc8 100644
--- a/expressions/aggregation/AggregationHandleMax.cpp
+++ b/expressions/aggregation/AggregationHandleMax.cpp
@@ -39,7 +39,7 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleMax::AggregationHandleMax(const Type &type)
-    : type_(type) {
+    : type_(type), block_update(false) {
   fast_comparator_.reset(ComparisonFactory::GetComparison(ComparisonID::kGreater)
                          .makeUncheckedComparatorForTypes(type,
                                                           type.getNonNullableVersion()));
@@ -135,7 +135,7 @@ ColumnVector* AggregationHandleMax::finalizeHashTable(
 
 AggregationState* AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
       AggregationHandleMax,
       AggregationStateMax>(
           distinctify_hash_table);
@@ -143,14 +143,12 @@ AggregationState* AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle
 
 void AggregationHandleMax::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+    AggregationStateHashTableBase *aggregation_hash_table, int index) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleMax,
-      AggregationStateMax,
-      AggregationStateHashTable<AggregationStateMax>>(
+      AggregationStateFastHashTable>(
           distinctify_hash_table,
-          AggregationStateMax(type_),
-          aggregation_hash_table);
+          aggregation_hash_table, index);
 }
 
 void AggregationHandleMax::mergeGroupByHashTables(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index 3c06fc4..82d6ebb 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -113,9 +113,18 @@ class AggregationHandleMax : public AggregationConcreteHandle {
   }
 
   inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+    if (block_update) return;
     iterateUnaryInlFast(arguments.front(), byte_ptr);
   }
 
+  void BlockUpdate() override {
+      block_update = true;
+  }
+
+  void AllowUpdate() override {
+      block_update = false;
+  }
+
   void initPayload(uint8_t *byte_ptr) override {
     TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
     TypedValue t1 = (type_.getNullableVersion().makeNullValue());
@@ -175,7 +184,7 @@ class AggregationHandleMax : public AggregationConcreteHandle {
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table) const override;
+      AggregationStateHashTableBase *aggregation_hash_table, int index) const override;
 
   void mergeGroupByHashTables(
       const AggregationStateHashTableBase &source_hash_table,
@@ -221,6 +230,8 @@ class AggregationHandleMax : public AggregationConcreteHandle {
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 
+  bool block_update;
+
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMax);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.cpp b/expressions/aggregation/AggregationHandleMin.cpp
index 9d5be72..70d6c1b 100644
--- a/expressions/aggregation/AggregationHandleMin.cpp
+++ b/expressions/aggregation/AggregationHandleMin.cpp
@@ -39,7 +39,7 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleMin::AggregationHandleMin(const Type &type)
-    : type_(type) {
+    : type_(type), block_update(false) {
   fast_comparator_.reset(ComparisonFactory::GetComparison(ComparisonID::kLess)
                          .makeUncheckedComparatorForTypes(type,
                                                           type.getNonNullableVersion()));
@@ -136,7 +136,7 @@ ColumnVector* AggregationHandleMin::finalizeHashTable(
 
 AggregationState* AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
       AggregationHandleMin,
       AggregationStateMin>(
           distinctify_hash_table);
@@ -144,14 +144,12 @@ AggregationState* AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle
 
 void AggregationHandleMin::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+    AggregationStateHashTableBase *aggregation_hash_table, int index) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleMin,
-      AggregationStateMin,
-      AggregationStateHashTable<AggregationStateMin>>(
+      AggregationStateFastHashTable>(
           distinctify_hash_table,
-          AggregationStateMin(type_),
-          aggregation_hash_table);
+          aggregation_hash_table, index);
 }
 
 void AggregationHandleMin::mergeGroupByHashTables(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index 6329cd7..0f5e3a1 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -110,16 +110,25 @@ class AggregationHandleMin : public AggregationConcreteHandle {
     compareAndUpdate(state, value);
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) {
+  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) const {
       DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
       TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
       compareAndUpdateFast(min_ptr, value);
   }
 
   inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+    if (block_update) return;
     iterateUnaryInlFast(arguments.front(), byte_ptr);
   }
 
+  void BlockUpdate() override {
+      block_update = true;
+  }
+
+  void AllowUpdate() override {
+      block_update = false;
+  }
+
   void initPayload(uint8_t *byte_ptr) override {
     TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
     TypedValue t1 = (type_.getNullableVersion().makeNullValue());
@@ -178,7 +187,7 @@ class AggregationHandleMin : public AggregationConcreteHandle {
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table) const override;
+      AggregationStateHashTableBase *aggregation_hash_table, int index) const override;
 
   void mergeGroupByHashTables(
       const AggregationStateHashTableBase &source_hash_table,
@@ -223,6 +232,8 @@ class AggregationHandleMin : public AggregationConcreteHandle {
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 
+  bool block_update;
+
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMin);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 7a16605..534db30 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -43,7 +43,7 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleSum::AggregationHandleSum(const Type &type)
-    : argument_type_(type) {
+    : argument_type_(type), block_update(false) {
   // We sum Int as Long and Float as Double so that we have more headroom when
   // adding many values.
   TypeID type_precision_id;
@@ -184,7 +184,7 @@ ColumnVector* AggregationHandleSum::finalizeHashTable(
 
 AggregationState* AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
       AggregationHandleSum,
       AggregationStateSum>(
           distinctify_hash_table);
@@ -192,14 +192,13 @@ AggregationState* AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle
 
 void AggregationHandleSum::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+    AggregationStateHashTableBase *aggregation_hash_table, int index) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
       AggregationHandleSum,
-      AggregationStateSum,
-      AggregationStateHashTable<AggregationStateSum>>(
+      AggregationStateFastHashTable>(
           distinctify_hash_table,
-          blank_state_,
-          aggregation_hash_table);
+          aggregation_hash_table,
+          index);
 }
 
 void AggregationHandleSum::mergeGroupByHashTables(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index 79f8331..3a2252d 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -118,7 +118,7 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     state->null_ = false;
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) {
+  inline void iterateUnaryInlFast(const TypedValue &value, uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
     if (value.isNull()) return;
     TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
@@ -128,9 +128,18 @@ class AggregationHandleSum : public AggregationConcreteHandle {
   }
 
   inline void iterateInlFast(const std::vector<TypedValue> &arguments, uint8_t *byte_ptr) override {
+     if (block_update) return;
      iterateUnaryInlFast(arguments.front(), byte_ptr);
   }
 
+  void BlockUpdate() override {
+      block_update = true;
+  }
+
+  void AllowUpdate() override {
+      block_update = false;
+  }
+
   void initPayload(uint8_t *byte_ptr) override {
     TypedValue *sum_ptr = reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset);
     bool *null_ptr = reinterpret_cast<bool *>(byte_ptr + blank_state_.null_offset);
@@ -188,7 +197,7 @@ class AggregationHandleSum : public AggregationConcreteHandle {
    */
   void aggregateOnDistinctifyHashTableForGroupBy(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table) const override;
+      AggregationStateHashTableBase *aggregation_hash_table, int index) const override;
 
   void mergeGroupByHashTables(
       const AggregationStateHashTableBase &source_hash_table,
@@ -214,6 +223,8 @@ class AggregationHandleSum : public AggregationConcreteHandle {
   std::unique_ptr<UncheckedBinaryOperator> fast_operator_;
   std::unique_ptr<UncheckedBinaryOperator> merge_operator_;
 
+  bool block_update;
+
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleSum);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 7d6d179..833b707 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -105,7 +105,8 @@ AggregationOperationState::AggregationOperationState(
         new HashTablePool(estimated_num_entries,
                           hash_table_impl_type,
                           group_by_types,
-                          handles_.back(),
+                          {1},
+                          handles_,
                           storage_manager)));
   } else {
     // Set up each individual aggregate in this operation.
@@ -142,8 +143,11 @@ AggregationOperationState::AggregationOperationState(
                               group_by_types,
                               handles_.back().get(),
                               storage_manager)));*/
+         if (*is_distinct_it) {
+            handles_.back()->BlockUpdate();
+         }
          group_by_handles.emplace_back(handles_.back());
-         payload_sizes.emplace_back(handles_.back()->getPayloadSize());
+         payload_sizes.emplace_back(group_by_handles.back()->getPayloadSize());
 
       } else {
         // Aggregation without GROUP BY: create a single global state.
@@ -186,26 +190,26 @@ AggregationOperationState::AggregationOperationState(
                 estimated_num_entries,
                 storage_manager));*/
 
-std::vector<AggregationHandle *> local;
-local.emplace_back(handles_.back());
+        std::vector<AggregationHandle *> local;
+        // local.emplace_back(handles_.back());
+        local.clear();
         distinctify_hashtables_.emplace_back(
-AggregationStateFastHashTableFactory::CreateResizable(
+        AggregationStateFastHashTableFactory::CreateResizable(
                 *distinctify_hash_table_impl_types_it,
                 key_types,
                 estimated_num_entries,
                 {0},
                 local,
                 storage_manager));
-
         ++distinctify_hash_table_impl_types_it;
       } else {
         distinctify_hashtables_.emplace_back(nullptr);
       }
     }
 
-      if (!group_by_handles.empty()) {
-        // Aggregation with GROUP BY: create a HashTable pool for per-group states.
-        group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
+    if (!group_by_handles.empty()) {
+      // Aggregation with GROUP BY: create a HashTable pool for per-group states.
+      group_by_hashtable_pools_.emplace_back(std::unique_ptr<HashTablePool>(
             new HashTablePool(estimated_num_entries,
                               hash_table_impl_type,
                               group_by_types,
@@ -213,7 +217,7 @@ AggregationStateFastHashTableFactory::CreateResizable(
                               group_by_handles,
                               storage_manager)));
       }
-  }
+    }
 }
 
 AggregationOperationState* AggregationOperationState::ReconstructFromProto(
@@ -442,13 +446,15 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
                                distinctify_hashtables_[agg_idx].get(),
                                &reuse_matches,
                                &reuse_group_by_vectors);
-    } else {
-      // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
-      // directly into the (threadsafe) shared global HashTable for this
-      // aggregate.
-      DCHECK(group_by_hashtable_pools_[0] != nullptr);
-      AggregationStateHashTableBase *agg_hash_table = group_by_hashtable_pools_[0]->getHashTableFast();
-      DCHECK(agg_hash_table != nullptr);
+    }
+  }
+
+  // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
+  // directly into the (threadsafe) shared global HashTable for this
+  // aggregate.
+  DCHECK(group_by_hashtable_pools_[0] != nullptr);
+  AggregationStateHashTableBase *agg_hash_table = group_by_hashtable_pools_[0]->getHashTableFast();
+  DCHECK(agg_hash_table != nullptr);
  /*     block->aggregateGroupBy(*handles_[agg_idx],
                               arguments_[agg_idx],
                               group_by_list_,
@@ -456,16 +462,13 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
                               agg_hash_table,
                               &reuse_matches,
                               &reuse_group_by_vectors);*/
-      block->aggregateGroupByFast(arguments_,
+  block->aggregateGroupByFast(arguments_,
                               group_by_list_,
                               predicate_.get(),
                               agg_hash_table,
                               &reuse_matches,
                               &reuse_group_by_vectors);
-      group_by_hashtable_pools_[0]->returnHashTable(agg_hash_table);
-      break;
-    }
-  }
+  group_by_hashtable_pools_[0]->returnHashTable(agg_hash_table);
 }
 
 void AggregationOperationState::finalizeSingleState(InsertDestination *output_destination) {
@@ -541,9 +544,11 @@ void AggregationOperationState::finalizeHashTable(InsertDestination *output_dest
       DCHECK(hash_tables->back() != nullptr);
       AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
       DCHECK(agg_hash_table != nullptr);
+      handles_[agg_idx]->AllowUpdate();
       handles_[agg_idx]->aggregateOnDistinctifyHashTableForGroupBy(
           *distinctify_hashtables_[agg_idx],
-          agg_hash_table);
+          agg_hash_table,
+          agg_idx);
     }
 
     auto *hash_tables = group_by_hashtable_pools_[0]->getAllHashTables();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index b6f2ef9..79a5b87 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -631,7 +631,6 @@ target_link_libraries(quickstep_storage_EvictionPolicy
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_FastHashTable
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationHandleAvg
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_StorageBlob

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
index 12e447f..cba039a 100644
--- a/storage/FastHashTable.hpp
+++ b/storage/FastHashTable.hpp
@@ -35,7 +35,6 @@
 #include "storage/TupleReference.hpp"
 #include "storage/ValueAccessor.hpp"
 #include "storage/ValueAccessorUtil.hpp"
-#include "expressions/aggregation/AggregationHandleAvg.hpp"
 #include "threading/SpinSharedMutex.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
@@ -434,6 +433,11 @@ class FastHashTable : public HashTableBase<resizable,
                           const 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);
+
   bool upsertCompositeKeyNewFast(const std::vector<TypedValue> &key,
                           const uint8_t *init_value_ptr,
                           const uint8_t *source_state);
@@ -1851,6 +1855,41 @@ 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, int index) {
+  DEBUG_ASSERT(!allow_duplicate_keys);
+  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);
+        if (value != nullptr) {
+          (*functor)(value+payload_offsets_[index]);
+          return true;
+        }
+      }
+      resize(0, variable_size);
+    }
+  } else {
+    uint8_t *value = upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
+    if (value == nullptr) {
+      return false;
+    } else {
+      (*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>
     ::upsertCompositeKeyNewFast(const std::vector<TypedValue> &key,
                          const std::uint8_t *init_value_ptr,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1665593e/storage/FastSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastSeparateChainingHashTable.hpp b/storage/FastSeparateChainingHashTable.hpp
index 64c4979..756d6e5 100644
--- a/storage/FastSeparateChainingHashTable.hpp
+++ b/storage/FastSeparateChainingHashTable.hpp
@@ -308,8 +308,11 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
           key_manager_(this->key_types_, kValueOffset + this->total_payload_size_),
           bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
   init_payload_ = static_cast<std::uint8_t *>(calloc(this->total_payload_size_, 1));
-  for (auto handle : handles)
-      handle->initPayload(init_payload_);
+  int k = 0;
+  for (auto handle : handles) {
+      handle->initPayload(init_payload_+this->payload_offsets_[k]);
+      k++;
+  }
   // Bucket size always rounds up to the alignment requirement of the atomic
   // size_t "next" pointer at the front or a ValueT, whichever is larger.
   //
@@ -437,8 +440,7 @@ FastSeparateChainingHashTable<resizable, serializable, force_key_copy, allow_dup
               true),
           kBucketAlignment(alignof(std::atomic<std::size_t>) < alignof(uint8_t) ? alignof(uint8_t)
                                                   : alignof(std::atomic<std::size_t>)),
-          kValueOffset((((sizeof(std::atomic<std::size_t>) + sizeof(std::size_t) - 1) /
-                                           alignof(uint8_t)) + 1) * alignof(uint8_t)),
+          kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
           key_manager_(this->key_types_, kValueOffset + sizeof(uint8_t)),
           bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
   // Bucket size always rounds up to the alignment requirement of the atomic
@@ -1046,7 +1048,6 @@ uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy,
     else
         memcpy(value, init_value_ptr, this->total_payload_size_);
 
-
   // Update the previous chain pointer to point to the new bucket.
   pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);
 
@@ -1168,10 +1169,11 @@ uint8_t* FastSeparateChainingHashTable<resizable, serializable, force_key_copy,
 //  uint8_t *value;
 //  value = static_cast<unsigned char*>(bucket) + kValueOffset;
     uint8_t *value = static_cast<unsigned char*>(bucket) + kValueOffset;
-    if (init_value_ptr == nullptr)
+    if (init_value_ptr == nullptr) {
         memcpy(value, init_payload_, this->total_payload_size_);
-    else
+    } else {
         memcpy(value, init_value_ptr, this->total_payload_size_);
+    }
 
   // Update the previous chaing pointer to point to the new bucket.
   pending_chain_ptr->store(pending_chain_ptr_finish_value, std::memory_order_release);



[08/28] incubator-quickstep git commit: Fixed cyclic dependencies. Removed Aggregation unit test. Other minor changes.

Posted by hb...@apache.org.
Fixed cyclic dependencies. Removed Aggregation unit test. Other minor changes.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/c510d24b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/c510d24b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/c510d24b

Branch: refs/heads/partitioned-aggregation
Commit: c510d24b089ce790f51988ed62350cf68751c6d9
Parents: 1665593
Author: rathijit <ra...@node-2.aggregation.quickstep-pg0.wisc.cloudlab.us>
Authored: Sun Aug 14 02:59:40 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:56:07 2016 -0500

----------------------------------------------------------------------
 expressions/aggregation/CMakeLists.txt    |  2 +-
 storage/CMakeLists.txt                    |  2 --
 storage/FastHashTable.hpp                 |  4 +---
 storage/FastSeparateChainingHashTable.hpp |  3 ---
 storage/HashTable.hpp                     | 10 ----------
 storage/HashTableBase.hpp                 | 18 +++++++++++++++++-
 storage/StorageBlock.cpp                  | 10 ++++------
 7 files changed, 23 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c510d24b/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index 98222df..9de6833 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -321,4 +321,4 @@ target_link_libraries(AggregationHandle_tests
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID)
-add_test(AggregationHandle_tests AggregationHandle_tests)
+#add_test(AggregationHandle_tests AggregationHandle_tests)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c510d24b/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 79a5b87..f05cc46 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -631,7 +631,6 @@ target_link_libraries(quickstep_storage_EvictionPolicy
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_FastHashTable
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_StorageBlob
                       quickstep_storage_StorageBlockInfo
@@ -968,7 +967,6 @@ target_link_libraries(quickstep_storage_StorageBlock
                       quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
                       quickstep_storage_CompressedPackedRowStoreTupleStorageSubBlock
                       quickstep_storage_CountedReference
-                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_IndexSubBlock
                       quickstep_storage_InsertDestinationInterface

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c510d24b/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
index cba039a..e7887ab 100644
--- a/storage/FastHashTable.hpp
+++ b/storage/FastHashTable.hpp
@@ -42,7 +42,6 @@
 #include "utility/BloomFilter.hpp"
 #include "utility/HashPair.hpp"
 #include "utility/Macros.hpp"
-#include "storage/HashTable.hpp"
 
 namespace quickstep {
 
@@ -561,7 +560,7 @@ class FastHashTable : public HashTableBase<resizable,
       const std::vector<std::vector<attribute_id>> &argument,
       ValueAccessor *accessor,
       const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys);
+      const bool check_for_null_keys) override;
 
   /**
    * @brief Determine the number of entries (key-value pairs) contained in this
@@ -1322,7 +1321,6 @@ class FastHashTable : public HashTableBase<resizable,
   // Used only when resizable is false:
   void *hash_table_memory_;
   const std::size_t hash_table_memory_size_;
-virtual size_t get_buckets_allocated() const {return 0;}
 
  private:
   // Assign '*key_vector' with the attribute values specified by 'key_attr_ids'

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c510d24b/storage/FastSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastSeparateChainingHashTable.hpp b/storage/FastSeparateChainingHashTable.hpp
index 756d6e5..49cea5b 100644
--- a/storage/FastSeparateChainingHashTable.hpp
+++ b/storage/FastSeparateChainingHashTable.hpp
@@ -170,9 +170,6 @@ class FastSeparateChainingHashTable : public FastHashTable<resizable,
   bool preallocateForBulkInsert(const std::size_t total_entries,
                                 const std::size_t total_variable_key_size,
                                 HashTablePreallocationState *prealloc_state) override;
-
-  size_t get_buckets_allocated() const override {return header_->buckets_allocated;}
-
  private:
   struct Header {
     std::size_t num_slots;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c510d24b/storage/HashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTable.hpp b/storage/HashTable.hpp
index 9fa41a2..f2dcb03 100644
--- a/storage/HashTable.hpp
+++ b/storage/HashTable.hpp
@@ -49,16 +49,6 @@ namespace quickstep {
  */
 
 /**
- * @brief Codes which indicate the result of a call to put() or
- *        putCompositeKey().
- **/
-enum class HashTablePutResult {
-  kOK = 0,
-  kDuplicateKey,
-  kOutOfSpace
-};
-
-/**
  * @brief Base class for hash table.
  *
  * This class is templated so that the core hash-table logic can be reused in

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c510d24b/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index 7eadae9..5526164 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -21,8 +21,10 @@
 #define QUICKSTEP_STORAGE_HASH_TABLE_BASE_HPP_
 
 #include <cstddef>
+#include <vector>
 
 #include "utility/Macros.hpp"
+#include "ValueAccessor.hpp"
 
 namespace quickstep {
 
@@ -52,6 +54,16 @@ struct HashTablePreallocationState {
 };
 
 /**
+ * @brief Codes which indicate the result of a call to put() or
+ *        putCompositeKey().
+ **/
+enum class HashTablePutResult {
+  kOK = 0,
+  kDuplicateKey,
+  kOutOfSpace
+};
+
+/**
  * @brief An ultra-minimal base class that HashTables with different ValueT
  *        parameters inherit from. This allows for a bit more type-safety than
  *        just passing around void* pointers (although casting will still be
@@ -66,7 +78,11 @@ class HashTableBase {
  public:
   virtual ~HashTableBase() {
   }
-virtual size_t get_buckets_allocated() const {return 0;}
+  virtual bool upsertValueAccessorCompositeKeyFast(
+      const std::vector<std::vector<attribute_id>> &argument,
+      ValueAccessor *accessor,
+      const std::vector<attribute_id> &key_attr_ids,
+      const bool check_for_null_keys) {return false;}
  protected:
   HashTableBase() {
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c510d24b/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 50732fd..8ff18b5 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -38,7 +38,6 @@
 #include "storage/CompressedPackedRowStoreTupleStorageSubBlock.hpp"
 #include "storage/CountedReference.hpp"
 #include "storage/HashTableBase.hpp"
-#include "storage/FastHashTable.hpp"
 #include "storage/IndexSubBlock.hpp"
 #include "storage/InsertDestinationInterface.hpp"
 #include "storage/PackedRowStoreTupleStorageSubBlock.hpp"
@@ -573,11 +572,10 @@ void StorageBlock::aggregateGroupByFast(
      }
   }
 
-  static_cast<AggregationStateFastHashTable *>(hash_table)->upsertValueAccessorCompositeKeyFast(
-      argument_ids,
-      &temp_result,
-      key_ids,
-      true);
+  hash_table->upsertValueAccessorCompositeKeyFast(argument_ids,
+                                                  &temp_result,
+                                                  key_ids,
+                                                  true);
 }
 
 


[07/28] incubator-quickstep git commit: reinterpreted byte to SpinMutex before locking. This removes the need to have an additional function accepting a pointer in the SpinMutex class

Posted by hb...@apache.org.
reinterpreted byte to SpinMutex before locking. This removes the need to have an additional function accepting a pointer in the SpinMutex class


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/ae789f98
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/ae789f98
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/ae789f98

Branch: refs/heads/partitioned-aggregation
Commit: ae789f981760de8d59b1d63e99e507a22210b6b1
Parents: 935d5d0
Author: rathijit <ra...@node-2.aggregation.quickstep-pg0.wisc.cloudlab.us>
Authored: Sun Aug 14 19:54:50 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Sep 20 12:56:07 2016 -0500

----------------------------------------------------------------------
 storage/FastHashTable.hpp | 12 ++++++------
 threading/SpinMutex.hpp   |  2 --
 2 files changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ae789f98/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
index 8d8d82b..c659a20 100644
--- a/storage/FastHashTable.hpp
+++ b/storage/FastHashTable.hpp
@@ -1900,7 +1900,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
         SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
         uint8_t *value = upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
         if (value != nullptr) {
-            SpinMutex lock(value);
+            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]);
             }
@@ -1914,7 +1914,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
     if (value == nullptr) {
       return false;
     } else {
-      SpinMutex lock(value);
+      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]);
       }
@@ -2017,7 +2017,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
               continuing = true;
               break;
             } else {
-              SpinMutex lock(value);
+              SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
               for (unsigned int k = 0; k < handles_.size(); ++k) {
                   local.clear();
                   if (argument_ids[k].size()) {
@@ -2044,7 +2044,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
         if (value == nullptr) {
           return false;
         } else {
-          SpinMutex lock(value);
+          SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
           for (unsigned int k = 0; k < handles_.size(); ++k) {
               local.clear();
               if (argument_ids[k].size()) {
@@ -2170,7 +2170,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
               continuing = true;
               break;
             } else {
-              SpinMutex lock(value);
+              SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
               for (unsigned int k = 0; k < handles_.size(); ++k) {
                   local.clear();
                   if (argument_ids[k].size()) {
@@ -2201,7 +2201,7 @@ bool FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys
         if (value == nullptr) {
           return false;
         } else {
-          SpinMutex lock(value);
+          SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
           for (unsigned int k = 0; k < handles_.size(); ++k) {
               local.clear();
               if (argument_ids[k].size()) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ae789f98/threading/SpinMutex.hpp
----------------------------------------------------------------------
diff --git a/threading/SpinMutex.hpp b/threading/SpinMutex.hpp
index 106ef13..5ed1405 100644
--- a/threading/SpinMutex.hpp
+++ b/threading/SpinMutex.hpp
@@ -44,8 +44,6 @@ class SpinMutex {
   SpinMutex() : locked_(false) {
   }
 
-  explicit SpinMutex(uint8_t *ptr): locked_(*ptr) {}
-
   /**
    * @note This call does NOT yield when contended. SpinMutex is intended
    *       mainly for cases where locks are held briefly and it is better to