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

[1/6] incubator-quickstep git commit: Updates

Repository: incubator-quickstep
Updated Branches:
  refs/heads/untyped-agg 140069b95 -> 9ccd5a311


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/HashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTablePool.hpp b/storage/HashTablePool.hpp
index faa0abc..124b4f5 100644
--- a/storage/HashTablePool.hpp
+++ b/storage/HashTablePool.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_STORAGE_HASH_TABLE_POOL_HPP_
 #define QUICKSTEP_STORAGE_HASH_TABLE_POOL_HPP_
 
+#include <algorithm>
 #include <chrono>
 #include <memory>
 #include <utility>
@@ -27,9 +28,7 @@
 
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
-#include "storage/AggregationHashTable.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/AggregationStateHashTable.hpp"
 #include "threading/SpinMutex.hpp"
 #include "utility/Macros.hpp"
 #include "utility/StringUtil.hpp"
@@ -57,36 +56,6 @@ class HashTablePool {
   /**
    * @brief Constructor.
    *
-   * @param estimated_num_entries The maximum number of entries in a hash table.
-   * @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.
-   **/
-  HashTablePool(const std::size_t estimated_num_entries,
-                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)),
-        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)) {}
-
-  /**
-   * @brief Constructor.
-   *
    * @note This constructor is relevant for HashTables specialized for
    *       aggregation.
    *
@@ -94,52 +63,29 @@ class HashTablePool {
    * @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 payload_sizes The sizes in bytes for the AggregationStates for the
-   *        respective AggregationHandles.
    * @param handles The AggregationHandles in this query.
    * @param storage_manager A pointer to the 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.
    *
-   * @return A hash table pointer.
-   **/
-  AggregationStateHashTableBase* getHashTable() {
-    {
-      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 createNewHashTable();
-  }
-
-  /**
-   * @brief Check out a hash table for insertion.
-   *
    * @note This method is relevant for specialized (for aggregation)
    *       hash table implementation.
    *
    * @return A hash table pointer.
    **/
-  AggregationStateHashTableBase* getHashTableFast() {
+  AggregationStateHashTableBase* getHashTable() {
     {
       SpinMutexLock lock(mutex_);
       if (!hash_tables_.empty()) {
@@ -150,7 +96,7 @@ class HashTablePool {
         return ret_hash_table.release();
       }
     }
-    return createNewHashTableFast();
+    return createNewHashTable();
   }
 
   /**
@@ -174,63 +120,38 @@ class HashTablePool {
    * @param All the hash tables in the pool.
    *
    **/
-  const std::vector<std::unique_ptr<AggregationStateHashTableBase>>*
-      getAllHashTables() {
+  std::vector<std::unique_ptr<AggregationStateHashTableBase>>* getAllHashTables() {
     return &hash_tables_;
   }
 
-  AggregationHashTableBase* createNewThreadPrivateHashTable() {
-    agg_hash_tables_.emplace_back(
-        std::unique_ptr<AggregationHashTableBase>(
-            new ThreadPrivateAggregationHashTable(
-               group_by_types_,
-               estimated_num_entries_,
-               handles_,
-               storage_manager_)));
-    return agg_hash_tables_.back().get();
+  // TODO: temporary
+  const HashTableImplType& hash_table_impl_type() const {
+    return hash_table_impl_type_;
   }
 
  private:
   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_);
+    return new ThreadPrivateAggregationStateHashTable(
+        group_by_types_,
+        estimated_num_entries_,
+        handles_,
+        storage_manager_);
   }
 
   inline std::size_t reduceEstimatedCardinality(
       const std::size_t original_estimate) const {
-    if (original_estimate < kEstimateReductionFactor) {
-      return original_estimate;
-    } else {
-      DCHECK_GT(kEstimateReductionFactor, 0u);
-      return original_estimate / kEstimateReductionFactor;
-    }
+    DCHECK_GT(kEstimateReductionFactor, 0u);
+    return std::max(16uL,  original_estimate / kEstimateReductionFactor);
   }
 
   static constexpr std::size_t kEstimateReductionFactor = 100;
 
   std::vector<std::unique_ptr<AggregationStateHashTableBase>> hash_tables_;
-  std::vector<std::unique_ptr<AggregationHashTableBase>> agg_hash_tables_;
 
   const std::size_t estimated_num_entries_;
   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_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/HashTableUntypedKeyManager.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableUntypedKeyManager.hpp b/storage/HashTableUntypedKeyManager.hpp
index dffb116..ee32412 100644
--- a/storage/HashTableUntypedKeyManager.hpp
+++ b/storage/HashTableUntypedKeyManager.hpp
@@ -46,10 +46,12 @@ class HashTableUntypedKeyManager {
   HashTableUntypedKeyManager(const std::vector<const Type *> &key_types,
                              const std::size_t key_start_in_bucket)
       : key_types_(key_types),
-        key_hasher_(MakeUntypedKeyHashFunctor(key_types)),
-        key_equality_checker_(MakeUntypedKeyEqualityFunctor(key_types)),
+        key_hasher_(MakeUntypedHashFunctor(key_types)),
+        key_equality_checker_(MakeUntypedEqualityFunctor(key_types)),
         key_start_in_bucket_(key_start_in_bucket),
-        fixed_key_size_(0) {
+        fixed_key_size_(0),
+        is_key_nullable_(false) {
+
     for (const Type *key_type : key_types) {
       key_sizes_.emplace_back(key_type->maximumByteLength());
       key_offsets_.emplace_back(fixed_key_size_);
@@ -57,6 +59,7 @@ class HashTableUntypedKeyManager {
 
       DCHECK(!key_type->isVariableLength());
       fixed_key_size_ += key_type->maximumByteLength();
+      is_key_nullable_ |= key_type->isNullable();
     }
   }
 
@@ -68,6 +71,10 @@ class HashTableUntypedKeyManager {
     return fixed_key_size_;
   }
 
+  inline bool isKeyNullable() const {
+    return is_key_nullable_;
+  }
+
   inline std::size_t hashUntypedKey(const void *key) const {
     return key_hasher_(key);
   }
@@ -77,8 +84,14 @@ class HashTableUntypedKeyManager {
     return key_equality_checker_(key, getUntypedKeyComponent(bucket));
   }
 
-  inline void writeUntypedKeyToBucket(const void *key, void *bucket) {
-    InlineMemcpy(getUntypedKeyComponent(bucket), key, fixed_key_size_);
+  inline void writeUntypedKeyToBucket(const void *key,
+                                      void *bucket) {
+    copyUntypedKey(getUntypedKeyComponent(bucket), key);
+  }
+
+  inline void copyUntypedKey(void *destination,
+                             const void *key) const {
+    InlineMemcpy(destination, key, fixed_key_size_);
   }
 
   template <typename ValueAccessorT>
@@ -125,21 +138,24 @@ class HashTableUntypedKeyManager {
     return static_cast<char *>(bucket) + key_offsets_in_bucket_[component_id];
   }
 
-  inline const UntypedKeyHashFunctor& getUntypedKeyHashFunctor() const {
+  inline const UntypedHashFunctor& getUntypedKeyHashFunctor() const {
     return key_hasher_;
   }
 
-  inline const UntypedKeyEqualityFunctor& getUntypedKeyEqualityFunctor() const {
+  inline const UntypedEqualityFunctor& getUntypedKeyEqualityFunctor() const {
     return key_equality_checker_;
   }
 
  private:
   const std::vector<const Type*> &key_types_;
-  const UntypedKeyHashFunctor key_hasher_;
-  const UntypedKeyEqualityFunctor key_equality_checker_;
+  const UntypedHashFunctor key_hasher_;
+  const UntypedEqualityFunctor key_equality_checker_;
+
 
   std::size_t key_start_in_bucket_;
   std::size_t fixed_key_size_;
+  bool is_key_nullable_;
+
   std::vector<std::size_t> key_sizes_;
   std::vector<std::size_t> key_offsets_;
   std::vector<std::size_t> key_offsets_in_bucket_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 5e83453..5b77631 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -33,6 +33,7 @@
 #include "query_execution/QueryExecutionMessages.pb.h"
 #include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/QueryExecutionUtil.hpp"
+#include "storage/AggregationResultIterator.hpp"
 #include "storage/InsertDestination.pb.h"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -221,6 +222,23 @@ void InsertDestination::bulkInsertTuples(ValueAccessor *accessor, bool always_ma
   });
 }
 
+void InsertDestination::bulkInsertAggregationResults(
+    AggregationResultIterator *results) {
+  results->beginIteration();
+  while (!results->iterationFinished()) {
+    MutableBlockReference output_block = this->getBlockForInsertion();
+    // FIXME(chasseur): Deal with TupleTooLargeForBlock exception.
+    if (output_block->bulkInsertAggregationResults(results) == 0) {
+      // output_block is full.
+      this->returnBlock(std::move(output_block), true);
+    } else {
+      // Bulk insert into output_block was successful. output_block
+      // will be rebuilt when there won't be any more insertions to it.
+      this->returnBlock(std::move(output_block), !results->iterationFinished());
+    }
+  }
+}
+
 void InsertDestination::bulkInsertTuplesWithRemappedAttributes(
     const std::vector<attribute_id> &attribute_map,
     ValueAccessor *accessor,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index 408e76b..ca02da3 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -52,6 +52,7 @@ namespace tmb { class MessageBus; }
 
 namespace quickstep {
 
+class AggregationResultIterator;
 class StorageManager;
 class ValueAccessor;
 
@@ -145,6 +146,8 @@ class InsertDestination : public InsertDestinationInterface {
 
   void insertTupleInBatch(const Tuple &tuple) override;
 
+  void bulkInsertAggregationResults(AggregationResultIterator *results);
+
   void bulkInsertTuples(ValueAccessor *accessor, bool always_mark_full = false) override;
 
   void bulkInsertTuplesWithRemappedAttributes(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/PackedRowStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreTupleStorageSubBlock.cpp b/storage/PackedRowStoreTupleStorageSubBlock.cpp
index 0ad4a4c..f09fabb 100644
--- a/storage/PackedRowStoreTupleStorageSubBlock.cpp
+++ b/storage/PackedRowStoreTupleStorageSubBlock.cpp
@@ -26,6 +26,7 @@
 #include "catalog/CatalogAttribute.hpp"
 #include "catalog/CatalogRelationSchema.hpp"
 #include "catalog/CatalogTypedefs.hpp"
+#include "storage/AggregationResultIterator.hpp"
 #include "storage/PackedRowStoreValueAccessor.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageBlockLayout.pb.h"
@@ -204,6 +205,35 @@ tuple_id PackedRowStoreTupleStorageSubBlock::bulkInsertTuples(ValueAccessor *acc
   return header_->num_tuples - original_num_tuples;
 }
 
+tuple_id PackedRowStoreTupleStorageSubBlock::bulkInsertAggregationResults(
+    AggregationResultIterator *results) {
+  const tuple_id original_num_tuples = header_->num_tuples;
+  char *dest_addr = static_cast<char*>(tuple_storage_)
+                        + header_->num_tuples * relation_.getFixedByteLength();
+
+  const std::size_t attrs_total_size = relation_.getMaximumByteLength();
+  const std::size_t key_size = results->getKeySize();
+  const std::size_t results_size = results->getResultsSize();
+  DEBUG_ASSERT(attrs_total_size == key_size + results_size);
+
+  if (results_size  == 0) {
+    while (this->hasSpaceToInsert<false>(1) && results->next()) {
+      results->writeKeyTo(dest_addr);
+      dest_addr += attrs_total_size;
+      ++(header_->num_tuples);
+    }
+  } else {
+    while (this->hasSpaceToInsert<false>(1) && results->next()) {
+      results->writeKeyTo(dest_addr);
+      results->writeResultsTo(dest_addr + key_size);
+      dest_addr += attrs_total_size;
+      ++(header_->num_tuples);
+    }
+  }
+
+  return header_->num_tuples - original_num_tuples;
+}
+
 tuple_id PackedRowStoreTupleStorageSubBlock::bulkInsertTuplesWithRemappedAttributes(
     const std::vector<attribute_id> &attribute_map,
     ValueAccessor *accessor) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/PackedRowStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreTupleStorageSubBlock.hpp b/storage/PackedRowStoreTupleStorageSubBlock.hpp
index 0cd41f4..9e9fe40 100644
--- a/storage/PackedRowStoreTupleStorageSubBlock.hpp
+++ b/storage/PackedRowStoreTupleStorageSubBlock.hpp
@@ -33,6 +33,7 @@
 
 namespace quickstep {
 
+class AggregationResultIterator;
 class CatalogRelationSchema;
 class ComparisonPredicate;
 class TupleStorageSubBlockDescription;
@@ -144,6 +145,8 @@ class PackedRowStoreTupleStorageSubBlock: public TupleStorageSubBlock {
 
   tuple_id bulkInsertTuples(ValueAccessor *accessor) override;
 
+  tuple_id bulkInsertAggregationResults(AggregationResultIterator *results) override;
+
   tuple_id bulkInsertTuplesWithRemappedAttributes(
       const std::vector<attribute_id> &attribute_map,
       ValueAccessor *accessor) override;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 4fce131..abb17f1 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -31,6 +31,7 @@
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
+#include "storage/AggregationResultIterator.hpp"
 #include "storage/BasicColumnStoreTupleStorageSubBlock.hpp"
 #include "storage/BloomFilterIndexSubBlock.hpp"
 #include "storage/CSBTreeIndexSubBlock.hpp"
@@ -60,6 +61,7 @@
 #include "types/containers/Tuple.hpp"
 #include "types/operations/comparisons/ComparisonUtil.hpp"
 #include "utility/Macros.hpp"
+#include "utility/ScopedBuffer.hpp"
 
 #include "glog/logging.h"
 
@@ -267,6 +269,19 @@ tuple_id StorageBlock::bulkInsertTuples(ValueAccessor *accessor) {
   return num_inserted;
 }
 
+tuple_id StorageBlock::bulkInsertAggregationResults(AggregationResultIterator *results) {
+  const tuple_id num_inserted = tuple_store_->bulkInsertAggregationResults(results);
+  if (num_inserted != 0) {
+    invalidateAllIndexes();
+    dirty_ = true;
+  } else if (tuple_store_->isEmpty()) {
+    if (!results->iterationFinished()) {
+      throw TupleTooLargeForBlock(0);
+    }
+  }
+  return num_inserted;
+}
+
 tuple_id StorageBlock::bulkInsertTuplesWithRemappedAttributes(
     const std::vector<attribute_id> &attribute_map,
     ValueAccessor *accessor) {
@@ -385,7 +400,7 @@ void StorageBlock::selectSimple(const std::vector<attribute_id> &selection,
                                                       accessor.get());
 }
 
-AggregationState* StorageBlock::aggregate(
+ScopedBuffer StorageBlock::aggregate(
     const AggregationHandle &handle,
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
     const std::vector<attribute_id> *arguments_as_attributes,
@@ -419,7 +434,6 @@ void StorageBlock::aggregateGroupBy(
     const std::vector<std::unique_ptr<const Scalar>> &group_by,
     const Predicate *predicate,
     AggregationStateHashTableBase *hash_table,
-    AggregationHashTableBase *tp_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)
@@ -492,115 +506,9 @@ void StorageBlock::aggregateGroupBy(
      }
   }
 
-  hash_table->upsertValueAccessorCompositeKeyFast(argument_ids,
-                                                  &temp_result,
-                                                  key_ids,
-                                                  true);
-
-  // NOTE: experiments here
-  if (tp_hash_table != nullptr) {
-    if (key_ids.size() == 1) {
-      tp_hash_table->upsertValueAccessor(&temp_result,
-                                         key_ids.front(),
-                                         argument_ids,
-                                         true);
-    } else {
-      tp_hash_table->upsertValueAccessorCompositeKey(&temp_result,
-                                                     key_ids,
-                                                     argument_ids,
-                                                     true);
-    }
-    tp_hash_table->print();
-  }
-}
-
-
-void StorageBlock::aggregateDistinct(
-    const AggregationHandle &handle,
-    const std::vector<std::unique_ptr<const Scalar>> &arguments,
-    const std::vector<attribute_id> *arguments_as_attributes,
-    const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const Predicate *predicate,
-    AggregationStateHashTableBase *distinctify_hash_table,
-    std::unique_ptr<TupleIdSequence> *reuse_matches,
-    std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
-  DCHECK_GT(arguments.size(), 0u)
-      << "Called aggregateDistinct() with zero argument expressions";
-  DCHECK((group_by.size() == 0 || reuse_group_by_vectors != nullptr));
-
-  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());
-    }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-    // If all the arguments to this aggregate are plain relation attributes,
-    // aggregate directly on a ValueAccessor from this block to avoid a copy.
-    if ((arguments_as_attributes != nullptr) && (!arguments_as_attributes->empty())) {
-      DCHECK_EQ(arguments.size(), arguments_as_attributes->size())
-          << "Mismatch between number of arguments and number of attribute_ids";
-      DCHECK_EQ(group_by.size(), 0u);
-      handle.insertValueAccessorIntoDistinctifyHashTable(
-          accessor.get(), *arguments_as_attributes, distinctify_hash_table);
-      return;
-    }
-#endif
-
-    SubBlocksReference sub_blocks_ref(*tuple_store_,
-                                      indices_,
-                                      indices_consistent_);
-    attribute_id attr_id = 0;
-
-    if (!group_by.empty()) {
-      // 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));
-      key_ids.push_back(attr_id++);
-    }
-  }
-
-  handle.insertValueAccessorIntoDistinctifyHashTable(
-      &temp_result, key_ids, distinctify_hash_table);
+  hash_table->upsertValueAccessorCompositeKey(&temp_result,
+                                              key_ids,
+                                              argument_ids);
 }
 
 // TODO(chasseur): Vectorization for updates.
@@ -1302,14 +1210,17 @@ std::unordered_map<attribute_id, TypedValue>* StorageBlock::generateUpdatedValue
   return update_map;
 }
 
-AggregationState* StorageBlock::aggregateHelperColumnVector(
+ScopedBuffer StorageBlock::aggregateHelperColumnVector(
     const AggregationHandle &handle,
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
     const TupleIdSequence *matches) const {
+  ScopedBuffer state = handle.createInitialState();
   if (arguments.empty()) {
     // Special case. This is a nullary aggregate (i.e. COUNT(*)).
-    return handle.accumulateNullary(matches == nullptr ? tuple_store_->numTuples()
-                                                       : matches->size());
+    handle.accumulateNullary(
+        state.get(),
+        matches == nullptr ? tuple_store_->numTuples()
+                           : matches->size());
   } else {
     // Set up a ValueAccessor that will be used when materializing argument
     // values below (possibly filtered based on the '*matches' to a filter
@@ -1332,12 +1243,13 @@ AggregationState* StorageBlock::aggregateHelperColumnVector(
     }
 
     // Have the AggregationHandle actually do the aggregation.
-    return handle.accumulateColumnVectors(column_vectors);
+    handle.accumulateColumnVectors(state.get(), column_vectors);
   }
+  return state;
 }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* StorageBlock::aggregateHelperValueAccessor(
+ScopedBuffer StorageBlock::aggregateHelperValueAccessor(
     const AggregationHandle &handle,
     const std::vector<attribute_id> &argument_ids,
     const TupleIdSequence *matches) const {
@@ -1351,9 +1263,11 @@ AggregationState* StorageBlock::aggregateHelperValueAccessor(
   }
 
   // Have the AggregationHandle actually do the aggregation.
-  return handle.accumulateValueAccessor(
-      accessor.get(),
-      argument_ids);
+  ScopedBuffer state = handle.createInitialState();
+  handle.accumulateValueAccessor(state.get(),
+                                 accessor.get(),
+                                 argument_ids);
+  return state;
 }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 567af33..08d81d0 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -36,11 +36,11 @@
 #include "storage/TupleStorageSubBlock.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
+#include "utility/ScopedBuffer.hpp"
 
 namespace quickstep {
 
 class AggregationHandle;
-class AggregationState;
 class CatalogRelationSchema;
 class ColumnVector;
 class InsertDestinationInterface;
@@ -282,6 +282,8 @@ class StorageBlock : public StorageBlockBase {
    **/
   tuple_id bulkInsertTuples(ValueAccessor *accessor);
 
+  tuple_id bulkInsertAggregationResults(AggregationResultIterator *results);
+
   /**
    * @brief Insert as many tuples as possible from a ValueAccessor (all of the
    *        tuples accessible or as many as will fit in this StorageBlock) as a
@@ -408,7 +410,7 @@ class StorageBlock : public StorageBlockBase {
    *         AggregationHandle::finalize() can be used to generate a final
    *         result.
    **/
-  AggregationState* aggregate(
+  ScopedBuffer aggregate(
       const AggregationHandle &handle,
       const std::vector<std::unique_ptr<const Scalar>> &arguments,
       const std::vector<attribute_id> *arguments_as_attributes,
@@ -463,55 +465,10 @@ class StorageBlock : public StorageBlockBase {
       const std::vector<std::unique_ptr<const Scalar>> &group_by,
       const Predicate *predicate,
       AggregationStateHashTableBase *hash_table,
-      AggregationHashTableBase *tp_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.
-   *
-   * This is the first step for DISTINCT aggregation. It populates the distinctify
-   * hash table so that arguments are distinctified within each GROUP BY group.
-   * Later, a second-round aggregation on the distinctify hash table will be
-   * performed to actually compute the aggregated result for each GROUP BY group.
-   *
-   * @param handle Aggregation handle to compute aggregates with.
-   * @param arguments The arguments to the aggregation function as Scalars.
-   * @param arguments_as_attributes If non-NULL, indicates a valid attribute_id
-   *        for each of the elements in arguments, and is used to elide a copy.
-   *        Has no effect if NULL, or if VECTOR_COPY_ELISION_LEVEL is NONE.
-   * @param group_by The list of GROUP BY attributes/expressions.
-   * @param predicate A predicate for selection. \c nullptr indicates that all
-   *        tuples should be aggregated on.
-   * @param distinctify_hash_table Hash table to store the arguments and GROUP
-   *        BY expressions together as hash table key and a bool constant \c true
-   *        as hash table value. (So the hash table actually serves as a hash set.)
-   * @param reuse_matches This parameter is used to store and reuse tuple-id
-   *        sequence of matches pre-computed in an earlier invocations of
-   *        aggregateGroupBy(). \c reuse_matches is never \c nullptr for ease of
-   *        use.  Current invocation of aggregateGroupBy() will reuse
-   *        TupleIdSequence if passed, otherwise computes a TupleIdSequence based
-   *        on \c predicate and stores in \c reuse_matches. We use
-   *        std::unique_ptr for each of use, since the caller will not have to
-   *        selective free.
-   * @param reuse_group_by_vectors This parameter is used to store and reuse
-   *        GROUP BY attribute vectors pre-computed in an earlier invocation of
-   *        aggregateGroupBy(). \c reuse_group_by_vectors is never \c nullptr
-   *        for ease of use. Current invocation of aggregateGroupBy() will reuse
-   *        ColumnVectors if non-empty, otherwise computes ColumnVectors based
-   *        on \c group_by and stores them in \c reuse_group_by_vectors.
-   */
-  void aggregateDistinct(const AggregationHandle &handle,
-                         const std::vector<std::unique_ptr<const Scalar>> &arguments,
-                         const std::vector<attribute_id> *arguments_as_attributes,
-                         const std::vector<std::unique_ptr<const Scalar>> &group_by,
-                         const Predicate *predicate,
-                         AggregationStateHashTableBase *distinctify_hash_table,
-                         std::unique_ptr<TupleIdSequence> *reuse_matches,
-                         std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
-
-  /**
    * @brief Perform an UPDATE query over the tuples in this StorageBlock.
    * @warning In some edge cases, calling this method may cause IndexSubBlocks
    *          in this block to become inconsistent (the TupleStorageSubBlock
@@ -635,13 +592,13 @@ class StorageBlock : public StorageBlockBase {
       const tuple_id tuple,
       const std::unordered_map<attribute_id, std::unique_ptr<const Scalar>> &assignments) const;
 
-  AggregationState* aggregateHelperColumnVector(
+  ScopedBuffer aggregateHelperColumnVector(
       const AggregationHandle &handle,
       const std::vector<std::unique_ptr<const Scalar>> &arguments,
       const TupleIdSequence *matches) const;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* aggregateHelperValueAccessor(
+  ScopedBuffer aggregateHelperValueAccessor(
       const AggregationHandle &handle,
       const std::vector<attribute_id> &argument_ids,
       const TupleIdSequence *matches) const;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/TupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/TupleStorageSubBlock.hpp b/storage/TupleStorageSubBlock.hpp
index aed6eea..3733165 100644
--- a/storage/TupleStorageSubBlock.hpp
+++ b/storage/TupleStorageSubBlock.hpp
@@ -30,10 +30,13 @@
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 class CatalogRelationSchema;
 class ComparisonPredicate;
+class AggregationResultIterator;
 class Tuple;
 class TupleStorageSubBlockDescription;
 class ValueAccessor;
@@ -245,6 +248,10 @@ class TupleStorageSubBlock {
    **/
   virtual tuple_id bulkInsertTuples(ValueAccessor *accessor) = 0;
 
+  virtual tuple_id bulkInsertAggregationResults(AggregationResultIterator *results) {
+    LOG(FATAL) << "Not implemented\n";
+  }
+
   /**
    * @brief Insert as many tuples as possible from a ValueAccessor (all of the
    *        tuples accessible or as many as will fit in this

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/CharType.hpp
----------------------------------------------------------------------
diff --git a/types/CharType.hpp b/types/CharType.hpp
index c7321f4..8ed903c 100644
--- a/types/CharType.hpp
+++ b/types/CharType.hpp
@@ -22,6 +22,7 @@
 
 #include <cstddef>
 #include <cstdio>
+#include <cstring>
 #include <string>
 
 #include "types/Type.hpp"
@@ -132,6 +133,17 @@ class CharType : public AsciiStringSuperType {
   TypedValue coerceValue(const TypedValue &original_value,
                          const Type &original_type) const override;
 
+  inline std::size_t getHash(const void *value_ptr) const {
+    const char *char_ptr = static_cast<const char *>(value_ptr);
+    return util::Hash(char_ptr, strnlen(char_ptr, length_));
+  }
+
+  inline void copyValue(void *dst, const void *src) const {
+    std::strncpy(static_cast<char *>(dst),
+                 static_cast<const char *>(src),
+                 length_);
+  }
+
  private:
   CharType(const std::size_t length, const bool nullable)
       : AsciiStringSuperType(kChar, nullable, length, length, length) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/DateType.hpp
----------------------------------------------------------------------
diff --git a/types/DateType.hpp b/types/DateType.hpp
index b83848e..e4f75a7 100644
--- a/types/DateType.hpp
+++ b/types/DateType.hpp
@@ -22,6 +22,7 @@
 
 #include <cstddef>
 #include <cstdio>
+#include <cstring>
 #include <string>
 
 #include "types/DatetimeLit.hpp"
@@ -126,8 +127,18 @@ class DateType : public Type {
     return true;
   }
 
+  void makeZeroValue(void *value_ptr) const override {
+    std::memset(value_ptr, 0, sizeof(cpptype));
+  }
+
   inline std::size_t getHash(const void *value_ptr) const {
-    return DateLit::getHash(reinterpret_cast<const DateLit *>(value_ptr));
+    return DateLit::getHash(static_cast<const DateLit *>(value_ptr));
+  }
+
+  inline void copyValue(void *dst, const void *src) const {
+    return DateLit::copyValue(
+        static_cast<DateLit *>(dst),
+        static_cast<const DateLit *>(src));
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/DatetimeIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeIntervalType.hpp b/types/DatetimeIntervalType.hpp
index 71e2842..d8b7bef 100644
--- a/types/DatetimeIntervalType.hpp
+++ b/types/DatetimeIntervalType.hpp
@@ -23,6 +23,7 @@
 #include <cstddef>
 #include <cstdint>
 #include <cstdio>
+#include <cstring>
 #include <string>
 
 #include "types/IntervalLit.hpp"
@@ -118,8 +119,17 @@ class DatetimeIntervalType : public Type {
     return true;
   }
 
+  void makeZeroValue(void *value_ptr) const override {
+    std::memset(value_ptr, 0, sizeof(cpptype));
+  }
+
   inline std::size_t getHash(const void *value_ptr) const {
-    return *reinterpret_cast<const DatetimeIntervalLit::cpptype *>(value_ptr);
+    return *static_cast<const DatetimeIntervalLit::cpptype *>(value_ptr);
+  }
+
+  inline void copyValue(void *dst, const void *src) const {
+    *static_cast<DatetimeIntervalLit::cpptype *>(dst) =
+        *static_cast<const DatetimeIntervalLit::cpptype *>(src);
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/DatetimeLit.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeLit.hpp b/types/DatetimeLit.hpp
index c528b1f..d43b4d8 100644
--- a/types/DatetimeLit.hpp
+++ b/types/DatetimeLit.hpp
@@ -105,6 +105,10 @@ struct DateLit {
         | static_cast<std::uint64_t>(date_lit->month) << 8
         | static_cast<std::uint64_t>(date_lit->day);
   }
+
+  static inline void copyValue(DateLit *dst, const DateLit *src) {
+    *dst = *src;
+  }
 };
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/DatetimeType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeType.hpp b/types/DatetimeType.hpp
index 974a277..b869aaf 100644
--- a/types/DatetimeType.hpp
+++ b/types/DatetimeType.hpp
@@ -22,6 +22,7 @@
 
 #include <cstddef>
 #include <cstdio>
+#include <cstring>
 #include <string>
 
 #include "types/DatetimeLit.hpp"
@@ -134,8 +135,17 @@ class DatetimeType : public Type {
     return true;
   }
 
+  void makeZeroValue(void *value_ptr) const override {
+    std::memset(value_ptr, 0, sizeof(cpptype));
+  }
+
   inline std::size_t getHash(const void *value_ptr) const {
-    return *reinterpret_cast<const DatetimeLit::cpptype *>(value_ptr);
+    return *static_cast<const DatetimeLit::cpptype *>(value_ptr);
+  }
+
+  inline void copyValue(void *dst, const void *src) const {
+    *static_cast<DatetimeLit::cpptype *>(dst) =
+        *static_cast<const DatetimeLit::cpptype *>(src);
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/NullType.hpp
----------------------------------------------------------------------
diff --git a/types/NullType.hpp b/types/NullType.hpp
index 309c826..90a8852 100644
--- a/types/NullType.hpp
+++ b/types/NullType.hpp
@@ -22,6 +22,7 @@
 
 #include <cstddef>
 #include <cstdio>
+#include <cstring>
 #include <string>
 
 #include "types/Type.hpp"
@@ -107,6 +108,9 @@ class NullType : public Type {
     return true;
   }
 
+  void makeZeroValue(void *value_ptr) const override {
+  }
+
  private:
   // NOTE(chasseur): NullType requires 0 bytes of inherent storage. It does,
   // however, require a bit in NULL bitmaps.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/NumericSuperType.hpp
----------------------------------------------------------------------
diff --git a/types/NumericSuperType.hpp b/types/NumericSuperType.hpp
index 76dfe33..37fd437 100644
--- a/types/NumericSuperType.hpp
+++ b/types/NumericSuperType.hpp
@@ -21,6 +21,7 @@
 #define QUICKSTEP_TYPES_NUMERIC_SUPER_TYPE_HPP_
 
 #include <cstddef>
+#include <cstring>
 
 #include "types/NullCoercibilityCheckMacro.hpp"
 #include "types/Type.hpp"
@@ -60,8 +61,16 @@ class NumericSuperType : public Type {
     return true;
   }
 
+  void makeZeroValue(void *value_ptr) const override {
+    std::memset(value_ptr, 0, sizeof(cpptype));
+  }
+
   inline std::size_t getHash(const void *value_ptr) const {
-    return *reinterpret_cast<const CppType *>(value_ptr);
+    return *static_cast<const CppType *>(value_ptr);
+  }
+
+  inline void copyValue(void *dst, const void *src) const {
+    *static_cast<CppType *>(dst) = *static_cast<const CppType *>(src);
   }
 
  protected:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/Type.hpp
----------------------------------------------------------------------
diff --git a/types/Type.hpp b/types/Type.hpp
index 041b6b8..13c0644 100644
--- a/types/Type.hpp
+++ b/types/Type.hpp
@@ -452,6 +452,8 @@ class Type {
 
   virtual bool canCheckEqualityWithMemcmp() const = 0;
 
+  virtual void makeZeroValue(void *value_ptr) const = 0;
+
  protected:
   Type(const SuperTypeID super_type_id,
        const TypeID type_id,
@@ -495,9 +497,10 @@ class AsciiStringSuperType : public Type {
     return false;
   }
 
-  inline std::size_t getHash(const void *value_ptr) const {
-    const char *char_ptr = reinterpret_cast<const char *>(value_ptr);
-    return util::Hash(char_ptr, strnlen(char_ptr, length_));
+  void makeZeroValue(void *value_ptr) const override {
+    if (maximum_byte_length_ > 0) {
+      *static_cast<char *>(value_ptr) = 0;
+    }
   }
 
  protected:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/TypeFunctors.cpp
----------------------------------------------------------------------
diff --git a/types/TypeFunctors.cpp b/types/TypeFunctors.cpp
index 84fcad2..bf49df8 100644
--- a/types/TypeFunctors.cpp
+++ b/types/TypeFunctors.cpp
@@ -73,8 +73,8 @@ typename Maker::FunctorType MakeHelper(const Type *type) {
   }
 }
 
-struct UntypedKeyHashFunctorMaker {
-  typedef UntypedKeyHashFunctor FunctorType;
+struct UntypedHashFunctorMaker {
+  typedef UntypedHashFunctor FunctorType;
 
   template <typename TypeName>
   static FunctorType MakeFunctor(const TypeName *type) {
@@ -84,22 +84,22 @@ struct UntypedKeyHashFunctorMaker {
   }
 };
 
-UntypedKeyHashFunctor MakeUntypedKeyHashFunctor(const Type *type) {
-  return MakeHelper<UntypedKeyHashFunctorMaker>(type);
+UntypedHashFunctor MakeUntypedHashFunctor(const Type *type) {
+  return MakeHelper<UntypedHashFunctorMaker>(type);
 }
 
-UntypedKeyHashFunctor MakeUntypedKeyHashFunctor(const std::vector<const Type *> &types) {
+UntypedHashFunctor MakeUntypedHashFunctor(const std::vector<const Type *> &types) {
   DCHECK_GE(types.size(), 1u);
 
   if (types.size() == 1u) {
-    return MakeUntypedKeyHashFunctor(types.front());
+    return MakeUntypedHashFunctor(types.front());
   }
 
-  std::vector<UntypedKeyHashFunctor> hashers;
+  std::vector<UntypedHashFunctor> hashers;
   std::vector<std::size_t> offsets;
   std::size_t accum_offset = 0;
   for (const Type *type : types) {
-    hashers.emplace_back(MakeUntypedKeyHashFunctor(type));
+    hashers.emplace_back(MakeUntypedHashFunctor(type));
     offsets.emplace_back(accum_offset);
     accum_offset += type->isVariableLength() ? sizeof(void *)
                                              : type->maximumByteLength();
@@ -115,44 +115,44 @@ UntypedKeyHashFunctor MakeUntypedKeyHashFunctor(const std::vector<const Type *>
   };
 }
 
-struct UntypedKeyEqualityFunctorMaker {
-  typedef UntypedKeyEqualityFunctor FunctorType;
+struct UntypedEqualityFunctorMaker {
+  typedef UntypedEqualityFunctor FunctorType;
 
   template <typename TypeName>
-  static UntypedKeyEqualityFunctor MakeFunctor(const TypeName *type) {
+  static UntypedEqualityFunctor MakeFunctor(const TypeName *type) {
     return STLLiteralEqual<typename TypeName::cpptype>();
   }
 };
 
 template <>
-UntypedKeyEqualityFunctor UntypedKeyEqualityFunctorMaker::MakeFunctor(
+UntypedEqualityFunctor UntypedEqualityFunctorMaker::MakeFunctor(
     const CharType *type) {
   return STLCharEqual(type->getStringLength());
 }
 
 template <>
-UntypedKeyEqualityFunctor UntypedKeyEqualityFunctorMaker::MakeFunctor(
+UntypedEqualityFunctor UntypedEqualityFunctorMaker::MakeFunctor(
     const VarCharType *type) {
   return STLVarCharEqual();
 }
 
-UntypedKeyEqualityFunctor MakeUntypedKeyEqualityFunctor(const Type *type) {
-  return MakeHelper<UntypedKeyEqualityFunctorMaker>(type);
+UntypedEqualityFunctor MakeUntypedEqualityFunctor(const Type *type) {
+  return MakeHelper<UntypedEqualityFunctorMaker>(type);
 }
 
-UntypedKeyEqualityFunctor MakeUntypedKeyEqualityFunctor(const std::vector<const Type *> &types) {
+UntypedEqualityFunctor MakeUntypedEqualityFunctor(const std::vector<const Type *> &types) {
   DCHECK_GE(types.size(), 1u);
 
   if (types.size() == 1u) {
-    return MakeUntypedKeyEqualityFunctor(types.front());
+    return MakeUntypedEqualityFunctor(types.front());
   }
 
-  std::vector<UntypedKeyEqualityFunctor> equality_checkers;
+  std::vector<UntypedEqualityFunctor> equality_checkers;
   std::vector<std::size_t> offsets;
   std::size_t accum_offset = 0;
   bool can_check_equality_with_memcmp = true;
   for (const Type *type : types) {
-    equality_checkers.emplace_back(MakeUntypedKeyEqualityFunctor(type));
+    equality_checkers.emplace_back(MakeUntypedEqualityFunctor(type));
     offsets.emplace_back(accum_offset);
     accum_offset += type->isVariableLength() ? sizeof(void *)
                                              : type->maximumByteLength();
@@ -160,7 +160,7 @@ UntypedKeyEqualityFunctor MakeUntypedKeyEqualityFunctor(const std::vector<const
   }
   if (can_check_equality_with_memcmp) {
     return [accum_offset](const void *left, const void *right) -> bool {
-      return memcmp(left, right, accum_offset);
+      return !std::memcmp(left, right, accum_offset);
     };
   } else {
     return [offsets, equality_checkers](const void *left, const void *right) -> bool {
@@ -175,5 +175,19 @@ UntypedKeyEqualityFunctor MakeUntypedKeyEqualityFunctor(const std::vector<const
   }
 }
 
+struct UntypedCopyFunctorMaker {
+  typedef UntypedCopyFunctor FunctorType;
+
+  template <typename TypeName>
+  static FunctorType MakeFunctor(const TypeName *type) {
+    return [type](void *dst, const void *src) -> void {
+      type->copyValue(dst, src);
+    };
+  }
+};
+
+UntypedCopyFunctor MakeUntypedCopyFunctor(const Type *type) {
+  return MakeHelper<UntypedCopyFunctorMaker>(type);
+}
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/TypeFunctors.hpp
----------------------------------------------------------------------
diff --git a/types/TypeFunctors.hpp b/types/TypeFunctors.hpp
index a8466c5..55f63e1 100644
--- a/types/TypeFunctors.hpp
+++ b/types/TypeFunctors.hpp
@@ -32,13 +32,16 @@ class Type;
  *  @{
  */
 
-typedef std::function<std::size_t (const void *)> UntypedKeyHashFunctor;
-UntypedKeyHashFunctor MakeUntypedKeyHashFunctor(const Type *type);
-UntypedKeyHashFunctor MakeUntypedKeyHashFunctor(const std::vector<const Type *> &types);
+typedef std::function<std::size_t (const void *)> UntypedHashFunctor;
+UntypedHashFunctor MakeUntypedHashFunctor(const Type *type);
+UntypedHashFunctor MakeUntypedHashFunctor(const std::vector<const Type *> &types);
 
-typedef std::function<bool (const void *, const void *)> UntypedKeyEqualityFunctor;
-UntypedKeyEqualityFunctor MakeUntypedKeyEqualityFunctor(const Type *type);
-UntypedKeyEqualityFunctor MakeUntypedKeyEqualityFunctor(const std::vector<const Type *> &types);
+typedef std::function<bool (const void *, const void *)> UntypedEqualityFunctor;
+UntypedEqualityFunctor MakeUntypedEqualityFunctor(const Type *type);
+UntypedEqualityFunctor MakeUntypedEqualityFunctor(const std::vector<const Type *> &types);
+
+typedef std::function<void (void *, const void *)> UntypedCopyFunctor;
+UntypedCopyFunctor MakeUntypedCopyFunctor(const Type *type);
 
 /** @} */
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/VarCharType.hpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.hpp b/types/VarCharType.hpp
index bb50e92..d35039f 100644
--- a/types/VarCharType.hpp
+++ b/types/VarCharType.hpp
@@ -22,6 +22,7 @@
 
 #include <cstddef>
 #include <cstdio>
+#include <cstring>
 #include <string>
 
 #include "types/Type.hpp"
@@ -30,6 +31,8 @@
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 /** \addtogroup Types
@@ -135,6 +138,15 @@ class VarCharType : public AsciiStringSuperType {
   TypedValue coerceValue(const TypedValue &original_value,
                          const Type &original_type) const override;
 
+  inline std::size_t getHash(const void *value_ptr) const {
+    LOG(FATAL) << "Not implemented";
+    return 0;
+  }
+
+  inline void copyValue(void *dst, const void *src) const {
+    LOG(FATAL) << "Not implemented";
+  }
+
  private:
   VarCharType(const std::size_t length, const bool nullable)
       : AsciiStringSuperType(kVarChar, nullable, 1, length + 1, length) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/YearMonthIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.hpp b/types/YearMonthIntervalType.hpp
index 6c54b10..c0f4723 100644
--- a/types/YearMonthIntervalType.hpp
+++ b/types/YearMonthIntervalType.hpp
@@ -22,6 +22,7 @@
 
 #include <cstddef>
 #include <cstdio>
+#include <cstring>
 #include <string>
 
 #include "types/IntervalLit.hpp"
@@ -117,8 +118,17 @@ class YearMonthIntervalType : public Type {
     return true;
   }
 
+  void makeZeroValue(void *value_ptr) const override {
+    std::memset(value_ptr, 0, sizeof(cpptype));
+  }
+
   inline std::size_t getHash(const void *value_ptr) const {
-    return *reinterpret_cast<const YearMonthIntervalLit::cpptype *>(value_ptr);
+    return *static_cast<const YearMonthIntervalLit::cpptype *>(value_ptr);
+  }
+
+  inline void copyValue(void *dst, const void *src) const {
+    *static_cast<YearMonthIntervalLit::cpptype *>(dst) =
+        *static_cast<const YearMonthIntervalLit::cpptype *>(src);
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp b/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
index 7224a0c..9238d4e 100644
--- a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
+++ b/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
@@ -172,6 +172,26 @@ class ArithmeticUncheckedBinaryOperator : public UncheckedBinaryOperator {
   ArithmeticUncheckedBinaryOperator(const ArithmeticUncheckedBinaryOperator &orig) = default;
   ~ArithmeticUncheckedBinaryOperator() = default;
 
+  BinaryOperatorFunctor getFunctor() const override {
+    OpFunctor<LeftCppType, RightCppType> op_functor;
+    return [op_functor](void *result, const void *left, const void *right) {
+      *static_cast<typename ResultType::cpptype *>(result) =
+          op_functor(
+              *static_cast<const LeftCppType *>(left),
+              *static_cast<const RightCppType *>(right));
+    };
+  }
+
+  BinaryOperatorMergeFunctor getMergeFunctor() const override {
+    OpFunctor<LeftCppType, RightCppType> op_functor;
+    return [op_functor](void *left, const void *right) {
+      *static_cast<typename ResultType::cpptype *>(left) =
+          op_functor(
+              *static_cast<const LeftCppType *>(left),
+              *static_cast<const RightCppType *>(right));
+    };
+  }
+
   inline TypedValue applyToTypedValues(const TypedValue &left,
                                        const TypedValue &right) const override {
     return applyToTypedValuesInl(left, right);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/types/operations/binary_operations/BinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperation.hpp b/types/operations/binary_operations/BinaryOperation.hpp
index 585a1c6..49cc342 100644
--- a/types/operations/binary_operations/BinaryOperation.hpp
+++ b/types/operations/binary_operations/BinaryOperation.hpp
@@ -21,6 +21,7 @@
 #define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_HPP_
 
 #include <cstddef>
+#include <functional>
 #include <string>
 #include <type_traits>
 #include <utility>
@@ -44,6 +45,9 @@ class ValueAccessor;
  *  @{
  */
 
+typedef std::function<void (void *, const void *, const void*)> BinaryOperatorFunctor;
+typedef std::function<void (void *, const void *)> BinaryOperatorMergeFunctor;
+
 /**
  * @brief A binary operator which can be quickly applied to data items WITHOUT
  *        checking their types.
@@ -56,6 +60,9 @@ class UncheckedBinaryOperator {
   virtual ~UncheckedBinaryOperator() {
   }
 
+  virtual BinaryOperatorFunctor getFunctor() const = 0;
+  virtual BinaryOperatorMergeFunctor getMergeFunctor() const = 0;
+
   /**
    * @brief Apply to two TypedValues without type-checking.
    *


[5/6] incubator-quickstep git commit: Updates

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.cpp b/expressions/aggregation/AggregationHandleMax.cpp
index c2d571b..d40ae9f 100644
--- a/expressions/aggregation/AggregationHandleMax.cpp
+++ b/expressions/aggregation/AggregationHandleMax.cpp
@@ -38,100 +38,100 @@ 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()));
-}
-
-AggregationStateHashTableBase* AggregationHandleMax::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<AggregationStateMax>::CreateResizable(
-      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
-}
-
-AggregationState* AggregationHandleMax::accumulateColumnVectors(
-    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
-  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()));
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleMax::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  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()));
-}
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-void AggregationHandleMax::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(1u, argument_ids.size())
-      << "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);
-
-  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);
-  }
-}
-
-ColumnVector* AggregationHandleMax::finalizeHashTable(
-    const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  return finalizeHashTableHelperFast<AggregationHandleMax,
-                                     AggregationStateFastHashTable>(
-      type_.getNullableVersion(), hash_table, group_by_keys, index);
-}
-
-AggregationState*
-AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
-    const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleMax,
-      AggregationStateMax>(distinctify_hash_table);
-}
-
-void AggregationHandleMax::aggregateOnDistinctifyHashTableForGroupBy(
-    const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleMax,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
-}
+AggregationHandleMax::AggregationHandleMax(const Type &type) {}
+//    : 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::size_t estimated_num_groups,
+//    StorageManager *storage_manager) const {
+//  return AggregationStateHashTableFactory<AggregationStateMax>::CreateResizable(
+//      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
+//}
+//
+//AggregationState* AggregationHandleMax::accumulateColumnVectors(
+//    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
+//  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()));
+//}
+//
+//#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+//AggregationState* AggregationHandleMax::accumulateValueAccessor(
+//    ValueAccessor *accessor,
+//    const std::vector<attribute_id> &accessor_ids) const {
+//  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()));
+//}
+//#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+//
+//void AggregationHandleMax::aggregateValueAccessorIntoHashTable(
+//    ValueAccessor *accessor,
+//    const std::vector<attribute_id> &argument_ids,
+//    const std::vector<attribute_id> &group_by_key_ids,
+//    AggregationStateHashTableBase *hash_table) const {
+//  DCHECK_EQ(1u, argument_ids.size())
+//      << "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);
+//
+//  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);
+//  }
+//}
+//
+//ColumnVector* AggregationHandleMax::finalizeHashTable(
+//    const AggregationStateHashTableBase &hash_table,
+//    std::vector<std::vector<TypedValue>> *group_by_keys,
+//    int index) const {
+//  return finalizeHashTableHelperFast<AggregationHandleMax,
+//                                     AggregationStateFastHashTable>(
+//      type_.getNullableVersion(), hash_table, group_by_keys, index);
+//}
+//
+//AggregationState*
+//AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
+//    const AggregationStateHashTableBase &distinctify_hash_table) const {
+//  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
+//      AggregationHandleMax,
+//      AggregationStateMax>(distinctify_hash_table);
+//}
+//
+//void AggregationHandleMax::aggregateOnDistinctifyHashTableForGroupBy(
+//    const AggregationStateHashTableBase &distinctify_hash_table,
+//    AggregationStateHashTableBase *aggregation_hash_table,
+//    std::size_t index) const {
+//  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
+//      AggregationHandleMax,
+//      AggregationStateFastHashTable>(
+//      distinctify_hash_table, aggregation_hash_table, index);
+//}
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index 5fb9f44..effc38f 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -26,9 +26,7 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
 #include "storage/HashTableBase.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
@@ -49,149 +47,12 @@ class ValueAccessor;
  */
 
 /**
- * @brief Aggregation state for max.
- */
-class AggregationStateMax : public AggregationState {
- public:
-  /**
-   * @brief Copy constructor (ignores mutex).
-   */
-  AggregationStateMax(const AggregationStateMax &orig) : max_(orig.max_) {}
-
-  /**
-   * @brief Destructor.
-   */
-  ~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()) {}
-
-  explicit AggregationStateMax(TypedValue &&value) : max_(std::move(value)) {}
-
-  TypedValue max_;
-  SpinMutex mutex_;
-};
-
-/**
  * @brief An aggregationhandle for max.
  **/
-class AggregationHandleMax : public AggregationConcreteHandle {
+class AggregationHandleMax : public AggregationHandle {
  public:
   ~AggregationHandleMax() override {}
 
-  AggregationState* createInitialState() const override {
-    return new AggregationStateMax(type_);
-  }
-
-  AggregationStateHashTableBase* 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 override;
-
-  /**
-   * @brief Iterate with max aggregation state.
-   */
-  inline void iterateUnaryInl(AggregationStateMax *state,
-                              const TypedValue &value) const {
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
-    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 updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
-
-  void blockUpdate() override { block_update_ = true; }
-
-  void allowUpdate() override { block_update_ = false; }
-
-  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;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_ids) const override;
-#endif
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  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_);
-  }
-
-  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 {
-    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,
-      int index) const override;
-
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for MAX aggregation.
-   */
-  AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
-
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        for MAX aggregation.
-   */
-  void aggregateOnDistinctifyHashTableForGroupBy(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const override;
-
-  std::size_t getPayloadSize() const override { return sizeof(TypedValue); }
-
  private:
   friend class AggregateFunctionMax;
 
@@ -202,37 +63,8 @@ class AggregationHandleMax : public AggregationConcreteHandle {
    **/
   explicit AggregationHandleMax(const Type &type);
 
-  /**
-   * @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 {
-    // 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_)) {
-      state->max_ = value;
-    }
-  }
-
-  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_;
-
-  bool block_update_;
+//  const Type &type_;
+//  std::unique_ptr<UncheckedComparator> fast_comparator_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMax);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.cpp b/expressions/aggregation/AggregationHandleMin.cpp
index a07f299..4765c93 100644
--- a/expressions/aggregation/AggregationHandleMin.cpp
+++ b/expressions/aggregation/AggregationHandleMin.cpp
@@ -38,101 +38,101 @@ 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()));
-}
-
-AggregationStateHashTableBase* AggregationHandleMin::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<AggregationStateMin>::CreateResizable(
-      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
-}
-
-AggregationState* AggregationHandleMin::accumulateColumnVectors(
-    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
-  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()));
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleMin::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  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()));
-}
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-void AggregationHandleMin::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(1u, argument_ids.size())
-      << "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);
-
-  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);
-
-  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,
-    int index) const {
-  return finalizeHashTableHelperFast<AggregationHandleMin,
-                                     AggregationStateFastHashTable>(
-      type_.getNonNullableVersion(), hash_table, group_by_keys, index);
-}
-
-AggregationState*
-AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
-    const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleMin,
-      AggregationStateMin>(distinctify_hash_table);
-}
-
-void AggregationHandleMin::aggregateOnDistinctifyHashTableForGroupBy(
-    const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleMin,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
-}
+AggregationHandleMin::AggregationHandleMin(const Type &type) {}
+//    : 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::size_t estimated_num_groups,
+//    StorageManager *storage_manager) const {
+//  return AggregationStateHashTableFactory<AggregationStateMin>::CreateResizable(
+//      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
+//}
+//
+//AggregationState* AggregationHandleMin::accumulateColumnVectors(
+//    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
+//  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()));
+//}
+//
+//#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+//AggregationState* AggregationHandleMin::accumulateValueAccessor(
+//    ValueAccessor *accessor,
+//    const std::vector<attribute_id> &accessor_ids) const {
+//  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()));
+//}
+//#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+//
+//void AggregationHandleMin::aggregateValueAccessorIntoHashTable(
+//    ValueAccessor *accessor,
+//    const std::vector<attribute_id> &argument_ids,
+//    const std::vector<attribute_id> &group_by_key_ids,
+//    AggregationStateHashTableBase *hash_table) const {
+//  DCHECK_EQ(1u, argument_ids.size())
+//      << "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);
+//
+//  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);
+//
+//  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,
+//    int index) const {
+//  return finalizeHashTableHelperFast<AggregationHandleMin,
+//                                     AggregationStateFastHashTable>(
+//      type_.getNonNullableVersion(), hash_table, group_by_keys, index);
+//}
+//
+//AggregationState*
+//AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
+//    const AggregationStateHashTableBase &distinctify_hash_table) const {
+//  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
+//      AggregationHandleMin,
+//      AggregationStateMin>(distinctify_hash_table);
+//}
+//
+//void AggregationHandleMin::aggregateOnDistinctifyHashTableForGroupBy(
+//    const AggregationStateHashTableBase &distinctify_hash_table,
+//    AggregationStateHashTableBase *aggregation_hash_table,
+//    std::size_t index) const {
+//  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
+//      AggregationHandleMin,
+//      AggregationStateFastHashTable>(
+//      distinctify_hash_table, aggregation_hash_table, index);
+//}
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index 173911d..64fddea 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -26,11 +26,8 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
 #include "storage/HashTableBase.hpp"
-#include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
@@ -49,151 +46,12 @@ class ValueAccessor;
  */
 
 /**
- * @brief Aggregation state for min.
- */
-class AggregationStateMin : public AggregationState {
- public:
-  /**
-   * @brief Copy constructor (ignores mutex).
-   */
-  AggregationStateMin(const AggregationStateMin &orig) : min_(orig.min_) {}
-
-  /**
-   * @brief Destructor.
-   */
-  ~AggregationStateMin() override {}
-
-  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;
-
-  explicit AggregationStateMin(const Type &type)
-      : min_(type.getNullableVersion().makeNullValue()) {}
-
-  explicit AggregationStateMin(TypedValue &&value) : min_(std::move(value)) {}
-
-  TypedValue min_;
-  SpinMutex mutex_;
-};
-
-/**
  * @brief An aggregationhandle for min.
  **/
-class AggregationHandleMin : public AggregationConcreteHandle {
+class AggregationHandleMin : public AggregationHandle {
  public:
   ~AggregationHandleMin() override {}
 
-  AggregationState* createInitialState() const override {
-    return new AggregationStateMin(type_);
-  }
-
-  AggregationStateHashTableBase* 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 override;
-
-  /**
-   * @brief Iterate with min aggregation state.
-   */
-  inline void iterateUnaryInl(AggregationStateMin *state,
-                              const TypedValue &value) const {
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
-    compareAndUpdate(state, 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 updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
-
-  void blockUpdate() override { block_update_ = true; }
-
-  void allowUpdate() override { block_update_ = false; }
-
-  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;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_ids) const override;
-#endif
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  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 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 {
-    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,
-      int index) const override;
-
-  /**
-   * @brief Implementation of
-   * AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for MIN aggregation.
-   */
-  AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
-
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        for MIN aggregation.
-   */
-  void aggregateOnDistinctifyHashTableForGroupBy(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const override;
-
-  std::size_t getPayloadSize() const override { return sizeof(TypedValue); }
-
  private:
   friend class AggregateFunctionMin;
 
@@ -204,36 +62,8 @@ class AggregationHandleMin : public AggregationConcreteHandle {
    **/
   explicit AggregationHandleMin(const Type &type);
 
-  /**
-   * @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 {
-    if (value.isNull()) return;
-
-    SpinMutexLock lock(state->mutex_);
-    if (state->min_.isNull() ||
-        fast_comparator_->compareTypedValues(value, state->min_)) {
-      state->min_ = value;
-    }
-  }
-
-  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_;
-
-  bool block_update_;
+//  const Type &type_;
+//  std::unique_ptr<UncheckedComparator> fast_comparator_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMin);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 642d88d..4e77ed0 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -20,6 +20,7 @@
 #include "expressions/aggregation/AggregationHandleSum.hpp"
 
 #include <cstddef>
+#include <cstring>
 #include <memory>
 #include <utility>
 #include <vector>
@@ -35,6 +36,7 @@
 #include "types/operations/binary_operations/BinaryOperation.hpp"
 #include "types/operations/binary_operations/BinaryOperationFactory.hpp"
 #include "types/operations/binary_operations/BinaryOperationID.hpp"
+#include "types/TypeFunctors.hpp"
 
 #include "glog/logging.h"
 
@@ -42,12 +44,11 @@ namespace quickstep {
 
 class StorageManager;
 
-AggregationHandleSum::AggregationHandleSum(const Type &type)
-    : argument_type_(type), block_update_(false) {
+AggregationHandleSum::AggregationHandleSum(const Type &argument_type) {
   // We sum Int as Long and Float as Double so that we have more headroom when
   // adding many values.
   TypeID type_precision_id;
-  switch (argument_type_.getTypeID()) {
+  switch (argument_type.getTypeID()) {
     case kInt:
     case kLong:
       type_precision_id = kLong;
@@ -57,134 +58,57 @@ AggregationHandleSum::AggregationHandleSum(const Type &type)
       type_precision_id = kDouble;
       break;
     default:
-      type_precision_id = type.getTypeID();
+      type_precision_id = argument_type.getTypeID();
       break;
   }
 
   const Type &sum_type = TypeFactory::GetType(type_precision_id);
-  blank_state_.sum_ = sum_type.makeZeroValue();
+  state_size_ = sum_type.maximumByteLength();
+  blank_state_.reset(state_size_, false);
+
+  tv_blank_state_ = sum_type.makeZeroValue();
 
   // Make operators to do arithmetic:
   // Add operator for summing argument values.
-  fast_operator_.reset(
+  accumulate_operator_.reset(
       BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-          .makeUncheckedBinaryOperatorForTypes(sum_type, argument_type_));
+          .makeUncheckedBinaryOperatorForTypes(sum_type, argument_type));
+  accumulate_functor_ = accumulate_operator_->getMergeFunctor();
+
   // Add operator for merging states.
   merge_operator_.reset(
       BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
           .makeUncheckedBinaryOperatorForTypes(sum_type, sum_type));
+  merge_functor_ = merge_operator_->getMergeFunctor();
 
-  // Result is nullable, because SUM() over 0 values (or all NULL values) is
-  // NULL.
-  result_type_ = &sum_type.getNullableVersion();
+  finalize_functor_ = MakeUntypedCopyFunctor(&sum_type);
+  result_type_ = &sum_type;
 }
 
-AggregationStateHashTableBase* AggregationHandleSum::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<AggregationStateSum>::CreateResizable(
-      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
-}
-
-AggregationState* AggregationHandleSum::accumulateColumnVectors(
+void AggregationHandleSum::accumulateColumnVectors(
+    void *state,
     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_, *column_vectors.front(), &num_tuples);
-  return new AggregationStateSum(std::move(cv_sum), num_tuples == 0);
+  TypedValue cv_sum = accumulate_operator_->accumulateColumnVector(
+      tv_blank_state_, *column_vectors.front(), &num_tuples);
+  cv_sum.copyInto(state);
 }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleSum::accumulateValueAccessor(
+void AggregationHandleSum::accumulateValueAccessor(
+    void *state,
     ValueAccessor *accessor,
     const std::vector<attribute_id> &accessor_ids) const {
   DCHECK_EQ(1u, accessor_ids.size())
       << "Got wrong number of attributes for SUM: " << accessor_ids.size();
 
   std::size_t num_tuples = 0;
-  TypedValue va_sum = fast_operator_->accumulateValueAccessor(
-      blank_state_.sum_, accessor, accessor_ids.front(), &num_tuples);
-  return new AggregationStateSum(std::move(va_sum), num_tuples == 0);
+  TypedValue va_sum = accumulate_operator_->accumulateValueAccessor(
+      tv_blank_state_, accessor, accessor_ids.front(), &num_tuples);
+  va_sum.copyInto(state);
 }
 #endif
 
-void AggregationHandleSum::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(1u, argument_ids.size())
-      << "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);
-
-  SpinMutexLock lock(sum_destination->mutex_);
-  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 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);
-  if (agg_state.null_) {
-    // SUM() over no values is NULL.
-    return result_type_->makeNullValue();
-  } else {
-    return agg_state.sum_;
-  }
-}
-
-ColumnVector* AggregationHandleSum::finalizeHashTable(
-    const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  return finalizeHashTableHelperFast<AggregationHandleSum,
-                                     AggregationStateFastHashTable>(
-      *result_type_, hash_table, group_by_keys, index);
-}
-
-AggregationState*
-AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle(
-    const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleSum,
-      AggregationStateSum>(distinctify_hash_table);
-}
-
-void AggregationHandleSum::aggregateOnDistinctifyHashTableForGroupBy(
-    const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleSum,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index 6c334a6..f45e87e 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -26,198 +26,39 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
-#include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
+#include "utility/ScopedBuffer.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
-class ColumnVector;
-class StorageManager;
-class ValueAccessor;
-
 /** \addtogroup Expressions
  *  @{
  */
 
 /**
- * @brief Aggregation state for sum.
- */
-class AggregationStateSum : public AggregationState {
- public:
-  /**
-   * @brief Copy constructor (ignores mutex).
-   */
-  AggregationStateSum(const AggregationStateSum &orig)
-      : sum_(orig.sum_),
-        null_(orig.null_),
-        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<std::uint8_t *>(&null_) -
-                     reinterpret_cast<std::uint8_t *>(&sum_)) {}
-
-  AggregationStateSum(TypedValue &&sum, const bool is_null)
-      : 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.
-  TypedValue sum_;
-  bool null_;
-  SpinMutex mutex_;
-
-  int sum_offset_, null_offset_;
-};
-
-/**
  * @brief An aggregationhandle for sum.
  **/
-class AggregationHandleSum : public AggregationConcreteHandle {
+class AggregationHandleSum : public AggregationHandle {
  public:
   ~AggregationHandleSum() override {}
 
-  AggregationState* createInitialState() const override {
-    return new AggregationStateSum(blank_state_);
-  }
-
-  AggregationStateHashTableBase* 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 override;
-
-  inline void iterateUnaryInl(AggregationStateSum *state,
-                              const TypedValue &value) const {
-    DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
-    if (value.isNull()) return;
-
-    SpinMutexLock lock(state->mutex_);
-    state->sum_ = fast_operator_->applyToTypedValues(state->sum_, value);
-    state->null_ = false;
-  }
-
-  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_);
-    *sum_ptr = fast_operator_->applyToTypedValues(*sum_ptr, value);
-    *null_ptr = false;
-  }
-
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
-
-  void blockUpdate() override { block_update_ = true; }
-
-  void allowUpdate() override { block_update_ = false; }
-
-  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;
+  void accumulateColumnVectors(
+      void *state,
+      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
+  void accumulateValueAccessor(
+      void *state,
       ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_id) const override;
+      const std::vector<attribute_id> &accessor_ids) const override;
 #endif
 
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  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;
-
-  inline TypedValue finalizeHashTableEntry(
-      const AggregationState &state) const {
-    return static_cast<const AggregationStateSum &>(state).sum_;
-  }
-
-  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;
-  }
-
-  ColumnVector* finalizeHashTable(
-      const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
-
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for SUM aggregation.
-   */
-  AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
-
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        for SUM aggregation.
-   */
-  void aggregateOnDistinctifyHashTableForGroupBy(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const override;
-
-  std::size_t getPayloadSize() const override {
-    return blank_state_.getPayloadSize();
-  }
-
  private:
   friend class AggregateFunctionSum;
 
@@ -226,15 +67,13 @@ class AggregationHandleSum : public AggregationConcreteHandle {
    *
    * @param type Type of the sum value.
    **/
-  explicit AggregationHandleSum(const Type &type);
+  explicit AggregationHandleSum(const Type &argument_type);
 
-  const Type &argument_type_;
-  const Type *result_type_;
-  AggregationStateSum blank_state_;
-  std::unique_ptr<UncheckedBinaryOperator> fast_operator_;
-  std::unique_ptr<UncheckedBinaryOperator> merge_operator_;
+  // TODO: temporary
+  TypedValue tv_blank_state_;
 
-  bool block_update_;
+  std::unique_ptr<UncheckedBinaryOperator> accumulate_operator_;
+  std::unique_ptr<UncheckedBinaryOperator> merge_operator_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleSum);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index e9503f7..7b369ae 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -43,9 +43,6 @@ add_library(quickstep_expressions_aggregation_AggregateFunctionMin
 add_library(quickstep_expressions_aggregation_AggregateFunctionSum
             AggregateFunctionSum.cpp
             AggregateFunctionSum.hpp)
-add_library(quickstep_expressions_aggregation_AggregationConcreteHandle
-            AggregationConcreteHandle.cpp
-            AggregationConcreteHandle.hpp)
 add_library(quickstep_expressions_aggregation_AggregationHandle
             ../../empty_src.cpp
             AggregationHandle.hpp)
@@ -55,9 +52,6 @@ add_library(quickstep_expressions_aggregation_AggregationHandleAvg
 add_library(quickstep_expressions_aggregation_AggregationHandleCount
             AggregationHandleCount.cpp
             AggregationHandleCount.hpp)
-add_library(quickstep_expressions_aggregation_AggregationHandleDistinct
-            AggregationHandleDistinct.cpp
-            AggregationHandleDistinct.hpp)
 add_library(quickstep_expressions_aggregation_AggregationHandleMax
             AggregationHandleMax.cpp
             AggregationHandleMax.hpp)
@@ -142,34 +136,20 @@ target_link_libraries(quickstep_expressions_aggregation_AggregateFunctionSum
                       quickstep_types_operations_binaryoperations_BinaryOperationFactory
                       quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_expressions_aggregation_AggregationConcreteHandle
-                      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)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandle
                       glog
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_HashTableBase
+                      quickstep_types_Type
                       quickstep_types_TypedValue
-                      quickstep_utility_Macros)
+                      quickstep_utility_Macros
+                      quickstep_utility_ScopedBuffer)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleAvg
                       glog
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
-                      quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
@@ -181,34 +161,23 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleAvg
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleCount
                       glog
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
                       quickstep_storage_ValueAccessor
                       quickstep_storage_ValueAccessorUtil
+                      quickstep_types_LongType
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorUtil
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_expressions_aggregation_AggregationHandleDistinct
-                      glog
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationConcreteHandle
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_types_TypedValue
-                      quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMax
                       glog
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
@@ -223,9 +192,7 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMax
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMin
                       glog
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
@@ -240,21 +207,21 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMin
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleSum
                       glog
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypeFactory
+                      quickstep_types_TypeFunctors
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_operations_binaryoperations_BinaryOperation
                       quickstep_types_operations_binaryoperations_BinaryOperationFactory
                       quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_Macros)
+                      quickstep_utility_Macros
+                      quickstep_utility_ScopedBuffer)
 
 # Submodule all-in-one library:
 add_library(quickstep_expressions_aggregation ../../empty_src.cpp)
@@ -267,11 +234,9 @@ target_link_libraries(quickstep_expressions_aggregation
                       quickstep_expressions_aggregation_AggregateFunctionMax
                       quickstep_expressions_aggregation_AggregateFunctionMin
                       quickstep_expressions_aggregation_AggregateFunctionSum
-                      quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
                       quickstep_expressions_aggregation_AggregationHandleAvg
                       quickstep_expressions_aggregation_AggregationHandleCount
-                      quickstep_expressions_aggregation_AggregationHandleDistinct
                       quickstep_expressions_aggregation_AggregationHandleMax
                       quickstep_expressions_aggregation_AggregationHandleMin
                       quickstep_expressions_aggregation_AggregationHandleSum

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/AggregationHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationHashTable.hpp b/storage/AggregationHashTable.hpp
deleted file mode 100644
index fca6d4c..0000000
--- a/storage/AggregationHashTable.hpp
+++ /dev/null
@@ -1,330 +0,0 @@
-/**
- * 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_STORAGE_AGGREGATION_HASH_TABLE_HPP_
-#define QUICKSTEP_STORAGE_AGGREGATION_HASH_TABLE_HPP_
-
-#include <algorithm>
-#include <atomic>
-#include <cstddef>
-#include <cstdlib>
-#include <cstring>
-#include <limits>
-#include <memory>
-#include <unordered_map>
-#include <utility>
-#include <vector>
-
-#include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/HashTableBase.hpp"
-#include "storage/HashTableUntypedKeyManager.hpp"
-#include "storage/StorageBlob.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageConstants.hpp"
-#include "storage/StorageManager.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#include "threading/SpinMutex.hpp"
-#include "threading/SpinSharedMutex.hpp"
-#include "types/Type.hpp"
-#include "types/TypeFunctors.hpp"
-#include "utility/Alignment.hpp"
-#include "utility/InlineMemcpy.hpp"
-#include "utility/Macros.hpp"
-#include "utility/PrimeNumber.hpp"
-
-namespace quickstep {
-
-/** \addtogroup Storage
- *  @{
- */
-
-template <bool use_mutex>
-class AggregationHashTablePayloadManager {
- public:
-  AggregationHashTablePayloadManager(const std::vector<AggregationHandle *> &handles)
-      : handles_(handles),
-        payload_size_in_bytes_(0) {
-    if (use_mutex) {
-      payload_size_in_bytes_ += sizeof(SpinMutex);
-    }
-    for (const AggregationHandle *handle : handles) {
-      const std::size_t state_size = handle->getStateSize();
-      agg_state_sizes_.emplace_back(state_size);
-      agg_state_offsets_.emplace_back(payload_size_in_bytes_);
-      payload_size_in_bytes_ += state_size;
-    }
-
-    initial_payload_ = std::malloc(payload_size_in_bytes_);
-    if (use_mutex) {
-      new(initial_payload_) Mutex;
-    }
-//    for (std::size_t i = 0; i < handles_.size(); ++i) {
-//      handles_[i]->initPayload(
-//          static_cast<std::uint8_t *>(initial_payload_) + agg_state_offsets_[i]);
-//    }
-  }
-
-  ~AggregationHashTablePayloadManager() {
-    std::free(initial_payload_);
-  }
-
-  inline std::size_t getPayloadSizeInBytes() const {
-    return payload_size_in_bytes_;
-  }
-
-  inline void updatePayload(void *payload) const {
-  }
-
-  inline void initPayload(void *payload) const {
-  }
-
- private:
-  std::vector<AggregationHandle *> handles_;
-
-  std::vector<std::size_t> agg_state_sizes_;
-  std::vector<std::size_t> agg_state_offsets_;
-  std::size_t payload_size_in_bytes_;
-
-  void *initial_payload_;
-
-  DISALLOW_COPY_AND_ASSIGN(AggregationHashTablePayloadManager);
-};
-
-class ThreadPrivateAggregationHashTable : public AggregationHashTableBase {
- public:
-  ThreadPrivateAggregationHashTable(const std::vector<const Type *> &key_types,
-                                    const std::size_t num_entries,
-                                    const std::vector<AggregationHandle *> &handles,
-                                    StorageManager *storage_manager)
-    : payload_manager_(handles),
-      key_types_(key_types),
-      key_manager_(this->key_types_, payload_manager_.getPayloadSizeInBytes()),
-      slots_(num_entries * kHashTableLoadFactor,
-             key_manager_.getUntypedKeyHashFunctor(),
-             key_manager_.getUntypedKeyEqualityFunctor()),
-      bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize(),
-                                     payload_manager_.getPayloadSizeInBytes())),
-      buckets_allocated_(0),
-      storage_manager_(storage_manager) {
-    std::size_t num_storage_slots =
-        this->storage_manager_->SlotsNeededForBytes(num_entries);
-
-    // 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);
-
-    buckets_ = this->blob_->getMemoryMutable();
-    num_buckets_ = num_storage_slots * kSlotSizeBytes / bucket_size_;
-  }
-
-  void resize() {
-    const std::size_t resized_memory_required = num_buckets_ * bucket_size_ * 2;
-    const std::size_t resized_storage_slots =
-        this->storage_manager_->SlotsNeededForBytes(resized_memory_required);
-    const block_id resized_blob_id =
-        this->storage_manager_->createBlob(resized_storage_slots);
-    MutableBlobReference resized_blob =
-        this->storage_manager_->getBlobMutable(resized_blob_id);
-
-    void *resized_buckets = resized_blob->getMemoryMutable();
-    std::memcpy(resized_buckets, buckets_, buckets_allocated_ * bucket_size_);
-
-    for (auto &pair : slots_) {
-      pair.second =
-           (static_cast<const char *>(pair.first) - static_cast<char *>(buckets_))
-           + static_cast<char *>(resized_buckets);
-    }
-
-    buckets_ = resized_buckets;
-    num_buckets_ = resized_storage_slots * kSlotSizeBytes / bucket_size_;
-    std::swap(this->blob_, resized_blob);
-  }
-
-  bool upsertValueAccessor(ValueAccessor *accessor,
-                           const attribute_id key_attr_id,
-                           const std::vector<attribute_id> &argument_ids,
-                           const bool check_for_null_keys) override {
-    if (check_for_null_keys) {
-      return upsertValueAccessorInternal<true>(
-          accessor, key_attr_id, argument_ids);
-    } else {
-      return upsertValueAccessorInternal<false>(
-          accessor, key_attr_id, argument_ids);
-    }
-  }
-
-  template <bool check_for_null_keys>
-  bool upsertValueAccessorInternal(ValueAccessor *accessor,
-                                   const attribute_id key_attr_id,
-                                   const std::vector<attribute_id> &argument_ids) {
-    return InvokeOnAnyValueAccessor(
-        accessor,
-        [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
-      accessor->beginIteration();
-      while (accessor->next()) {
-        const void *key = accessor->template getUntypedValue<check_for_null_keys>(key_attr_id);
-        if (check_for_null_keys && key == nullptr) {
-          continue;
-        }
-        bool is_empty;
-        void *bucket = locateBucket(key, &is_empty);
-        if (is_empty) {
-          payload_manager_.initPayload(bucket);
-        } else {
-          payload_manager_.updatePayload(bucket);
-        }
-      }
-      return true;
-    });
-  }
-
-  bool upsertValueAccessorCompositeKey(ValueAccessor *accessor,
-                                       const std::vector<attribute_id> &key_attr_ids,
-                                       const std::vector<attribute_id> &argument_ids,
-                                       const bool check_for_null_keys) override {
-    if (check_for_null_keys) {
-      return upsertValueAccessorCompositeKeyInternal<true>(
-          accessor, key_attr_ids, argument_ids);
-    } else {
-      return upsertValueAccessorCompositeKeyInternal<false>(
-          accessor, key_attr_ids, argument_ids);
-    }
-  }
-
-  template <bool check_for_null_keys>
-  bool upsertValueAccessorCompositeKeyInternal(ValueAccessor *accessor,
-                                               const std::vector<attribute_id> &key_attr_ids,
-                                               const std::vector<attribute_id> &argument_ids) {
-    return InvokeOnAnyValueAccessor(
-        accessor,
-        [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
-      accessor->beginIteration();
-      void *prealloc_bucket = allocateBucket();
-      while (accessor->next()) {
-        if (check_for_null_keys) {
-          const bool is_null =
-              key_manager_.writeNullableUntypedKeyFromValueAccessorToBucket(
-                  accessor,
-                  key_attr_ids,
-                  prealloc_bucket);
-          if (is_null) {
-            continue;
-          }
-        } else {
-          key_manager_.writeUntypedKeyFromValueAccessorToBucket(
-              accessor,
-              key_attr_ids,
-              prealloc_bucket);
-        }
-        void *bucket = locateBucketWithPrealloc(prealloc_bucket);
-        if (bucket != prealloc_bucket) {
-          payload_manager_.initPayload(bucket);
-          prealloc_bucket = allocateBucket();
-        } else {
-          payload_manager_.updatePayload(bucket);
-        }
-      }
-      // Reclaim the last unused bucket
-      --buckets_allocated_;
-      return true;
-    });
-  }
-
-  inline void* locateBucket(const void *key, bool *is_empty) {
-    auto slot_it = slots_.find(key);
-    if (slot_it == slots_.end()) {
-      void *bucket = allocateBucket();
-      key_manager_.writeUntypedKeyToBucket(key, bucket);
-      slots_.emplace(key_manager_.getUntypedKeyComponent(bucket), bucket);
-      *is_empty = true;
-      return bucket;
-    } else {
-      *is_empty = false;
-      return slot_it->second;
-    }
-  }
-
-  inline void* locateBucketWithPrealloc(void *prealloc_bucket) {
-    const void *key = key_manager_.getUntypedKeyComponent(prealloc_bucket);
-    auto slot_it = slots_.find(key);
-    if (slot_it == slots_.end()) {
-      slots_.emplace(key, prealloc_bucket);
-      return prealloc_bucket;
-    } else {
-      return slot_it->second;
-    }
-  }
-
-  inline void* allocateBucket() {
-    if (buckets_allocated_ >= num_buckets_) {
-      resize();
-    }
-    void *bucket = static_cast<char *>(buckets_) + buckets_allocated_ * bucket_size_;
-    ++buckets_allocated_;
-    return bucket;
-  }
-
-  void print() const override {
-    std::cerr << "Bucket size = " << bucket_size_ << "\n";
-    std::cerr << "Buckets: \n";
-    for (const auto &pair : slots_) {
-      std::cerr << pair.first << " -- " << pair.second << "\n";
-      std::cerr << *static_cast<const int *>(pair.second) << "\n";
-    }
-  }
-
- private:
-  // Helper object to manage hash table payloads (i.e. aggregation states).
-  AggregationHashTablePayloadManager<false> payload_manager_;
-
-  // Type(s) of keys.
-  const std::vector<const Type*> key_types_;
-
-  // Helper object to manage key storage.
-  HashTableUntypedKeyManager key_manager_;
-
-  // Round bucket size up to a multiple of kBucketAlignment.
-  static std::size_t ComputeBucketSize(const std::size_t fixed_key_size,
-                                       const std::size_t total_payload_size) {
-    constexpr std::size_t kBucketAlignment = 4;
-    return (((fixed_key_size + total_payload_size - 1)
-               / kBucketAlignment) + 1) * kBucketAlignment;
-  }
-
-  std::unordered_map<const void *, void *,
-                     UntypedKeyHashFunctor,
-                     UntypedKeyEqualityFunctor> slots_;
-
-  void *buckets_;
-  const std::size_t bucket_size_;
-  std::size_t num_buckets_;
-  std::size_t buckets_allocated_;
-
-  StorageManager *storage_manager_;
-  MutableBlobReference blob_;
-
-  DISALLOW_COPY_AND_ASSIGN(ThreadPrivateAggregationHashTable);
-};
-
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_STORAGE_AGGREGATION_HASH_TABLE_HPP_
-

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index fe16fc4..50e7c06 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -34,13 +34,11 @@
 #include "expressions/aggregation/AggregateFunction.hpp"
 #include "expressions/aggregation/AggregateFunctionFactory.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "expressions/aggregation/AggregationHandleDistinct.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
-#include "storage/AggregationHashTable.hpp"
 #include "storage/AggregationOperationState.pb.h"
-#include "storage/HashTable.hpp"
+#include "storage/AggregationStateHashTable.hpp"
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTableFactory.hpp"
 #include "storage/InsertDestination.hpp"
@@ -88,122 +86,66 @@ AggregationOperationState::AggregationOperationState(
   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.
-    DCHECK_GT(group_by_list_.size(), 0u);
-
-    handles_.emplace_back(new AggregationHandleDistinct());
-    arguments_.push_back({});
-    is_distinct_.emplace_back(false);
-    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 =
-        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<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.
-      std::vector<const Type *> argument_types;
-      for (const std::unique_ptr<const Scalar> &argument : *args_it) {
-        argument_types.emplace_back(&argument->getType());
-      }
-
-      // Sanity checks: aggregate function exists and can apply to the specified
-      // arguments.
-      DCHECK(*agg_func_it != nullptr);
-      DCHECK((*agg_func_it)->canApplyToTypes(argument_types));
-
-      // Have the AggregateFunction create an AggregationHandle that we can use
-      // to do actual aggregate computation.
-      handles_.emplace_back((*agg_func_it)->createHandle(argument_types));
+  // Set up each individual aggregate in this operation.
+  for (std::size_t i = 0; i < aggregate_functions.size(); ++i) {
+    // Get the Types of this aggregate's arguments so that we can create an
+    // AggregationHandle.
+    std::vector<const Type *> argument_types;
+    for (const std::unique_ptr<const Scalar> &argument : arguments[i]) {
+      argument_types.emplace_back(&argument->getType());
+    }
 
-      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());
-      } else {
-        // Aggregation without GROUP BY: create a single global state.
-        single_states_.emplace_back(handles_.back()->createInitialState());
+    // Sanity checks: aggregate function exists and can apply to the specified
+    // arguments.
+    const AggregateFunction *agg_func = aggregate_functions[i];
+    DCHECK(agg_func != nullptr);
+    DCHECK(agg_func->canApplyToTypes(argument_types));
+
+    // Have the AggregateFunction create an AggregationHandle that we can use
+    // to do actual aggregate computation.
+    handles_.emplace_back(agg_func->createHandle(argument_types));
+
+    if (!group_by_list_.empty()) {
+      // TODO(jianqiao): handle DISTINCT aggregation.
+      // if (is_distinct[i]) {
+      // }
+      group_by_handles.emplace_back(handles_.back());
+    } else {
+      // Aggregation without GROUP BY: create a single global state.
+      single_states_.emplace_back(handles_.back()->createInitialState());
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-        // See if all of this aggregate's arguments are attributes in the input
-        // relation. If so, remember the attribute IDs so that we can do copy
-        // elision when actually performing the aggregation.
-        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();
-          if (argument_id == -1) {
-            local_arguments_as_attributes.clear();
-            break;
-          } else {
-            DCHECK_EQ(input_relation_.getID(),
-                      argument->getRelationIdForValueAccessor());
-            local_arguments_as_attributes.push_back(argument_id);
-          }
+      // See if all of this aggregate's arguments are attributes in the input
+      // relation. If so, remember the attribute IDs so that we can do copy
+      // elision when actually performing the aggregation.
+      std::vector<attribute_id> local_arguments_as_attributes;
+      local_arguments_as_attributes.reserve(arguments[i].size());
+      for (const std::unique_ptr<const Scalar> &argument : arguments[i]) {
+        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());
+          local_arguments_as_attributes.push_back(argument_id);
         }
-
-        arguments_as_attributes_.emplace_back(
-            std::move(local_arguments_as_attributes));
-#endif
       }
 
-      // 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
-        // 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(
-            AggregationStateFastHashTableFactory::CreateResizable(
-                *distinctify_hash_table_impl_types_it,
-                key_types,
-                estimated_num_entries,
-                {0},
-                {},
-                storage_manager));
-        ++distinctify_hash_table_impl_types_it;
-      } else {
-        distinctify_hashtables_.emplace_back(nullptr);
-      }
+      arguments_as_attributes_.emplace_back(
+          std::move(local_arguments_as_attributes));
+#endif
     }
+  }
 
-    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));
-    }
+  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,
+                                                     group_by_handles,
+                                                     storage_manager));
   }
 }
 
@@ -352,12 +294,12 @@ void AggregationOperationState::finalizeAggregate(
 }
 
 void AggregationOperationState::mergeSingleState(
-    const std::vector<std::unique_ptr<AggregationState>> &local_state) {
+    const std::vector<ScopedBuffer> &local_state) {
   DEBUG_ASSERT(local_state.size() == single_states_.size());
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
     if (!is_distinct_[agg_idx]) {
-      handles_[agg_idx]->mergeStates(*local_state[agg_idx],
-                                     single_states_[agg_idx].get());
+      handles_[agg_idx]->mergeStates(single_states_[agg_idx].get(),
+                                     local_state[agg_idx].get());
     }
   }
 }
@@ -365,7 +307,7 @@ void AggregationOperationState::mergeSingleState(
 void AggregationOperationState::aggregateBlockSingleState(
     const block_id input_block) {
   // Aggregate per-block state for each aggregate.
-  std::vector<std::unique_ptr<AggregationState>> local_state;
+  std::vector<ScopedBuffer> local_state;
 
   BlockReference block(
       storage_manager_->getBlock(input_block, input_relation_));
@@ -386,14 +328,7 @@ void AggregationOperationState::aggregateBlockSingleState(
       // Call StorageBlock::aggregateDistinct() to put the arguments as keys
       // directly into the (threadsafe) shared global distinctify HashTable
       // for this aggregate.
-      block->aggregateDistinct(*handles_[agg_idx],
-                               arguments_[agg_idx],
-                               local_arguments_as_attributes,
-                               {}, /* group_by */
-                               predicate_.get(),
-                               distinctify_hashtables_[agg_idx].get(),
-                               &reuse_matches,
-                               nullptr /* reuse_group_by_vectors */);
+      // TODO(jianqiao): handle DISTINCT aggregation.
       local_state.emplace_back(nullptr);
     } else {
       // Call StorageBlock::aggregate() to actually do the aggregation.
@@ -426,18 +361,10 @@ void AggregationOperationState::aggregateBlockHashTable(
 
   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],
-                               arguments_[agg_idx],
-                               nullptr, /* arguments_as_attributes */
-                               group_by_list_,
-                               predicate_.get(),
-                               distinctify_hashtables_[agg_idx].get(),
-                               &reuse_matches,
-                               &reuse_group_by_vectors);
+      // TODO(jianqiao): handle DISTINCT aggregation.
     }
   }
 
@@ -445,16 +372,13 @@ void AggregationOperationState::aggregateBlockHashTable(
   // 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();
+  auto *agg_hash_table = group_by_hashtable_pool_->getHashTable();
   DCHECK(agg_hash_table != nullptr);
 
   block->aggregateGroupBy(arguments_,
                           group_by_list_,
                           predicate_.get(),
                           agg_hash_table,
-                          group_by_hashtable_pool_->createNewThreadPrivateHashTable(),
-//                          nullptr,
                           &reuse_matches,
                           &reuse_group_by_vectors);
   group_by_hashtable_pool_->returnHashTable(agg_hash_table);
@@ -468,23 +392,23 @@ void AggregationOperationState::finalizeSingleState(
 
   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]));
+      // TODO(jianqiao): handle DISTINCT aggregation
     }
 
     attribute_values.emplace_back(
-        handles_[agg_idx]->finalize(*single_states_[agg_idx]));
+        handles_[agg_idx]->finalize(single_states_[agg_idx].get()));
   }
 
   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);
+    AggregationStateHashTableBase *destination_hash_table,
+    const AggregationStateHashTableBase *source_hash_table) {
+  static_cast<ThreadPrivateAggregationStateHashTable *>(
+      destination_hash_table)->mergeHashTable(
+          static_cast<const ThreadPrivateAggregationStateHashTable *>(
+              source_hash_table));
 }
 
 void AggregationOperationState::finalizeHashTable(
@@ -501,103 +425,22 @@ void AggregationOperationState::finalizeHashTable(
   // e.g. Keep merging entries from smaller hash tables to larger.
 
   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);
-         ++hash_table_index) {
-      // Merge each hash table to the last hash table.
-      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) {
-    if (is_distinct_[agg_idx]) {
-      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_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();
-      DCHECK(agg_hash_table != nullptr);
-      handles_[agg_idx]->allowUpdate();
-      handles_[agg_idx]->aggregateOnDistinctifyHashTableForGroupBy(
-          *distinctify_hashtables_[agg_idx], agg_hash_table, agg_idx);
-    }
-
-    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_pool_->getHashTableFast();
-      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);
-    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);
-    }
-  }
-
-  // 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;
+  if (hash_tables->size() == 0) {
+    return;
   }
 
-  // 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());
+  std::unique_ptr<AggregationStateHashTableBase> final_hash_table(
+      hash_tables->back().release());
+  for (std::size_t i = 0; i < hash_tables->size() - 1; ++i) {
+    std::unique_ptr<AggregationStateHashTableBase> hash_table(
+        hash_tables->at(i).release());
+    mergeGroupByHashTables(final_hash_table.get(), hash_table.get());
   }
 
   // Bulk-insert the complete result.
-  output_destination->bulkInsertTuples(&complete_result);
+  std::unique_ptr<AggregationResultIterator> results(
+      final_hash_table->createResultIterator());
+  output_destination->bulkInsertAggregationResults(results.get());
 }
 
 }  // namespace quickstep



[6/6] incubator-quickstep git commit: Updates

Posted by ji...@apache.org.
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/9ccd5a31
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/9ccd5a31
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/9ccd5a31

Branch: refs/heads/untyped-agg
Commit: 9ccd5a31189823f0e63b0a34ec02978c8695b25f
Parents: 140069b
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Oct 17 16:55:49 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Mon Oct 17 16:55:49 2016 -0500

----------------------------------------------------------------------
 .../aggregation/AggregationConcreteHandle.cpp   |   68 -
 .../aggregation/AggregationConcreteHandle.hpp   |  325 ---
 expressions/aggregation/AggregationHandle.hpp   |  405 +--
 .../aggregation/AggregationHandleAvg.cpp        |  321 ++-
 .../aggregation/AggregationHandleAvg.hpp        |  200 +-
 .../aggregation/AggregationHandleCount.cpp      |  134 +-
 .../aggregation/AggregationHandleCount.hpp      |  183 +-
 .../aggregation/AggregationHandleDistinct.cpp   |   81 -
 .../aggregation/AggregationHandleDistinct.hpp   |  130 -
 .../aggregation/AggregationHandleMax.cpp        |  190 +-
 .../aggregation/AggregationHandleMax.hpp        |  174 +-
 .../aggregation/AggregationHandleMin.cpp        |  192 +-
 .../aggregation/AggregationHandleMin.hpp        |  176 +-
 .../aggregation/AggregationHandleSum.cpp        |  128 +-
 .../aggregation/AggregationHandleSum.hpp        |  187 +-
 expressions/aggregation/CMakeLists.txt          |   49 +-
 storage/AggregationHashTable.hpp                |  330 ---
 storage/AggregationOperationState.cpp           |  317 +--
 storage/AggregationOperationState.hpp           |   20 +-
 storage/AggregationResultIterator.hpp           |  104 +
 storage/AggregationStateHashTable.hpp           |  338 +++
 storage/AggregationStateManager.hpp             |  181 ++
 storage/CMakeLists.txt                          |   98 +-
 storage/FastHashTable.hpp                       | 2515 ------------------
 storage/FastHashTableFactory.hpp                |  257 --
 storage/FastSeparateChainingHashTable.hpp       | 1734 ------------
 storage/HashTableBase.hpp                       |   43 +-
 storage/HashTablePool.hpp                       |  109 +-
 storage/HashTableUntypedKeyManager.hpp          |   34 +-
 storage/InsertDestination.cpp                   |   18 +
 storage/InsertDestination.hpp                   |    3 +
 storage/PackedRowStoreTupleStorageSubBlock.cpp  |   30 +
 storage/PackedRowStoreTupleStorageSubBlock.hpp  |    3 +
 storage/StorageBlock.cpp                        |  152 +-
 storage/StorageBlock.hpp                        |   55 +-
 storage/TupleStorageSubBlock.hpp                |    7 +
 types/CharType.hpp                              |   12 +
 types/DateType.hpp                              |   13 +-
 types/DatetimeIntervalType.hpp                  |   12 +-
 types/DatetimeLit.hpp                           |    4 +
 types/DatetimeType.hpp                          |   12 +-
 types/NullType.hpp                              |    4 +
 types/NumericSuperType.hpp                      |   11 +-
 types/Type.hpp                                  |    9 +-
 types/TypeFunctors.cpp                          |   54 +-
 types/TypeFunctors.hpp                          |   15 +-
 types/VarCharType.hpp                           |   12 +
 types/YearMonthIntervalType.hpp                 |   12 +-
 .../ArithmeticBinaryOperators.hpp               |   20 +
 .../binary_operations/BinaryOperation.hpp       |    7 +
 50 files changed, 1592 insertions(+), 7896 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationConcreteHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.cpp b/expressions/aggregation/AggregationConcreteHandle.cpp
deleted file mode 100644
index e3fb520..0000000
--- a/expressions/aggregation/AggregationConcreteHandle.cpp
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * 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.
- **/
-
-#include "expressions/aggregation/AggregationConcreteHandle.hpp"
-
-#include <cstddef>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
-
-namespace quickstep {
-
-class StorageManager;
-class Type;
-class ValueAccessor;
-
-AggregationStateHashTableBase* AggregationConcreteHandle::createDistinctifyHashTable(
-    const HashTableImplType hash_table_impl,
-    const std::vector<const Type*> &key_types,
-    const std::size_t estimated_num_distinct_keys,
-    StorageManager *storage_manager) const {
-  // Create a hash table with key types as key_types and value type as bool.
-  return AggregationStateHashTableFactory<bool>::CreateResizable(
-      hash_table_impl,
-      key_types,
-      estimated_num_distinct_keys,
-      storage_manager);
-}
-
-void AggregationConcreteHandle::insertValueAccessorIntoDistinctifyHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &key_ids,
-    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.
-
-  AggregationStateFastHashTable *hash_table =
-      static_cast<AggregationStateFastHashTable *>(distinctify_hash_table);
-  if (key_ids.size() == 1) {
-    hash_table->upsertValueAccessorFast(
-        key_ids, accessor, key_ids[0], true /* check_for_null_keys */);
-  } else {
-    std::vector<attribute_id> empty_args {kInvalidAttributeID};
-    hash_table->upsertValueAccessorCompositeKeyFast(
-        empty_args, accessor, key_ids, true /* check_for_null_keys */);
-  }
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationConcreteHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.hpp b/expressions/aggregation/AggregationConcreteHandle.hpp
deleted file mode 100644
index 398a032..0000000
--- a/expressions/aggregation/AggregationConcreteHandle.hpp
+++ /dev/null
@@ -1,325 +0,0 @@
-/**
- * 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_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_
-#define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_
-
-#include <cstddef>
-#include <utility>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationHandle.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 "glog/logging.h"
-
-namespace quickstep {
-
-class StorageManager;
-class Type;
-class ValueAccessor;
-
-/** \addtogroup Expressions
- *  @{
- */
-
-/**
- * @brief An upserter class for modifying the destination hash table while
- *        merging two group by hash tables.
- **/
-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 std::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()(std::uint8_t *destination_state) {
-    handle_.mergeStatesFast(source_state_, destination_state);
-  }
-
- private:
-  const HandleT &handle_;
-  const std::uint8_t *source_state_;
-
-  DISALLOW_COPY_AND_ASSIGN(HashTableStateUpserterFast);
-};
-
-/**
- * @brief The helper intermediate subclass of AggregationHandle that provides
- *        virtual method implementations as well as helper methods that are
- *        shared among all its subclasses.
- *
- * @note The reason that we have this intermediate class instead of putting
- *       everything inside AggregationHandle is to avoid cyclic dependency, e.g.
- *       when HashTable has to be used.
- **/
-class AggregationConcreteHandle : public AggregationHandle {
- public:
-  /**
-   * @brief Default implementaion for AggregationHandle::accumulateNullary().
-   */
-  AggregationState* accumulateNullary(
-      const std::size_t num_tuples) const override {
-    LOG(FATAL) << "Called accumulateNullary on an AggregationHandle that "
-               << "takes at least one argument.";
-  }
-
-  /**
-   * @brief Implementaion for AggregationHandle::createDistinctifyHashTable()
-   *        that creates a new HashTable for the distinctify step for
-   *        DISTINCT aggregation.
-   */
-  AggregationStateHashTableBase* createDistinctifyHashTable(
-      const HashTableImplType hash_table_impl,
-      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
-   * as keys into the distinctify hash table.
-   */
-  void insertValueAccessorIntoDistinctifyHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_ids,
-      AggregationStateHashTableBase *distinctify_hash_table) const override;
-
- protected:
-  AggregationConcreteHandle() {}
-
-  template <typename HandleT, typename StateT>
-  StateT* aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
-      const AggregationStateHashTableBase &distinctify_hash_table) const;
-
-  template <typename HandleT, typename HashTableT>
-  void aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *hash_table,
-      std::size_t index) 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 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 HashTableT>
-  void mergeGroupByHashTablesHelperFast(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const;
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(AggregationConcreteHandle);
-};
-
-/**
- * @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
- *        collecting the GROUP BY key values and the final aggregate values in
- *        a ColumnVector.
- **/
-template <typename HandleT, typename ColumnVectorT>
-class HashTableAggregateFinalizer {
- public:
-  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) {}
-
-  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));
-  }
-
-  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_;
-  ColumnVectorT *output_column_vector_;
-};
-
-/** @} */
-
-// ----------------------------------------------------------------------------
-// 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());
-
-  // 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 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 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);
-    };
-
-    target_hash_table->upsertCompositeKeyFast(key, nullptr, &upserter, index);
-  };
-
-  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::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;
-    }
-  }
-}
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index 19f28ff..40d5e26 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -21,14 +21,19 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_HPP_
 
 #include <cstddef>
+#include <cstring>
+#include <functional>
 #include <memory>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTableBase.hpp"
+#include "types/Type.hpp"
 #include "types/TypedValue.hpp"
+#include "utility/ScopedBuffer.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 class ColumnVector;
@@ -40,66 +45,10 @@ class ValueAccessor;
  *  @{
  */
 
-/**
- * @brief Abstract base class for aggregation state.
- **/
-class AggregationState {
- public:
-  /**
-   * @brief Default constructor.
-   **/
-  AggregationState() {}
+typedef std::function<void (void *, const void *)> AggregationStateAccumulateFunctor;
+typedef std::function<void (void *, const void *)> AggregationStateMergeFunctor;
+typedef std::function<void (void *, const void *)> AggregationStateFinalizeFunctor;
 
-  /**
-   * @brief Pure virtual destructor.
-   **/
-  virtual ~AggregationState() = 0;
-};
-
-// Destructor should be defined. This will be called when derived class
-// destructor is called.
-inline AggregationState::~AggregationState() {}
-
-/**
- * @brief AggregationHandle encapsulates logic for actually computing
- *        aggregates with particular argument(s).
- * @note See also AggregateFunction, which represents a SQL aggregate function
- *       in the abstract sense.
- *
- * An AggregationHandle is created by calling
- * AggregateFunction::createHandle(). The AggregationHandle object provides
- * methods that are used to actually compute the aggregate, storing
- * intermediate results in AggregationState objects.
- *
- * I. The work-flow for computing an aggregate without GROUP BY is as follows:
- *     1. Create a global state for the aggregate with createInitialState().
- *     2. For each block in a relation (parallelizable):
- *        a. Call StorageBlock::aggregate() to accumulate results from the
- *           block (under the covers, this calls either
- *           accumulateColumnVectors() or accumulateValueAccessor() to do the
- *           actual per-block aggregation in a vectorized fashion).
- *        b. Merge the per-block results back with the global state by calling
- *           mergeStates() (this is threadsafe).
- *     3. Generate the final result by calling finalize() on the global state.
- *
- * II. The work-flow for computing an aggregate with GROUP BY is as follows:
- *     1. Create a HashTable to hold per-group states by calling
- *        createGroupByHashTable().
- *     2. For each block in a relation (parallelizable):
- *        a. Call StorageBlock::aggregateGroupBy() to update the states in the
- *           HashTable according to the values in the block (under the covers,
- *           this calls aggregateValueAccessorIntoHashTable() to aggregate over
- *           all the values/groups in a block in one shot; this is threadsafe).
- *     3. Generate the final set of groups and their corresponding results by
- *        calling finalizeHashTable().
- *
- * See also AggregationOperationState, which holds 1 or more global states or
- * HashTables for an aggregate query, and has some logic to re-use common
- * information across multiple aggregates being computed on the same block
- * (e.g. the set of matches for a predicate, or the values of computed GROUP BY
- * expressions). AggregationOperationState also has a method to write out
- * finalized aggregate values to an InsertDestination.
- **/
 class AggregationHandle {
  public:
   /**
@@ -109,67 +58,25 @@ class AggregationHandle {
   virtual ~AggregationHandle() {}
 
   /**
-   * @brief Create an initial "blank" state for this aggregation.
-   *
-   * @return An initial "blank" state for this particular aggregation.
-   **/
-  virtual AggregationState* createInitialState() const = 0;
-
-  virtual std::size_t getStateSize() const {
-    return 0;
-  }
-
-  /**
-   * @brief Create a new HashTable for aggregation with GROUP BY.
-   *
-   * @param hash_table_impl The choice of which concrete HashTable
-   *        implementation to use.
-   * @param group_by_types The types of the GROUP BY columns/expressions. These
-   *        correspond to the (composite) key type for the HashTable.
-   * @param estimated_num_groups The estimated number of distinct groups for
-   *        the GROUP BY aggregation. This is used to size the initial
-   *        HashTable. 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 storage.
-   * @return A new HashTable instance with the appropriate state type for this
-   *         aggregate.
-   **/
-  virtual AggregationStateHashTableBase* 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 = 0;
-
-  /**
    * @brief Accumulate over tuples for a nullary aggregate function (one that
    *        has zero arguments, i.e. COUNT(*)).
-   *
-   * @param num_tuples The number of tuples to "accumulate". No actual tuple
-   *        data is accessed, the only thing that a nullary aggeregate can know
-   *        about input is its cardinality.
-   * @return A new AggregationState which contains the accumulated results from
-   *         applying the (nullary) aggregate to the specified number of
-   *         tuples.
    **/
-  virtual AggregationState* accumulateNullary(
-      const std::size_t num_tuples) const = 0;
+  virtual void accumulateNullary(void *state,
+                                 const std::size_t num_tuples) const {
+    LOG(FATAL) << "Called accumulateNullary on an AggregationHandle that "
+               << "takes at least one argument.";
+  }
 
   /**
    * @brief Accumulate (iterate over) all values in one or more ColumnVectors
    *        and return a new AggregationState which can be merged with other
    *        states or finalized.
-   *
-   * @param column_vectors One or more ColumnVectors that the aggregate will be
-   *        applied to. These correspond to the aggregate function's arguments,
-   *        in order.
-   * @return A new AggregationState which contains the accumulated results from
-   *         applying the aggregate to column_vectors. Caller is responsible
-   *         for deleting the returned AggregationState.
    **/
-  virtual AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const = 0;
+  virtual void accumulateColumnVectors(
+      void *state,
+      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
+    LOG(FATAL) << "Not implemented\n";
+  }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
   /**
@@ -186,243 +93,79 @@ class AggregationHandle {
    *         applying the aggregate to the specified columns in accessor.
    *         Caller is responsible for deleting the returned AggregationState.
    **/
-  virtual AggregationState* accumulateValueAccessor(
+  virtual void accumulateValueAccessor(
+      void *state,
       ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_ids) const = 0;
+      const std::vector<attribute_id> &accessor_ids) const {
+    LOG(FATAL) << "Not implemented\n";
+  }
 #endif
 
   /**
-   * @brief Perform an aggregation with GROUP BY over all the tuples accessible
-   *        through a ValueAccessor, upserting states in a HashTable.
-   *
-   * @note Implementations of this method are threadsafe with respect to
-   *       hash_table, and can be called concurrently from multiple threads
-   *       with the same HashTable object.
-   *
-   * @param accessor The ValueAccessor that will be iterated over to read
-   *        tuples.
-   * @param argument_ids The attribute_ids of the arguments to this aggregate
-   *        in accessor, in order.
-   * @param group_by_key_ids The attribute_ids of the group-by
-   *        columns/expressions in accessor.
-   * @param hash_table The HashTable to upsert AggregationStates in. This
-   *        should have been created by calling createGroupByHashTable() on
-   *        this same AggregationHandle.
-   **/
-  virtual void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const = 0;
-
-  /**
-   * @brief Merge two AggregationStates, updating one in-place. This computes a
-   *        new intermediate result and stores it in the destination
-   *        AggregationState.
-   *
-   * @note The order of arguments (i.e. which AggregationState is source and
-   *       which is destination) is not important for the result of the merging
-   *       process, but it does determine which of the two states is
-   *       overwritten (destination) and which remains unchanged (source).
-   * @note Implementations of this method are threadsafe with respect to the
-   *       destination state, and can be called concurrently from multiple
-   *       threads with the same state object.
-   *
-   * @param source The AggregationState to merge "from".
-   * @param destination The AggregationState to merge "to". The internal state
-   *        will be overwritten with the merged result.
-   **/
-  virtual void mergeStates(const AggregationState &source,
-                           AggregationState *destination) const = 0;
-
-  /**
-   * @brief Computes and returns the resulting aggregate by using intermediate
-   *        result saved in this handle.
-   *
-   * @note Except for count, SQL89 aggeregates return NULL when no rows are
-   *       selected.
-   * @warning It is dangerous to assume that calling mergeStates() or some
-   *          iterate method after previous finalize call will work correctly
-   *          when the aggregate function is not one of SQL89 aggregates (SUM,
-   *          COUNT, MIN, MAX, AVG).
-   *
-   * @return The result of this aggregation.
+   * @brief Get the number of bytes needed to store the aggregation handle's
+   *        state.
    **/
-  virtual TypedValue finalize(const AggregationState &state) const = 0;
+  inline std::size_t getStateSize() const {
+    return state_size_;
+  }
 
-  /**
-   * @brief Compute and return finalized aggregates for all groups in a
-   *        HashTable.
-   *
-   * @param hash_table The HashTable to finalize states from. This should have
-   *        have been created by calling createGroupByHashTable() on this same
-   *        AggregationHandle.
-   * @param group_by_keys A pointer to a vector of vectors of GROUP BY keys. If
-   *        this is initially empty, it will be filled in with the GROUP BY
-   *        keys visited by this method in the same order as the finalized
-   *        values returned in the ColumnVector. If this is already filled in,
-   *        then this method will visit the GROUP BY keys in the exact order
-   *        specified.
-   * @param index The index of the AggregationHandle to be finalized.
-   *
-   * @return A ColumnVector containing each group's finalized aggregate value.
-   **/
-  virtual ColumnVector* finalizeHashTable(
-      const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const = 0;
+  inline void initializeState(void *state) const {
+    std::memcpy(state, blank_state_.get(), state_size_);
+  }
 
-  /**
-   * @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
-   * into the distinctify hash table, so that arguments are distinctified within
-   * each GROUP BY group. Later, a second-round aggregation on the distinctify
-   * hash table will be performed to actually compute the aggregated result for
-   * each GROUP BY group.
-   *
-   * In the case of single aggregation where there is no GROUP BY expressions,
-   * 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 to use.
-   * @param key_types The types of the GROUP BY expressions together with the
-   *        types of the aggregation arguments.
-   * @param estimated_num_distinct_keys The estimated number of distinct keys
-   *        (i.e. GROUP BY expressions together with aggregation arguments) for
-   *        the distinctify step. This is used to size the initial HashTable.
-   *        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 storage.
-   *
-   * @return A new HashTable instance with the appropriate state type for this
-   *         aggregate.
-   */
-  virtual AggregationStateHashTableBase* createDistinctifyHashTable(
-      const HashTableImplType hash_table_impl,
-      const std::vector<const Type *> &key_types,
-      const std::size_t estimated_num_distinct_keys,
-      StorageManager *storage_manager) const = 0;
+  inline ScopedBuffer createInitialState() const {
+    ScopedBuffer state(state_size_, false);
+    initializeState(state.get());
+    return state;
+  }
 
-  /**
-   * @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 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 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 createDistinctifyHashTable();
-   */
-  virtual void insertValueAccessorIntoDistinctifyHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_ids,
-      AggregationStateHashTableBase *distinctify_hash_table) const = 0;
+  inline void mergeStates(void *destination_state,
+                          const void *source_state) const {
+    merge_functor_(destination_state, source_state);
+  }
 
-  /**
-   * @brief Perform single (i.e. without GROUP BY) aggregation on the keys from
-   * the distinctify hash table to actually compute the aggregated results.
-   *
-   * @param distinctify_hash_table Hash table which stores the distinctified
-   *        aggregation arguments as hash table keys. This should have been
-   *        created by calling createDistinctifyHashTable();
-   * @return A new AggregationState which contains the aggregated results from
-   *         applying the aggregate to the distinctify hash table.
-   *         Caller is responsible for deleting the returned AggregationState.
-   */
-  virtual AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table) const = 0;
+  inline std::size_t getResultSize() const {
+    return result_type_->maximumByteLength();
+  }
 
-  /**
-   * @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 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
-   *        this same AggregationHandle.
-   * @param index The index of the distinctify hash table for which we perform
-   *        the DISTINCT aggregation.
-   */
-  virtual void aggregateOnDistinctifyHashTableForGroupBy(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const = 0;
+  inline void finalize(void *value, const void *state) const {
+    finalize_functor_(value, state);
+  }
 
-  /**
-   * @brief Get the number of bytes needed to store the aggregation handle's
-   *        state.
-   **/
-  virtual std::size_t getPayloadSize() const { return 1; }
+  inline TypedValue finalize(const void *state) const {
+    ScopedBuffer value(state_size_, false);
+    finalize(value.get(), state);
+    TypedValue result = result_type_->makeValue(value.get());
+    result.ensureNotReference();
+    return result;
+  }
 
-  /**
-   * @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 {}
+  inline const AggregationStateMergeFunctor& getStateAccumulateFunctor() const {
+    DCHECK(accumulate_functor_);
+    return accumulate_functor_;
+  }
 
-  /**
-   * @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 {}
+  inline const AggregationStateMergeFunctor& getStateMergeFunctor() const {
+    DCHECK(merge_functor_);
+    return merge_functor_;
+  }
 
-  /**
-   * @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 {}
+  inline const AggregationStateFinalizeFunctor& getStateFinalizeFunctor() const {
+    DCHECK(finalize_functor_);
+    return finalize_functor_;
+  }
 
-  /**
-   * @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 {}
+ protected:
+  AggregationHandle() {}
 
-  /**
-   * @brief Inform the aggregation handle to block (prohibit) updates on the
-   *        aggregation state.
-   **/
-  virtual void blockUpdate() {}
+  std::size_t state_size_;
+  ScopedBuffer blank_state_;
 
-  /**
-   * @brief Inform the aggregation handle to allow updates on the
-   *        aggregation state.
-   **/
-  virtual void allowUpdate() {}
+  AggregationStateAccumulateFunctor accumulate_functor_;
+  AggregationStateMergeFunctor merge_functor_;
 
- protected:
-  AggregationHandle() {}
+  const Type *result_type_;
+  AggregationStateFinalizeFunctor finalize_functor_;
 
  private:
   DISALLOW_COPY_AND_ASSIGN(AggregationHandle);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index 2481092..47f3f41 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.cpp
@@ -26,7 +26,6 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "storage/HashTable.hpp"
 #include "storage/HashTableFactory.hpp"
-#include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
@@ -41,165 +40,165 @@ namespace quickstep {
 
 class StorageManager;
 
-AggregationHandleAvg::AggregationHandleAvg(const 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;
-  switch (type.getTypeID()) {
-    case kInt:
-    case kLong:
-      type_precision_id = kLong;
-      break;
-    case kFloat:
-    case kDouble:
-      type_precision_id = kDouble;
-      break;
-    default:
-      type_precision_id = type.getTypeID();
-      break;
-  }
-
-  const Type &sum_type = TypeFactory::GetType(type_precision_id);
-  blank_state_.sum_ = sum_type.makeZeroValue();
-  blank_state_.count_ = 0;
-
-  // Make operators to do arithmetic:
-  // Add operator for summing argument values.
-  fast_add_operator_.reset(
-      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-          .makeUncheckedBinaryOperatorForTypes(sum_type, argument_type_));
-  // Add operator for merging states.
-  merge_add_operator_.reset(
-      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-          .makeUncheckedBinaryOperatorForTypes(sum_type, sum_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)));
-
-  // 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());
-}
-
-AggregationStateHashTableBase* AggregationHandleAvg::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<AggregationStateAvg>::CreateResizable(
-      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
-}
-
-AggregationState* AggregationHandleAvg::accumulateColumnVectors(
-    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
-  DCHECK_EQ(1u, column_vectors.size())
-      << "Got wrong number of ColumnVectors for AVG: " << column_vectors.size();
-
-  AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
-  std::size_t count = 0;
-  state->sum_ = fast_add_operator_->accumulateColumnVector(
-      state->sum_, *column_vectors.front(), &count);
-  state->count_ = count;
-  return state;
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleAvg::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  DCHECK_EQ(1u, accessor_ids.size())
-      << "Got wrong number of attributes for AVG: " << accessor_ids.size();
-
-  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->count_ = count;
-  return state;
-}
-#endif
-
-void AggregationHandleAvg::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(1u, argument_ids.size())
-      << "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);
-
-  SpinMutexLock lock(avg_destination->mutex_);
-  avg_destination->count_ += avg_source.count_;
-  avg_destination->sum_ = merge_add_operator_->applyToTypedValues(
-      avg_destination->sum_, avg_source.sum_);
-}
-
-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);
-  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_)));
-  }
-}
-
-ColumnVector* AggregationHandleAvg::finalizeHashTable(
-    const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  return finalizeHashTableHelperFast<AggregationHandleAvg,
-                                     AggregationStateFastHashTable>(
-      *result_type_, hash_table, group_by_keys, index);
-}
-
-AggregationState*
-AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
-    const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleAvg,
-      AggregationStateAvg>(distinctify_hash_table);
-}
-
-void AggregationHandleAvg::aggregateOnDistinctifyHashTableForGroupBy(
-    const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleAvg,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
-}
+AggregationHandleAvg::AggregationHandleAvg(const 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;
+//  switch (type.getTypeID()) {
+//    case kInt:
+//    case kLong:
+//      type_precision_id = kLong;
+//      break;
+//    case kFloat:
+//    case kDouble:
+//      type_precision_id = kDouble;
+//      break;
+//    default:
+//      type_precision_id = type.getTypeID();
+//      break;
+//  }
+//
+//  const Type &sum_type = TypeFactory::GetType(type_precision_id);
+//  blank_state_.sum_ = sum_type.makeZeroValue();
+//  blank_state_.count_ = 0;
+//
+//  // Make operators to do arithmetic:
+//  // Add operator for summing argument values.
+//  fast_add_operator_.reset(
+//      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
+//          .makeUncheckedBinaryOperatorForTypes(sum_type, argument_type_));
+//  // Add operator for merging states.
+//  merge_add_operator_.reset(
+//      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
+//          .makeUncheckedBinaryOperatorForTypes(sum_type, sum_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)));
+//
+//  // 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());
+//}
+//
+//AggregationStateHashTableBase* AggregationHandleAvg::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<AggregationStateAvg>::CreateResizable(
+//      hash_table_impl, group_by_types, estimated_num_groups, storage_manager);
+//}
+//
+//AggregationState* AggregationHandleAvg::accumulateColumnVectors(
+//    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
+//  DCHECK_EQ(1u, column_vectors.size())
+//      << "Got wrong number of ColumnVectors for AVG: " << column_vectors.size();
+//
+//  AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
+//  std::size_t count = 0;
+//  state->sum_ = fast_add_operator_->accumulateColumnVector(
+//      state->sum_, *column_vectors.front(), &count);
+//  state->count_ = count;
+//  return state;
+//}
+//
+//#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+//AggregationState* AggregationHandleAvg::accumulateValueAccessor(
+//    ValueAccessor *accessor,
+//    const std::vector<attribute_id> &accessor_ids) const {
+//  DCHECK_EQ(1u, accessor_ids.size())
+//      << "Got wrong number of attributes for AVG: " << accessor_ids.size();
+//
+//  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->count_ = count;
+//  return state;
+//}
+//#endif
+//
+//void AggregationHandleAvg::aggregateValueAccessorIntoHashTable(
+//    ValueAccessor *accessor,
+//    const std::vector<attribute_id> &argument_ids,
+//    const std::vector<attribute_id> &group_by_key_ids,
+//    AggregationStateHashTableBase *hash_table) const {
+//  DCHECK_EQ(1u, argument_ids.size())
+//      << "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);
+//
+//  SpinMutexLock lock(avg_destination->mutex_);
+//  avg_destination->count_ += avg_source.count_;
+//  avg_destination->sum_ = merge_add_operator_->applyToTypedValues(
+//      avg_destination->sum_, avg_source.sum_);
+//}
+//
+//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);
+//  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_)));
+//  }
+//}
+//
+//ColumnVector* AggregationHandleAvg::finalizeHashTable(
+//    const AggregationStateHashTableBase &hash_table,
+//    std::vector<std::vector<TypedValue>> *group_by_keys,
+//    int index) const {
+//  return finalizeHashTableHelperFast<AggregationHandleAvg,
+//                                     AggregationStateFastHashTable>(
+//      *result_type_, hash_table, group_by_keys, index);
+//}
+//
+//AggregationState*
+//AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
+//    const AggregationStateHashTableBase &distinctify_hash_table) const {
+//  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
+//      AggregationHandleAvg,
+//      AggregationStateAvg>(distinctify_hash_table);
+//}
+//
+//void AggregationHandleAvg::aggregateOnDistinctifyHashTableForGroupBy(
+//    const AggregationStateHashTableBase &distinctify_hash_table,
+//    AggregationStateHashTableBase *aggregation_hash_table,
+//    std::size_t index) const {
+//  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
+//      AggregationHandleAvg,
+//      AggregationStateFastHashTable>(
+//      distinctify_hash_table, aggregation_hash_table, index);
+//}
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index 3c6e0c2..cc5adc8 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -26,11 +26,8 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
 #include "storage/HashTableBase.hpp"
-#include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
@@ -49,193 +46,12 @@ class ValueAccessor;
  */
 
 /**
- * @brief Aggregation state for average.
- */
-class AggregationStateAvg : public AggregationState {
- public:
-  /**
-   * @brief Copy constructor (ignores mutex).
-   */
-  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_) {}
-
-  /**
-   * @brief Destructor.
-   */
-  ~AggregationStateAvg() override {}
-
-  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<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.
-  TypedValue sum_;
-  std::int64_t count_;
-  SpinMutex mutex_;
-
-  int sum_offset_, count_offset_, mutex_offset_;
-};
-
-/**
  * @brief An aggregationhandle for avg.
  **/
-class AggregationHandleAvg : public AggregationConcreteHandle {
+class AggregationHandleAvg : public AggregationHandle {
  public:
   ~AggregationHandleAvg() override {}
 
-  AggregationState* createInitialState() const override {
-    return new AggregationStateAvg(blank_state_);
-  }
-
-  AggregationStateHashTableBase* 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 override;
-
-  /**
-   * @brief Iterate method with average aggregation state.
-   **/
-  inline void iterateUnaryInl(AggregationStateAvg *state,
-                              const TypedValue &value) const {
-    DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
-    if (value.isNull()) return;
-
-    SpinMutexLock lock(state->mutex_);
-    state->sum_ = fast_add_operator_->applyToTypedValues(state->sum_, value);
-    ++state->count_;
-  }
-
-  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_);
-    *sum_ptr = fast_add_operator_->applyToTypedValues(*sum_ptr, value);
-    ++(*count_ptr);
-  }
-
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
-
-  void blockUpdate() override { block_update_ = true; }
-
-  void allowUpdate() override { block_update_ = false; }
-
-  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;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_id) const override;
-#endif
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  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;
-
-  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_)));
-  }
-
-  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(
-      const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
-
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for AVG aggregation.
-   */
-  AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
-
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        for AVG aggregation.
-   */
-  void aggregateOnDistinctifyHashTableForGroupBy(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const override;
-
-  std::size_t getPayloadSize() const override {
-    return blank_state_.getPayloadSize();
-  }
-
  private:
   friend class AggregateFunctionAvg;
 
@@ -246,14 +62,12 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
    **/
   explicit AggregationHandleAvg(const Type &type);
 
-  const Type &argument_type_;
-  const Type *result_type_;
-  AggregationStateAvg blank_state_;
-  std::unique_ptr<UncheckedBinaryOperator> fast_add_operator_;
-  std::unique_ptr<UncheckedBinaryOperator> merge_add_operator_;
-  std::unique_ptr<UncheckedBinaryOperator> divide_operator_;
-
-  bool block_update_;
+//  const Type &argument_type_;
+//  const Type *result_type_;
+//  AggregationStateAvg blank_state_;
+//  std::unique_ptr<UncheckedBinaryOperator> fast_add_operator_;
+//  std::unique_ptr<UncheckedBinaryOperator> merge_add_operator_;
+//  std::unique_ptr<UncheckedBinaryOperator> divide_operator_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleAvg);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.cpp b/expressions/aggregation/AggregationHandleCount.cpp
index 034c942..c095a82 100644
--- a/expressions/aggregation/AggregationHandleCount.cpp
+++ b/expressions/aggregation/AggregationHandleCount.cpp
@@ -21,12 +21,11 @@
 
 #include <atomic>
 #include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "storage/ValueAccessor.hpp"
@@ -48,22 +47,37 @@ class Type;
 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>::AggregationHandleCount() {
+  state_size_ = sizeof(ResultCppType);
+  blank_state_.reset(state_size_, true);
+
+  accumulate_functor_ = [](void *state, const void *value) {
+    *static_cast<ResultCppType *>(state) += 1;
+  };
+
+  merge_functor_ = [](void *state, const void *value) {
+    *static_cast<ResultCppType *>(state) +=
+        *static_cast<const ResultCppType *>(value);
+  };
+
+  finalize_functor_ = [](void *result, const void *state) {
+    *static_cast<ResultCppType *>(result) =
+        *static_cast<const ResultCppType *>(state);
+  };
+
+  result_type_ = &TypeFactory::GetType(ResultType::kStaticTypeID);
+}
+
+template <bool count_star, bool nullable_type>
+void AggregationHandleCount<count_star, nullable_type>::accumulateNullary(
+      void *state,
+      const std::size_t num_tuples) const {
+  *static_cast<ResultCppType *>(state) = num_tuples;
 }
 
 template <bool count_star, bool nullable_type>
-AggregationState*
-AggregationHandleCount<count_star, nullable_type>::accumulateColumnVectors(
+void AggregationHandleCount<count_star, nullable_type>::accumulateColumnVectors(
+    void *state,
     const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
   DCHECK(!count_star)
       << "Called non-nullary accumulation method on an AggregationHandleCount "
@@ -84,20 +98,22 @@ AggregationHandleCount<count_star, nullable_type>::accumulateColumnVectors(
           // 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();
+            if (column_vector.getUntypedValue(pos) != nullptr) {
+              ++count;
+            }
           }
         } else {
           count = column_vector.size();
         }
       });
 
-  return new AggregationStateCount(count);
+  *static_cast<ResultCppType *>(state) = count;
 }
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 template <bool count_star, bool nullable_type>
-AggregationState*
-AggregationHandleCount<count_star, nullable_type>::accumulateValueAccessor(
+void AggregationHandleCount<count_star, nullable_type>::accumulateValueAccessor(
+    void *state,
     ValueAccessor *accessor,
     const std::vector<attribute_id> &accessor_ids) const {
   DCHECK(!count_star)
@@ -114,91 +130,19 @@ AggregationHandleCount<count_star, nullable_type>::accumulateValueAccessor(
       [&accessor_id, &count](auto *accessor) -> void {  // NOLINT(build/c++11)
         if (nullable_type) {
           while (accessor->next()) {
-            count += !accessor->getTypedValue(accessor_id).isNull();
+            if (accessor->getUntypedValue(accessor_id) != nullptr) {
+              ++count;
+            }
           }
         } else {
           count = accessor->getNumTuples();
         }
       });
 
-  return new AggregationStateCount(count);
+  *static_cast<ResultCppType *>(state) = count;
 }
 #endif
 
-template <bool count_star, bool nullable_type>
-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();
-  } else {
-    DCHECK_EQ(1u, argument_ids.size())
-        << "Got wrong number of arguments for COUNT: " << argument_ids.size();
-  }
-}
-
-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);
-}
-
-template <bool count_star, bool nullable_type>
-void AggregationHandleCount<count_star, nullable_type>::mergeStatesFast(
-    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);
-}
-
-template <bool count_star, bool nullable_type>
-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);
-}
-
-template <bool count_star, bool nullable_type>
-void AggregationHandleCount<count_star, nullable_type>::
-    aggregateOnDistinctifyHashTableForGroupBy(
-        const AggregationStateHashTableBase &distinctify_hash_table,
-        AggregationStateHashTableBase *aggregation_hash_table,
-        std::size_t index) const {
-  DCHECK_EQ(count_star, false);
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleCount<count_star, nullable_type>,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
-}
-
 // 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/9ccd5a31/expressions/aggregation/AggregationHandleCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index 6aab0cd..629cf11 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.hpp
@@ -27,11 +27,8 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
-#include "types/TypedValue.hpp"
+#include "types/LongType.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -49,42 +46,6 @@ class AggregationHandleCount;
  */
 
 /**
- * @brief Aggregation state of count.
- */
-class AggregationStateCount : public AggregationState {
- public:
-  /**
-   * @brief Copy constructor.
-   */
-  AggregationStateCount(const AggregationStateCount &state)
-      : count_(state.count_.load(std::memory_order_relaxed)) {}
-
-  /**
-   * @brief Destructor.
-   */
-  ~AggregationStateCount() override {}
-
-  std::size_t getPayloadSize() const { return sizeof(count_); }
-
-  const std::uint8_t* getPayloadAddress() const {
-    return reinterpret_cast<const uint8_t *>(&count_);
-  }
-
- private:
-  friend class AggregationHandleCount<false, false>;
-  friend class AggregationHandleCount<false, true>;
-  friend class AggregationHandleCount<true, false>;
-  friend class AggregationHandleCount<true, true>;
-
-  AggregationStateCount() : count_(0) {}
-
-  explicit AggregationStateCount(const std::int64_t initial_count)
-      : count_(initial_count) {}
-
-  std::atomic<std::int64_t> count_;
-};
-
-/**
  * @brief An aggregationhandle for count.
  *
  * @param count_star If true, this AggregationHandleCount is for nullary
@@ -94,151 +55,35 @@ class AggregationStateCount : public AggregationState {
  *        not nullable and NULL-checks can safely be skipped.
  **/
 template <bool count_star, bool nullable_type>
-class AggregationHandleCount : public AggregationConcreteHandle {
+class AggregationHandleCount : public AggregationHandle {
  public:
   ~AggregationHandleCount() override {}
 
-  AggregationState* createInitialState() const override {
-    return new AggregationStateCount();
-  }
-
-  AggregationStateHashTableBase* 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 override;
+  void accumulateNullary(
+      void *state,
+      const std::size_t num_tuples) const override;
 
-  inline void iterateNullaryInl(AggregationStateCount *state) const {
-    state->count_.fetch_add(1, std::memory_order_relaxed);
-  }
-
-  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 {
-    if ((!nullable_type) || (!value.isNull())) {
-      state->count_.fetch_add(1, std::memory_order_relaxed);
-    }
-  }
-
-  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 updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
-
-  inline void updateStateNullary(std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateNullaryInlFast(byte_ptr);
-    }
-  }
-
-  void blockUpdate() override { block_update_ = true; }
-
-  void allowUpdate() override { block_update_ = false; }
-
-  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 {
-    return new AggregationStateCount(num_tuples);
-  }
-
-  AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
-      const override;
+  void accumulateColumnVectors(
+      void *state,
+      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const override;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
+  void accumulateValueAccessor(
+      void *state,
       ValueAccessor *accessor,
       const std::vector<attribute_id> &accessor_id) const override;
 #endif
 
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  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 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 std::uint8_t *byte_ptr) const {
-    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,
-      int index) const override;
-
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for SUM aggregation.
-   */
-  AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
-
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        for SUM aggregation.
-   */
-  void aggregateOnDistinctifyHashTableForGroupBy(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *aggregation_hash_table,
-      std::size_t index) const override;
-
-  std::size_t getPayloadSize() const override { return sizeof(std::int64_t); }
-
  private:
   friend class AggregateFunctionCount;
 
+  typedef LongType ResultType;
+  typedef ResultType::cpptype ResultCppType;
+
   /**
    * @brief Constructor.
    **/
-  AggregationHandleCount() : block_update_(false) {}
-
-  bool block_update_;
+  AggregationHandleCount();
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleCount);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleDistinct.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.cpp b/expressions/aggregation/AggregationHandleDistinct.cpp
deleted file mode 100644
index 0dc8b56..0000000
--- a/expressions/aggregation/AggregationHandleDistinct.cpp
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * 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.
- **/
-
-#include "expressions/aggregation/AggregationHandleDistinct.hpp"
-
-#include <cstddef>
-#include <memory>
-#include <vector>
-#include <utility>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-
-#include "types/TypedValue.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-class ColumnVector;
-class StorageManager;
-class Type;
-class ValueAccessor;
-
-AggregationStateHashTableBase* AggregationHandleDistinct::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 createDistinctifyHashTable(
-      hash_table_impl,
-      group_by_types,
-      estimated_num_groups,
-      storage_manager);
-}
-
-void AggregationHandleDistinct::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(argument_ids.size(), 0u);
-
-  insertValueAccessorIntoDistinctifyHashTable(
-      accessor,
-      group_by_key_ids,
-      hash_table);
-}
-
-ColumnVector* AggregationHandleDistinct::finalizeHashTable(
-    const AggregationStateHashTableBase &hash_table,
-    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).forEachCompositeKeyFast(&keys_retriever);
-
-  return nullptr;
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/expressions/aggregation/AggregationHandleDistinct.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.hpp b/expressions/aggregation/AggregationHandleDistinct.hpp
deleted file mode 100644
index 838bfdd..0000000
--- a/expressions/aggregation/AggregationHandleDistinct.hpp
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * 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_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_DISTINCT_HPP_
-#define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_DISTINCT_HPP_
-
-#include <cstddef>
-#include <memory>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationConcreteHandle.hpp"
-#include "storage/HashTableBase.hpp"
-#include "types/TypedValue.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-class AggregationState;
-class ColumnVector;
-class StorageManager;
-class Type;
-class ValueAccessor;
-
-/** \addtogroup Expressions
- *  @{
- */
-
-class AggregationHandleDistinct : public AggregationConcreteHandle {
- public:
-  /**
-   * @brief Constructor.
-   **/
-  AggregationHandleDistinct() {}
-
-  AggregationState* createInitialState() const override {
-    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* 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().";
-  }
-#endif
-
-  void mergeStates(const AggregationState &source,
-                   AggregationState *destination) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support mergeStates().";
-  }
-
-  TypedValue finalize(const AggregationState &state) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support finalize().";
-  }
-
-  AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support "
-               << "aggregateOnDistinctifyHashTableForSingle().";
-  }
-
-  void aggregateOnDistinctifyHashTableForGroupBy(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *groupby_hash_table,
-      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::size_t estimated_num_groups,
-      StorageManager *storage_manager) const override;
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  ColumnVector* finalizeHashTable(
-      const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(AggregationHandleDistinct);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_DISTINCT_HPP_


[3/6] incubator-quickstep git commit: Updates

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
deleted file mode 100644
index 4a95cd9..0000000
--- a/storage/FastHashTable.hpp
+++ /dev/null
@@ -1,2515 +0,0 @@
-/**
- *   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 "threading/SpinMutex.hpp"
-#include "threading/SpinSharedMutex.hpp"
-#include "types/Type.hpp"
-#include "types/TypedValue.hpp"
-#include "utility/BloomFilter.hpp"
-#include "utility/HashPair.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-/** \addtogroup Storage
- *  @{
- */
-
-/**
- * @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).
- *
- * 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
- *         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.
- *
- * @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 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.");
-
- public:
-  // Shadow template parameters. This is useful for shared test harnesses.
-  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 std::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 std::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 std::uint8_t *initial_value_ptr,
-              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 upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                              const std::uint8_t *init_value_ptr,
-                              FunctorT *functor);
-
-  template <typename FunctorT>
-  bool upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                              const std::uint8_t *init_value_ptr,
-                              FunctorT *functor,
-                              int index);
-
-  bool upsertCompositeKeyFast(const std::vector<TypedValue> &key,
-                              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
-   *        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).
-   **/
-  bool upsertValueAccessorFast(
-      const 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).
-   **/
-  bool upsertValueAccessorCompositeKeyFast(
-      const std::vector<attribute_id> &argument,
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys) override;
-
-  /**
-   * @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 std::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 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.
-   *
-   * @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 std::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 std::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 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),
-        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;
-    }
-  }
-
-  /**
-   * @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 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
-  // 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 std::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 *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 std::uint8_t **value,
-                            std::size_t *entry_num) const = 0;
-  virtual bool getNextEntryCompositeKey(std::vector<TypedValue> *key,
-                                        const std::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 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 std::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 unsigned int num_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_;
-
- 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 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) {
-      {
-        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 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 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;
-  if (resizable) {
-    for (;;) {
-      {
-        SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
-        std::uint8_t *value =
-            upsertInternalFast(key, variable_size, initial_value_ptr);
-        if (value != nullptr) {
-          (*functor)(value);
-          return true;
-        }
-      }
-      resize(0, force_key_copy && !scalar_key_inline_ ? key.getDataSize() : 0);
-    }
-  } else {
-    std::uint8_t *value =
-        upsertInternalFast(key, variable_size, initial_value_ptr);
-    if (value == nullptr) {
-      return false;
-    } else {
-      (*functor)(value);
-      return true;
-    }
-  }
-}
-
-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.
-   **/
-  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.
-   *
-   * @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 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
-      // 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, source_state));
-    } else {
-      destination_hash_table_->putCompositeKey(group_by_key, source_state);
-    }
-  }
-
- private:
-  FastHashTable<true, false, true, false> *destination_hash_table_;
-
-  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) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  const std::size_t variable_size =
-      calculateVariableLengthCompositeKeyCopySize(key);
-  if (resizable) {
-    for (;;) {
-      {
-        SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
-        std::uint8_t *value =
-            upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
-        if (value != nullptr) {
-          (*functor)(value);
-          return true;
-        }
-      }
-      resize(0, variable_size);
-    }
-  } else {
-    std::uint8_t *value =
-        upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
-    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>::
-    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_);
-        std::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 {
-    std::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>::
-    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);
-  if (resizable) {
-    for (;;) {
-      {
-        SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
-        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 < num_handles_; ++k) {
-            handles_[k]->mergeStatesFast(source_state + payload_offsets_[k],
-                                         value + payload_offsets_[k]);
-          }
-          return true;
-        }
-      }
-      resize(0, variable_size);
-    }
-  } else {
-    std::uint8_t *value =
-        upsertCompositeKeyInternalFast(key, init_value_ptr, variable_size);
- 

<TRUNCATED>


[2/6] incubator-quickstep git commit: Updates

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/FastHashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTableFactory.hpp b/storage/FastHashTableFactory.hpp
deleted file mode 100644
index 6d0b693..0000000
--- a/storage/FastHashTableFactory.hpp
+++ /dev/null
@@ -1,257 +0,0 @@
-/**
- *   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 payload_sizes The sizes in bytes for the AggregationStates for the
-   *        respective AggregationHandles.
-   * @param handles The AggregationHandles used in this HashTable.
-   * @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);
-      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);
-      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_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/FastSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastSeparateChainingHashTable.hpp b/storage/FastSeparateChainingHashTable.hpp
deleted file mode 100644
index a41535c..0000000
--- a/storage/FastSeparateChainingHashTable.hpp
+++ /dev/null
@@ -1,1734 +0,0 @@
-/**
- *   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/FastHashTable.hpp"
-#include "storage/HashTable.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 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 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 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 std::uint8_t **value,
-                    std::size_t *entry_num) const override;
-  bool getNextEntryCompositeKey(std::vector<TypedValue> *key,
-                                const std::uint8_t **value,
-                                std::size_t *entry_num) const override;
-
-  bool getNextEntryForKey(const TypedValue &key,
-                          const std::size_t hash_code,
-                          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 std::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;
-
- 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));
-  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.
-  //
-  // 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(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(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_'.
-  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 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()));
-
-  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 std::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 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);
-  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 std::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 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);
-  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 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);
-  }
-
-  // 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 std::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 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);
-  }
-}
-
-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 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);
-  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 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);
-  }
-}
-
-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 std::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) 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);
-
-  // 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 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) {
-      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);
-
-  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);
-
-  // We're all done.
-  return HashTablePutResult::kOK;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-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()));
-
-  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<std::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.
-  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);
-
-  // Return the value.
-  return value;
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-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());
-
-  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<std::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);
-
-  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);
-
-  // 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 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_;
-    *key = key_manager_.getKeyComponentTyped(bucket, 0);
-    *value = reinterpret_cast<const std::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 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;
-         key_idx < this->key_types_.size();
-         ++key_idx) {
-      key->emplace_back(key_manager_.getKeyComponentTyped(bucket, key_idx));
-    }
-    *value = reinterpret_cast<const std::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 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()));
-
-  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 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.
-        *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 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);
-  } 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 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.
-        *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.
-  // NOTE(harshad) - Regarding point 4 above, as this is a specialized
-  // hash table implemented for aggregation, the values are trivially copyable,
-  // therefore we don't need to invoke payload values' copy/move constructors.
-  std::memcpy(resized_buckets, buckets_, original_buckets_used * 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<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;
-    }
-  }
-}
-
-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/9ccd5a31/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index f98c645..f823494 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -28,6 +28,8 @@
 
 namespace quickstep {
 
+class AggregationResultIterator;
+
 /** \addtogroup Storage
  *  @{
  */
@@ -74,31 +76,6 @@ 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<attribute_id> &argument,
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys) {
-    return false;
-  }
-
  protected:
   HashTableBase() {}
 
@@ -106,24 +83,24 @@ class HashTableBase {
   DISALLOW_COPY_AND_ASSIGN(HashTableBase);
 };
 
-typedef HashTableBase<true, false, true, false> AggregationStateHashTableBase;
-
-class AggregationHashTableBase {
+class AggregationStateHashTableBase {
  public:
+  virtual ~AggregationStateHashTableBase() {}
+
   virtual bool upsertValueAccessor(ValueAccessor *accessor,
                                    const attribute_id key_attr_id,
-                                   const std::vector<attribute_id> &argument_ids,
-                                   const bool check_for_null_keys) = 0;
+                                   const std::vector<attribute_id> &argument_ids) = 0;
 
   virtual bool upsertValueAccessorCompositeKey(ValueAccessor *accessor,
                                                const std::vector<attribute_id> &key_attr_ids,
-                                               const std::vector<attribute_id> &argument_ids,
-                                               const bool check_for_null_keys) = 0;
+                                               const std::vector<attribute_id> &argument_ids) = 0;
+
+  virtual AggregationResultIterator* createResultIterator() const = 0;
 
   virtual void print() const = 0;
 
  protected:
-  AggregationHashTableBase() {}
+  AggregationStateHashTableBase() {}
 };
 
 



[4/6] incubator-quickstep git commit: Updates

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index cbbfc22..9fa3bd2 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -34,6 +34,7 @@
 #include "storage/HashTablePool.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "utility/Macros.hpp"
+#include "utility/ScopedBuffer.hpp"
 
 namespace quickstep {
 
@@ -167,8 +168,8 @@ class AggregationOperationState {
    **/
   void finalizeAggregate(InsertDestination *output_destination);
 
-  static void mergeGroupByHashTables(AggregationStateHashTableBase *src,
-                                     AggregationStateHashTableBase *dst);
+  static void mergeGroupByHashTables(AggregationStateHashTableBase *destination_hash_table,
+                                     const AggregationStateHashTableBase *source_hash_table);
 
   int dflag;
 
@@ -176,7 +177,7 @@ class AggregationOperationState {
   // Merge locally (per storage block) aggregated states with global aggregation
   // states.
   void mergeSingleState(
-      const std::vector<std::unique_ptr<AggregationState>> &local_state);
+      const std::vector<ScopedBuffer> &local_state);
 
   // Aggregate on input block.
   void aggregateBlockSingleState(const block_id input_block);
@@ -201,10 +202,6 @@ class AggregationOperationState {
   // arguments.
   std::vector<bool> is_distinct_;
 
-  // Hash table for obtaining distinct (i.e. unique) arguments.
-  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
   // 'input_relation_', then this caches the attribute IDs of those arguments.
@@ -212,14 +209,7 @@ class AggregationOperationState {
 #endif
 
   // Per-aggregate global states for aggregation without GROUP BY.
-  std::vector<std::unique_ptr<AggregationState>> single_states_;
-
-  // Per-aggregate HashTables for aggregation with GROUP BY.
-  //
-  // 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<ScopedBuffer> single_states_;
 
   // A vector of group by hash table pools.
   std::unique_ptr<HashTablePool> group_by_hashtable_pool_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/AggregationResultIterator.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationResultIterator.hpp b/storage/AggregationResultIterator.hpp
new file mode 100644
index 0000000..259c533
--- /dev/null
+++ b/storage/AggregationResultIterator.hpp
@@ -0,0 +1,104 @@
+/**
+ * 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_STORAGE_AGGREGATION_RESULT_ITERATOR_HPP_
+#define QUICKSTEP_STORAGE_AGGREGATION_RESULT_ITERATOR_HPP_
+
+#include <cstddef>
+#include <vector>
+
+#include "storage/AggregationStateManager.hpp"
+#include "storage/HashTableUntypedKeyManager.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+class AggregationResultIterator {
+ public:
+  AggregationResultIterator(const void *buckets,
+                            const std::size_t bucket_size,
+                            const std::size_t num_entries,
+                            const HashTableUntypedKeyManager &key_manager,
+                            const AggregationStateManager<false> &state_manager)
+      : buckets_(buckets),
+        bucket_size_(bucket_size),
+        num_entries_(num_entries),
+        key_manager_(key_manager),
+        state_manager_(state_manager) {}
+
+  inline std::size_t getKeySize() const {
+    return key_manager_.getFixedKeySize();
+  }
+
+  inline std::size_t getResultsSize() const {
+    return state_manager_.getResultsSizeInBytes();
+  }
+
+  inline void beginIteration() {
+    current_position_ = std::numeric_limits<std::size_t>::max();
+  }
+
+  inline bool iterationFinished() const {
+    return current_position_ + 1 >= num_entries_;
+  }
+
+  inline bool next() {
+    ++current_position_;
+    return current_position_ < num_entries_;
+  }
+
+  inline void previous() {
+    --current_position_;
+  }
+
+  inline void writeKeyTo(void *destination) const {
+    key_manager_.copyUntypedKey(
+        destination,
+        key_manager_.getUntypedKeyComponent(getCurrentBucket()));
+  }
+
+  inline void writeResultsTo(void *destination) const {
+    state_manager_.finalizeStates(destination, getCurrentBucket());
+  }
+
+ private:
+  inline const void* getCurrentBucket() const {
+    return static_cast<const char *>(buckets_) + current_position_ * bucket_size_;
+  }
+
+  friend class ThreadPrivateAggregationStateHashTable;
+
+  std::size_t current_position_;
+
+  const void *buckets_;
+  const std::size_t bucket_size_;
+  const std::size_t num_entries_;
+  const HashTableUntypedKeyManager &key_manager_;
+  const AggregationStateManager<false> &state_manager_;
+
+  DISALLOW_COPY_AND_ASSIGN(AggregationResultIterator);
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_AGGREGATION_RESULT_ITERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/AggregationStateHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationStateHashTable.hpp b/storage/AggregationStateHashTable.hpp
new file mode 100644
index 0000000..85a6bdc
--- /dev/null
+++ b/storage/AggregationStateHashTable.hpp
@@ -0,0 +1,338 @@
+/**
+ * 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_STORAGE_AGGREGATION_STATE_HASH_TABLE_HPP_
+#define QUICKSTEP_STORAGE_AGGREGATION_STATE_HASH_TABLE_HPP_
+
+#include <algorithm>
+#include <atomic>
+#include <cstddef>
+#include <cstdlib>
+#include <cstring>
+#include <limits>
+#include <memory>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "storage/AggregationResultIterator.hpp"
+#include "storage/AggregationStateManager.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/HashTableUntypedKeyManager.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "threading/SpinMutex.hpp"
+#include "threading/SpinSharedMutex.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFunctors.hpp"
+#include "utility/Alignment.hpp"
+#include "utility/InlineMemcpy.hpp"
+#include "utility/Macros.hpp"
+#include "utility/PrimeNumber.hpp"
+#include "utility/ScopedBuffer.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+class ThreadPrivateAggregationStateHashTable : public AggregationStateHashTableBase {
+ public:
+  ThreadPrivateAggregationStateHashTable(const std::vector<const Type *> &key_types,
+                                         const std::size_t num_entries,
+                                         const std::vector<AggregationHandle *> &handles,
+                                         StorageManager *storage_manager)
+    : payload_manager_(handles),
+      key_types_(key_types),
+      key_manager_(this->key_types_, payload_manager_.getStatesSizeInBytes()),
+      slots_(num_entries * kHashTableLoadFactor,
+             key_manager_.getUntypedKeyHashFunctor(),
+             key_manager_.getUntypedKeyEqualityFunctor()),
+      bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize(),
+                                     payload_manager_.getStatesSizeInBytes())),
+      buckets_allocated_(0),
+      storage_manager_(storage_manager) {
+    std::size_t num_storage_slots =
+        this->storage_manager_->SlotsNeededForBytes(num_entries);
+
+    // 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);
+
+    buckets_ = this->blob_->getMemoryMutable();
+    num_buckets_ = num_storage_slots * kSlotSizeBytes / bucket_size_;
+  }
+
+  ~ThreadPrivateAggregationStateHashTable() {}
+
+  inline std::size_t numEntries() const {
+    return buckets_allocated_;
+  }
+
+  inline std::size_t getKeySizeInBytes() const {
+    return key_manager_.getFixedKeySize();
+  }
+
+  inline std::size_t getStatesSizeInBytes() const {
+    return payload_manager_.getStatesSizeInBytes();
+  }
+
+  inline std::size_t getResultsSizeInBytes() const {
+    return payload_manager_.getResultsSizeInBytes();
+  }
+
+  AggregationResultIterator* createResultIterator() const override {
+    return new AggregationResultIterator(buckets_,
+                                         bucket_size_,
+                                         buckets_allocated_,
+                                         key_manager_,
+                                         payload_manager_);
+  }
+
+  bool upsertValueAccessor(ValueAccessor *accessor,
+                           const attribute_id key_attr_id,
+                           const std::vector<attribute_id> &argument_ids) override {
+    if (key_manager_.isKeyNullable()) {
+      return upsertValueAccessorInternal<true>(
+          accessor, key_attr_id, argument_ids);
+    } else {
+      return upsertValueAccessorInternal<false>(
+          accessor, key_attr_id, argument_ids);
+    }
+  }
+
+  template <bool check_for_null_keys>
+  bool upsertValueAccessorInternal(ValueAccessor *accessor,
+                                   const attribute_id key_attr_id,
+                                   const std::vector<attribute_id> &argument_ids) {
+    return InvokeOnAnyValueAccessor(
+        accessor,
+        [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
+      accessor->beginIteration();
+      while (accessor->next()) {
+        const void *key = accessor->template getUntypedValue<check_for_null_keys>(key_attr_id);
+        if (check_for_null_keys && key == nullptr) {
+          continue;
+        }
+        bool is_empty;
+        void *bucket = locateBucket(key, &is_empty);
+        if (is_empty) {
+          payload_manager_.initializeStates(bucket);
+        } else {
+          payload_manager_.template updateStates<check_for_null_keys>(
+              bucket, accessor, argument_ids);
+        }
+      }
+      return true;
+    });
+  }
+
+  bool upsertValueAccessorCompositeKey(ValueAccessor *accessor,
+                                       const std::vector<attribute_id> &key_attr_ids,
+                                       const std::vector<attribute_id> &argument_ids) override {
+    if (key_manager_.isKeyNullable()) {
+      return upsertValueAccessorCompositeKeyInternal<true>(
+          accessor, key_attr_ids, argument_ids);
+    } else {
+      return upsertValueAccessorCompositeKeyInternal<false>(
+          accessor, key_attr_ids, argument_ids);
+    }
+  }
+
+  template <bool check_for_null_keys>
+  bool upsertValueAccessorCompositeKeyInternal(ValueAccessor *accessor,
+                                               const std::vector<attribute_id> &key_attr_ids,
+                                               const std::vector<attribute_id> &argument_ids) {
+    return InvokeOnAnyValueAccessor(
+        accessor,
+        [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
+      accessor->beginIteration();
+      void *prealloc_bucket = allocateBucket();
+      while (accessor->next()) {
+        if (check_for_null_keys) {
+          const bool is_null =
+              key_manager_.writeNullableUntypedKeyFromValueAccessorToBucket(
+                  accessor,
+                  key_attr_ids,
+                  prealloc_bucket);
+          if (is_null) {
+            continue;
+          }
+        } else {
+          key_manager_.writeUntypedKeyFromValueAccessorToBucket(
+              accessor,
+              key_attr_ids,
+              prealloc_bucket);
+        }
+        void *bucket = locateBucketWithPrealloc(prealloc_bucket);
+        if (bucket == prealloc_bucket) {
+          payload_manager_.initializeStates(bucket);
+          prealloc_bucket = allocateBucket();
+        } else {
+          payload_manager_.template updateStates<check_for_null_keys>(
+              bucket, accessor, argument_ids);
+        }
+      }
+      // Reclaim the last unused bucket
+      --buckets_allocated_;
+      return true;
+    });
+  }
+
+  void mergeHashTable(const ThreadPrivateAggregationStateHashTable *source_hash_table) {
+    source_hash_table->forEachKeyAndStates(
+        [&](const void *source_key, const void *source_states) -> void {
+          bool is_empty;
+          void *bucket = locateBucket(source_key, &is_empty);
+          if (is_empty) {
+            payload_manager_.copyStates(bucket, source_states);
+          } else {
+            payload_manager_.mergeStates(bucket, source_states);
+          }
+        });
+  }
+
+  template <typename FunctorT>
+  inline void forEachKey(const FunctorT &functor) const {
+    for (std::size_t i = 0; i < buckets_allocated_; ++i) {
+      functor(key_manager_.getUntypedKeyComponent(locateBucket(i)));
+    }
+  }
+
+  template <typename FunctorT>
+  inline void forEachKeyAndStates(const FunctorT &functor) const {
+    for (std::size_t i = 0; i < buckets_allocated_; ++i) {
+      const char *bucket = static_cast<const char *>(locateBucket(i));
+      functor(key_manager_.getUntypedKeyComponent(bucket), bucket);
+    }
+  }
+
+  inline void* locateBucket(const std::size_t bucket_id) const {
+    return static_cast<char *>(buckets_) + bucket_id * bucket_size_;
+  }
+
+  inline void* locateBucket(const void *key, bool *is_empty) {
+    auto slot_it = slots_.find(key);
+    if (slot_it == slots_.end()) {
+      void *bucket = allocateBucket();
+      key_manager_.writeUntypedKeyToBucket(key, bucket);
+      slots_.emplace(key_manager_.getUntypedKeyComponent(bucket), bucket);
+      *is_empty = true;
+      return bucket;
+    } else {
+      *is_empty = false;
+      return slot_it->second;
+    }
+  }
+
+  inline void* locateBucketWithPrealloc(void *prealloc_bucket) {
+    const void *key = key_manager_.getUntypedKeyComponent(prealloc_bucket);
+    auto slot_it = slots_.find(key);
+    if (slot_it == slots_.end()) {
+      slots_.emplace(key, prealloc_bucket);
+      return prealloc_bucket;
+    } else {
+      return slot_it->second;
+    }
+  }
+
+  inline void* allocateBucket() {
+    if (buckets_allocated_ >= num_buckets_) {
+      resize();
+    }
+    void *bucket = locateBucket(buckets_allocated_);
+    ++buckets_allocated_;
+    return bucket;
+  }
+
+  void resize() {
+    const std::size_t resized_memory_required = num_buckets_ * bucket_size_ * 2;
+    const std::size_t resized_storage_slots =
+        this->storage_manager_->SlotsNeededForBytes(resized_memory_required);
+    const block_id resized_blob_id =
+        this->storage_manager_->createBlob(resized_storage_slots);
+    MutableBlobReference resized_blob =
+        this->storage_manager_->getBlobMutable(resized_blob_id);
+
+    void *resized_buckets = resized_blob->getMemoryMutable();
+    std::memcpy(resized_buckets, buckets_, buckets_allocated_ * bucket_size_);
+
+    for (auto &pair : slots_) {
+      pair.second =
+           (static_cast<const char *>(pair.first) - static_cast<char *>(buckets_))
+           + static_cast<char *>(resized_buckets);
+    }
+
+    buckets_ = resized_buckets;
+    num_buckets_ = resized_storage_slots * kSlotSizeBytes / bucket_size_;
+    std::swap(this->blob_, resized_blob);
+  }
+
+  void print() const override {
+    std::cerr << "Bucket size = " << bucket_size_ << "\n";
+    std::cerr << "Buckets: \n";
+    for (const auto &pair : slots_) {
+      std::cerr << pair.first << " -- " << pair.second << "\n";
+      std::cerr << *static_cast<const int *>(pair.second) << "\n";
+    }
+  }
+
+ private:
+  // Helper object to manage hash table payloads (i.e. aggregation states).
+  AggregationStateManager<false> payload_manager_;
+
+  // Type(s) of keys.
+  const std::vector<const Type*> key_types_;
+
+  // Helper object to manage key storage.
+  HashTableUntypedKeyManager key_manager_;
+
+  // Round bucket size up to a multiple of kBucketAlignment.
+  static std::size_t ComputeBucketSize(const std::size_t fixed_key_size,
+                                       const std::size_t total_payload_size) {
+    constexpr std::size_t kBucketAlignment = 4;
+    return (((fixed_key_size + total_payload_size - 1)
+               / kBucketAlignment) + 1) * kBucketAlignment;
+  }
+
+  std::unordered_map<const void *, void *,
+                     UntypedHashFunctor,
+                     UntypedEqualityFunctor> slots_;
+
+  void *buckets_;
+  const std::size_t bucket_size_;
+  std::size_t num_buckets_;
+  std::size_t buckets_allocated_;
+
+  StorageManager *storage_manager_;
+  MutableBlobReference blob_;
+
+  DISALLOW_COPY_AND_ASSIGN(ThreadPrivateAggregationStateHashTable);
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_AGGREGATION_STATE_HASH_TABLE_HPP_
+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/AggregationStateManager.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationStateManager.hpp b/storage/AggregationStateManager.hpp
new file mode 100644
index 0000000..98dca90
--- /dev/null
+++ b/storage/AggregationStateManager.hpp
@@ -0,0 +1,181 @@
+/**
+ * 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_STORAGE_AGGREGATION_STATE_MANAGER_HPP_
+#define QUICKSTEP_STORAGE_AGGREGATION_STATE_MANAGER_HPP_
+
+#include <cstddef>
+#include <cstring>
+#include <vector>
+
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "catalog/CatalogTypedefs.hpp"
+#include "threading/SpinMutex.hpp"
+#include "threading/SpinSharedMutex.hpp"
+#include "utility/InlineMemcpy.hpp"
+#include "utility/Macros.hpp"
+#include "utility/ScopedBuffer.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+template <bool use_mutex>
+class AggregationStateManager {
+ public:
+  AggregationStateManager(const std::vector<AggregationHandle *> &handles)
+      : handles_(handles),
+        states_size_in_bytes_(0),
+        results_size_in_bytes_(0) {
+    if (use_mutex) {
+      states_size_in_bytes_ += sizeof(SpinMutex);
+    }
+    for (const AggregationHandle *handle : handles) {
+      const std::size_t state_size = handle->getStateSize();
+      state_sizes_.emplace_back(state_size);
+      state_offsets_.emplace_back(states_size_in_bytes_);
+      states_size_in_bytes_ += state_size;
+
+      const std::size_t result_size = handle->getResultSize();
+      result_sizes_.emplace_back(result_size);
+      result_offsets_.emplace_back(results_size_in_bytes_);
+      results_size_in_bytes_ += result_size;
+
+      accumulate_functors_.emplace_back(handle->getStateAccumulateFunctor());
+      merge_functors_.emplace_back(handle->getStateMergeFunctor());
+      finalize_functors_.emplace_back(handle->getStateFinalizeFunctor());
+    }
+
+    initial_states_.reset(states_size_in_bytes_, false);
+    if (use_mutex) {
+      new(initial_states_.get()) Mutex;
+    }
+    for (std::size_t i = 0; i < handles_.size(); ++i) {
+      handles_[i]->initializeState(
+          static_cast<char *>(initial_states_.get()) + state_offsets_[i]);
+    }
+  }
+
+  inline std::size_t getStatesSizeInBytes() const {
+    return states_size_in_bytes_;
+  }
+
+  inline std::size_t getResultsSizeInBytes() const {
+    return results_size_in_bytes_;
+  }
+
+  inline void initializeStates(void *states) const {
+    copyStates(states, initial_states_.get());
+  }
+
+  template <bool check_for_null_keys, typename ValueAccessorT>
+  inline void updateState(void *states,
+                          ValueAccessorT *accessor,
+                          const attribute_id argument_id) const {
+    // TODO: templates on whether to check invalid attribute id
+    DCHECK_NE(argument_id, kInvalidAttributeID);
+
+    const void *value =
+        accessor->template getUntypedValue<check_for_null_keys>(argument_id);
+    if (check_for_null_keys && value == nullptr) {
+      return;
+    }
+    accumulate_functors_.front()(states, value);
+  }
+
+  template <bool check_for_null_keys, typename ValueAccessorT>
+  inline void updateStates(void *states,
+                           ValueAccessorT *accessor,
+                           const std::vector<attribute_id> &argument_ids) const {
+    for (std::size_t i = 0; i < argument_ids.size(); ++i) {
+      // TODO: templates on whether to check invalid attribute id
+      DCHECK_NE(argument_ids[i], kInvalidAttributeID);
+
+      const void *value =
+          accessor->template getUntypedValue<check_for_null_keys>(argument_ids[i]);
+      if (check_for_null_keys && value == nullptr) {
+        return;
+      }
+      accumulate_functors_[i](getStateComponent(states, i), value);
+    }
+  }
+
+  inline void copyStates(void *destination_states,
+                         const void *source_states) const {
+    InlineMemcpy(destination_states, source_states, states_size_in_bytes_);
+  }
+
+  inline void mergeStates(void *destination_states,
+                          const void *source_states) const {
+    for (std::size_t i = 0; i < merge_functors_.size(); ++i) {
+      merge_functors_[i](getStateComponent(destination_states, i),
+                         getStateComponent(source_states, i));
+    }
+  }
+
+  inline void finalizeStates(void *results, const void *states) const {
+    for (std::size_t i = 0; i < merge_functors_.size(); ++i) {
+      finalize_functors_[i](getResultComponent(results, i),
+                            getStateComponent(states, i));
+    }
+  }
+
+  inline const void* getStateComponent(const void *states,
+                                       const std::size_t component_id) const {
+    return static_cast<const char *>(states) + state_offsets_[component_id];
+  }
+
+  inline void* getStateComponent(void *states,
+                                 const std::size_t component_id) const {
+    return static_cast<char *>(states) + state_offsets_[component_id];
+  }
+
+  inline void* getResultComponent(void *results,
+                                  const std::size_t component_id) const {
+    return static_cast<char *>(results) + result_offsets_[component_id];
+  }
+
+ private:
+  std::vector<AggregationHandle *> handles_;
+
+  std::vector<std::size_t> state_sizes_;
+  std::vector<std::size_t> state_offsets_;
+  std::size_t states_size_in_bytes_;
+
+  std::vector<std::size_t> result_sizes_;
+  std::vector<std::size_t> result_offsets_;
+  std::size_t results_size_in_bytes_;
+
+  std::vector<AggregationStateAccumulateFunctor> accumulate_functors_;
+  std::vector<AggregationStateMergeFunctor> merge_functors_;
+  std::vector<AggregationStateFinalizeFunctor> finalize_functors_;
+
+  ScopedBuffer initial_states_;
+
+  DISALLOW_COPY_AND_ASSIGN(AggregationStateManager);
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_AGGREGATION_STATE_MANAGER_HPP_
+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9ccd5a31/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index bdc7596..0aaaca4 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -145,11 +145,13 @@ if (ENABLE_DISTRIBUTED)
 endif()
 
 # Declare micro-libs:
-add_library(quickstep_storage_AggregationHashTable ../empty_src.cpp AggregationHashTable.hpp)
+add_library(quickstep_storage_AggregationStateHashTable ../empty_src.cpp AggregationStateHashTable.hpp)
+add_library(quickstep_storage_AggregationStateManager ../empty_src.cpp AggregationStateManager.hpp)
 add_library(quickstep_storage_AggregationOperationState
             AggregationOperationState.cpp
             AggregationOperationState.hpp)
 add_library(quickstep_storage_AggregationOperationState_proto ${storage_AggregationOperationState_proto_srcs})
+add_library(quickstep_storage_AggregationResultIterator ../empty_src.cpp AggregationResultIterator.hpp)
 add_library(quickstep_storage_BasicColumnStoreTupleStorageSubBlock
             BasicColumnStoreTupleStorageSubBlock.cpp
             BasicColumnStoreTupleStorageSubBlock.hpp)
@@ -199,9 +201,6 @@ 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)
@@ -271,9 +270,11 @@ add_library(quickstep_storage_WindowAggregationOperationState_proto ${storage_Wi
 
 
 # Link dependencies:
-target_link_libraries(quickstep_storage_AggregationHashTable
+target_link_libraries(quickstep_storage_AggregationStateHashTable
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_AggregationResultIterator
+                      quickstep_storage_AggregationStateManager
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableUntypedKeyManager
                       quickstep_storage_StorageBlob
@@ -290,8 +291,18 @@ target_link_libraries(quickstep_storage_AggregationHashTable
                       quickstep_types_TypedValue
                       quickstep_utility_Alignment
                       quickstep_utility_HashPair
+                      quickstep_utility_InlineMemcpy
                       quickstep_utility_Macros
-                      quickstep_utility_PrimeNumber)
+                      quickstep_utility_PrimeNumber
+                      quickstep_utility_ScopedBuffer)
+target_link_libraries(quickstep_storage_AggregationStateManager
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_threading_SpinMutex
+                      quickstep_threading_SpinSharedMutex
+                      quickstep_utility_InlineMemcpy
+                      quickstep_utility_Macros
+                      quickstep_utility_ScopedBuffer)
 target_link_libraries(quickstep_storage_AggregationOperationState
                       glog
                       quickstep_catalog_CatalogDatabaseLite
@@ -302,13 +313,11 @@ target_link_libraries(quickstep_storage_AggregationOperationState
                       quickstep_expressions_aggregation_AggregateFunction
                       quickstep_expressions_aggregation_AggregateFunctionFactory
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_expressions_aggregation_AggregationHandleDistinct
                       quickstep_expressions_aggregation_AggregationID
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
-                      quickstep_storage_AggregationHashTable
+                      quickstep_storage_AggregationStateHashTable
                       quickstep_storage_AggregationOperationState_proto
-                      quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
                       quickstep_storage_HashTablePool
@@ -321,12 +330,17 @@ target_link_libraries(quickstep_storage_AggregationOperationState
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorsValueAccessor
                       quickstep_types_containers_Tuple
-                      quickstep_utility_Macros)
+                      quickstep_utility_Macros
+                      quickstep_utility_ScopedBuffer)
 target_link_libraries(quickstep_storage_AggregationOperationState_proto
                       quickstep_expressions_Expressions_proto
                       quickstep_expressions_aggregation_AggregateFunction_proto
                       quickstep_storage_HashTable_proto
                       ${PROTOBUF_LIBRARY})
+target_link_libraries(quickstep_storage_AggregationResultIterator
+                      quickstep_storage_AggregationStateManager
+                      quickstep_storage_HashTableUntypedKeyManager
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_BasicColumnStoreTupleStorageSubBlock
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelationSchema
@@ -654,53 +668,6 @@ 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_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
@@ -786,9 +753,7 @@ target_link_libraries(quickstep_storage_HashTableKeyManager
 target_link_libraries(quickstep_storage_HashTablePool
                       glog
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_AggregationHashTable
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_FastHashTableFactory
+                      quickstep_storage_AggregationStateHashTable
                       quickstep_storage_HashTableBase
                       quickstep_threading_SpinMutex
                       quickstep_utility_Macros
@@ -799,6 +764,7 @@ target_link_libraries(quickstep_storage_HashTableUntypedKeyManager
                       quickstep_types_Type
                       quickstep_types_TypeFunctors
                       quickstep_types_TypedValue
+                      quickstep_utility_InlineMemcpy
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_IndexSubBlock
                       quickstep_catalog_CatalogTypedefs
@@ -820,6 +786,7 @@ target_link_libraries(quickstep_storage_InsertDestination
                       quickstep_queryexecution_QueryExecutionMessages_proto
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
+                      quickstep_storage_AggregationResultIterator
                       quickstep_storage_InsertDestinationInterface
                       quickstep_storage_InsertDestination_proto
                       quickstep_storage_StorageBlock
@@ -859,6 +826,7 @@ target_link_libraries(quickstep_storage_PackedRowStoreTupleStorageSubBlock
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_predicate_PredicateCost
+                      quickstep_storage_AggregationResultIterator
                       quickstep_storage_PackedRowStoreValueAccessor
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageBlockLayout_proto
@@ -994,6 +962,7 @@ target_link_libraries(quickstep_storage_StorageBlock
                       quickstep_expressions_aggregation_AggregationHandle
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
+                      quickstep_storage_AggregationResultIterator
                       quickstep_storage_BasicColumnStoreTupleStorageSubBlock
                       quickstep_storage_BloomFilterIndexSubBlock
                       quickstep_storage_CSBTreeIndexSubBlock
@@ -1022,7 +991,8 @@ target_link_libraries(quickstep_storage_StorageBlock
                       quickstep_types_containers_Tuple
                       quickstep_types_operations_comparisons_ComparisonUtil
                       quickstep_utility_Macros
-                      quickstep_utility_PtrVector)
+                      quickstep_utility_PtrVector
+                      quickstep_utility_ScopedBuffer)
 # CMAKE_VALIDATE_IGNORE_BEGIN
 if(QUICKSTEP_HAVE_BITWEAVING)
   target_link_libraries(quickstep_storage_StorageBlock
@@ -1184,9 +1154,6 @@ 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
@@ -1208,6 +1175,9 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_SimpleScalarSeparateChainingHashTable
                       quickstep_storage_SplitRowStoreTupleStorageSubBlock
                       quickstep_storage_SplitRowStoreValueAccessor
+                      quickstep_storage_AggregationResultIterator
+                      quickstep_storage_AggregationStateHashTable
+                      quickstep_storage_AggregationStateManager
                       quickstep_storage_StorageBlob
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockBase