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 2017/01/31 23:19:55 UTC

[01/13] incubator-quickstep git commit: Minor refactor for InsertDestinations. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/collision-free-agg af6cf5119 -> b46bc73c8 (forced update)


Minor refactor for InsertDestinations.


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

Branch: refs/heads/collision-free-agg
Commit: f2e77266edeaff38a60650b48836ff6ddb3b84ca
Parents: 0f4938c
Author: Zuyu Zhang <zu...@apache.org>
Authored: Mon Jan 30 15:24:03 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jan 30 15:24:03 2017 -0800

----------------------------------------------------------------------
 storage/InsertDestination.cpp          | 17 ++++-------------
 storage/InsertDestination.hpp          |  4 +++-
 storage/InsertDestinationInterface.hpp |  2 +-
 storage/StorageBlock.hpp               |  2 +-
 4 files changed, 9 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f2e77266/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 944998f..714e6e5 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -290,7 +290,6 @@ void InsertDestination::bulkInsertTuplesFromValueAccessors(
       ValueAccessor *accessor = p.first;
       std::vector<attribute_id> attribute_map = p.second;
 
-
       InvokeOnAnyValueAccessor(
           accessor,
           [&](auto *accessor) -> void {  // NOLINT(build/c++11)
@@ -621,11 +620,10 @@ void PartitionAwareInsertDestination::bulkInsertTuples(ValueAccessor *accessor,
        &always_mark_full,
        &num_partitions](auto *accessor) -> void {  // NOLINT(build/c++11)
     std::vector<std::unique_ptr<TupleIdSequence>> partition_membership;
-    partition_membership.resize(num_partitions);
 
     // Create a tuple-id sequence for each partition.
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
-      partition_membership[partition].reset(new TupleIdSequence(accessor->getEndPosition()));
+      partition_membership.emplace_back(std::make_unique<TupleIdSequence>(accessor->getEndPosition()));
     }
 
     // Iterate over ValueAccessor for each tuple,
@@ -641,9 +639,8 @@ void PartitionAwareInsertDestination::bulkInsertTuples(ValueAccessor *accessor,
     // TupleIdSequence.
     std::vector<std::unique_ptr<typename std::remove_pointer<
         decltype(accessor->createSharedTupleIdSequenceAdapter(*partition_membership.front()))>::type>> adapter;
-    adapter.resize(num_partitions);
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
-      adapter[partition].reset(accessor->createSharedTupleIdSequenceAdapter(*partition_membership[partition]));
+      adapter.emplace_back(accessor->createSharedTupleIdSequenceAdapter(*partition_membership[partition]));
     }
 
     // Bulk-insert into a block belonging to the partition.
@@ -678,11 +675,10 @@ void PartitionAwareInsertDestination::bulkInsertTuplesWithRemappedAttributes(
        &always_mark_full,
        &num_partitions](auto *accessor) -> void {  // NOLINT(build/c++11)
     std::vector<std::unique_ptr<TupleIdSequence>> partition_membership;
-    partition_membership.resize(num_partitions);
 
     // Create a tuple-id sequence for each partition.
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
-      partition_membership[partition].reset(new TupleIdSequence(accessor->getEndPosition()));
+      partition_membership.emplace_back(std::make_unique<TupleIdSequence>(accessor->getEndPosition()));
     }
 
     // Iterate over ValueAccessor for each tuple,
@@ -698,9 +694,8 @@ void PartitionAwareInsertDestination::bulkInsertTuplesWithRemappedAttributes(
     // TupleIdSequence.
     std::vector<std::unique_ptr<typename std::remove_pointer<
         decltype(accessor->createSharedTupleIdSequenceAdapter(*partition_membership.front()))>::type>> adapter;
-    adapter.resize(num_partitions);
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
-      adapter[partition].reset(accessor->createSharedTupleIdSequenceAdapter(*partition_membership[partition]));
+      adapter.emplace_back(accessor->createSharedTupleIdSequenceAdapter(*partition_membership[partition]));
     }
 
     // Bulk-insert into a block belonging to the partition.
@@ -742,10 +737,6 @@ void PartitionAwareInsertDestination::insertTuplesFromVector(std::vector<Tuple>:
   }
 }
 
-MutableBlockReference PartitionAwareInsertDestination::getBlockForInsertion() {
-  FATAL_ERROR("PartitionAwareInsertDestination::getBlockForInsertion needs a partition id as an argument.");
-}
-
 MutableBlockReference PartitionAwareInsertDestination::getBlockForInsertionInPartition(const partition_id part_id) {
   DCHECK_LT(part_id, partition_scheme_header_->getNumPartitions());
   SpinMutexLock lock(mutexes_for_partition_[part_id]);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f2e77266/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index c3c40bd..6707192 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -539,7 +539,9 @@ class PartitionAwareInsertDestination : public InsertDestination {
                               std::vector<Tuple>::const_iterator end) override;
 
  protected:
-  MutableBlockReference getBlockForInsertion() override;
+  MutableBlockReference getBlockForInsertion() override {
+    LOG(FATAL) << "PartitionAwareInsertDestination::getBlockForInsertion needs a partition id as an argument.";
+  }
 
   /**
    * @brief Get a block to use for insertion from a partition.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f2e77266/storage/InsertDestinationInterface.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestinationInterface.hpp b/storage/InsertDestinationInterface.hpp
index b62d3e5..be6b0c2 100644
--- a/storage/InsertDestinationInterface.hpp
+++ b/storage/InsertDestinationInterface.hpp
@@ -131,7 +131,7 @@ class InsertDestinationInterface {
    *
    * @param accessor_attribute_map A vector of pairs of ValueAccessor and
    *        corresponding attribute map
-   *        The i-th attribute ID in the attr map for a value accessor is "n" 
+   *        The i-th attribute ID in the attr map for a value accessor is "n"
    *        if the attribute_id "i" in the output relation
    *        is the attribute_id "n" in corresponding input value accessor.
    *        Set the i-th element to kInvalidCatalogId if it doesn't come from

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f2e77266/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index ed252c5..16ea50f 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -325,7 +325,7 @@ class StorageBlock : public StorageBlockBase {
    *       function with the appropriate attribute_map for each value
    *       accessor (InsertDestination::bulkInsertTuplesFromValueAccessors
    *       handles all the details) to insert tuples without an extra temp copy.
-   * 
+   *
    * @warning Must call bulkInsertPartialTuplesFinalize() to update the header,
    *          until which point, the insertion is not visible to others.
    * @warning The inserted tuples may be placed in sub-optimal locations in this


[12/13] incubator-quickstep git commit: Initial commit.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 642d88d..00b229e 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -25,8 +25,8 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/PackedPayloadAggregationStateHashTable.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
@@ -43,7 +43,8 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleSum::AggregationHandleSum(const Type &type)
-    : argument_type_(type), block_update_(false) {
+    : AggregationConcreteHandle(AggregationID::kSum),
+      argument_type_(type) {
   // We sum Int as Long and Float as Double so that we have more headroom when
   // adding many values.
   TypeID type_precision_id;
@@ -79,47 +80,26 @@ AggregationHandleSum::AggregationHandleSum(const Type &type)
   result_type_ = &sum_type.getNullableVersion();
 }
 
-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::accumulate(
+    ValueAccessor *accessor,
+    ColumnVectorsValueAccessor *aux_accessor,
+    const std::vector<attribute_id> &argument_ids) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for SUM: " << argument_ids.size();
 
-AggregationState* AggregationHandleSum::accumulateColumnVectors(
-    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
-  DCHECK_EQ(1u, column_vectors.size())
-      << "Got wrong number of ColumnVectors for SUM: " << column_vectors.size();
-  std::size_t num_tuples = 0;
-  TypedValue cv_sum = fast_operator_->accumulateColumnVector(
-      blank_state_.sum_, *column_vectors.front(), &num_tuples);
-  return new AggregationStateSum(std::move(cv_sum), num_tuples == 0);
-}
+  const attribute_id argument_id = argument_ids.front();
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleSum::accumulateValueAccessor(
-    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();
+  ValueAccessor *target_accessor =
+      argument_id >= 0 ? accessor : aux_accessor;
+  const attribute_id target_argument_id =
+      argument_id >= 0 ? argument_id : -(argument_id+2);
 
   std::size_t num_tuples = 0;
   TypedValue va_sum = fast_operator_->accumulateValueAccessor(
-      blank_state_.sum_, accessor, accessor_ids.front(), &num_tuples);
+        blank_state_.sum_, target_accessor, target_argument_id, &num_tuples);
   return new AggregationStateSum(std::move(va_sum), num_tuples == 0);
 }
-#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 {
@@ -134,8 +114,8 @@ void AggregationHandleSum::mergeStates(const AggregationState &source,
   sum_destination->null_ = sum_destination->null_ && sum_source.null_;
 }
 
-void AggregationHandleSum::mergeStatesFast(const std::uint8_t *source,
-                                           std::uint8_t *destination) const {
+void AggregationHandleSum::mergeStates(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 =
@@ -164,27 +144,10 @@ 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<
+  return finalizeHashTableHelper<
       AggregationHandleSum,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+      PackedPayloadSeparateChainingAggregationStateHashTable>(
+          *result_type_, hash_table, group_by_keys, index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index f0d23e1..9fb7706 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -28,7 +28,6 @@
 #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"
@@ -41,6 +40,7 @@
 namespace quickstep {
 
 class ColumnVector;
+class ColumnVectorsValueAccessor;
 class StorageManager;
 class ValueAccessor;
 
@@ -101,16 +101,18 @@ class AggregationHandleSum : public AggregationConcreteHandle {
  public:
   ~AggregationHandleSum() override {}
 
+  std::vector<const Type *> getArgumentTypes() const override {
+    return {&argument_type_};
+  }
+
+  const Type* getResultType() const override {
+    return result_type_;
+  }
+
   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()));
@@ -121,28 +123,19 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     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;
-  }
+  AggregationState* accumulate(
+      ValueAccessor *accessor,
+      ColumnVectorsValueAccessor *aux_accessor,
+      const std::vector<attribute_id> &argument_ids) const override;
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  TypedValue finalize(const AggregationState &state) const override;
 
-  void allowUpdate() override { block_update_ = false; }
+  std::size_t getPayloadSize() const override {
+    return blank_state_.getPayloadSize();
+  }
 
   void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *sum_ptr =
@@ -161,41 +154,23 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     }
   }
 
-  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 {
-    return static_cast<const AggregationStateSum &>(state).sum_;
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    DCHECK(argument.isPlausibleInstanceOf(argument_type_.getSignature()));
+    if (argument.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, argument);
+    *null_ptr = false;
   }
 
-  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;
+  void mergeStates(const std::uint8_t *source,
+                   std::uint8_t *destination) const override;
+
+  inline TypedValue finalizeHashTableEntry(const std::uint8_t *byte_ptr) const {
+    return *reinterpret_cast<const TypedValue *>(byte_ptr + blank_state_.sum_offset_);
   }
 
   ColumnVector* finalizeHashTable(
@@ -203,29 +178,6 @@ class AggregationHandleSum : public AggregationConcreteHandle {
       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;
 
@@ -242,8 +194,6 @@ class AggregationHandleSum : public AggregationConcreteHandle {
   std::unique_ptr<UncheckedBinaryOperator> fast_operator_;
   std::unique_ptr<UncheckedBinaryOperator> merge_operator_;
 
-  bool block_update_;
-
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleSum);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationID.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationID.hpp b/expressions/aggregation/AggregationID.hpp
index 1efb35c..cd18d47 100644
--- a/expressions/aggregation/AggregationID.hpp
+++ b/expressions/aggregation/AggregationID.hpp
@@ -32,9 +32,11 @@ namespace quickstep {
 enum class AggregationID {
   kAvg = 0,
   kCount,
+  kDistinct,
   kMax,
   kMin,
-  kSum
+  kSum,
+  kUnknown
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index e9503f7..bd239d4 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -146,10 +146,8 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationConcreteHandl
                       glog
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
                       quickstep_threading_SpinMutex
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
@@ -157,6 +155,7 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationConcreteHandl
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandle
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
                       quickstep_types_TypedValue
                       quickstep_utility_Macros)
@@ -165,10 +164,9 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleAvg
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_storage_PackedPayloadAggregationStateHashTable
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypeFactory
@@ -183,12 +181,12 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleCount
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_storage_PackedPayloadAggregationStateHashTable
                       quickstep_storage_ValueAccessor
                       quickstep_storage_ValueAccessorUtil
+                      quickstep_types_LongType
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
@@ -199,8 +197,9 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleDistinc
                       glog
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
-                      quickstep_storage_HashTable
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
+                      quickstep_storage_PackedPayloadAggregationStateHashTable
                       quickstep_types_TypedValue
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMax
@@ -208,10 +207,9 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMax
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_storage_PackedPayloadAggregationStateHashTable
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypedValue
@@ -225,10 +223,9 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMin
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_storage_PackedPayloadAggregationStateHashTable
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypedValue
@@ -242,10 +239,9 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleSum
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_storage_PackedPayloadAggregationStateHashTable
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypeFactory

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/query_execution/QueryContext.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.hpp b/query_execution/QueryContext.hpp
index 895c2ea..ed0f99c 100644
--- a/query_execution/QueryContext.hpp
+++ b/query_execution/QueryContext.hpp
@@ -200,20 +200,6 @@ class QueryContext {
   }
 
   /**
-   * @brief Destroy the payloads from the aggregation hash tables.
-   *
-   * @warning After calling these methods, the hash table will be in an invalid
-   *          state. No other operation should be performed on them.
-   *
-   * @param id The ID of the AggregationOperationState.
-   **/
-  inline void destroyAggregationHashTablePayload(const aggregation_state_id id) {
-    DCHECK_LT(id, aggregation_states_.size());
-    DCHECK(aggregation_states_[id]);
-    aggregation_states_[id]->destroyAggregationHashTablePayload();
-  }
-
-  /**
    * @brief Whether the given GeneratorFunctionHandle id is valid.
    *
    * @param id The GeneratorFunctionHandle id.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 0ca971d..ba19e58 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -64,6 +64,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_expressions_Expressions_proto
                       quickstep_expressions_aggregation_AggregateFunction
                       quickstep_expressions_aggregation_AggregateFunction_proto
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_expressions_scalar_ScalarAttribute
@@ -123,6 +124,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_relationaloperators_DropTableOperator
                       quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_relationaloperators_HashJoinOperator
+                      quickstep_relationaloperators_InitializeAggregationStateOperator
                       quickstep_relationaloperators_InsertOperator
                       quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_relationaloperators_RelationalOperator
@@ -143,6 +145,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_storage_StorageBlockLayout_proto
                       quickstep_storage_SubBlockTypeRegistry
                       quickstep_types_Type
+                      quickstep_types_TypeID
                       quickstep_types_Type_proto
                       quickstep_types_TypedValue
                       quickstep_types_TypedValue_proto

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index e25b8ad..960fe67 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -49,6 +49,7 @@
 #include "expressions/Expressions.pb.h"
 #include "expressions/aggregation/AggregateFunction.hpp"
 #include "expressions/aggregation/AggregateFunction.pb.h"
+#include "expressions/aggregation/AggregationID.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "expressions/scalar/ScalarAttribute.hpp"
@@ -103,6 +104,7 @@
 #include "relational_operators/DropTableOperator.hpp"
 #include "relational_operators/FinalizeAggregationOperator.hpp"
 #include "relational_operators/HashJoinOperator.hpp"
+#include "relational_operators/InitializeAggregationStateOperator.hpp"
 #include "relational_operators/InsertOperator.hpp"
 #include "relational_operators/NestedLoopsJoinOperator.hpp"
 #include "relational_operators/RelationalOperator.hpp"
@@ -124,6 +126,7 @@
 #include "storage/SubBlockTypeRegistry.hpp"
 #include "types/Type.hpp"
 #include "types/Type.pb.h"
+#include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "types/TypedValue.pb.h"
 #include "types/containers/Tuple.pb.h"
@@ -366,6 +369,91 @@ void ExecutionGenerator::dropAllTemporaryRelations() {
   }
 }
 
+bool ExecutionGenerator::canUseCollisionFreeAggregation(
+    const P::AggregatePtr &aggregate,
+    const std::size_t estimated_num_groups,
+    std::size_t *exact_num_groups) const {
+  if (aggregate->grouping_expressions().size() != 1) {
+    return false;
+  }
+
+  E::AttributeReferencePtr group_by_key_attr;
+  const E::ExpressionPtr agg_expr = aggregate->grouping_expressions().front();
+  if (!E::SomeAttributeReference::MatchesWithConditionalCast(agg_expr, &group_by_key_attr)) {
+    return false;
+  }
+
+  bool min_value_stat_is_exact;
+  bool max_value_stat_is_exact;
+  const TypedValue min_value =
+      cost_model_for_aggregation_->findMinValueStat(
+          aggregate, group_by_key_attr, &min_value_stat_is_exact);
+  const TypedValue max_value =
+      cost_model_for_aggregation_->findMaxValueStat(
+          aggregate, group_by_key_attr, &max_value_stat_is_exact);
+  if (min_value.isNull() || max_value.isNull() ||
+      (!min_value_stat_is_exact) || (!max_value_stat_is_exact)) {
+    return false;
+ }
+
+  std::int64_t min_cpp_value;
+  std::int64_t max_cpp_value;
+  switch (group_by_key_attr->getValueType().getTypeID()) {
+    case TypeID::kInt: {
+      min_cpp_value = min_value.getLiteral<int>();
+      max_cpp_value = max_value.getLiteral<int>();
+      break;
+    }
+    case TypeID::kLong: {
+      min_cpp_value = min_value.getLiteral<std::int64_t>();
+      max_cpp_value = max_value.getLiteral<std::int64_t>();
+      break;
+    }
+    default:
+      return false;
+  }
+
+  // TODO
+  if (min_cpp_value < 0 ||
+      max_cpp_value > 1000000000 ||
+      max_cpp_value / static_cast<double>(estimated_num_groups) > 256.0) {
+    return false;
+  }
+
+
+  for (const auto &agg_expr : aggregate->aggregate_expressions()) {
+    const E::AggregateFunctionPtr agg_func =
+        std::static_pointer_cast<const E::AggregateFunction>(agg_expr->expression());
+    switch (agg_func->getAggregate().getAggregationID()) {
+      case AggregationID::kCount:  // Fall through
+      case AggregationID::kSum:
+        break;
+      default:
+        return false;
+    }
+
+    const auto &arguments = agg_func->getArguments();
+    if (arguments.size() > 1) {
+      return false;
+    }
+
+    if (arguments.size() == 1) {
+      switch (arguments.front()->getValueType().getTypeID()) {
+        case TypeID::kInt:  // Fall through
+        case TypeID::kLong:
+        case TypeID::kFloat:
+        case TypeID::kDouble:
+          break;
+        default:
+          return false;
+      }
+    }
+  }
+
+  *exact_num_groups = static_cast<std::size_t>(max_cpp_value) + 1;
+  return true;
+}
+
 void ExecutionGenerator::convertNamedExpressions(
     const std::vector<E::NamedExpressionPtr> &named_expressions,
     S::QueryContext::ScalarGroup *scalar_group_proto) {
@@ -1392,6 +1480,8 @@ void ExecutionGenerator::convertAggregate(
       findRelationInfoOutputByPhysical(physical_plan->input());
   aggr_state_proto->set_relation_id(input_relation_info->relation->getID());
 
+  bool use_parallel_initialization = false;
+
   std::vector<const Type*> group_by_types;
   for (const E::NamedExpressionPtr &grouping_expression : physical_plan->grouping_expressions()) {
     unique_ptr<const Scalar> execution_group_by_expression;
@@ -1412,9 +1502,34 @@ void ExecutionGenerator::convertAggregate(
   }
 
   if (!group_by_types.empty()) {
-    // Right now, only SeparateChaining is supported.
-    aggr_state_proto->set_hash_table_impl_type(
-        serialization::HashTableImplType::SEPARATE_CHAINING);
+    const std::size_t estimated_num_groups =
+        cost_model_for_aggregation_->estimateNumGroupsForAggregate(physical_plan);
+
+    std::size_t exact_num_groups;
+    const bool can_use_collision_free_aggregation =
+        canUseCollisionFreeAggregation(physical_plan,
+                                       estimated_num_groups,
+                                       &exact_num_groups);
+
+    if (can_use_collision_free_aggregation) {
+      aggr_state_proto->set_hash_table_impl_type(
+          serialization::HashTableImplType::COLLISION_FREE_VECTOR);
+      std::cout << "Use collision free aggregation!\n"
+                << "Size = " << exact_num_groups << "\n";
+
+      aggr_state_proto->set_estimated_num_entries(exact_num_groups);
+      use_parallel_initialization = true;
+    } else {
+      // Otherwise, use SeparateChaining.
+      aggr_state_proto->set_hash_table_impl_type(
+          serialization::HashTableImplType::SEPARATE_CHAINING);
+      std::cout << "Use normal aggregation\n"
+                << "Size = " << estimated_num_groups << "\n";
+
+      aggr_state_proto->set_estimated_num_entries(std::max(16uL, estimated_num_groups));
+    }
+  } else {
+    aggr_state_proto->set_estimated_num_entries(1uL);
   }
 
   for (const E::AliasPtr &named_aggregate_expression : physical_plan->aggregate_expressions()) {
@@ -1452,10 +1567,6 @@ void ExecutionGenerator::convertAggregate(
     aggr_state_proto->mutable_predicate()->CopyFrom(predicate->getProto());
   }
 
-  const std::size_t estimated_num_groups =
-      cost_model_for_aggregation_->estimateNumGroupsForAggregate(physical_plan);
-  aggr_state_proto->set_estimated_num_entries(std::max(16uL, estimated_num_groups));
-
   const QueryPlan::DAGNodeIndex aggregation_operator_index =
       execution_plan_->addRelationalOperator(
           new AggregationOperator(
@@ -1470,6 +1581,18 @@ void ExecutionGenerator::convertAggregate(
                                          false /* is_pipeline_breaker */);
   }
 
+  if (use_parallel_initialization) {
+    const QueryPlan::DAGNodeIndex initialize_aggregation_state_operator_index =
+        execution_plan_->addRelationalOperator(
+            new InitializeAggregationStateOperator(
+                query_handle_->query_id(),
+                aggr_state_index));
+
+    execution_plan_->addDirectDependency(aggregation_operator_index,
+                                         initialize_aggregation_state_operator_index,
+                                         true);
+  }
+
   // Create InsertDestination proto.
   const CatalogRelation *output_relation = nullptr;
   const QueryContext::insert_destination_id insert_destination_index =

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index 55197c9..411fd4e 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_EXECUTION_GENERATOR_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_EXECUTION_GENERATOR_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <string>
 #include <unordered_map>
@@ -37,6 +38,7 @@
 #include "query_optimizer/QueryHandle.hpp"
 #include "query_optimizer/QueryPlan.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/NamedExpression.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
@@ -202,6 +204,10 @@ class ExecutionGenerator {
    */
   std::string getNewRelationName();
 
+  bool canUseCollisionFreeAggregation(const physical::AggregatePtr &aggregate,
+                                      const std::size_t estimated_num_groups,
+                                      std::size_t *exact_num_groups) const;
+
   /**
    * @brief Sets up the info of the CatalogRelation represented by TableReference.
    *        TableReference is not converted to any operator.
@@ -419,7 +425,7 @@ class ExecutionGenerator {
   /**
    * @brief The cost model to use for estimating aggregation hash table size.
    */
-  std::unique_ptr<cost::CostModel> cost_model_for_aggregation_;
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_for_aggregation_;
 
   /**
    * @brief The cost model to use for estimating join hash table size.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/query_optimizer/cost_model/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CMakeLists.txt b/query_optimizer/cost_model/CMakeLists.txt
index 90133e7..d76a6b3 100644
--- a/query_optimizer/cost_model/CMakeLists.txt
+++ b/query_optimizer/cost_model/CMakeLists.txt
@@ -49,6 +49,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationStatistics
+                      quickstep_catalog_CatalogTypedefs
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ComparisonExpression
@@ -72,6 +73,8 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_physical_WindowAggregate
+                      quickstep_types_NullType
+                      quickstep_types_TypedValue
                       quickstep_utility_Macros)
 
 # Module all-in-one library:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 75b1b2b..b9606a2 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -21,11 +21,11 @@
 
 #include <algorithm>
 #include <memory>
-#include <unordered_map>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationStatistics.hpp"
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
@@ -48,6 +48,8 @@
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "types/TypedValue.hpp"
+#include "types/NullType.hpp"
 
 #include "glog/logging.h"
 
@@ -383,18 +385,124 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
 std::size_t StarSchemaSimpleCostModel::getNumDistinctValues(
     const E::ExprId attribute_id,
     const P::TableReferencePtr &table_reference) {
-  const CatalogRelation &relation = *table_reference->relation();
-  const std::vector<E::AttributeReferencePtr> &attributes = table_reference->attribute_list();
-  for (std::size_t i = 0; i < attributes.size(); ++i) {
-    if (attributes[i]->id() == attribute_id) {
-      const CatalogRelationStatistics &stat = relation.getStatistics();
-      if (stat.hasNumDistinctValues(i)) {
-        return stat.getNumDistinctValues(i);
+  const auto rel_attr_id =
+      findCatalogRelationAttributeId(table_reference, attribute_id);
+  if (rel_attr_id != kInvalidAttributeID) {
+    const CatalogRelationStatistics &stat =
+        table_reference->relation()->getStatistics();
+    if (stat.hasNumDistinctValues(rel_attr_id)) {
+      return stat.getNumDistinctValues(rel_attr_id);
+    }
+  }
+  return estimateCardinalityForTableReference(table_reference);
+}
+
+bool StarSchemaSimpleCostModel::impliesUniqueAttributes(
+    const P::PhysicalPtr &physical_plan,
+    const std::vector<E::AttributeReferencePtr> &attributes) {
+  switch (physical_plan->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr &aggregate =
+          std::static_pointer_cast<const P::Aggregate>(physical_plan);
+      return E::SubsetOfExpressions(aggregate->grouping_expressions(), attributes);
+    }
+    case P::PhysicalType::kHashJoin: {
+      const P::HashJoinPtr &hash_join =
+          std::static_pointer_cast<const P::HashJoin>(physical_plan);
+      bool unique_from_left =
+          impliesUniqueAttributes(hash_join->right(), hash_join->right_join_attributes())
+              && impliesUniqueAttributes(hash_join->left(), attributes);
+      bool unique_from_right =
+          impliesUniqueAttributes(hash_join->left(), hash_join->left_join_attributes())
+              && impliesUniqueAttributes(hash_join->right(), attributes);
+      return unique_from_left || unique_from_right;
+    }
+    case P::PhysicalType::kTableReference: {
+      const P::TableReferencePtr &table_reference =
+          std::static_pointer_cast<const P::TableReference>(physical_plan);
+      const CatalogRelationStatistics &stat =
+          table_reference->relation()->getStatistics();
+      if (stat.hasNumTuples()) {
+        const std::size_t num_tuples = stat.getNumTuples();
+        for (const auto &attr : attributes) {
+          const attribute_id rel_attr_id =
+              findCatalogRelationAttributeId(table_reference, attr->id());
+          if (rel_attr_id != kInvalidAttributeID &&
+              stat.hasNumDistinctValues(rel_attr_id) &&
+              stat.getNumDistinctValues(rel_attr_id) == num_tuples) {
+            return true;
+          }
+        }
       }
+      return false;
+    }
+    case P::PhysicalType::kSample:  // Fall through
+    case P::PhysicalType::kSelection:
+    case P::PhysicalType::kSort: {
+      DCHECK_EQ(physical_plan->getNumChildren(), 1u);
+      return impliesUniqueAttributes(physical_plan->children()[0], attributes);
+    }
+    default:
       break;
+  }
+  return false;
+}
+
+TypedValue StarSchemaSimpleCostModel::findCatalogRelationStat(
+    const P::PhysicalPtr &physical_plan,
+    const E::ExprId attr_id,
+    const StatType stat_type,
+    bool *is_exact_stat) {
+  P::TableReferencePtr table_reference;
+  if (P::SomeTableReference::MatchesWithConditionalCast(physical_plan, &table_reference)) {
+    const attribute_id rel_attr_id =
+        findCatalogRelationAttributeId(table_reference, attr_id);
+    if (rel_attr_id != kInvalidAttributeID) {
+      const CatalogRelationStatistics &stat =
+          table_reference->relation()->getStatistics();
+
+      if (is_exact_stat != nullptr) {
+        *is_exact_stat = stat.isExact();
+      }
+
+      switch (stat_type) {
+        case StatType::kMin: {
+          if (stat.hasMinValue(rel_attr_id)) {
+            return stat.getMinValue(rel_attr_id);
+          }
+          break;
+        }
+        case StatType::kMax: {
+          if (stat.hasMaxValue(rel_attr_id)) {
+            return stat.getMaxValue(rel_attr_id);
+          }
+          break;
+        }
+        default:
+          break;
+      }
+      return NullType::InstanceNullable().makeNullValue();
     }
   }
-  return estimateCardinalityForTableReference(table_reference);
+
+  for (const auto &child : physical_plan->children()) {
+    if (E::ContainsExprId(child->getOutputAttributes(), attr_id)) {
+      return findCatalogRelationStat(child, attr_id, stat_type, is_exact_stat);
+    }
+  }
+  return NullType::InstanceNullable().makeNullValue();
+}
+
+attribute_id StarSchemaSimpleCostModel::findCatalogRelationAttributeId(
+    const physical::TableReferencePtr &table_reference,
+    const expressions::ExprId expr_id) {
+  const auto &attribute_list = table_reference->attribute_list();
+  for (std::size_t i = 0; i < attribute_list.size(); ++i) {
+    if (attribute_list[i]->id() == expr_id) {
+      return i;
+    }
+  }
+  return kInvalidAttributeID;
 }
 
 }  // namespace cost

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index 6f6aa29..8d3ef7b 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -23,7 +23,9 @@
 #include <cstddef>
 #include <vector>
 
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
@@ -36,6 +38,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
 #include "query_optimizer/physical/WindowAggregate.hpp"
+#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -105,6 +108,63 @@ class StarSchemaSimpleCostModel : public CostModel {
   double estimateSelectivityForFilterPredicate(
       const physical::PhysicalPtr &physical_plan);
 
+  /**
+   * @brief Check whether a set of attributes are unique (i.e. have distinct
+   *        values) for a relation.
+   *
+   * @param physical_plan The physical plan that corresponds to a relation.
+   * @param attributes The set of attributes to be checked. Note that each
+   *        attribute in this set must be an output attribute of the physical
+   *        plan.
+   * @return True if it is guaranteed that the attributes are unique; false
+   *         otherwise.
+   */
+  bool impliesUniqueAttributes(
+      const physical::PhysicalPtr &physical_plan,
+      const std::vector<expressions::AttributeReferencePtr> &attributes);
+
+  /**
+   * @brief For a physical plan attribute, find its correponding catalog attribute's
+   *        MIN statistic. Returns Null value if there is no corresponding catalog
+   *        attribute for the physical plan attribute.
+   *
+   * @param physical_plan The physical plan.
+   * @param attribute The attribute. Must be an output attribute of the given
+   *        physical plan.
+   * @param is_exact_stat If this pointer is not null, its pointed content will
+   *        be modified by this method to indicate whether the returned statistic
+   *        is EXACT for the stored relation (i.e. not outdated or estimated).
+   * @return The MIN statistic for the attribute.
+   */
+  TypedValue findMinValueStat(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::AttributeReferencePtr &attribute,
+      bool *is_exact_stat = nullptr) {
+    return findCatalogRelationStat(
+        physical_plan, attribute->id(), StatType::kMin, is_exact_stat);
+  }
+
+  /**
+   * @brief For a physical plan attribute, find its correponding catalog attribute's
+   *        MAX statistic. Returns Null value if there is no corresponding catalog
+   *        attribute for the physical plan attribute.
+   *
+   * @param physical_plan The physical plan.
+   * @param attribute The attribute. Must be an output attribute of the given
+   *        physical plan.
+   * @param is_exact_stat If this pointer is not null, its pointed content will
+   *        be modified by this method to indicate whether the returned statistic
+   *        is EXACT for the stored relation (i.e. not not outdated or estimated).
+   * @return The MAX statistic for the attribute.
+   */
+  TypedValue findMaxValueStat(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::AttributeReferencePtr &attribute,
+      bool *is_exact_stat = nullptr) {
+    return findCatalogRelationStat(
+        physical_plan, attribute->id(), StatType::kMax, is_exact_stat);
+  }
+
  private:
   std::size_t estimateCardinalityForAggregate(
       const physical::AggregatePtr &physical_plan);
@@ -144,6 +204,25 @@ class StarSchemaSimpleCostModel : public CostModel {
   std::size_t getNumDistinctValues(const expressions::ExprId attribute_id,
                                    const physical::TableReferencePtr &table_reference);
 
+  enum class StatType {
+    kMax = 0,
+    kMin
+  };
+
+  // For a physical plan attribute, find its correponding catalog attribute's
+  // min/max statistics. Returns Null value if there is no corresponding catalog
+  // attribute for the physical plan attribute (e.g. the attribute is the result
+  // of an expression).
+  TypedValue findCatalogRelationStat(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::ExprId expr_id,
+      const StatType stat_type,
+      bool *is_exact_stat);
+
+  // For a table reference attribute, find its correponding catalog attribute.
+  attribute_id findCatalogRelationAttributeId(
+      const physical::TableReferencePtr &table_reference,
+      const expressions::ExprId expr_id);
 
   DISALLOW_COPY_AND_ASSIGN(StarSchemaSimpleCostModel);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/query_optimizer/expressions/ExpressionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionUtil.hpp b/query_optimizer/expressions/ExpressionUtil.hpp
index 422d5ab..6b8666e 100644
--- a/query_optimizer/expressions/ExpressionUtil.hpp
+++ b/query_optimizer/expressions/ExpressionUtil.hpp
@@ -122,12 +122,12 @@ bool ContainsExprId(
  *              contain the other operand).
  * @return True if \p left is a subset of \p right.
  */
-template <class NamedExpressionType>
+template <class LeftNamedExpressionType, class RightNamedExpressionType>
 bool SubsetOfExpressions(
-    const std::vector<std::shared_ptr<const NamedExpressionType>> &left,
-    const std::vector<std::shared_ptr<const NamedExpressionType>> &right) {
+    const std::vector<std::shared_ptr<const LeftNamedExpressionType>> &left,
+    const std::vector<std::shared_ptr<const RightNamedExpressionType>> &right) {
   UnorderedNamedExpressionSet supset(right.begin(), right.end());
-  for (const std::shared_ptr<const NamedExpressionType> &expr : left) {
+  for (const std::shared_ptr<const LeftNamedExpressionType> &expr : left) {
     if (supset.find(expr) == supset.end()) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index c8447f3..110bf0e 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -46,6 +46,9 @@ add_library(quickstep_relationaloperators_FinalizeAggregationOperator
             FinalizeAggregationOperator.cpp
             FinalizeAggregationOperator.hpp)
 add_library(quickstep_relationaloperators_HashJoinOperator HashJoinOperator.cpp HashJoinOperator.hpp)
+add_library(quickstep_relationaloperators_InitializeAggregationStateOperator
+            InitializeAggregationStateOperator.cpp
+            InitializeAggregationStateOperator.hpp)
 add_library(quickstep_relationaloperators_InsertOperator InsertOperator.cpp InsertOperator.hpp)
 add_library(quickstep_relationaloperators_NestedLoopsJoinOperator
             NestedLoopsJoinOperator.cpp
@@ -232,6 +235,17 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_utility_lipfilter_LIPFilterAdaptiveProber
                       quickstep_utility_lipfilter_LIPFilterUtil
                       tmb)
+target_link_libraries(quickstep_relationaloperators_InitializeAggregationStateOperator
+                      glog
+                      quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
+                      quickstep_queryexecution_WorkOrdersContainer
+                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
+                      quickstep_storage_AggregationOperationState
+                      quickstep_utility_Macros
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_InsertOperator
                       glog
                       quickstep_catalog_CatalogRelation
@@ -524,6 +538,7 @@ target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_DropTableOperator
                       quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_relationaloperators_HashJoinOperator
+                      quickstep_relationaloperators_InitializeAggregationStateOperator
                       quickstep_relationaloperators_InsertOperator
                       quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_relationaloperators_RebuildWorkOrder

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/relational_operators/DestroyAggregationStateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyAggregationStateOperator.cpp b/relational_operators/DestroyAggregationStateOperator.cpp
index 49be43d..62ca9e7 100644
--- a/relational_operators/DestroyAggregationStateOperator.cpp
+++ b/relational_operators/DestroyAggregationStateOperator.cpp
@@ -58,13 +58,6 @@ bool DestroyAggregationStateOperator::getAllWorkOrderProtos(WorkOrderProtosConta
 }
 
 void DestroyAggregationStateWorkOrder::execute() {
-  // NOTE(harshad) : The destroyAggregationHashTablePayload call is separate
-  // from the destroyAggregationState call. The reason is that the aggregation
-  // hash tables don't own the AggregationHandle objects. However the hash table
-  // class requires the handles for destroying the payload (see the
-  // destroyPayload methods in AggregationHandle classes). Therefore, we first
-  // destroy the payloads in the hash table and then destroy the hash table.
-  query_context_->destroyAggregationHashTablePayload(aggr_state_index_);
   query_context_->destroyAggregationState(aggr_state_index_);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 0cbf635..b66030b 100644
--- a/relational_operators/FinalizeAggregationOperator.cpp
+++ b/relational_operators/FinalizeAggregationOperator.cpp
@@ -44,15 +44,15 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
     AggregationOperationState *agg_state =
         query_context->getAggregationState(aggr_state_index_);
     DCHECK(agg_state != nullptr);
-    for (int part_id = 0;
-         part_id < static_cast<int>(agg_state->getNumPartitions());
-         ++part_id) {
+    for (std::size_t partition_id = 0;
+         partition_id < agg_state->getNumPartitions();
+         ++partition_id) {
       container->addNormalWorkOrder(
           new FinalizeAggregationWorkOrder(
               query_id_,
+              partition_id,
               agg_state,
-              query_context->getInsertDestination(output_destination_index_),
-              part_id),
+              query_context->getInsertDestination(output_destination_index_)),
           op_index_);
     }
   }
@@ -80,11 +80,7 @@ bool FinalizeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer
 }
 
 void FinalizeAggregationWorkOrder::execute() {
-  if (state_->isAggregatePartitioned()) {
-    state_->finalizeAggregatePartitioned(part_id_, output_destination_);
-  } else {
-    state_->finalizeAggregate(output_destination_);
-  }
+  state_->finalizeAggregate(partition_id_, output_destination_);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index ae7127a..3c209b1 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -116,29 +116,29 @@ class FinalizeAggregationWorkOrder : public WorkOrder {
    * @note InsertWorkOrder takes ownership of \c state.
    *
    * @param query_id The ID of the query to which this operator belongs.
+   * @param partition_id The partition ID for which the Finalize aggregation
+   *        work order is issued.
    * @param state The AggregationState to use.
    * @param output_destination The InsertDestination to insert aggregation
    *        results.
-   * @param part_id The partition ID for which the Finalize aggregation work
-   *        order is issued. Ignore if aggregation is not partitioned.
    */
   FinalizeAggregationWorkOrder(const std::size_t query_id,
+                               const std::size_t partition_id,
                                AggregationOperationState *state,
-                               InsertDestination *output_destination,
-                               const int part_id = -1)
+                               InsertDestination *output_destination)
       : WorkOrder(query_id),
+        partition_id_(partition_id),
         state_(DCHECK_NOTNULL(state)),
-        output_destination_(DCHECK_NOTNULL(output_destination)),
-        part_id_(part_id) {}
+        output_destination_(DCHECK_NOTNULL(output_destination)) {}
 
   ~FinalizeAggregationWorkOrder() override {}
 
   void execute() override;
 
  private:
+  const std::size_t partition_id_;
   AggregationOperationState *state_;
   InsertDestination *output_destination_;
-  const int part_id_;
 
   DISALLOW_COPY_AND_ASSIGN(FinalizeAggregationWorkOrder);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/relational_operators/InitializeAggregationStateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeAggregationStateOperator.cpp b/relational_operators/InitializeAggregationStateOperator.cpp
new file mode 100644
index 0000000..dfee459
--- /dev/null
+++ b/relational_operators/InitializeAggregationStateOperator.cpp
@@ -0,0 +1,68 @@
+/**
+ * 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 "relational_operators/InitializeAggregationStateOperator.hpp"
+
+#include <vector>
+
+#include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
+#include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
+#include "storage/AggregationOperationState.hpp"
+
+#include "tmb/id_typedefs.h"
+
+namespace quickstep {
+
+bool InitializeAggregationStateOperator::getAllWorkOrders(
+    WorkOrdersContainer *container,
+    QueryContext *query_context,
+    StorageManager *storage_manager,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus) {
+  if (!started_) {
+    AggregationOperationState *agg_state =
+        query_context->getAggregationState(aggr_state_index_);
+    DCHECK(agg_state != nullptr);
+
+    for (std::size_t part_id = 0;
+         part_id < agg_state->getNumInitializationPartitions();
+         ++part_id) {
+      container->addNormalWorkOrder(
+          new InitializeAggregationStateWorkOrder(query_id_,
+                                                  part_id,
+                                                  agg_state),
+          op_index_);
+    }
+    started_ = true;
+  }
+  return started_;
+}
+
+bool InitializeAggregationStateOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  // TODO
+  LOG(FATAL) << "Not implemented";
+}
+
+void InitializeAggregationStateWorkOrder::execute() {
+  state_->initializeState(partition_id_);
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/relational_operators/InitializeAggregationStateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeAggregationStateOperator.hpp b/relational_operators/InitializeAggregationStateOperator.hpp
new file mode 100644
index 0000000..10403b3
--- /dev/null
+++ b/relational_operators/InitializeAggregationStateOperator.hpp
@@ -0,0 +1,103 @@
+/**
+ * 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_RELATIONAL_OPERATORS_INITIALIZE_AGGREGATION_STATE_OPERATOR_HPP_
+#define QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_AGGREGATION_STATE_OPERATOR_HPP_
+
+#include <string>
+
+#include "query_execution/QueryContext.hpp"
+#include "relational_operators/RelationalOperator.hpp"
+#include "relational_operators/WorkOrder.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb { class MessageBus; }
+
+namespace quickstep {
+
+class AggregationOperationState;
+class StorageManager;
+class WorkOrderProtosContainer;
+class WorkOrdersContainer;
+
+namespace serialization { class WorkOrder; }
+
+/** \addtogroup RelationalOperators
+ *  @{
+ */
+
+class InitializeAggregationStateOperator : public RelationalOperator {
+ public:
+  InitializeAggregationStateOperator(const std::size_t query_id,
+                                     const QueryContext::aggregation_state_id aggr_state_index)
+      : RelationalOperator(query_id),
+        aggr_state_index_(aggr_state_index),
+        started_(false) {}
+
+  ~InitializeAggregationStateOperator() override {}
+
+  std::string getName() const override {
+    return "InitializeAggregationStateOperator";
+  }
+
+  bool getAllWorkOrders(WorkOrdersContainer *container,
+                        QueryContext *query_context,
+                        StorageManager *storage_manager,
+                        const tmb::client_id scheduler_client_id,
+                        tmb::MessageBus *bus) override;
+
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
+ private:
+  const QueryContext::aggregation_state_id aggr_state_index_;
+  bool started_;
+
+  DISALLOW_COPY_AND_ASSIGN(InitializeAggregationStateOperator);
+};
+
+class InitializeAggregationStateWorkOrder : public WorkOrder {
+ public:
+  InitializeAggregationStateWorkOrder(const std::size_t query_id,
+                                      const std::size_t partition_id,
+                                      AggregationOperationState *state)
+      : WorkOrder(query_id),
+        partition_id_(partition_id),
+        state_(DCHECK_NOTNULL(state)) {}
+
+  ~InitializeAggregationStateWorkOrder() override {}
+
+  void execute() override;
+
+ private:
+  const std::size_t partition_id_;
+
+  AggregationOperationState *state_;
+
+  DISALLOW_COPY_AND_ASSIGN(InitializeAggregationStateWorkOrder);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_AGGREGATION_STATE_OPERATOR_HPP_


[02/13] incubator-quickstep git commit: Minor refactor for HashJoinInnerJoin.

Posted by ji...@apache.org.
Minor refactor for HashJoinInnerJoin.


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

Branch: refs/heads/collision-free-agg
Commit: 23e14b8e078f42a8d3e5f6c0c4885dee271d99aa
Parents: f2e7726
Author: Zuyu Zhang <zu...@apache.org>
Authored: Mon Jan 30 15:28:49 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jan 30 20:21:23 2017 -0800

----------------------------------------------------------------------
 relational_operators/CMakeLists.txt       |  1 +
 relational_operators/HashJoinOperator.cpp | 42 ++++++++++++++------------
 2 files changed, 23 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/23e14b8e/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index c1caaa3..b2e08cf 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -199,6 +199,7 @@ target_link_libraries(quickstep_relationaloperators_FinalizeAggregationOperator
 target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       ${GFLAGS_LIB_NAME}
                       glog
+                      quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogTypedefs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/23e14b8e/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index fd3841f..7394554 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -25,6 +25,7 @@
 #include <utility>
 #include <vector>
 
+#include "catalog/CatalogAttribute.hpp"
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationSchema.hpp"
 #include "catalog/CatalogTypedefs.hpp"
@@ -165,6 +166,12 @@ class OuterJoinTupleCollector {
   TupleIdSequence *filter_;
 };
 
+// For InnerJoin.
+constexpr std::size_t kNumValueAccessors = 3u;
+constexpr std::size_t kBuildValueAccessorIndex = 0,
+                      kProbeValueAccessorIndex = 1u,
+                      kTempResultValueAccessorIndex = 2u;
+
 }  // namespace
 
 bool HashJoinOperator::getAllWorkOrders(
@@ -565,31 +572,27 @@ void HashInnerJoinWorkOrder::execute() {
         });
     }
 
-
     // We also need a temp value accessor to store results of any scalar expressions.
     ColumnVectorsValueAccessor temp_result;
 
     // Create a map of ValueAccessors and what attributes we want to pick from them
-    std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> accessor_attribute_map;
-    const std::vector<ValueAccessor *> accessors{
-        ordered_build_accessor.get(), ordered_probe_accessor.get(), &temp_result};
-    const unsigned int build_index = 0, probe_index = 1, temp_index = 2;
-    for (auto &accessor : accessors) {
-      accessor_attribute_map.push_back(std::make_pair(
-          accessor,
-          std::vector<attribute_id>(selection_.size(), kInvalidCatalogId)));
-    }
+    std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> accessor_attribute_map(
+        kNumValueAccessors, std::make_pair(nullptr,  // A late binding ValueAccessor.
+                                           vector<attribute_id>(selection_.size(), kInvalidCatalogId)));
 
-    attribute_id dest_attr = 0;
-    std::vector<std::pair<tuple_id, tuple_id>> zipped_joined_tuple_ids;
+    accessor_attribute_map[kBuildValueAccessorIndex].first = ordered_build_accessor.get();
+    accessor_attribute_map[kProbeValueAccessorIndex].first = ordered_probe_accessor.get();
+    accessor_attribute_map[kTempResultValueAccessorIndex].first = &temp_result;
 
+    attribute_id dest_attr = 0;
     for (auto &selection_cit : selection_) {
       // If the Scalar (column) is not an attribute in build/probe blocks, then
       // insert it into a ColumnVectorsValueAccessor.
       if (selection_cit->getDataSource() != Scalar::ScalarDataSource::kAttribute) {
         // Current destination attribute maps to the column we'll create now.
-        accessor_attribute_map[temp_index].second[dest_attr] = temp_result.getNumColumns();
+        accessor_attribute_map[kTempResultValueAccessorIndex].second[dest_attr] = temp_result.getNumColumns();
 
+        std::vector<std::pair<tuple_id, tuple_id>> zipped_joined_tuple_ids;
         if (temp_result.getNumColumns() == 0) {
           // The getAllValuesForJoin function below needs joined tuple IDs as
           // a vector of pair of (build-tuple-ID, probe-tuple-ID), and we have
@@ -599,9 +602,8 @@ void HashInnerJoinWorkOrder::execute() {
           // they don't have scalar expressions with attributes from both
           // build and probe relations (other expressions would have been
           // pushed down to before the join).
-          zipped_joined_tuple_ids.reserve(build_tids.size());
           for (std::size_t i = 0; i < build_tids.size(); ++i) {
-            zipped_joined_tuple_ids.push_back(std::make_pair(build_tids[i], probe_tids[i]));
+            zipped_joined_tuple_ids.emplace_back(build_tids[i], probe_tids[i]);
           }
         }
         temp_result.addColumn(
@@ -610,12 +612,12 @@ void HashInnerJoinWorkOrder::execute() {
                                       probe_relation_id, probe_accessor.get(),
                                       zipped_joined_tuple_ids));
       } else {
-        auto scalar_attr = static_cast<const ScalarAttribute *>(selection_cit.get());
-        const attribute_id attr_id = scalar_attr->getAttribute().getID();
-        if (scalar_attr->getAttribute().getParent().getID() == build_relation_id) {
-          accessor_attribute_map[build_index].second[dest_attr] = attr_id;
+        const CatalogAttribute &attr = static_cast<const ScalarAttribute *>(selection_cit.get())->getAttribute();
+        const attribute_id attr_id = attr.getID();
+        if (attr.getParent().getID() == build_relation_id) {
+          accessor_attribute_map[kBuildValueAccessorIndex].second[dest_attr] = attr_id;
         } else {
-          accessor_attribute_map[probe_index].second[dest_attr] = attr_id;
+          accessor_attribute_map[kProbeValueAccessorIndex].second[dest_attr] = attr_id;
         }
       }
       ++dest_attr;


[03/13] incubator-quickstep git commit: Reorder output attribute order to improve copy performance.

Posted by ji...@apache.org.
Reorder output attribute order to improve copy performance.


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

Branch: refs/heads/collision-free-agg
Commit: 6d83b46af25b35fb0b3a23452b6fbd2842b33793
Parents: 23e14b8
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Jan 12 18:41:17 2017 -0600
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jan 31 00:10:45 2017 -0800

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/PhysicalGenerator.cpp           |  12 +
 query_optimizer/rules/CMakeLists.txt            |  14 +
 query_optimizer/rules/ReorderColumns.cpp        | 214 ++++++++++++++++
 query_optimizer/rules/ReorderColumns.hpp        |  75 ++++++
 query_optimizer/tests/OptimizerTextTest.cpp     |   6 +-
 relational_operators/CMakeLists.txt             |   1 +
 relational_operators/HashJoinOperator.cpp       | 254 +++++++++++--------
 relational_operators/HashJoinOperator.hpp       |   4 +
 storage/SplitRowStoreValueAccessor.hpp          |   5 +
 storage/ValueAccessor.hpp                       |  30 +++
 types/containers/ColumnVectorsValueAccessor.hpp |   5 +
 12 files changed, 515 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index b6c794d..e8bc21c 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -207,6 +207,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_AttachLIPFilters
                       quickstep_queryoptimizer_rules_PruneColumns
+                      quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_strategy_Aggregate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 7cb97dc..e12f8be 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -28,6 +28,7 @@
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
+#include "query_optimizer/rules/ReorderColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
 #include "query_optimizer/strategy/Aggregate.hpp"
@@ -44,6 +45,10 @@
 namespace quickstep {
 namespace optimizer {
 
+DEFINE_bool(reorder_columns, true,
+            "Adjust the ordering of intermediate relations' columns to improve "
+            "copy performance.");
+
 DEFINE_bool(reorder_hash_joins, true,
             "If true, apply hash join order optimization to each group of hash "
             "joins. The optimization applies a greedy algorithm to favor smaller "
@@ -109,6 +114,13 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   } else {
     rules.emplace_back(new SwapProbeBuild());
   }
+  if (FLAGS_reorder_columns) {
+    // NOTE(jianqiao): This optimization relies on the fact that the intermediate
+    // relations all have SPLIT_ROW_STORE layouts. If this fact gets changed, the
+    // optimization algorithm may need to be updated and the performance impact
+    // should be re-evaluated.
+    rules.emplace_back(new ReorderColumns());
+  }
   if (FLAGS_use_lip_filters) {
     rules.emplace_back(new AttachLIPFilters());
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 7fffadc..fe2fd17 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -25,6 +25,7 @@ add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp Gener
 add_library(quickstep_queryoptimizer_rules_PruneColumns PruneColumns.cpp PruneColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownFilter PushDownFilter.cpp PushDownFilter.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin PushDownSemiAntiJoin.cpp PushDownSemiAntiJoin.hpp)
+add_library(quickstep_queryoptimizer_rules_ReorderColumns ReorderColumns.cpp ReorderColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_Rule ../../empty_src.cpp Rule.hpp)
 add_library(quickstep_queryoptimizer_rules_RuleHelper RuleHelper.cpp RuleHelper.hpp)
 add_library(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
@@ -118,6 +119,18 @@ target_link_libraries(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_logical_PatternMatcher
                       quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_ReorderColumns
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_physical_TableReference
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_Rule
                       glog
                       quickstep_utility_Macros)
@@ -213,6 +226,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
+                      quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/rules/ReorderColumns.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReorderColumns.cpp b/query_optimizer/rules/ReorderColumns.cpp
new file mode 100644
index 0000000..f7e58d5
--- /dev/null
+++ b/query_optimizer/rules/ReorderColumns.cpp
@@ -0,0 +1,214 @@
+/**
+ * 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 "query_optimizer/rules/ReorderColumns.hpp"
+
+#include <algorithm>
+#include <cstddef>
+#include <limits>
+#include <unordered_map>
+#include <vector>
+
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/Selection.hpp"
+#include "query_optimizer/physical/TableReference.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr ReorderColumns::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr &input,
+                                             const bool lock_ordering) {
+  // We have to guarantee that the top level ordering of the columns remain
+  // unchanged so that the output columns are ordered as specified by the user.
+  // So here we use the flag "lock_ordering" to skip the first transformable
+  // node (i.e. the first Selection or HashJoin).
+  const bool is_not_transformable = !IsTransformable(input);
+  const bool skip_transform = lock_ordering || is_not_transformable;
+
+  if (skip_transform) {
+    std::vector<P::PhysicalPtr> new_children;
+    for (const P::PhysicalPtr &child : input->children()) {
+      new_children.emplace_back(applyInternal(child, lock_ordering && is_not_transformable));
+    }
+
+    if (new_children != input->children()) {
+      return input->copyWithNewChildren(new_children);
+    } else {
+      return input;
+    }
+  }
+
+  // Collect the maximal chain of transformable nodes.
+  std::vector<P::PhysicalPtr> nodes;
+  for (P::PhysicalPtr node = input; IsTransformable(node); node = node->children().front()) {
+    nodes.emplace_back(node);
+  }
+  // Arrange the nodes with bottom-up order.
+  std::reverse(nodes.begin(), nodes.end());
+
+  // A greedy algorithm that reorders the output attributes based on the GEN/KILL
+  // intervals. This algorithm works well with SSB/TPCH queries and is not likely
+  // to make the plans worse for whatever queries.
+  //
+  // Here is a brief explanation of the three data structure base/gen/kill.
+  //   (1) base: maps each attribute's id to its position in the BASE relation's
+  //             output attributes. Note that the base relation is the child
+  //             relation of nodes[0].
+  //   (2) gen:  maps each attribute's id to the MINIMUM index i such that the
+  //             attribute is among nodes[i]'s output attributes. I.e. node i
+  //             GENERATEs the attribute.
+  //   (3) kill: maps each attribute's id to the MAXIMUM index i such that the
+  //             attribute is among nodes[i]'s output attributes. I.e. node i+1
+  //             KILLs the attribute.
+  std::unordered_map<E::ExprId, std::size_t> base, gen, kill;
+
+  const P::PhysicalPtr base_node =
+      applyInternal(nodes.front()->children().front(), false);
+  const std::vector<E::AttributeReferencePtr> base_attrs =
+      base_node->getOutputAttributes();
+  for (std::size_t i = 0; i < base_attrs.size(); ++i) {
+    base.emplace(base_attrs[i]->id(), i);
+  }
+
+  for (std::size_t i = 0; i < nodes.size(); ++i) {
+    for (const auto &attr : nodes[i]->getOutputAttributes()) {
+      const E::ExprId attr_id = attr->id();
+      if (gen.find(attr_id) == gen.end()) {
+        gen.emplace(attr_id, i);
+      }
+      kill[attr_id] = i;
+    }
+  }
+
+  // TODO(jianqiao): implement this comparator as a standalone and well-documented
+  // struct.
+  const auto comparator = [&gen, &kill, &base](const E::NamedExpressionPtr &lhs,
+                                               const E::NamedExpressionPtr &rhs) -> bool {
+    const E::ExprId lhs_id = lhs->id();
+    const E::ExprId rhs_id = rhs->id();
+
+    // Sort the attributes first by GEN location.
+    const std::size_t lhs_gen = gen.at(lhs_id);
+    const std::size_t rhs_gen = gen.at(rhs_id);
+    if (lhs_gen != rhs_gen) {
+      return lhs_gen < rhs_gen;
+    }
+
+    // Then by KILL location.
+    const std::size_t lhs_kill = kill.at(lhs_id);
+    const std::size_t rhs_kill = kill.at(rhs_id);
+    if (lhs_kill != rhs_kill) {
+      return lhs_kill < rhs_kill;
+    }
+
+    // Finally by the ordering in the base relaton.
+    const auto lhs_base_it = base.find(lhs_id);
+    const auto rhs_base_it = base.find(rhs_id);
+    const std::size_t lhs_base =
+        lhs_base_it == base.end() ? std::numeric_limits<std::size_t>::max()
+                                  : lhs_base_it->second;
+    const std::size_t rhs_base =
+        rhs_base_it == base.end() ? std::numeric_limits<std::size_t>::max()
+                                  : rhs_base_it->second;
+    if (lhs_base != rhs_base) {
+      return lhs_base < rhs_base;
+    }
+
+    return lhs_id < rhs_id;
+  };
+
+  P::PhysicalPtr output = base_node;
+
+  for (const auto &node : nodes) {
+    std::vector<E::NamedExpressionPtr> project_expressions;
+    switch (node->getPhysicalType()) {
+      case P::PhysicalType::kHashJoin: {
+        project_expressions =
+            std::static_pointer_cast<const P::HashJoin>(node)->project_expressions();
+        break;
+      }
+      case P::PhysicalType::kSelection: {
+        project_expressions =
+            std::static_pointer_cast<const P::Selection>(node)->project_expressions();
+        break;
+      }
+      default:
+        LOG(FATAL) << "Unsupported physical type";
+    }
+
+    std::sort(project_expressions.begin(), project_expressions.end(), comparator);
+
+    switch (node->getPhysicalType()) {
+      case P::PhysicalType::kHashJoin: {
+        const P::HashJoinPtr old_node =
+            std::static_pointer_cast<const P::HashJoin>(node);
+        output = P::HashJoin::Create(output,
+                                     applyInternal(old_node->right(), false),
+                                     old_node->left_join_attributes(),
+                                     old_node->right_join_attributes(),
+                                     old_node->residual_predicate(),
+                                     project_expressions,
+                                     old_node->join_type());
+        break;
+      }
+      case P::PhysicalType::kSelection: {
+        const P::SelectionPtr old_node =
+            std::static_pointer_cast<const P::Selection>(node);
+        output = P::Selection::Create(output,
+                                      project_expressions,
+                                      old_node->filter_predicate());
+        break;
+      }
+      default:
+        LOG(FATAL) << "Unsupported physical type";
+    }
+  }
+
+  return output;
+}
+
+bool ReorderColumns::IsTransformable(const physical::PhysicalPtr &input) {
+  switch (input->getPhysicalType()) {
+    case P::PhysicalType::kHashJoin:  // Fall through
+    case P::PhysicalType::kSelection:
+      return true;
+    default:
+      return false;
+  }
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/rules/ReorderColumns.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReorderColumns.hpp b/query_optimizer/rules/ReorderColumns.hpp
new file mode 100644
index 0000000..36fa183
--- /dev/null
+++ b/query_optimizer/rules/ReorderColumns.hpp
@@ -0,0 +1,75 @@
+/**
+ * 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_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_
+
+#include <string>
+
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief Rule that applies to a physical plan to adjust the orderings of some
+ *        intermediate nodes' output attributes to improve copy performance.
+ *
+ * @note This optimization is based on the fact that the intermediate relations
+ *       all have SPLIT_ROW_STORE layouts. If this fact gets changed, the rule's
+ *       algorithm may need to be updated and the performance impact should be
+ *       re-evaluated.
+ */
+class ReorderColumns : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  ReorderColumns() {}
+
+  ~ReorderColumns() override {}
+
+  std::string getName() const override {
+    return "ReorderColumns";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input,
+                                      const bool lock_ordering);
+
+  // Whether the physical node can
+  inline static bool IsTransformable(const physical::PhysicalPtr &input);
+
+  DISALLOW_COPY_AND_ASSIGN(ReorderColumns);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/query_optimizer/tests/OptimizerTextTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTest.cpp b/query_optimizer/tests/OptimizerTextTest.cpp
index 759c173..e17f5c4 100644
--- a/query_optimizer/tests/OptimizerTextTest.cpp
+++ b/query_optimizer/tests/OptimizerTextTest.cpp
@@ -31,6 +31,7 @@
 namespace quickstep {
 namespace optimizer {
 
+DECLARE_bool(reorder_columns);
 DECLARE_bool(reorder_hash_joins);
 DECLARE_bool(use_lip_filters);
 
@@ -58,8 +59,9 @@ int main(int argc, char** argv) {
   test_driver->registerOptions(
       quickstep::optimizer::OptimizerTextTestRunner::kTestOptions);
 
-  // Turn off join order optimization and LIPFilter for optimizer test since
-  // it is up to change and affects a large number of test cases.
+  // Turn off some optimization rules for optimizer test since they are up to
+  // change and affects a large number of test cases.
+  quickstep::optimizer::FLAGS_reorder_columns = false;
   quickstep::optimizer::FLAGS_reorder_hash_joins = false;
   quickstep::optimizer::FLAGS_use_lip_filters = false;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index b2e08cf..c8447f3 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -207,6 +207,7 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_catalog_PartitionSchemeHeader
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
+                      quickstep_expressions_scalar_ScalarAttribute
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 7394554..0e75411 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -31,6 +31,7 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
+#include "expressions/scalar/ScalarAttribute.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
@@ -64,6 +65,9 @@ namespace quickstep {
 
 namespace {
 
+typedef std::vector<std::pair<tuple_id, tuple_id>> VectorOfTupleIdPair;
+typedef std::pair<std::vector<tuple_id>, std::vector<tuple_id>> PairOfTupleIdVector;
+
 // Functor passed to HashTable::getAllFromValueAccessor() to collect matching
 // tuples from the inner relation. It stores matching tuple ID pairs
 // in an unordered_map keyed by inner block ID and a vector of
@@ -83,8 +87,7 @@ class VectorsOfPairsJoinedTuplesCollector {
   // key is inner block_id, values are vectors of joined tuple ID pairs with
   // tuple ID from the inner block on the left and the outer block on the
   // right.
-  inline std::unordered_map<block_id, std::vector<std::pair<tuple_id, tuple_id>>>*
-      getJoinedTuples() {
+  inline std::unordered_map<block_id, VectorOfTupleIdPair>* getJoinedTuples() {
     return &joined_tuples_;
   }
 
@@ -94,7 +97,7 @@ class VectorsOfPairsJoinedTuplesCollector {
   // cross-product of all tuples from both blocks, but simply using pairs of
   // tuple-IDs is expected to be more space efficient if the result set is less
   // than 1/64 the cardinality of the cross-product.
-  std::unordered_map<block_id, std::vector<std::pair<tuple_id, tuple_id>>> joined_tuples_;
+  std::unordered_map<block_id, VectorOfTupleIdPair> joined_tuples_;
 };
 
 // Another collector using an unordered_map keyed on inner block just like above,
@@ -107,15 +110,15 @@ class PairsOfVectorsJoinedTuplesCollector {
   template <typename ValueAccessorT>
   inline void operator()(const ValueAccessorT &accessor,
                          const TupleReference &tref) {
-    joined_tuples_[tref.block].first.push_back(tref.tuple);
-    joined_tuples_[tref.block].second.push_back(accessor.getCurrentPosition());
+    auto &entry = joined_tuples_[tref.block];
+    entry.first.emplace_back(tref.tuple);
+    entry.second.emplace_back(accessor.getCurrentPosition());
   }
 
   // Get a mutable pointer to the collected map of joined tuple ID pairs. The
   // key is inner block_id, value is a pair consisting of
   // inner block tuple IDs (first) and outer block tuple IDs (second).
-  inline std::unordered_map< block_id, std::pair<std::vector<tuple_id>, std::vector<tuple_id>>>*
-      getJoinedTuples() {
+  inline std::unordered_map<block_id, PairOfTupleIdVector>* getJoinedTuples() {
     return &joined_tuples_;
   }
 
@@ -166,12 +169,6 @@ class OuterJoinTupleCollector {
   TupleIdSequence *filter_;
 };
 
-// For InnerJoin.
-constexpr std::size_t kNumValueAccessors = 3u;
-constexpr std::size_t kBuildValueAccessorIndex = 0,
-                      kProbeValueAccessorIndex = 1u,
-                      kTempResultValueAccessorIndex = 2u;
-
 }  // namespace
 
 bool HashJoinOperator::getAllWorkOrders(
@@ -473,16 +470,93 @@ void HashInnerJoinWorkOrder::execute() {
         base_accessor->createSharedTupleIdSequenceAdapterVirtual(*existence_map));
   }
 
+  if (probe_accessor->getImplementationType() == ValueAccessor::Implementation::kSplitRowStore) {
+    executeWithCopyElision(probe_accessor.get());
+  } else {
+    executeWithoutCopyElision(probe_accessor.get());
+  }
+}
+
+void HashInnerJoinWorkOrder::executeWithoutCopyElision(ValueAccessor *probe_accessor) {
+  VectorsOfPairsJoinedTuplesCollector collector;
+  if (join_key_attributes_.size() == 1) {
+    hash_table_.getAllFromValueAccessor(
+        probe_accessor,
+        join_key_attributes_.front(),
+        any_join_key_attributes_nullable_,
+        &collector);
+  } else {
+    hash_table_.getAllFromValueAccessorCompositeKey(
+        probe_accessor,
+        join_key_attributes_,
+        any_join_key_attributes_nullable_,
+        &collector);
+  }
+
+  const relation_id build_relation_id = build_relation_.getID();
+  const relation_id probe_relation_id = probe_relation_.getID();
+
+  for (std::pair<const block_id, VectorOfTupleIdPair>
+           &build_block_entry : *collector.getJoinedTuples()) {
+    BlockReference build_block =
+        storage_manager_->getBlock(build_block_entry.first, build_relation_);
+    const TupleStorageSubBlock &build_store = build_block->getTupleStorageSubBlock();
+    std::unique_ptr<ValueAccessor> build_accessor(build_store.createValueAccessor());
+
+    // Evaluate '*residual_predicate_', if any.
+    //
+    // TODO(chasseur): We might consider implementing true vectorized
+    // evaluation for join predicates that are not equijoins (although in
+    // general that would require evaluating and materializing some expressions
+    // over the cross-product of all tuples in a pair of blocks in order to
+    // evaluate the predicate). We could use a heuristic where we only do the
+    // vectorized materialization and evaluation if the set of matches from the
+    // hash join is below a reasonable threshold so that we don't blow up
+    // temporary memory requirements to an unreasonable degree.
+    if (residual_predicate_ != nullptr) {
+      VectorOfTupleIdPair filtered_matches;
+
+      for (const std::pair<tuple_id, tuple_id> &hash_match
+           : build_block_entry.second) {
+        if (residual_predicate_->matchesForJoinedTuples(*build_accessor,
+                                                        build_relation_id,
+                                                        hash_match.first,
+                                                        *probe_accessor,
+                                                        probe_relation_id,
+                                                        hash_match.second)) {
+          filtered_matches.emplace_back(hash_match);
+        }
+      }
+
+      build_block_entry.second = std::move(filtered_matches);
+    }
+
+    ColumnVectorsValueAccessor temp_result;
+    for (auto selection_cit = selection_.begin();
+         selection_cit != selection_.end();
+         ++selection_cit) {
+      temp_result.addColumn((*selection_cit)->getAllValuesForJoin(build_relation_id,
+                                                                  build_accessor.get(),
+                                                                  probe_relation_id,
+                                                                  probe_accessor,
+                                                                  build_block_entry.second));
+    }
+
+    output_destination_->bulkInsertTuples(&temp_result);
+  }
+}
+
+void HashInnerJoinWorkOrder::executeWithCopyElision(ValueAccessor *probe_accessor) {
   PairsOfVectorsJoinedTuplesCollector collector;
   if (join_key_attributes_.size() == 1) {
     hash_table_.getAllFromValueAccessor(
-        probe_accessor.get(),
+        probe_accessor,
         join_key_attributes_.front(),
         any_join_key_attributes_nullable_,
         &collector);
   } else {
     hash_table_.getAllFromValueAccessorCompositeKey(
-        probe_accessor.get(),
+        probe_accessor,
         join_key_attributes_,
         any_join_key_attributes_nullable_,
         &collector);
@@ -491,7 +565,37 @@ void HashInnerJoinWorkOrder::execute() {
   const relation_id build_relation_id = build_relation_.getID();
   const relation_id probe_relation_id = probe_relation_.getID();
 
-  for (std::pair<const block_id, std::pair<std::vector<tuple_id>, std::vector<tuple_id>>>
+  constexpr std::size_t kNumIndexes = 3u;
+  constexpr std::size_t kBuildIndex = 0, kProbeIndex = 1u, kTempIndex = 2u;
+
+  // Create a map of ValueAccessors and what attributes we want to pick from them.
+  std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> accessor_attribute_map(
+      kNumIndexes, std::make_pair(nullptr /* late binding ValueAccessor */,
+                                  vector<attribute_id>(selection_.size(), kInvalidCatalogId)));
+
+  std::vector<const Scalar *> non_trivial_expressions;
+  attribute_id dest_attr = 0;
+
+  for (const auto &scalar : selection_) {
+    // If the Scalar (column) is not an attribute in build/probe blocks, we will
+    // insert it into a ColumnVectorsValueAccessor.
+    if (scalar->getDataSource() != Scalar::ScalarDataSource::kAttribute) {
+      // Current destination attribute maps to the column we'll create now.
+      accessor_attribute_map[kTempIndex].second[dest_attr] = non_trivial_expressions.size();
+      non_trivial_expressions.emplace_back(scalar.get());
+    } else {
+      const CatalogAttribute &attr = static_cast<const ScalarAttribute *>(scalar.get())->getAttribute();
+      const attribute_id attr_id = attr.getID();
+      if (attr.getParent().getID() == build_relation_id) {
+        accessor_attribute_map[kBuildIndex].second[dest_attr] = attr_id;
+      } else {
+        accessor_attribute_map[kProbeIndex].second[dest_attr] = attr_id;
+      }
+    }
+    ++dest_attr;
+  }
+
+  for (std::pair<const block_id, PairOfTupleIdVector>
            &build_block_entry : *collector.getJoinedTuples()) {
     BlockReference build_block =
         storage_manager_->getBlock(build_block_entry.first, build_relation_);
@@ -511,7 +615,8 @@ void HashInnerJoinWorkOrder::execute() {
     // hash join is below a reasonable threshold so that we don't blow up
     // temporary memory requirements to an unreasonable degree.
     if (residual_predicate_ != nullptr) {
-      std::pair<std::vector<tuple_id>, std::vector<tuple_id>> filtered_matches;
+      PairOfTupleIdVector filtered_matches;
+
       for (std::size_t i = 0; i < build_tids.size(); ++i) {
         if (residual_predicate_->matchesForJoinedTuples(*build_accessor,
                                                         build_relation_id,
@@ -519,110 +624,51 @@ void HashInnerJoinWorkOrder::execute() {
                                                         *probe_accessor,
                                                         probe_relation_id,
                                                         probe_tids[i])) {
-          filtered_matches.first.push_back(build_tids[i]);
-          filtered_matches.second.push_back(probe_tids[i]);
+          filtered_matches.first.emplace_back(build_tids[i]);
+          filtered_matches.second.emplace_back(probe_tids[i]);
         }
       }
 
       build_block_entry.second = std::move(filtered_matches);
     }
 
-    // TODO(chasseur): If all the output expressions are ScalarAttributes,
-    // we could implement a similar fast-path to StorageBlock::selectSimple()
-    // that avoids a copy.
-    //
     // TODO(chasseur): See TODO in NestedLoopsJoinOperator.cpp about limiting
     // the size of materialized temporary results. In common usage, this
     // probably won't be an issue for hash-joins, but in the worst case a hash
     // join can still devolve into a cross-product.
-    //
-    // NOTE(chasseur): We could also create one big ColumnVectorsValueAccessor
-    // and accumulate all the results across multiple block pairs into it
-    // before inserting anything into output blocks, but this would require
-    // some significant API extensions to the expressions system for a dubious
-    // benefit (probably only a real performance win when there are very few
-    // matching tuples in each individual inner block but very many inner
-    // blocks with at least one match).
-
-    // We now create ordered value accessors for both build and probe side,
-    // using the joined tuple TIDs. Note that we have to use this Lambda-based
-    // invocation method here because the accessors don't have a virtual
-    // function that creates such an OrderedTupleIdSequenceAdapterValueAccessor.
-    std::unique_ptr<ValueAccessor> ordered_build_accessor, ordered_probe_accessor;
-    InvokeOnValueAccessorNotAdapter(
-        build_accessor.get(),
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-          ordered_build_accessor.reset(
-              accessor->createSharedOrderedTupleIdSequenceAdapter(build_tids));
-        });
-
-    if (probe_accessor->isTupleIdSequenceAdapter()) {
-      InvokeOnTupleIdSequenceAdapterValueAccessor(
-        probe_accessor.get(),
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-          ordered_probe_accessor.reset(
-            accessor->createSharedOrderedTupleIdSequenceAdapter(probe_tids));
-        });
-    } else {
-      InvokeOnValueAccessorNotAdapter(
-        probe_accessor.get(),
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-          ordered_probe_accessor.reset(
-            accessor->createSharedOrderedTupleIdSequenceAdapter(probe_tids));
-        });
-    }
 
     // We also need a temp value accessor to store results of any scalar expressions.
     ColumnVectorsValueAccessor temp_result;
+    if (!non_trivial_expressions.empty()) {
+      // The getAllValuesForJoin function below needs joined tuple IDs as a
+      // vector of pair of (build-tuple-ID, probe-tuple-ID), and we have a pair
+      // of (build-tuple-IDs-vector, probe-tuple-IDs-vector). So we'll have to
+      // zip our two vectors together.
+      VectorOfTupleIdPair zipped_joined_tuple_ids;
+      for (std::size_t i = 0; i < build_tids.size(); ++i) {
+        zipped_joined_tuple_ids.emplace_back(build_tids[i], probe_tids[i]);
+      }
 
-    // Create a map of ValueAccessors and what attributes we want to pick from them
-    std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> accessor_attribute_map(
-        kNumValueAccessors, std::make_pair(nullptr,  // A late binding ValueAccessor.
-                                           vector<attribute_id>(selection_.size(), kInvalidCatalogId)));
-
-    accessor_attribute_map[kBuildValueAccessorIndex].first = ordered_build_accessor.get();
-    accessor_attribute_map[kProbeValueAccessorIndex].first = ordered_probe_accessor.get();
-    accessor_attribute_map[kTempResultValueAccessorIndex].first = &temp_result;
-
-    attribute_id dest_attr = 0;
-    for (auto &selection_cit : selection_) {
-      // If the Scalar (column) is not an attribute in build/probe blocks, then
-      // insert it into a ColumnVectorsValueAccessor.
-      if (selection_cit->getDataSource() != Scalar::ScalarDataSource::kAttribute) {
-        // Current destination attribute maps to the column we'll create now.
-        accessor_attribute_map[kTempResultValueAccessorIndex].second[dest_attr] = temp_result.getNumColumns();
-
-        std::vector<std::pair<tuple_id, tuple_id>> zipped_joined_tuple_ids;
-        if (temp_result.getNumColumns() == 0) {
-          // The getAllValuesForJoin function below needs joined tuple IDs as
-          // a vector of pair of (build-tuple-ID, probe-tuple-ID), and we have
-          // a pair of (build-tuple-IDs-vector, probe-tuple-IDs-vector). So
-          // we'll have to zip our two vectors together. We do this inside
-          // the loop because most queries don't exercise this code since
-          // they don't have scalar expressions with attributes from both
-          // build and probe relations (other expressions would have been
-          // pushed down to before the join).
-          for (std::size_t i = 0; i < build_tids.size(); ++i) {
-            zipped_joined_tuple_ids.emplace_back(build_tids[i], probe_tids[i]);
-          }
-        }
-        temp_result.addColumn(
-            selection_cit
-                ->getAllValuesForJoin(build_relation_id, build_accessor.get(),
-                                      probe_relation_id, probe_accessor.get(),
-                                      zipped_joined_tuple_ids));
-      } else {
-        const CatalogAttribute &attr = static_cast<const ScalarAttribute *>(selection_cit.get())->getAttribute();
-        const attribute_id attr_id = attr.getID();
-        if (attr.getParent().getID() == build_relation_id) {
-          accessor_attribute_map[kBuildValueAccessorIndex].second[dest_attr] = attr_id;
-        } else {
-          accessor_attribute_map[kProbeValueAccessorIndex].second[dest_attr] = attr_id;
-        }
+      for (const Scalar *scalar : non_trivial_expressions) {
+        temp_result.addColumn(scalar->getAllValuesForJoin(build_relation_id,
+                                                          build_accessor.get(),
+                                                          probe_relation_id,
+                                                          probe_accessor,
+                                                          zipped_joined_tuple_ids));
       }
-      ++dest_attr;
     }
 
+    // We now create ordered value accessors for both build and probe side,
+    // using the joined tuple IDs.
+    std::unique_ptr<ValueAccessor> ordered_build_accessor(
+        build_accessor->createSharedOrderedTupleIdSequenceAdapterVirtual(build_tids));
+    std::unique_ptr<ValueAccessor> ordered_probe_accessor(
+        probe_accessor->createSharedOrderedTupleIdSequenceAdapterVirtual(probe_tids));
+
+    accessor_attribute_map[kBuildIndex].first = ordered_build_accessor.get();
+    accessor_attribute_map[kProbeIndex].first = ordered_probe_accessor.get();
+    accessor_attribute_map[kTempIndex].first = &temp_result;
+
     // NOTE(chasseur): calling the bulk-insert method of InsertDestination once
     // for each pair of joined blocks incurs some extra overhead that could be
     // avoided by keeping checked-out MutableBlockReferences across iterations

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index acfe3d2..5e9c5d8 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -423,6 +423,10 @@ class HashInnerJoinWorkOrder : public WorkOrder {
   }
 
  private:
+  void executeWithoutCopyElision(ValueAccessor *probe_accesor);
+
+  void executeWithCopyElision(ValueAccessor *probe_accessor);
+
   const CatalogRelationSchema &build_relation_;
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/storage/SplitRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreValueAccessor.hpp b/storage/SplitRowStoreValueAccessor.hpp
index 951a20a..46367b3 100644
--- a/storage/SplitRowStoreValueAccessor.hpp
+++ b/storage/SplitRowStoreValueAccessor.hpp
@@ -318,6 +318,11 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/storage/ValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessor.hpp b/storage/ValueAccessor.hpp
index 654bbf9..f183efe 100644
--- a/storage/ValueAccessor.hpp
+++ b/storage/ValueAccessor.hpp
@@ -305,6 +305,21 @@ class ValueAccessor {
       const TupleIdSequence &id_sequence) = 0;
 
   /**
+   * @brief Create a new OrderedTupleIdSequenceAdapterValueAccessor that wraps
+   *        this ValueAccessor.
+   * @warning The newly-created adapter does NOT take ownership of this
+   *          ValueAccessor nor the provided OrderedTupleIdSequence. Both must
+   *          remain valid so long as the adapter will be used.
+   *
+   * @param id_sequence An OrderedTupleIdSequence specifying some subset of the
+   *        tuples for this ValueAccessor that the adapter will iterate over.
+   * @return A new OrderedTupleIdSequenceAdapterValueAccessor that will iterate
+   *         over only the tuples specified in id_sequence.
+   **/
+  virtual ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) = 0;
+
+  /**
    * @brief Get a TupleIdSequence indicating which positions this ValueAccessor
    *        is iterating over.
    *
@@ -512,6 +527,11 @@ class TupleIdSequenceAdapterValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }
@@ -718,6 +738,11 @@ class OrderedTupleIdSequenceAdapterValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }
@@ -944,6 +969,11 @@ class PackedTupleStorageSubBlockValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6d83b46a/types/containers/ColumnVectorsValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVectorsValueAccessor.hpp b/types/containers/ColumnVectorsValueAccessor.hpp
index fbbdc1b..6dc1124 100644
--- a/types/containers/ColumnVectorsValueAccessor.hpp
+++ b/types/containers/ColumnVectorsValueAccessor.hpp
@@ -290,6 +290,11 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }



[09/13] incubator-quickstep git commit: Initial commit.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
deleted file mode 100644
index 4a82a62..0000000
--- a/storage/FastHashTable.hpp
+++ /dev/null
@@ -1,2403 +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_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/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;
-
- 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;
-
-  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);
-          }
-        }
-        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);
-                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);
-            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>
-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);
-    if (value == nullptr) {
-      return false;
-    } else {
-      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;
-    }
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
-bool FastHashTable<resizable,
-                   serializable,
-                   force_key_copy,
-                   allow_duplicate_keys>::
-    upsertValueAccessorFast(
-        const std::vector<attribute_id> &argument_ids,
-        ValueAccessor *accessor,
-        const attribute_id key_attr_id,
-        const bool check_for_null_keys) {
-  DEBUG_ASSERT(!allow_duplicate_keys);
-  std::size_t variable_size;
-  return InvokeOnAnyValueAccessor(
-      accessor,
-      [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
-        if (resizable) {
-          bool continuing = true;
-          while (continuing) {
-            {
-              continuing = false;
-              SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
-              while (accessor->next()) {
-                TypedValue key = accessor->getTypedValue(key_attr_id);
-                if (check_for_null_keys && key.isNull()) {
-                  continue;
-                }
-                variable_size = (force_key_copy && !scalar_key_inline_)
-                                    ? key.getDataSize()
-                                    : 0;
-                std::uint8_t *value =
-                    this->upsertInternalFast(key, variable_size, nullptr);
-                if (value == nullptr) {
-                  continuing = true;
-                  break;
-                } else {
-                  SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
-                  for (unsigned int k = 0; k < num_handles_; ++k) {
-                    if (argument_ids[k] != kInvalidAttributeID) {
-                      handles_[k]->updateStateUnary(
-                          accessor->getTypedValue(argument_ids[k]),
-                          value + payload_offsets_[k]);
-                    } else {
-                      handles_[k]->updateStateNullary(value +
-                                                      payload_offsets_[k]);
-                    }
-                  }
-                }
-              }
-            }
-            if (continuing) {
-              this->resize(0, variable_size);
-              accessor->previous();
-            }
-          }
-        } else {
-          while (accessor->next()) {
-            TypedValue key = accessor->getTypedValue(key_attr_id);
-            if (check_for_null_keys && key.isNull()) {
-              continue;
-            }
-            variable_size =
-                (force_key_copy && !scalar_key_inline_) ? key.getDataSize() : 0;
-            std::uint8_t *value =
-                this->upsertInternalFast(key, variable_size, nullptr);
-            if (value == nullptr) {
-              return false;
-            } else {
-              SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
-              for (unsigned int k = 0; k < num_handles_; ++k) {
-                if (argument_ids[k] != kInvalidAttributeID) {
-                  handles_[k]->updateStateUnary(
-                      accessor->getTypedValue(argument_ids[k]),
-                      value + payload_offsets_[k]);
-                } else {
-                  handles_[k]->updateStateNullary(value +
-            

<TRUNCATED>


[06/13] incubator-quickstep git commit: Enabled some checks for the distributed version in the release build.

Posted by ji...@apache.org.
Enabled some checks for the distributed version in the release build.


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

Branch: refs/heads/collision-free-agg
Commit: 5ffdaaf9f9d42cb25ffcbaf59cfafc049dcaca27
Parents: dff4a14
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 31 14:45:27 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jan 31 14:45:27 2017 -0800

----------------------------------------------------------------------
 cli/distributed/Cli.cpp | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5ffdaaf9/cli/distributed/Cli.cpp
----------------------------------------------------------------------
diff --git a/cli/distributed/Cli.cpp b/cli/distributed/Cli.cpp
index 01f824d..5af70e6 100644
--- a/cli/distributed/Cli.cpp
+++ b/cli/distributed/Cli.cpp
@@ -95,13 +95,13 @@ void Cli::init() {
   tmb::MessageStyle style;
 
   TaggedMessage cli_reg_message(kDistributedCliRegistrationMessage);
-  DCHECK(tmb::MessageBus::SendStatus::kOK ==
+  CHECK(tmb::MessageBus::SendStatus::kOK ==
       bus_.Send(cli_id_, all_addresses, style, move(cli_reg_message)));
 
   // Wait for Conductor to response.
   const AnnotatedMessage cli_reg_response_message(bus_.Receive(cli_id_, 0, true));
-  DCHECK_EQ(kDistributedCliRegistrationResponseMessage,
-            cli_reg_response_message.tagged_message.message_type());
+  CHECK_EQ(kDistributedCliRegistrationResponseMessage,
+           cli_reg_response_message.tagged_message.message_type());
   conductor_client_id_ = cli_reg_response_message.sender;
 
   DLOG(INFO) << "DistributedCli received typed '" << kDistributedCliRegistrationResponseMessage


[08/13] incubator-quickstep git commit: Initial commit.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/FastHashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTableFactory.hpp b/storage/FastHashTableFactory.hpp
deleted file mode 100644
index 682cc2a..0000000
--- a/storage/FastHashTableFactory.hpp
+++ /dev/null
@@ -1,224 +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/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).
-   * @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) {
-    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);
-    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/b46bc73c/storage/FastSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastSeparateChainingHashTable.hpp b/storage/FastSeparateChainingHashTable.hpp
deleted file mode 100644
index 2435d45..0000000
--- a/storage/FastSeparateChainingHashTable.hpp
+++ /dev/null
@@ -1,1551 +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 "expressions/aggregation/AggregationHandle.hpp"
-#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() override {
-    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;
-
-  void destroyPayload() override {
-    const std::size_t num_buckets =
-        header_->buckets_allocated.load(std::memory_order_relaxed);
-    void *bucket_ptr = static_cast<char *>(buckets_) + kValueOffset;
-    for (std::size_t bucket_num = 0; bucket_num < num_buckets; ++bucket_num) {
-      for (std::size_t handle_id = 0; handle_id < num_handles_; ++handle_id) {
-        void *value_internal_ptr =
-            static_cast<char *>(bucket_ptr) + this->payload_offsets_[handle_id];
-        handles_[handle_id]->destroyPayload(static_cast<std::uint8_t *>(value_internal_ptr));
-      }
-      bucket_ptr = static_cast<char *>(bucket_ptr) + bucket_size_;
-    }
-  }
-
- 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;
-  }
-
-  // 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;
-
-  const std::vector<AggregationHandle *> &handles_;
-  const std::size_t num_handles_;
-
-  // 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)),
-      handles_(handles),
-      num_handles_(handles.size()),
-      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));
-  DCHECK(init_payload_ != nullptr);
-  int k = 0;
-  for (auto handle : this->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>
-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.
-  destroyPayload();
-
-  // 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);
-  }
-
-  destroyPayload();
-
-  // 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>
-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/b46bc73c/storage/HashTable.proto
----------------------------------------------------------------------
diff --git a/storage/HashTable.proto b/storage/HashTable.proto
index 1d4ccb0..6839ebc 100644
--- a/storage/HashTable.proto
+++ b/storage/HashTable.proto
@@ -22,9 +22,10 @@ package quickstep.serialization;
 import "types/Type.proto";
 
 enum HashTableImplType {
-  LINEAR_OPEN_ADDRESSING = 0;
-  SEPARATE_CHAINING = 1;
-  SIMPLE_SCALAR_SEPARATE_CHAINING = 2;
+  COLLISION_FREE_VECTOR = 0;
+  LINEAR_OPEN_ADDRESSING = 1;
+  SEPARATE_CHAINING = 2;
+  SIMPLE_SCALAR_SEPARATE_CHAINING = 3;
 }
 
 // NOTE(chasseur): This proto describes the run-time parameters for a resizable

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index a3180bb..3d34600 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -23,11 +23,13 @@
 #include <cstddef>
 #include <vector>
 
-#include "ValueAccessor.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
+class ColumnVectorsValueAccessor;
+class ValueAccessor;
+
 /** \addtogroup Storage
  *  @{
  */
@@ -38,6 +40,7 @@ namespace quickstep {
  *        HashTableFactory to create a HashTable.
  **/
 enum class HashTableImplType {
+  kCollisionFreeVector,
   kLinearOpenAddressing,
   kSeparateChaining,
   kSimpleScalarSeparateChaining
@@ -75,6 +78,23 @@ class HashTableBase {
   virtual ~HashTableBase() {}
 
   /**
+   * @brief Destroy the payload stored in the hash table.
+   **/
+  virtual void destroyPayload() {
+  }
+
+ protected:
+  HashTableBase() {}
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(HashTableBase);
+};
+
+class AggregationStateHashTableBase {
+ public:
+  virtual ~AggregationStateHashTableBase() {}
+
+  /**
    * 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:
@@ -91,29 +111,21 @@ class HashTableBase {
    * Optionally, we can also remove the AggregationStateHashTableBase
    * specialization from this file.
    **/
-  virtual bool upsertValueAccessorCompositeKeyFast(
-      const std::vector<attribute_id> &argument,
-      ValueAccessor *accessor,
+  virtual bool upsertValueAccessor(
+      const std::vector<std::vector<attribute_id>> &argument_ids,
       const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys) {
-    return false;
-  }
+      ValueAccessor *accessor,
+      ColumnVectorsValueAccessor *aux_accessor = nullptr) = 0;
 
-  /**
-   * @brief Destroy the payload stored in the hash table.
-   **/
-  virtual void destroyPayload() {
-  }
+  virtual void destroyPayload() = 0;
 
  protected:
-  HashTableBase() {}
+  AggregationStateHashTableBase() {}
 
  private:
-  DISALLOW_COPY_AND_ASSIGN(HashTableBase);
+  DISALLOW_COPY_AND_ASSIGN(AggregationStateHashTableBase);
 };
 
-typedef HashTableBase<true, false, true, false> AggregationStateHashTableBase;
-
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/HashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.hpp b/storage/HashTableFactory.hpp
index d690557..d95362c 100644
--- a/storage/HashTableFactory.hpp
+++ b/storage/HashTableFactory.hpp
@@ -24,10 +24,12 @@
 #include <string>
 #include <vector>
 
+#include "storage/CollisionFreeAggregationStateHashTable.hpp"
 #include "storage/HashTable.hpp"
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTable.pb.h"
 #include "storage/LinearOpenAddressingHashTable.hpp"
+#include "storage/PackedPayloadAggregationStateHashTable.hpp"
 #include "storage/SeparateChainingHashTable.hpp"
 #include "storage/SimpleScalarSeparateChainingHashTable.hpp"
 #include "storage/TupleReference.hpp"
@@ -113,6 +115,8 @@ serialization::HashTableImplType SimplifyHashTableImplTypeProto(
 inline HashTableImplType HashTableImplTypeFromProto(
     const serialization::HashTableImplType proto_type) {
   switch (proto_type) {
+    case serialization::HashTableImplType::COLLISION_FREE_VECTOR:
+      return HashTableImplType::kCollisionFreeVector;
     case serialization::HashTableImplType::LINEAR_OPEN_ADDRESSING:
       return HashTableImplType::kLinearOpenAddressing;
     case serialization::HashTableImplType::SEPARATE_CHAINING:
@@ -324,19 +328,43 @@ class HashTableFactory {
 };
 
 /**
- * @brief Convenient alias that provides a HashTableFactory whose only template
- *        parameter is the aggregate state type.
- **/
-template <typename ValueT>
-using AggregationStateHashTableFactory
-    = HashTableFactory<ValueT, true, false, true, false>;
-
-/**
  * @brief Convenient alias for a HashTableFactory that makes JoinHashTables.
  **/
 typedef HashTableFactory<TupleReference, true, false, false, true>
     JoinHashTableFactory;
 
+/**
+ * @brief TODO
+ */
+class AggregationStateHashTableFactory {
+ public:
+  static AggregationStateHashTableBase* CreateResizable(
+      const HashTableImplType hash_table_type,
+      const std::vector<const Type*> &key_types,
+      const std::size_t num_entries,
+      const std::vector<AggregationHandle *> &handles,
+      StorageManager *storage_manager) {
+    switch (hash_table_type) {
+      case HashTableImplType::kSeparateChaining:
+        return new PackedPayloadSeparateChainingAggregationStateHashTable(
+            key_types, num_entries, handles, storage_manager);
+      case HashTableImplType::kCollisionFreeVector:
+        return new CollisionFreeAggregationStateHashTable(
+            key_types, num_entries, handles, storage_manager);
+      default: {
+        LOG(FATAL) << "Unrecognized HashTableImplType in "
+                   << "AggregationStateHashTableFactory::createResizable()\n";
+      }
+    }
+  }
+
+ private:
+  // Class is all-static and should not be instantiated.
+  AggregationStateHashTableFactory();
+
+  DISALLOW_COPY_AND_ASSIGN(AggregationStateHashTableFactory);
+};
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/HashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTablePool.hpp b/storage/HashTablePool.hpp
index 96cf849..5ba703b 100644
--- a/storage/HashTablePool.hpp
+++ b/storage/HashTablePool.hpp
@@ -27,8 +27,7 @@
 
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/HashTableFactory.hpp"
 #include "threading/SpinMutex.hpp"
 #include "utility/Macros.hpp"
 #include "utility/StringUtil.hpp"
@@ -56,36 +55,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.
    *
@@ -93,52 +62,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()) {
@@ -149,7 +95,7 @@ class HashTablePool {
         return ret_hash_table.release();
       }
     }
-    return createNewHashTableFast();
+    return createNewHashTable();
   }
 
   /**
@@ -180,18 +126,10 @@ class HashTablePool {
 
  private:
   AggregationStateHashTableBase* createNewHashTable() {
-    return agg_handle_->createGroupByHashTable(hash_table_impl_type_,
-                                               group_by_types_,
-                                               estimated_num_entries_,
-                                               storage_manager_);
-  }
-
-  AggregationStateHashTableBase* createNewHashTableFast() {
-    return AggregationStateFastHashTableFactory::CreateResizable(
+    return AggregationStateHashTableFactory::CreateResizable(
                 hash_table_impl_type_,
                 group_by_types_,
                 estimated_num_entries_,
-                payload_sizes_,
                 handles_,
                 storage_manager_);
   }
@@ -214,10 +152,6 @@ class HashTablePool {
   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_;
 


[04/13] incubator-quickstep git commit: Push down low cost disjunctive predicates to filter the stored relations early

Posted by ji...@apache.org.
Push down low cost disjunctive predicates to filter the stored relations early


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

Branch: refs/heads/collision-free-agg
Commit: 259cd5e731ead6e38f546c66211aceb3c20f6f4d
Parents: 6d83b46
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Jan 30 01:02:19 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Tue Jan 31 10:59:08 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/PhysicalGenerator.cpp           |  15 ++
 query_optimizer/rules/CMakeLists.txt            |  24 ++
 .../PushDownLowCostDisjunctivePredicate.cpp     | 225 +++++++++++++++++++
 .../PushDownLowCostDisjunctivePredicate.hpp     | 116 ++++++++++
 5 files changed, 381 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/259cd5e7/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index e8bc21c..0ca971d 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -207,6 +207,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_AttachLIPFilters
                       quickstep_queryoptimizer_rules_PruneColumns
+                      quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
                       quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/259cd5e7/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index e12f8be..bd05267 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -28,6 +28,7 @@
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
+#include "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp"
 #include "query_optimizer/rules/ReorderColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
@@ -108,12 +109,22 @@ P::PhysicalPtr PhysicalGenerator::generateInitialPlan(
 P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   std::vector<std::unique_ptr<Rule<P::Physical>>> rules;
   rules.emplace_back(new PruneColumns());
+
+  // TODO(jianqiao): It is possible for PushDownLowCostDisjunctivePredicate to
+  // generate two chaining Selection nodes that can actually be fused into one.
+  // Note that currently it is okay to have the two Selections because they are
+  // applied to a small cardinality stored relation, which is very light-weight.
+  // However it is better to have a FuseSelection optimization (or even a more
+  // general FusePhysical optimization) in the future.
+  rules.emplace_back(new PushDownLowCostDisjunctivePredicate());
+
   if (FLAGS_reorder_hash_joins) {
     rules.emplace_back(new StarSchemaHashJoinOrderOptimization());
     rules.emplace_back(new PruneColumns());
   } else {
     rules.emplace_back(new SwapProbeBuild());
   }
+
   if (FLAGS_reorder_columns) {
     // NOTE(jianqiao): This optimization relies on the fact that the intermediate
     // relations all have SPLIT_ROW_STORE layouts. If this fact gets changed, the
@@ -121,6 +132,10 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
     // should be re-evaluated.
     rules.emplace_back(new ReorderColumns());
   }
+
+  // NOTE(jianqiao): Adding rules after AttachLIPFilters requires extra handling
+  // of LIPFilterConfiguration for transformed nodes. So currently it is suggested
+  // that all the new rules be placed before this point.
   if (FLAGS_use_lip_filters) {
     rules.emplace_back(new AttachLIPFilters());
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/259cd5e7/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index fe2fd17..86d1ef7 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -24,6 +24,9 @@ add_library(quickstep_queryoptimizer_rules_CollapseProject CollapseProject.cpp C
 add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp GenerateJoins.hpp)
 add_library(quickstep_queryoptimizer_rules_PruneColumns PruneColumns.cpp PruneColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownFilter PushDownFilter.cpp PushDownFilter.hpp)
+add_library(quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
+            PushDownLowCostDisjunctivePredicate.cpp
+            PushDownLowCostDisjunctivePredicate.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin PushDownSemiAntiJoin.cpp PushDownSemiAntiJoin.hpp)
 add_library(quickstep_queryoptimizer_rules_ReorderColumns ReorderColumns.cpp ReorderColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_Rule ../../empty_src.cpp Rule.hpp)
@@ -111,6 +114,26 @@ target_link_libraries(quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
+                      ${GFLAGS_LIB_NAME}
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_LogicalAnd
+                      quickstep_queryoptimizer_expressions_LogicalOr
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_NestedLoopsJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_physical_TableReference
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ExpressionUtil
@@ -225,6 +248,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_GenerateJoins
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownFilter
+                      quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_Rule

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/259cd5e7/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp
new file mode 100644
index 0000000..e39f155
--- /dev/null
+++ b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp
@@ -0,0 +1,225 @@
+/**
+ * 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 "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp"
+
+#include <cstddef>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/LogicalAnd.hpp"
+#include "query_optimizer/expressions/LogicalOr.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/Selection.hpp"
+#include "query_optimizer/physical/TableReference.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+
+#include "gflags/gflags.h"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+DEFINE_uint64(push_down_disjunctive_predicate_cardinality_threshold, 100u,
+              "The cardinality threshold for a stored relation for the "
+              "PushDownLowCostDisjunctivePredicate optimization rule to push "
+              "down a disjunctive predicate to pre-filter that relation.");
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr PushDownLowCostDisjunctivePredicate::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  const P::TopLevelPlanPtr top_level_plan =
+     std::static_pointer_cast<const P::TopLevelPlan>(input);
+  cost_model_.reset(
+      new cost::StarSchemaSimpleCostModel(
+          top_level_plan->shared_subplans()));
+
+  collectApplicablePredicates(input);
+
+  if (!applicable_predicates_.empty()) {
+    // Apply the selected predicates to stored relations.
+    return attachPredicates(input);
+  } else {
+    return input;
+  }
+}
+
+void PushDownLowCostDisjunctivePredicate::collectApplicablePredicates(
+    const physical::PhysicalPtr &input) {
+  P::TableReferencePtr table_reference;
+  if (P::SomeTableReference::MatchesWithConditionalCast(input, &table_reference)) {
+    // Consider only stored relations with small cardinality as targets.
+    if (cost_model_->estimateCardinality(input) <=
+            FLAGS_push_down_disjunctive_predicate_cardinality_threshold) {
+      applicable_nodes_.emplace_back(input, &table_reference->attribute_list());
+    }
+    return;
+  }
+
+  for (const auto &child : input->children()) {
+    collectApplicablePredicates(child);
+  }
+
+  E::PredicatePtr filter_predicate = nullptr;
+  switch (input->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      filter_predicate =
+          std::static_pointer_cast<const P::Aggregate>(input)->filter_predicate();
+      break;
+    }
+    case P::PhysicalType::kHashJoin: {
+      const P::HashJoinPtr hash_join =
+          std::static_pointer_cast<const P::HashJoin>(input);
+      if (hash_join->join_type() == P::HashJoin::JoinType::kInnerJoin) {
+        filter_predicate = hash_join->residual_predicate();
+      }
+      break;
+    }
+    case P::PhysicalType::kNestedLoopsJoin: {
+      filter_predicate =
+          std::static_pointer_cast<const P::NestedLoopsJoin>(input)->join_predicate();
+      break;
+    }
+    case P::PhysicalType::kSelection: {
+      filter_predicate =
+          std::static_pointer_cast<const P::Selection>(input)->filter_predicate();
+      break;
+    }
+    default:
+      break;
+  }
+
+  E::LogicalOrPtr disjunctive_predicate;
+  if (filter_predicate == nullptr ||
+      !E::SomeLogicalOr::MatchesWithConditionalCast(filter_predicate, &disjunctive_predicate)) {
+    return;
+  }
+
+  // Consider only disjunctive normal form, i.e. disjunction of conjunctions.
+  // Divide the disjunctive components into groups.
+  std::vector<std::vector<E::PredicatePtr>> candidate_predicates;
+  std::vector<std::vector<std::vector<E::AttributeReferencePtr>>> candidate_attributes;
+  for (const auto &conjunctive_predicate : disjunctive_predicate->operands()) {
+    candidate_predicates.emplace_back();
+    candidate_attributes.emplace_back();
+    E::LogicalAndPtr logical_and;
+    if (E::SomeLogicalAnd::MatchesWithConditionalCast(conjunctive_predicate, &logical_and)) {
+      for (const auto &predicate : logical_and->operands()) {
+        candidate_predicates.back().emplace_back(predicate);
+        candidate_attributes.back().emplace_back(
+            predicate->getReferencedAttributes());
+      }
+    } else {
+      candidate_predicates.back().emplace_back(conjunctive_predicate);
+      candidate_attributes.back().emplace_back(
+          conjunctive_predicate->getReferencedAttributes());
+    }
+  }
+
+  // Check whether the conditions are met for pushing down part of the predicates
+  // to each small-cardinality stored relation.
+  for (const auto &node_pair : applicable_nodes_) {
+    const std::vector<E::AttributeReferencePtr> &target_attributes = *node_pair.second;
+    std::vector<E::PredicatePtr> selected_disj_preds;
+    for (std::size_t i = 0; i < candidate_predicates.size(); ++i) {
+      const auto &cand_preds = candidate_predicates[i];
+      const auto &cand_attrs = candidate_attributes[i];
+
+      std::vector<E::PredicatePtr> selected_conj_preds;
+      for (std::size_t j = 0; j < cand_preds.size(); ++j) {
+        if (E::SubsetOfExpressions(cand_attrs[j], target_attributes)) {
+          selected_conj_preds.emplace_back(cand_preds[j]);
+        }
+      }
+      if (selected_conj_preds.empty()) {
+        // Not every disjunctive component contains a predicate that can be applied
+        // to the table reference node -- condition failed, exit.
+        selected_disj_preds.clear();
+        break;
+      } else {
+        selected_disj_preds.emplace_back(
+            CreateConjunctive(selected_conj_preds));
+      }
+    }
+    if (!selected_disj_preds.empty()) {
+      applicable_predicates_[node_pair.first].add(
+          CreateDisjunctive(selected_disj_preds));
+    }
+  }
+}
+
+P::PhysicalPtr PushDownLowCostDisjunctivePredicate::attachPredicates(
+    const P::PhysicalPtr &input) const {
+  std::vector<P::PhysicalPtr> new_children;
+  for (const P::PhysicalPtr &child : input->children()) {
+    const P::PhysicalPtr new_child = attachPredicates(child);
+    new_children.push_back(new_child);
+  }
+
+  const P::PhysicalPtr output =
+      new_children == input->children() ? input
+                                        : input->copyWithNewChildren(new_children);
+
+  const auto &node_it = applicable_predicates_.find(input);
+  if (node_it != applicable_predicates_.end()) {
+    const E::PredicatePtr filter_predicate =
+        CreateConjunctive(node_it->second.predicates);
+    return P::Selection::Create(output,
+                                E::ToNamedExpressions(output->getOutputAttributes()),
+                                filter_predicate);
+  }
+
+  return output;
+}
+
+E::PredicatePtr PushDownLowCostDisjunctivePredicate::CreateConjunctive(
+    const std::vector<E::PredicatePtr> predicates) {
+  DCHECK_GE(predicates.size(), 1u);
+  if (predicates.size() == 1) {
+    return predicates.front();
+  } else {
+    return E::LogicalAnd::Create(predicates);
+  }
+}
+
+E::PredicatePtr PushDownLowCostDisjunctivePredicate::CreateDisjunctive(
+    const std::vector<E::PredicatePtr> predicates) {
+  DCHECK_GE(predicates.size(), 1u);
+  if (predicates.size() == 1) {
+    return predicates.front();
+  } else {
+    return E::LogicalOr::Create(predicates);
+  }
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/259cd5e7/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp
new file mode 100644
index 0000000..3e4b602
--- /dev/null
+++ b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp
@@ -0,0 +1,116 @@
+/**
+ * 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_QUERY_OPTIMIZER_RULES_PUSH_DOWN_LOW_COST_DISJUNCTIVE_PREDICATE_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_PUSH_DOWN_LOW_COST_DISJUNCTIVE_PREDICATE_HPP_
+
+#include <cstddef>
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief Rule that applies to a physical plan to push down low-cost disjunctive
+ *        predicate when proper conditions are met.
+ *
+ * Here we elaborate the conditions.
+ *
+ * Let
+ *   P = p_{1,1} AND ... AND p_{1, m_1} OR ... OR p_{n,1} AND ... AND p_{n, m_n}
+ * be a predicate in disjunctive normal form.
+ *
+ * Now consider each small-cardinality relation R, if for each i in 1..n, there
+ * exists at least one predicate p_{i, k_i} that is applicable to R. Then we can
+ * construct a new predicate
+ *   P' = p_{1, k_1} OR ... OR p_{n, k_n}
+ * and push down P' to be applied to R.
+ *
+ * Also, if any conjunctive component in P contains more than one predicate that
+ * is applicable to R, then we can combine all these applicable predicates as a
+ * conjunctive component in P'.
+ *
+ * Finally, note that if there exists a conjunctive component that contains no
+ * predicate applicable to R. Then the condition fails and we cannot do a push
+ * down for R.
+ */
+class PushDownLowCostDisjunctivePredicate : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  PushDownLowCostDisjunctivePredicate() {}
+
+  ~PushDownLowCostDisjunctivePredicate() override {}
+
+  std::string getName() const override {
+    return "PushDownLowCostDisjunctivePredicate";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  struct PredicateInfo {
+    PredicateInfo() {}
+    inline void add(expressions::PredicatePtr predicate) {
+      predicates.emplace_back(predicate);
+    }
+    std::vector<expressions::PredicatePtr> predicates;
+  };
+
+  void collectApplicablePredicates(const physical::PhysicalPtr &input);
+
+  physical::PhysicalPtr attachPredicates(const physical::PhysicalPtr &input) const;
+
+  static expressions::PredicatePtr CreateConjunctive(
+      const std::vector<expressions::PredicatePtr> predicates);
+
+  static expressions::PredicatePtr CreateDisjunctive(
+      const std::vector<expressions::PredicatePtr> predicates);
+
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+
+  std::vector<std::pair<physical::PhysicalPtr,
+                        const std::vector<expressions::AttributeReferencePtr> *>> applicable_nodes_;
+  std::map<physical::PhysicalPtr, PredicateInfo> applicable_predicates_;
+
+  DISALLOW_COPY_AND_ASSIGN(PushDownLowCostDisjunctivePredicate);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_PUSH_DOWN_LOW_COST_DISJUNCTIVE_PREDICATE_HPP_


[11/13] incubator-quickstep git commit: Initial commit.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index b942c1b..5de2653 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -39,15 +39,17 @@
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/AggregationOperationState.pb.h"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableBase.hpp"
 #include "storage/HashTableFactory.hpp"
+#include "storage/HashTableBase.hpp"
 #include "storage/InsertDestination.hpp"
+#include "storage/PackedPayloadAggregationStateHashTable.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/SubBlocksReference.hpp"
 #include "storage/TupleIdSequence.hpp"
 #include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
@@ -80,50 +82,63 @@ AggregationOperationState::AggregationOperationState(
     const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
     StorageManager *storage_manager)
     : input_relation_(input_relation),
-      is_aggregate_partitioned_(checkAggregatePartitioned(
-          estimated_num_entries, is_distinct, group_by, aggregate_functions)),
+      is_aggregate_collision_free_(false),
+      is_aggregate_partitioned_(false),
       predicate_(predicate),
-      group_by_list_(std::move(group_by)),
-      arguments_(std::move(arguments)),
       is_distinct_(std::move(is_distinct)),
       storage_manager_(storage_manager) {
+  if (!group_by.empty()) {
+    if (hash_table_impl_type == HashTableImplType::kCollisionFreeVector) {
+      is_aggregate_collision_free_ = true;
+    } else {
+      is_aggregate_partitioned_ = checkAggregatePartitioned(
+          estimated_num_entries, is_distinct_, group_by, aggregate_functions);
+    }
+  }
+
   // Sanity checks: each aggregate has a corresponding list of arguments.
-  DCHECK(aggregate_functions.size() == arguments_.size());
+  DCHECK(aggregate_functions.size() == arguments.size());
 
   // Get the types of GROUP BY expressions for creating HashTables below.
-  std::vector<const Type *> group_by_types;
-  for (const std::unique_ptr<const Scalar> &group_by_element : group_by_list_) {
-    group_by_types.emplace_back(&group_by_element->getType());
+  for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
+    group_by_types_.emplace_back(&group_by_element->getType());
   }
 
-  std::vector<AggregationHandle *> group_by_handles;
-  group_by_handles.clear();
+  // Prepare group-by element attribute ids and non-trivial expressions.
+  for (std::unique_ptr<const Scalar> &group_by_element : group_by) {
+    const attribute_id attr_id =
+        group_by_element->getAttributeIdForValueAccessor();
+    if (attr_id == kInvalidAttributeID) {
+      const attribute_id non_trivial_attr_id =
+          -(static_cast<attribute_id>(non_trivial_expressions_.size()) + 2);
+      non_trivial_expressions_.emplace_back(group_by_element.release());
+      group_by_key_ids_.emplace_back(non_trivial_attr_id);
+    } else {
+      group_by_key_ids_.emplace_back(attr_id);
+    }
+  }
 
   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);
+    DCHECK_GT(group_by_key_ids_.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_,
+                                                     group_by_types_,
+                                                     {handles_.front().get()},
                                                      storage_manager));
   } else {
+    std::vector<AggregationHandle *> group_by_handles;
+
     // 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<std::vector<std::unique_ptr<const Scalar>>>::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
@@ -133,6 +148,22 @@ AggregationOperationState::AggregationOperationState(
         argument_types.emplace_back(&argument->getType());
       }
 
+      // Prepare argument attribute ids and non-trivial expressions.
+      std::vector<attribute_id> argument_ids;
+      for (std::unique_ptr<const Scalar> &argument : *args_it) {
+        const attribute_id attr_id =
+            argument->getAttributeIdForValueAccessor();
+        if (attr_id == kInvalidAttributeID) {
+          const attribute_id non_trivial_attr_id =
+              -(static_cast<attribute_id>(non_trivial_expressions_.size()) + 2);
+          non_trivial_expressions_.emplace_back(argument.release());
+          argument_ids.emplace_back(non_trivial_attr_id);
+        } else {
+          argument_ids.emplace_back(attr_id);
+        }
+      }
+      argument_ids_.emplace_back(std::move(argument_ids));
+
       // Sanity checks: aggregate function exists and can apply to the specified
       // arguments.
       DCHECK(*agg_func_it != nullptr);
@@ -142,85 +173,43 @@ AggregationOperationState::AggregationOperationState(
       // to do actual aggregate computation.
       handles_.emplace_back((*agg_func_it)->createHandle(argument_types));
 
-      if (!group_by_list_.empty()) {
+      if (!group_by_key_ids_.empty()) {
         // Aggregation with GROUP BY: combined payload is partially updated in
         // the presence of DISTINCT.
         if (*is_distinct_it) {
-          handles_.back()->blockUpdate();
+          LOG(FATAL) << "Distinct aggregation not supported";
         }
-        group_by_handles.emplace_back(handles_.back());
-        payload_sizes.emplace_back(group_by_handles.back()->getPayloadSize());
+        group_by_handles.emplace_back(handles_.back().get());
       } 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);
-          }
-        }
-
-        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's
-        // worth.
-        distinctify_hashtables_.emplace_back(
-            AggregationStateFastHashTableFactory::CreateResizable(
-                *distinctify_hash_table_impl_types_it,
-                key_types,
+    // Aggregation with GROUP BY: create a HashTable pool.
+    if (!group_by_key_ids_.empty()) {
+      if (is_aggregate_collision_free_) {
+        collision_free_hashtable_.reset(
+            AggregationStateHashTableFactory::CreateResizable(
+                hash_table_impl_type,
+                group_by_types_,
                 estimated_num_entries,
-                {0},
-                {},
+                group_by_handles,
                 storage_manager));
-        ++distinctify_hash_table_impl_types_it;
-      } else {
-        distinctify_hashtables_.emplace_back(nullptr);
-      }
-    }
-
-    if (!group_by_handles.empty()) {
-      // Aggregation with GROUP BY: create a HashTable pool.
-      if (!is_aggregate_partitioned_) {
-        group_by_hashtable_pool_.reset(new HashTablePool(estimated_num_entries,
-                                                         hash_table_impl_type,
-                                                         group_by_types,
-                                                         payload_sizes,
-                                                         group_by_handles,
-                                                         storage_manager));
-      } else {
+      } else if (is_aggregate_partitioned_) {
         partitioned_group_by_hashtable_pool_.reset(
             new PartitionedHashTablePool(estimated_num_entries,
                                          FLAGS_num_aggregation_partitions,
                                          hash_table_impl_type,
-                                         group_by_types,
-                                         payload_sizes,
+                                         group_by_types_,
                                          group_by_handles,
                                          storage_manager));
+      } else {
+        group_by_hashtable_pool_.reset(new HashTablePool(estimated_num_entries,
+                                                         hash_table_impl_type,
+                                                         group_by_types_,
+                                                         group_by_handles,
+                                                         storage_manager));
       }
     }
   }
@@ -269,7 +258,7 @@ AggregationOperationState* AggregationOperationState::ReconstructFromProto(
         proto.group_by_expressions(group_by_idx), database));
   }
 
-  unique_ptr<Predicate> predicate;
+  std::unique_ptr<Predicate> predicate;
   if (proto.has_predicate()) {
     predicate.reset(
         PredicateFactory::ReconstructFromProto(proto.predicate(), database));
@@ -353,33 +342,72 @@ bool AggregationOperationState::ProtoIsValid(
   return true;
 }
 
-void AggregationOperationState::aggregateBlock(const block_id input_block,
-                                               LIPFilterAdaptiveProber *lip_filter_adaptive_prober) {
-  if (group_by_list_.empty()) {
-    aggregateBlockSingleState(input_block);
+std::size_t AggregationOperationState::getNumPartitions() const {
+  if (is_aggregate_collision_free_) {
+    return static_cast<CollisionFreeAggregationStateHashTable *>(
+        collision_free_hashtable_.get())->getNumFinalizationPartitions();
+  } else if (is_aggregate_partitioned_) {
+    return partitioned_group_by_hashtable_pool_->getNumPartitions();
+  } else  {
+    return 1u;
+  }
+}
+
+std::size_t AggregationOperationState::getNumInitializationPartitions() const {
+  if (is_aggregate_collision_free_) {
+    return static_cast<CollisionFreeAggregationStateHashTable *>(
+        collision_free_hashtable_.get())->getNumInitializationPartitions();
   } else {
-    aggregateBlockHashTable(input_block, lip_filter_adaptive_prober);
+    return 0u;
   }
 }
 
-void AggregationOperationState::finalizeAggregate(
-    InsertDestination *output_destination) {
-  if (group_by_list_.empty()) {
-    finalizeSingleState(output_destination);
+void AggregationOperationState::initializeState(const std::size_t partition_id) {
+  if (is_aggregate_collision_free_) {
+    static_cast<CollisionFreeAggregationStateHashTable *>(
+        collision_free_hashtable_.get())->initialize(partition_id);
   } else {
-    finalizeHashTable(output_destination);
+    LOG(FATAL) << "AggregationOperationState::initializeState() "
+               << "is not supported by this aggregation";
   }
 }
 
-void AggregationOperationState::mergeSingleState(
-    const std::vector<std::unique_ptr<AggregationState>> &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());
+bool AggregationOperationState::checkAggregatePartitioned(
+    const std::size_t estimated_num_groups,
+    const std::vector<bool> &is_distinct,
+    const std::vector<std::unique_ptr<const Scalar>> &group_by,
+    const std::vector<const AggregateFunction *> &aggregate_functions) const {
+  // If there's no aggregation, return false.
+  if (aggregate_functions.empty()) {
+    return false;
+  }
+  // Check if there's a distinct operation involved in any aggregate, if so
+  // the aggregate can't be partitioned.
+  for (auto distinct : is_distinct) {
+    if (distinct) {
+      return false;
     }
   }
+  // There's no distinct aggregation involved, Check if there's at least one
+  // GROUP BY operation.
+  if (group_by.empty()) {
+    return false;
+  }
+  // There are GROUP BYs without DISTINCT. Check if the estimated number of
+  // groups is large enough to warrant a partitioned aggregation.
+  return estimated_num_groups >
+         static_cast<std::size_t>(
+             FLAGS_partition_aggregation_num_groups_threshold);
+  return false;
+}
+
+void AggregationOperationState::aggregateBlock(const block_id input_block,
+                                               LIPFilterAdaptiveProber *lip_filter_adaptive_prober) {
+  if (group_by_key_ids_.empty()) {
+    aggregateBlockSingleState(input_block);
+  } else {
+    aggregateBlockHashTable(input_block, lip_filter_adaptive_prober);
+  }
 }
 
 void AggregationOperationState::aggregateBlockSingleState(
@@ -392,114 +420,137 @@ void AggregationOperationState::aggregateBlockSingleState(
 
   std::unique_ptr<TupleIdSequence> matches;
   if (predicate_ != nullptr) {
-    std::unique_ptr<ValueAccessor> accessor(
-        block->getTupleStorageSubBlock().createValueAccessor());
     matches.reset(block->getMatchesForPredicate(predicate_.get(), matches.get()));
   }
 
-  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    const std::vector<attribute_id> *local_arguments_as_attributes = nullptr;
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-    // If all arguments are attributes of the input relation, elide a copy.
-    if (!arguments_as_attributes_[agg_idx].empty()) {
-      local_arguments_as_attributes = &(arguments_as_attributes_[agg_idx]);
+  const auto &tuple_store = block->getTupleStorageSubBlock();
+  std::unique_ptr<ValueAccessor> accessor(
+      tuple_store.createValueAccessor(matches.get()));
+
+  ColumnVectorsValueAccessor non_trivial_results;
+  if (!non_trivial_expressions_.empty()) {
+    SubBlocksReference sub_blocks_ref(tuple_store,
+                                      block->getIndices(),
+                                      block->getIndicesConsistent());
+    for (const auto &expression : non_trivial_expressions_) {
+      non_trivial_results.addColumn(
+          expression->getAllValues(accessor.get(), &sub_blocks_ref));
     }
-#endif
-    if (is_distinct_[agg_idx]) {
-      // 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 */
-                               matches.get(),
-                               distinctify_hashtables_[agg_idx].get(),
-                               nullptr /* reuse_group_by_vectors */);
-      local_state.emplace_back(nullptr);
+  }
+
+  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
+    const auto &argument_ids = argument_ids_[agg_idx];
+    const auto &handle = handles_[agg_idx];
+
+    AggregationState *state;
+    if (argument_ids.empty()) {
+      // Special case. This is a nullary aggregate (i.e. COUNT(*)).
+      state = handle->accumulateNullary(matches == nullptr ? tuple_store.numTuples()
+                                                           : matches->size());
     } else {
-      // Call StorageBlock::aggregate() to actually do the aggregation.
-      local_state.emplace_back(block->aggregate(*handles_[agg_idx],
-                                                arguments_[agg_idx],
-                                                local_arguments_as_attributes,
-                                                matches.get()));
+      // Have the AggregationHandle actually do the aggregation.
+      state = handle->accumulate(accessor.get(), &non_trivial_results, argument_ids);
     }
+    local_state.emplace_back(state);
   }
 
   // Merge per-block aggregation states back with global state.
   mergeSingleState(local_state);
 }
 
+void AggregationOperationState::mergeSingleState(
+    const std::vector<std::unique_ptr<AggregationState>> &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());
+    }
+  }
+}
+
+void AggregationOperationState::mergeGroupByHashTables(
+    AggregationStateHashTableBase *src, AggregationStateHashTableBase *dst) const {
+  HashTableMergerFast merger(dst);
+  static_cast<PackedPayloadSeparateChainingAggregationStateHashTable *>(src)
+      ->forEach(&merger);
+}
+
 void AggregationOperationState::aggregateBlockHashTable(
     const block_id input_block,
     LIPFilterAdaptiveProber *lip_filter_adaptive_prober) {
   BlockReference block(
       storage_manager_->getBlock(input_block, input_relation_));
+  const auto &tuple_store = block->getTupleStorageSubBlock();
+  std::unique_ptr<ValueAccessor> base_accessor(tuple_store.createValueAccessor());
+  std::unique_ptr<ValueAccessor> shared_accessor;
+  ValueAccessor *accessor = base_accessor.get();
 
   // Apply the predicate first, then the LIPFilters, to generate a TupleIdSequence
   // as the existence map for the tuples.
   std::unique_ptr<TupleIdSequence> matches;
   if (predicate_ != nullptr) {
     matches.reset(block->getMatchesForPredicate(predicate_.get()));
+    shared_accessor.reset(
+        base_accessor->createSharedTupleIdSequenceAdapterVirtual(*matches));
+    accessor = shared_accessor.get();
   }
   if (lip_filter_adaptive_prober != nullptr) {
-    std::unique_ptr<ValueAccessor> accessor(
-        block->getTupleStorageSubBlock().createValueAccessor(matches.get()));
-    matches.reset(lip_filter_adaptive_prober->filterValueAccessor(accessor.get()));
+    matches.reset(lip_filter_adaptive_prober->filterValueAccessor(accessor));
+    shared_accessor.reset(
+        base_accessor->createSharedTupleIdSequenceAdapterVirtual(*matches));
+    accessor = shared_accessor.get();
   }
 
-  // This holds values of all the GROUP BY attributes so that the can be reused
-  // across multiple aggregates (i.e. we only pay the cost of evaluatin the
-  // GROUP BY expressions once).
-  std::vector<std::unique_ptr<ColumnVector>> reuse_group_by_vectors;
-
-  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    if (is_distinct_[agg_idx]) {
-      // 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_,
-                               matches.get(),
-                               distinctify_hashtables_[agg_idx].get(),
-                               &reuse_group_by_vectors);
+  std::unique_ptr<ColumnVectorsValueAccessor> non_trivial_results;
+  if (!non_trivial_expressions_.empty()) {
+    non_trivial_results.reset(new ColumnVectorsValueAccessor());
+    SubBlocksReference sub_blocks_ref(tuple_store,
+                                      block->getIndices(),
+                                      block->getIndicesConsistent());
+    for (const auto &expression : non_trivial_expressions_) {
+      non_trivial_results->addColumn(
+          expression->getAllValues(accessor, &sub_blocks_ref));
     }
   }
 
-  if (!is_aggregate_partitioned_) {
-    // Call StorageBlock::aggregateGroupBy() to aggregate this block's values
-    // directly into the (threadsafe) shared global HashTable for this
-    // aggregate.
-    DCHECK(group_by_hashtable_pool_ != nullptr);
-    AggregationStateHashTableBase *agg_hash_table =
-      group_by_hashtable_pool_->getHashTableFast();
-    DCHECK(agg_hash_table != nullptr);
-    block->aggregateGroupBy(arguments_,
-                            group_by_list_,
-                            matches.get(),
-                            agg_hash_table,
-                            &reuse_group_by_vectors);
-    group_by_hashtable_pool_->returnHashTable(agg_hash_table);
+  accessor->beginIterationVirtual();
+
+  // TODO
+  if (is_aggregate_collision_free_) {
+    aggregateBlockHashTableImplCollisionFree(
+        accessor, non_trivial_results.get());
+  } else if (is_aggregate_partitioned_) {
+    aggregateBlockHashTableImplPartitioned(
+        accessor, non_trivial_results.get());
   } else {
-    ColumnVectorsValueAccessor temp_result;
-    // IDs of 'arguments' as attributes in the ValueAccessor we create below.
-    std::vector<attribute_id> argument_ids;
+    aggregateBlockHashTableImplThreadPrivate(
+        accessor, non_trivial_results.get());
+  }
+}
 
-    // IDs of GROUP BY key element(s) in the ValueAccessor we create below.
-    std::vector<attribute_id> key_ids;
+void AggregationOperationState::aggregateBlockHashTableImplCollisionFree(
+    ValueAccessor *accessor,
+    ColumnVectorsValueAccessor *aux_accessor) {
+  DCHECK(collision_free_hashtable_ != nullptr);
+
+  collision_free_hashtable_->upsertValueAccessor(argument_ids_,
+                                                 group_by_key_ids_,
+                                                 accessor,
+                                                 aux_accessor);
+}
+
+void AggregationOperationState::aggregateBlockHashTableImplPartitioned(
+    ValueAccessor *accessor,
+    ColumnVectorsValueAccessor *aux_accessor) {
+  DCHECK(partitioned_group_by_hashtable_pool_ != nullptr);
+
+  InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+      accessor,
+      [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+    // TODO(jianqiao): handle the situation when keys in non_trivial_results
     const std::size_t num_partitions = partitioned_group_by_hashtable_pool_->getNumPartitions();
-    block->aggregateGroupByPartitioned(
-        arguments_,
-        group_by_list_,
-        matches.get(),
-        num_partitions,
-        &temp_result,
-        &argument_ids,
-        &key_ids,
-        &reuse_group_by_vectors);
+
     // Compute the partitions for the tuple formed by group by values.
     std::vector<std::unique_ptr<TupleIdSequence>> partition_membership;
     partition_membership.resize(num_partitions);
@@ -507,32 +558,57 @@ void AggregationOperationState::aggregateBlockHashTable(
     // Create a tuple-id sequence for each partition.
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
       partition_membership[partition].reset(
-          new TupleIdSequence(temp_result.getEndPosition()));
+          new TupleIdSequence(accessor->getEndPosition()));
     }
 
     // Iterate over ValueAccessor for each tuple,
     // set a bit in the appropriate TupleIdSequence.
-    temp_result.beginIteration();
-    while (temp_result.next()) {
+    while (accessor->next()) {
       // We need a unique_ptr because getTupleWithAttributes() uses "new".
-      std::unique_ptr<Tuple> curr_tuple(temp_result.getTupleWithAttributes(key_ids));
+      std::unique_ptr<Tuple> curr_tuple(
+          accessor->getTupleWithAttributes(group_by_key_ids_));
       const std::size_t curr_tuple_partition_id =
           curr_tuple->getTupleHash() % num_partitions;
       partition_membership[curr_tuple_partition_id]->set(
-          temp_result.getCurrentPosition(), true);
+          accessor->getCurrentPosition(), true);
     }
-    // For each partition, create an adapter around Value Accessor and
-    // TupleIdSequence.
-    std::vector<std::unique_ptr<
-        TupleIdSequenceAdapterValueAccessor<ColumnVectorsValueAccessor>>> adapter;
-    adapter.resize(num_partitions);
+    // Aggregate each partition.
     for (std::size_t partition = 0; partition < num_partitions; ++partition) {
-      adapter[partition].reset(temp_result.createSharedTupleIdSequenceAdapter(
-          *(partition_membership)[partition]));
+      std::unique_ptr<ValueAccessor> adapter(
+          accessor->createSharedTupleIdSequenceAdapter(
+              *(partition_membership)[partition]));
       partitioned_group_by_hashtable_pool_->getHashTable(partition)
-          ->upsertValueAccessorCompositeKeyFast(
-              argument_ids, adapter[partition].get(), key_ids, true);
+          ->upsertValueAccessor(argument_ids_,
+                                group_by_key_ids_,
+                                adapter.get(),
+                                aux_accessor);
     }
+  });
+}
+
+void AggregationOperationState::aggregateBlockHashTableImplThreadPrivate(
+    ValueAccessor *accessor,
+    ColumnVectorsValueAccessor *aux_accessor) {
+  DCHECK(group_by_hashtable_pool_ != nullptr);
+
+  AggregationStateHashTableBase *agg_hash_table =
+      group_by_hashtable_pool_->getHashTable();
+
+  agg_hash_table->upsertValueAccessor(argument_ids_,
+                                      group_by_key_ids_,
+                                      accessor,
+                                      aux_accessor);
+  group_by_hashtable_pool_->returnHashTable(agg_hash_table);
+}
+
+void AggregationOperationState::finalizeAggregate(
+    const std::size_t partition_id,
+    InsertDestination *output_destination) {
+  if (group_by_key_ids_.empty()) {
+    DCHECK_EQ(0u, partition_id);
+    finalizeSingleState(output_destination);
+  } else {
+    finalizeHashTable(partition_id, output_destination);
   }
 }
 
@@ -543,12 +619,6 @@ void AggregationOperationState::finalizeSingleState(
   std::vector<TypedValue> attribute_values;
 
   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]));
-    }
-
     attribute_values.emplace_back(
         handles_[agg_idx]->finalize(*single_states_[agg_idx]));
   }
@@ -556,80 +626,79 @@ void AggregationOperationState::finalizeSingleState(
   output_destination->insertTuple(Tuple(std::move(attribute_values)));
 }
 
-void AggregationOperationState::mergeGroupByHashTables(
-    AggregationStateHashTableBase *src, AggregationStateHashTableBase *dst) {
-  HashTableMergerFast merger(dst);
-  (static_cast<FastHashTable<true, false, true, false> *>(src))
-      ->forEachCompositeKeyFast(&merger);
+void AggregationOperationState::finalizeHashTable(
+    const std::size_t partition_id,
+    InsertDestination *output_destination) {
+  if (is_aggregate_collision_free_) {
+    finalizeHashTableImplCollisionFree(partition_id, output_destination);
+  } else if (is_aggregate_partitioned_) {
+    finalizeHashTableImplPartitioned(partition_id, output_destination);
+  } else {
+    DCHECK_EQ(0u, partition_id);
+    finalizeHashTableImplThreadPrivate(output_destination);
+  }
 }
 
-void AggregationOperationState::finalizeHashTable(
+void AggregationOperationState::finalizeHashTableImplCollisionFree(
+    const std::size_t partition_id,
     InsertDestination *output_destination) {
-  // Each element of 'group_by_keys' is a vector of values for a particular
-  // group (which is also the prefix of the finalized Tuple for that group).
-  std::vector<std::vector<TypedValue>> group_by_keys;
+  std::vector<std::unique_ptr<ColumnVector>> final_values;
+  CollisionFreeAggregationStateHashTable *hash_table =
+      static_cast<CollisionFreeAggregationStateHashTable *>(
+          collision_free_hashtable_.get());
 
-  // TODO(harshad) - The merge phase may be slower when each hash table contains
-  // large number of entries. We should find ways in which we can perform a
-  // parallel merge.
+  // TODO
+  const std::size_t max_length =
+      hash_table->getNumTuplesInPartition(partition_id);
+  ColumnVectorsValueAccessor complete_result;
 
-  // TODO(harshad) - Find heuristics for faster merge, even in a single thread.
-  // e.g. Keep merging entries from smaller hash tables to larger.
+  DCHECK_EQ(1u, group_by_types_.size());
+  const Type *key_type = group_by_types_.front();
+  DCHECK(NativeColumnVector::UsableForType(*key_type));
 
-  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());
+  std::unique_ptr<NativeColumnVector> key_cv(
+      new NativeColumnVector(*key_type, max_length));
+  hash_table->finalizeKey(partition_id, key_cv.get());
+  complete_result.addColumn(key_cv.release());
+
+  for (std::size_t i = 0; i < handles_.size(); ++i) {
+    if (handles_[i]->getAggregationID() == AggregationID::kDistinct) {
+      DCHECK_EQ(1u, handles_.size());
+      break;
     }
+
+    const Type *result_type = handles_[i]->getResultType();
+    DCHECK(NativeColumnVector::UsableForType(*result_type));
+
+    std::unique_ptr<NativeColumnVector> result_cv(
+        new NativeColumnVector(*result_type, max_length));
+    hash_table->finalizeState(partition_id, i, result_cv.get());
+    complete_result.addColumn(result_cv.release());
   }
 
+  // Bulk-insert the complete result.
+  output_destination->bulkInsertTuples(&complete_result);
+}
+
+void AggregationOperationState::finalizeHashTableImplPartitioned(
+    const std::size_t partition_id,
+    InsertDestination *output_destination) {
+  // Each element of 'group_by_keys' is a vector of values for a particular
+  // group (which is also the prefix of the finalized Tuple for that group).
+  std::vector<std::vector<TypedValue>> group_by_keys;
+
   // Collect per-aggregate finalized values.
   std::vector<std::unique_ptr<ColumnVector>> final_values;
+  AggregationStateHashTableBase *hash_table =
+      partitioned_group_by_hashtable_pool_->getHashTable(partition_id);
   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);
+        *hash_table, &group_by_keys, agg_idx);
     if (agg_result_col != nullptr) {
       final_values.emplace_back(agg_result_col);
     }
   }
+  hash_table->destroyPayload();
 
   // Reorganize 'group_by_keys' in column-major order so that we can make a
   // ColumnVectorsValueAccessor to bulk-insert results.
@@ -640,23 +709,20 @@ void AggregationOperationState::finalizeHashTable(
   // 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)) {
+  for (const Type *group_by_type : group_by_types_) {
+    if (NativeColumnVector::UsableForType(*group_by_type)) {
       NativeColumnVector *element_cv =
-          new NativeColumnVector(group_by_type, group_by_keys.size());
+          new NativeColumnVector(*group_by_type, group_by_keys.size());
       group_by_cvs.emplace_back(element_cv);
       for (std::vector<TypedValue> &group_key : group_by_keys) {
-        element_cv->appendTypedValue(
-            std::move(group_key[group_by_element_idx]));
+        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
       }
     } else {
       IndirectColumnVector *element_cv =
-          new IndirectColumnVector(group_by_type, group_by_keys.size());
+          new IndirectColumnVector(*group_by_type, group_by_keys.size());
       group_by_cvs.emplace_back(element_cv);
       for (std::vector<TypedValue> &group_key : group_by_keys) {
-        element_cv->appendTypedValue(
-            std::move(group_key[group_by_element_idx]));
+        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
       }
     }
     ++group_by_element_idx;
@@ -676,42 +742,44 @@ void AggregationOperationState::finalizeHashTable(
   output_destination->bulkInsertTuples(&complete_result);
 }
 
-void AggregationOperationState::destroyAggregationHashTablePayload() {
-  std::vector<std::unique_ptr<AggregationStateHashTableBase>> *all_hash_tables =
-      nullptr;
-  if (!is_aggregate_partitioned_) {
-    if (group_by_hashtable_pool_ != nullptr) {
-      all_hash_tables = group_by_hashtable_pool_->getAllHashTables();
-    }
-  } else {
-    if (partitioned_group_by_hashtable_pool_ != nullptr) {
-      all_hash_tables = partitioned_group_by_hashtable_pool_->getAllHashTables();
-    }
-  }
-  if (all_hash_tables != nullptr) {
-    for (std::size_t ht_index = 0; ht_index < all_hash_tables->size(); ++ht_index) {
-      (*all_hash_tables)[ht_index]->destroyPayload();
-    }
-  }
-}
-
-void AggregationOperationState::finalizeAggregatePartitioned(
-    const std::size_t partition_id, InsertDestination *output_destination) {
+void AggregationOperationState::finalizeHashTableImplThreadPrivate(
+    InsertDestination *output_destination) {
   // Each element of 'group_by_keys' is a vector of values for a particular
   // group (which is also the prefix of the finalized Tuple for that group).
   std::vector<std::vector<TypedValue>> group_by_keys;
 
+  // TODO(harshad) - The merge phase may be slower when each hash table contains
+  // large number of entries. We should find ways in which we can perform a
+  // parallel merge.
+
+  // TODO(harshad) - Find heuristics for faster merge, even in a single thread.
+  // e.g. Keep merging entries from smaller hash tables to larger.
+
+  auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
+  DCHECK(hash_tables != nullptr);
+  if (hash_tables->empty()) {
+    return;
+  }
+
+  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(hash_table.get(), final_hash_table.get());
+    hash_table->destroyPayload();
+  }
+
   // Collect per-aggregate finalized values.
   std::vector<std::unique_ptr<ColumnVector>> final_values;
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    AggregationStateHashTableBase *hash_table =
-        partitioned_group_by_hashtable_pool_->getHashTable(partition_id);
     ColumnVector *agg_result_col = handles_[agg_idx]->finalizeHashTable(
-        *hash_table, &group_by_keys, agg_idx);
+        *final_hash_table, &group_by_keys, agg_idx);
     if (agg_result_col != nullptr) {
       final_values.emplace_back(agg_result_col);
     }
   }
+  final_hash_table->destroyPayload();
 
   // Reorganize 'group_by_keys' in column-major order so that we can make a
   // ColumnVectorsValueAccessor to bulk-insert results.
@@ -722,19 +790,22 @@ void AggregationOperationState::finalizeAggregatePartitioned(
   // 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());
+  for (const Type *group_by_type : group_by_types_) {
+    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]));
+        element_cv->appendTypedValue(
+            std::move(group_key[group_by_element_idx]));
       }
     } else {
-      IndirectColumnVector *element_cv = new IndirectColumnVector(group_by_type, group_by_keys.size());
+      IndirectColumnVector *element_cv =
+          new IndirectColumnVector(*group_by_type, group_by_keys.size());
       group_by_cvs.emplace_back(element_cv);
       for (std::vector<TypedValue> &group_key : group_by_keys) {
-        element_cv->appendTypedValue(std::move(group_key[group_by_element_idx]));
+        element_cv->appendTypedValue(
+            std::move(group_key[group_by_element_idx]));
       }
     }
     ++group_by_element_idx;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 591e3a1..44803fc 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -33,7 +33,9 @@
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTablePool.hpp"
 #include "storage/PartitionedHashTablePool.hpp"
+#include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
+#include "utility/ConcurrentBitVector.hpp"
 #include "utility/Macros.hpp"
 
 #include "gflags/gflags.h"
@@ -43,9 +45,11 @@ namespace quickstep {
 class AggregateFunction;
 class CatalogDatabaseLite;
 class CatalogRelationSchema;
+class ColumnVectorsValueAccessor;
 class InsertDestination;
 class LIPFilterAdaptiveProber;
 class StorageManager;
+class TupleIdSequence;
 
 DECLARE_int32(num_aggregation_partitions);
 DECLARE_int32(partition_aggregation_num_groups_threshold);
@@ -166,127 +170,99 @@ class AggregationOperationState {
    *        the block.
    **/
   void aggregateBlock(const block_id input_block,
-                      LIPFilterAdaptiveProber *lip_filter_adaptive_prober);
+                      LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr);
 
   /**
    * @brief Generate the final results for the aggregates managed by this
    *        AggregationOperationState and write them out to StorageBlock(s).
    *
+   * @param partition_id The partition id of this finalize operation.
    * @param output_destination An InsertDestination where the finalized output
    *        tuple(s) from this aggregate are to be written.
    **/
-  void finalizeAggregate(InsertDestination *output_destination);
-
-  /**
-   * @brief Destroy the payloads in the aggregation hash tables.
-   **/
-  void destroyAggregationHashTablePayload();
-
-  /**
-   * @brief Generate the final results for the aggregates managed by this
-   *        AggregationOperationState and write them out to StorageBlock(s).
-   *        In this implementation, each thread picks a hash table belonging to
-   *        a partition and writes its values to StorageBlock(s). There is no
-   *        need to merge multiple hash tables in one, because there is no
-   *        overlap in the keys across two hash tables.
-   *
-   * @param partition_id The ID of the partition for which finalize is being
-   *        performed.
-   * @param output_destination An InsertDestination where the finalized output
-   *        tuple(s) from this aggregate are to be written.
-   **/
-  void finalizeAggregatePartitioned(
-      const std::size_t partition_id, InsertDestination *output_destination);
-
-  static void mergeGroupByHashTables(AggregationStateHashTableBase *src,
-                                     AggregationStateHashTableBase *dst);
-
-  bool isAggregatePartitioned() const {
-    return is_aggregate_partitioned_;
-  }
+  void finalizeAggregate(const std::size_t partition_id,
+                         InsertDestination *output_destination);
 
   /**
    * @brief Get the number of partitions to be used for the aggregation.
    *        For non-partitioned aggregations, we return 1.
    **/
-  std::size_t getNumPartitions() const {
-    return is_aggregate_partitioned_
-               ? partitioned_group_by_hashtable_pool_->getNumPartitions()
-               : 1;
-  }
+  std::size_t getNumPartitions() const;
 
-  int dflag;
+  std::size_t getNumInitializationPartitions() const;
+
+  void initializeState(const std::size_t partition_id);
 
  private:
-  // Merge locally (per storage block) aggregated states with global aggregation
-  // states.
-  void mergeSingleState(
-      const std::vector<std::unique_ptr<AggregationState>> &local_state);
+  bool checkAggregatePartitioned(
+      const std::size_t estimated_num_groups,
+      const std::vector<bool> &is_distinct,
+      const std::vector<std::unique_ptr<const Scalar>> &group_by,
+      const std::vector<const AggregateFunction *> &aggregate_functions) const;
 
   // Aggregate on input block.
   void aggregateBlockSingleState(const block_id input_block);
   void aggregateBlockHashTable(const block_id input_block,
                                LIPFilterAdaptiveProber *lip_filter_adaptive_prober);
 
-  void finalizeSingleState(InsertDestination *output_destination);
-  void finalizeHashTable(InsertDestination *output_destination);
+  // Merge locally (per storage block) aggregated states with global aggregation
+  // states.
+  void mergeSingleState(
+      const std::vector<std::unique_ptr<AggregationState>> &local_state);
+  void mergeGroupByHashTables(AggregationStateHashTableBase *src,
+                              AggregationStateHashTableBase *dst) const;
 
-  bool checkAggregatePartitioned(
-      const std::size_t estimated_num_groups,
-      const std::vector<bool> &is_distinct,
-      const std::vector<std::unique_ptr<const Scalar>> &group_by,
-      const std::vector<const AggregateFunction *> &aggregate_functions) const {
-    // If there's no aggregation, return false.
-    if (aggregate_functions.empty()) {
-      return false;
-    }
-    // Check if there's a distinct operation involved in any aggregate, if so
-    // the aggregate can't be partitioned.
-    for (auto distinct : is_distinct) {
-      if (distinct) {
-        return false;
-      }
-    }
-    // There's no distinct aggregation involved, Check if there's at least one
-    // GROUP BY operation.
-    if (group_by.empty()) {
-      return false;
-    }
-    // There are GROUP BYs without DISTINCT. Check if the estimated number of
-    // groups is large enough to warrant a partitioned aggregation.
-    return estimated_num_groups >
-           static_cast<std::size_t>(
-               FLAGS_partition_aggregation_num_groups_threshold);
-  }
+  // Finalize the aggregation results into output_destination.
+  void finalizeSingleState(InsertDestination *output_destination);
+  void finalizeHashTable(const std::size_t partition_id,
+                         InsertDestination *output_destination);
+
+  // Specialized implementations for aggregateBlockHashTable.
+  void aggregateBlockHashTableImplCollisionFree(ValueAccessor *accessor,
+                                                ColumnVectorsValueAccessor *aux_accessor);
+  void aggregateBlockHashTableImplPartitioned(ValueAccessor *accessor,
+                                              ColumnVectorsValueAccessor *aux_accessor);
+  void aggregateBlockHashTableImplThreadPrivate(ValueAccessor *accessor,
+                                                ColumnVectorsValueAccessor *aux_accessor);
+
+  // Specialized implementations for finalizeHashTable.
+  void finalizeHashTableImplCollisionFree(const std::size_t partition_id,
+                                          InsertDestination *output_destination);
+  void finalizeHashTableImplPartitioned(const std::size_t partition_id,
+                                        InsertDestination *output_destination);
+  void finalizeHashTableImplThreadPrivate(InsertDestination *output_destination);
 
   // Common state for all aggregates in this operation: the input relation, the
   // filter predicate (if any), and the list of GROUP BY expressions (if any).
   const CatalogRelationSchema &input_relation_;
 
+  // Whether the aggregation is collision free or not.
+  bool is_aggregate_collision_free_;
+
   // Whether the aggregation is partitioned or not.
-  const bool is_aggregate_partitioned_;
+  bool is_aggregate_partitioned_;
 
   std::unique_ptr<const Predicate> predicate_;
-  std::vector<std::unique_ptr<const Scalar>> group_by_list_;
 
   // Each individual aggregate in this operation has an AggregationHandle and
-  // some number of Scalar arguments.
-  std::vector<AggregationHandle *> handles_;
-  std::vector<std::vector<std::unique_ptr<const Scalar>>> arguments_;
+  // zero (indicated by -1) or one argument.
+  std::vector<std::unique_ptr<AggregationHandle>> handles_;
 
   // For each aggregate, whether DISTINCT should be applied to the aggregate's
   // arguments.
   std::vector<bool> is_distinct_;
 
-  // Hash table for obtaining distinct (i.e. unique) arguments.
-  std::vector<std::unique_ptr<AggregationStateHashTableBase>>
-      distinctify_hashtables_;
+  // Non-trivial group-by/argument expressions that need to be evaluated.
+  std::vector<std::unique_ptr<const Scalar>> non_trivial_expressions_;
 
-#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.
-  std::vector<std::vector<attribute_id>> arguments_as_attributes_;
-#endif
+  std::vector<attribute_id> group_by_key_ids_;
+  std::vector<std::vector<attribute_id>> argument_ids_;
+
+  std::vector<const Type *> group_by_types_;
+
+  // Hash table for obtaining distinct (i.e. unique) arguments.
+//  std::vector<std::unique_ptr<AggregationStateHashTableBase>>
+//      distinctify_hashtables_;
 
   // Per-aggregate global states for aggregation without GROUP BY.
   std::vector<std::unique_ptr<AggregationState>> single_states_;
@@ -303,6 +279,8 @@ class AggregationOperationState {
 
   std::unique_ptr<PartitionedHashTablePool> partitioned_group_by_hashtable_pool_;
 
+  std::unique_ptr<AggregationStateHashTableBase> collision_free_hashtable_;
+
   StorageManager *storage_manager_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationOperationState);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index fddea1f..c7bc28f 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -165,6 +165,9 @@ if(QUICKSTEP_HAVE_BITWEAVING)
               bitweaving/BitWeavingVIndexSubBlock.hpp)
 endif()
 # CMAKE_VALIDATE_IGNORE_END
+add_library(quickstep_storage_CollisionFreeAggregationStateHashTable
+            CollisionFreeAggregationStateHashTable.cpp
+            CollisionFreeAggregationStateHashTable.hpp)
 add_library(quickstep_storage_ColumnStoreUtil ColumnStoreUtil.cpp ColumnStoreUtil.hpp)
 add_library(quickstep_storage_CompressedBlockBuilder CompressedBlockBuilder.cpp CompressedBlockBuilder.hpp)
 add_library(quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
@@ -194,9 +197,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)
@@ -225,6 +225,9 @@ add_library(quickstep_storage_InsertDestination_proto
 add_library(quickstep_storage_LinearOpenAddressingHashTable
             ../empty_src.cpp
             LinearOpenAddressingHashTable.hpp)
+add_library(quickstep_storage_PackedPayloadAggregationStateHashTable
+            PackedPayloadAggregationStateHashTable.cpp
+            PackedPayloadAggregationStateHashTable.hpp)
 add_library(quickstep_storage_PartitionedHashTablePool ../empty_src.cpp PartitionedHashTablePool.hpp)
 add_library(quickstep_storage_PreloaderThread PreloaderThread.cpp PreloaderThread.hpp)
 add_library(quickstep_storage_SMAIndexSubBlock SMAIndexSubBlock.cpp SMAIndexSubBlock.hpp)
@@ -276,22 +279,25 @@ target_link_libraries(quickstep_storage_AggregationOperationState
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_storage_AggregationOperationState_proto
-                      quickstep_storage_HashTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
                       quickstep_storage_HashTablePool
                       quickstep_storage_InsertDestination
                       quickstep_storage_PartitionedHashTablePool
+                      quickstep_storage_PackedPayloadAggregationStateHashTable
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
+                      quickstep_storage_SubBlocksReference
                       quickstep_storage_TupleIdSequence
                       quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorsValueAccessor
                       quickstep_types_containers_Tuple
                       quickstep_utility_Macros
+                      quickstep_utility_ConcurrentBitVector
                       quickstep_utility_lipfilter_LIPFilterAdaptiveProber)
 target_link_libraries(quickstep_storage_AggregationOperationState_proto
                       quickstep_expressions_Expressions_proto
@@ -429,6 +435,24 @@ if(QUICKSTEP_HAVE_BITWEAVING)
                         quickstep_utility_Macros)
 endif()
 # CMAKE_VALIDATE_IGNORE_END
+target_link_libraries(quickstep_storage_CollisionFreeAggregationStateHashTable
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_StorageBlob
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_StorageManager
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_ConcurrentBitVector
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_ColumnStoreUtil
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelationSchema
@@ -626,52 +650,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_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_Macros)
-target_link_libraries(quickstep_storage_FastSeparateChainingHashTable
-                      quickstep_expressions_aggregation_AggregationHandle
-                      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
@@ -734,10 +712,12 @@ target_link_libraries(quickstep_storage_HashTable_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_storage_HashTableFactory
                       glog
+                      quickstep_storage_CollisionFreeAggregationStateHashTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTable_proto
                       quickstep_storage_HashTableBase
                       quickstep_storage_LinearOpenAddressingHashTable
+                      quickstep_storage_PackedPayloadAggregationStateHashTable
                       quickstep_storage_SeparateChainingHashTable
                       quickstep_storage_SimpleScalarSeparateChainingHashTable
                       quickstep_storage_TupleReference
@@ -757,9 +737,8 @@ target_link_libraries(quickstep_storage_HashTableKeyManager
 target_link_libraries(quickstep_storage_HashTablePool
                       glog
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_FastHashTableFactory
                       quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableFactory
                       quickstep_threading_SpinMutex
                       quickstep_utility_Macros
                       quickstep_utility_StringUtil)
@@ -817,12 +796,32 @@ target_link_libraries(quickstep_storage_LinearOpenAddressingHashTable
                       quickstep_utility_Alignment
                       quickstep_utility_Macros
                       quickstep_utility_PrimeNumber)
+target_link_libraries(quickstep_storage_PackedPayloadAggregationStateHashTable
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableKeyManager
+                      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_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_Alignment
+                      quickstep_utility_HashPair
+                      quickstep_utility_Macros
+                      quickstep_utility_PrimeNumber)
 target_link_libraries(quickstep_storage_PartitionedHashTablePool
                       glog
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_FastHashTableFactory
                       quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableFactory
                       quickstep_utility_Macros
                       quickstep_utility_StringUtil)
 target_link_libraries(quickstep_storage_PreloaderThread
@@ -933,7 +932,6 @@ target_link_libraries(quickstep_storage_StorageBlock
                       glog
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationHandle
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_storage_BasicColumnStoreTupleStorageSubBlock
@@ -942,7 +940,6 @@ target_link_libraries(quickstep_storage_StorageBlock
                       quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
                       quickstep_storage_CompressedPackedRowStoreTupleStorageSubBlock
                       quickstep_storage_CountedReference
-                      quickstep_storage_HashTableBase
                       quickstep_storage_IndexSubBlock
                       quickstep_storage_InsertDestinationInterface
                       quickstep_storage_SMAIndexSubBlock
@@ -1111,6 +1108,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_BasicColumnStoreValueAccessor
                       quickstep_storage_BloomFilterIndexSubBlock
                       quickstep_storage_CSBTreeIndexSubBlock
+                      quickstep_storage_CollisionFreeAggregationStateHashTable
                       quickstep_storage_ColumnStoreUtil
                       quickstep_storage_CompressedBlockBuilder
                       quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
@@ -1123,9 +1121,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
@@ -1139,6 +1134,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_InsertDestination_proto
                       quickstep_storage_LinearOpenAddressingHashTable
                       quickstep_storage_PartitionedHashTablePool
+                      quickstep_storage_PackedPayloadAggregationStateHashTable
                       quickstep_storage_PreloaderThread
                       quickstep_storage_SMAIndexSubBlock
                       quickstep_storage_SeparateChainingHashTable

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/CollisionFreeAggregationStateHashTable.cpp
----------------------------------------------------------------------
diff --git a/storage/CollisionFreeAggregationStateHashTable.cpp b/storage/CollisionFreeAggregationStateHashTable.cpp
new file mode 100644
index 0000000..15d4dfe
--- /dev/null
+++ b/storage/CollisionFreeAggregationStateHashTable.cpp
@@ -0,0 +1,254 @@
+/**
+ * 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 "storage/CollisionFreeAggregationStateHashTable.hpp"
+
+#include <algorithm>
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <cstdlib>
+#include <map>
+#include <memory>
+#include <vector>
+
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+
+namespace quickstep {
+
+CollisionFreeAggregationStateHashTable::CollisionFreeAggregationStateHashTable(
+    const std::vector<const Type *> &key_types,
+    const std::size_t num_entries,
+    const std::vector<AggregationHandle *> &handles,
+    StorageManager *storage_manager)
+    : key_type_(key_types.front()),
+      num_entries_(num_entries),
+      num_handles_(handles.size()),
+      handles_(handles),
+      num_finalize_partitions_(std::min((num_entries_ >> 12u) + 1u, 80uL)),
+      storage_manager_(storage_manager) {
+  CHECK_EQ(1u, key_types.size());
+  DCHECK_GT(num_entries, 0u);
+
+  std::map<std::string, std::size_t> memory_offsets;
+  std::size_t required_memory = 0;
+
+  memory_offsets.emplace("existence_map", required_memory);
+  required_memory +=
+      CacheLineAlignedBytes(ConcurrentBitVector::BytesNeeded(num_entries));
+
+  for (std::size_t i = 0; i < num_handles_; ++i) {
+    const AggregationHandle *handle = handles_[i];
+    const std::vector<const Type *> argument_types = handle->getArgumentTypes();
+
+    std::size_t state_size = 0;
+    switch (handle->getAggregationID()) {
+      case AggregationID::kCount: {
+        state_size = sizeof(std::atomic<std::size_t>);
+        break;
+      }
+      case AggregationID::kSum: {
+        CHECK_EQ(1u, argument_types.size());
+        switch (argument_types.front()->getTypeID()) {
+          case TypeID::kInt:  // Fall through
+          case TypeID::kLong:
+            state_size = sizeof(std::atomic<std::int64_t>);
+            break;
+          case TypeID::kFloat:  // Fall through
+          case TypeID::kDouble:
+            state_size = sizeof(std::atomic<double>);
+            break;
+          default:
+            LOG(FATAL) << "Not implemented";
+        }
+        break;
+      }
+      default:
+        LOG(FATAL) << "Not implemented";
+    }
+
+    memory_offsets.emplace(std::string("state") + std::to_string(i),
+                           required_memory);
+    required_memory += CacheLineAlignedBytes(state_size * num_entries);
+  }
+
+  const std::size_t num_storage_slots =
+      storage_manager_->SlotsNeededForBytes(required_memory);
+
+  const block_id blob_id = storage_manager_->createBlob(num_storage_slots);
+  blob_ = storage_manager_->getBlobMutable(blob_id);
+
+  void *memory_start = blob_->getMemoryMutable();
+  existence_map_.reset(new ConcurrentBitVector(
+      reinterpret_cast<char *>(memory_start) + memory_offsets.at("existence_map"),
+      num_entries,
+      false /* initialize */));
+
+  for (std::size_t i = 0; i < num_handles_; ++i) {
+    vec_tables_.emplace_back(
+        reinterpret_cast<char *>(memory_start) +
+            memory_offsets.at(std::string("state") + std::to_string(i)));
+  }
+
+  memory_size_ = required_memory;
+  num_init_partitions_ = std::min(memory_size_ / (4uL * 1024u * 1024u), 80uL);
+}
+
+CollisionFreeAggregationStateHashTable::~CollisionFreeAggregationStateHashTable() {
+  const block_id blob_id = blob_->getID();
+  blob_.release();
+  storage_manager_->deleteBlockOrBlobFile(blob_id);
+}
+
+void CollisionFreeAggregationStateHashTable::destroyPayload() {
+}
+
+bool CollisionFreeAggregationStateHashTable::upsertValueAccessor(
+    const std::vector<std::vector<attribute_id>> &argument_ids,
+    const std::vector<attribute_id> &key_attr_ids,
+    ValueAccessor *base_accessor,
+    ColumnVectorsValueAccessor *aux_accessor) {
+  DCHECK_EQ(1u, key_attr_ids.size());
+
+  const attribute_id key_attr_id = key_attr_ids.front();
+  const bool is_key_nullable = key_type_->isNullable();
+
+  for (std::size_t i = 0; i < num_handles_; ++i) {
+    DCHECK_LE(argument_ids[i].size(), 1u);
+
+    const attribute_id argument_id =
+        argument_ids[i].empty() ? kInvalidAttributeID : argument_ids[i].front();
+
+    const AggregationHandle *handle = handles_[i];
+    const auto &argument_types = handle->getArgumentTypes();
+
+    const Type *argument_type;
+    bool is_argument_nullable;
+    if (argument_types.empty()) {
+      argument_type = nullptr;
+      is_argument_nullable = false;
+    } else {
+      argument_type = argument_types.front();
+      is_argument_nullable = argument_type->isNullable();
+    }
+
+    InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        base_accessor,
+        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+      if (key_attr_id >= 0) {
+        if (argument_id >= 0) {
+          upsertValueAccessorDispatchHelper<false>(is_key_nullable,
+                                                   is_argument_nullable,
+                                                   key_type_,
+                                                   argument_type,
+                                                   handle->getAggregationID(),
+                                                   key_attr_id,
+                                                   argument_id,
+                                                   vec_tables_[i],
+                                                   accessor,
+                                                   accessor);
+        } else {
+          upsertValueAccessorDispatchHelper<true>(is_key_nullable,
+                                                  is_argument_nullable,
+                                                  key_type_,
+                                                  argument_type,
+                                                  handle->getAggregationID(),
+                                                  key_attr_id,
+                                                  -(argument_id+2),
+                                                  vec_tables_[i],
+                                                  accessor,
+                                                  aux_accessor);
+        }
+      } else {
+        if (argument_id >= 0) {
+          upsertValueAccessorDispatchHelper<true>(is_key_nullable,
+                                                  is_argument_nullable,
+                                                  key_type_,
+                                                  argument_type,
+                                                  handle->getAggregationID(),
+                                                  -(key_attr_id+2),
+                                                  argument_id,
+                                                  vec_tables_[i],
+                                                  aux_accessor,
+                                                  accessor);
+        } else {
+          upsertValueAccessorDispatchHelper<false>(is_key_nullable,
+                                                   is_argument_nullable,
+                                                   key_type_,
+                                                   argument_type,
+                                                   handle->getAggregationID(),
+                                                   -(key_attr_id+2),
+                                                   -(argument_id+2),
+                                                   vec_tables_[i],
+                                                   aux_accessor,
+                                                   aux_accessor);
+        }
+      }
+    });
+  }
+  return true;
+}
+
+void CollisionFreeAggregationStateHashTable::finalizeKey(
+    const std::size_t partition_id,
+    NativeColumnVector *output_cv) const {
+  const std::size_t start_position =
+      calculatePartitionStartPosition(partition_id);
+  const std::size_t end_position =
+      calculatePartitionEndPosition(partition_id);
+
+  switch (key_type_->getTypeID()) {
+    case TypeID::kInt:
+      finalizeKeyInternal<int>(start_position, end_position, output_cv);
+      return;
+    case TypeID::kLong:
+      finalizeKeyInternal<std::int64_t>(start_position, end_position, output_cv);
+      return;
+    default:
+      LOG(FATAL) << "Not supported";
+  }
+}
+
+void CollisionFreeAggregationStateHashTable::finalizeState(
+    const std::size_t partition_id,
+    std::size_t handle_id,
+    NativeColumnVector *output_cv) const {
+  const std::size_t start_position =
+      calculatePartitionStartPosition(partition_id);
+  const std::size_t end_position =
+      calculatePartitionEndPosition(partition_id);
+
+  const AggregationHandle *handle = handles_[handle_id];
+  const auto &argument_types = handle->getArgumentTypes();
+  const Type *argument_type =
+      argument_types.empty() ? nullptr : argument_types.front();
+
+  finalizeStateDispatchHelper(handle->getAggregationID(),
+                              argument_type,
+                              vec_tables_[handle_id],
+                              start_position,
+                              end_position,
+                              output_cv);
+}
+
+}  // namespace quickstep


[10/13] incubator-quickstep git commit: Initial commit.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/CollisionFreeAggregationStateHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/CollisionFreeAggregationStateHashTable.hpp b/storage/CollisionFreeAggregationStateHashTable.hpp
new file mode 100644
index 0000000..f3edfd8
--- /dev/null
+++ b/storage/CollisionFreeAggregationStateHashTable.hpp
@@ -0,0 +1,568 @@
+/**
+ * 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_COLLISION_FREE_AGGREGATION_STATE_HASH_TABLE_HPP_
+#define QUICKSTEP_STORAGE_COLLISION_FREE_AGGREGATION_STATE_HASH_TABLE_HPP_
+
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <cstring>
+#include <memory>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "utility/ConcurrentBitVector.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class ColumnVectorsValueAccessor;
+class StorageMnager;
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+class CollisionFreeAggregationStateHashTable : public AggregationStateHashTableBase {
+ public:
+  CollisionFreeAggregationStateHashTable(
+      const std::vector<const Type *> &key_types,
+      const std::size_t num_entries,
+      const std::vector<AggregationHandle *> &handles,
+      StorageManager *storage_manager);
+
+  ~CollisionFreeAggregationStateHashTable() override;
+
+  void destroyPayload() override;
+
+  inline std::size_t getNumInitializationPartitions() const {
+    return num_init_partitions_;
+  }
+
+  inline std::size_t getNumFinalizationPartitions() const {
+    return num_finalize_partitions_;
+  }
+
+  inline std::size_t getNumTuplesInPartition(
+      const std::size_t partition_id) const {
+    const std::size_t start_position =
+        calculatePartitionStartPosition(partition_id);
+    const std::size_t end_position =
+        calculatePartitionEndPosition(partition_id);
+    return existence_map_->onesCount(start_position, end_position);
+  }
+
+  inline void initialize(const std::size_t partition_id) {
+    const std::size_t memory_segment_size =
+        (memory_size_ + num_init_partitions_ - 1) / num_init_partitions_;
+    const std::size_t memory_start = memory_segment_size * partition_id;
+    std::memset(reinterpret_cast<char *>(blob_->getMemoryMutable()) + memory_start,
+                0,
+                std::min(memory_segment_size, memory_size_ - memory_start));
+  }
+
+  bool upsertValueAccessor(
+      const std::vector<std::vector<attribute_id>> &argument_ids,
+      const std::vector<attribute_id> &key_attr_ids,
+      ValueAccessor *base_accessor,
+      ColumnVectorsValueAccessor *aux_accessor = nullptr) override;
+
+  void finalizeKey(const std::size_t partition_id,
+                   NativeColumnVector *output_cv) const;
+
+  void finalizeState(const std::size_t partition_id,
+                     std::size_t handle_id,
+                     NativeColumnVector *output_cv) const;
+
+ private:
+  inline static std::size_t CacheLineAlignedBytes(const std::size_t actual_bytes) {
+    return (actual_bytes + kCacheLineBytes - 1) / kCacheLineBytes * kCacheLineBytes;
+  }
+
+  inline std::size_t calculatePartitionLength() const {
+    const std::size_t partition_length =
+        (num_entries_ + num_finalize_partitions_ - 1) / num_finalize_partitions_;
+    DCHECK_GE(partition_length, 0u);
+    return partition_length;
+  }
+
+  inline std::size_t calculatePartitionStartPosition(
+      const std::size_t partition_id) const {
+    return calculatePartitionLength() * partition_id;
+  }
+
+  inline std::size_t calculatePartitionEndPosition(
+      const std::size_t partition_id) const {
+    return std::min(calculatePartitionLength() * (partition_id + 1),
+                    num_entries_);
+  }
+
+  template <bool use_two_accessors, typename ...ArgTypes>
+  inline void upsertValueAccessorDispatchHelper(
+      const bool is_key_nullable,
+      const bool is_argument_nullable,
+      ArgTypes &&...args);
+
+  template <bool ...bool_values, typename ...ArgTypes>
+  inline void upsertValueAccessorDispatchHelper(
+      const Type *key_type,
+      ArgTypes &&...args);
+
+  template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+            typename KeyT, typename ...ArgTypes>
+  inline void upsertValueAccessorDispatchHelper(
+      const Type *argument_type,
+      const AggregationID agg_id,
+      ArgTypes &&...args);
+
+  template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+            typename KeyT, typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+  inline void upsertValueAccessorCountHelper(
+      const attribute_id key_attr_id,
+      const attribute_id argument_id,
+      void *vec_table,
+      KeyValueAccessorT *key_accessor,
+      ArgumentValueAccessorT *argument_accessor);
+
+  template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+            typename KeyT, typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+  inline void upsertValueAccessorSumHelper(
+      const Type *argument_type,
+      const attribute_id key_attr_id,
+      const attribute_id argument_id,
+      void *vec_table,
+      KeyValueAccessorT *key_accessor,
+      ArgumentValueAccessorT *argument_accessor);
+
+  template <bool is_key_nullable, typename KeyT, typename KeyValueAccessorT>
+  inline void upsertValueAccessorCountNullary(
+      const attribute_id key_attr_id,
+      std::atomic<std::size_t> *vec_table,
+      KeyValueAccessorT *key_accessor);
+
+  template <bool use_two_accessors, bool is_key_nullable, typename KeyT,
+            typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+  inline void upsertValueAccessorCountUnary(
+      const attribute_id key_attr_id,
+      const attribute_id argument_id,
+      std::atomic<std::size_t> *vec_table,
+      KeyValueAccessorT *key_accessor,
+      ArgumentValueAccessorT *argument_accessor);
+
+  template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+            typename KeyT, typename ArgumentT, typename StateT,
+            typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+  inline void upsertValueAccessorIntegerSum(
+      const attribute_id key_attr_id,
+      const attribute_id argument_id,
+      std::atomic<StateT> *vec_table,
+      KeyValueAccessorT *key_accessor,
+      ArgumentValueAccessorT *argument_accessor);
+
+  template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+            typename KeyT, typename ArgumentT, typename StateT,
+            typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+  inline void upsertValueAccessorGenericSum(
+      const attribute_id key_attr_id,
+      const attribute_id argument_id,
+      std::atomic<StateT> *vec_table,
+      KeyValueAccessorT *key_accessor,
+      ArgumentValueAccessorT *argument_accessor);
+
+  template <typename KeyT>
+  inline void finalizeKeyInternal(const std::size_t start_position,
+                                  const std::size_t end_position,
+                                  NativeColumnVector *output_cv) const {
+    std::size_t loc = start_position - 1;
+    while ((loc = existence_map_->nextOne(loc)) < end_position) {
+      *static_cast<KeyT *>(output_cv->getPtrForDirectWrite()) = loc;
+    }
+  }
+
+  template <typename ...ArgTypes>
+  inline void finalizeStateDispatchHelper(const AggregationID agg_id,
+                                          const Type *argument_type,
+                                          const void *vec_table,
+                                          ArgTypes &&...args) const {
+    switch (agg_id) {
+       case AggregationID::kCount:
+         finalizeStateCount(static_cast<const std::atomic<std::size_t> *>(vec_table),
+                            std::forward<ArgTypes>(args)...);
+         return;
+       case AggregationID::kSum:
+         finalizeStateSumHelper(argument_type,
+                                vec_table,
+                                std::forward<ArgTypes>(args)...);
+         return;
+       default:
+         LOG(FATAL) << "Not supported";
+    }
+  }
+
+  template <typename ...ArgTypes>
+  inline void finalizeStateSumHelper(const Type *argument_type,
+                                     const void *vec_table,
+                                     ArgTypes &&...args) const {
+    DCHECK(argument_type != nullptr);
+
+    switch (argument_type->getTypeID()) {
+      case TypeID::kInt:    // Fall through
+      case TypeID::kLong:
+        finalizeStateSum<std::int64_t>(
+            static_cast<const std::atomic<std::int64_t> *>(vec_table),
+            std::forward<ArgTypes>(args)...);
+        return;
+      case TypeID::kFloat:  // Fall through
+      case TypeID::kDouble:
+        finalizeStateSum<double>(
+            static_cast<const std::atomic<double> *>(vec_table),
+            std::forward<ArgTypes>(args)...);
+        return;
+      default:
+        LOG(FATAL) << "Not supported";
+    }
+  }
+
+  inline void finalizeStateCount(const std::atomic<std::size_t> *vec_table,
+                                 const std::size_t start_position,
+                                 const std::size_t end_position,
+                                 NativeColumnVector *output_cv) const {
+    std::size_t loc = start_position - 1;
+    while ((loc = existence_map_->nextOne(loc)) < end_position) {
+      *static_cast<std::int64_t *>(output_cv->getPtrForDirectWrite()) =
+          vec_table[loc].load(std::memory_order_relaxed);
+    }
+  }
+
+  template <typename ResultT, typename StateT>
+  inline void finalizeStateSum(const std::atomic<StateT> *vec_table,
+                               const std::size_t start_position,
+                               const std::size_t end_position,
+                               NativeColumnVector *output_cv) const {
+    std::size_t loc = start_position - 1;
+    while ((loc = existence_map_->nextOne(loc)) < end_position) {
+      *static_cast<ResultT *>(output_cv->getPtrForDirectWrite()) =
+          vec_table[loc].load(std::memory_order_relaxed);
+    }
+  }
+
+  const Type *key_type_;
+  const std::size_t num_entries_;
+
+  const std::size_t num_handles_;
+  const std::vector<AggregationHandle *> handles_;
+
+  std::unique_ptr<ConcurrentBitVector> existence_map_;
+  std::vector<void *> vec_tables_;
+
+  const std::size_t num_finalize_partitions_;
+
+  StorageManager *storage_manager_;
+  MutableBlobReference blob_;
+
+  std::size_t memory_size_;
+  std::size_t num_init_partitions_;
+
+  DISALLOW_COPY_AND_ASSIGN(CollisionFreeAggregationStateHashTable);
+};
+
+// ----------------------------------------------------------------------------
+// Implementations of template methods follow.
+
+template <bool use_two_accessors, typename ...ArgTypes>
+inline void CollisionFreeAggregationStateHashTable
+    ::upsertValueAccessorDispatchHelper(
+        const bool is_key_nullable,
+        const bool is_argument_nullable,
+        ArgTypes &&...args) {
+  if (is_key_nullable) {
+    if (is_argument_nullable) {
+      upsertValueAccessorDispatchHelper<use_two_accessors, true, true>(
+          std::forward<ArgTypes>(args)...);
+    } else {
+      upsertValueAccessorDispatchHelper<use_two_accessors, true, false>(
+          std::forward<ArgTypes>(args)...);
+    }
+  } else {
+    if (is_argument_nullable) {
+      upsertValueAccessorDispatchHelper<use_two_accessors, false, true>(
+          std::forward<ArgTypes>(args)...);
+    } else {
+      upsertValueAccessorDispatchHelper<use_two_accessors, false, false>(
+          std::forward<ArgTypes>(args)...);
+    }
+  }
+}
+
+template <bool ...bool_values, typename ...ArgTypes>
+inline void CollisionFreeAggregationStateHashTable
+    ::upsertValueAccessorDispatchHelper(
+        const Type *key_type,
+        ArgTypes &&...args) {
+  switch (key_type->getTypeID()) {
+    case TypeID::kInt:
+      upsertValueAccessorDispatchHelper<bool_values..., int>(
+          std::forward<ArgTypes>(args)...);
+      return;
+    case TypeID::kLong:
+      upsertValueAccessorDispatchHelper<bool_values..., std::int64_t>(
+          std::forward<ArgTypes>(args)...);
+      return;
+    default:
+      LOG(FATAL) << "Not supported";
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename ...ArgTypes>
+inline void CollisionFreeAggregationStateHashTable
+    ::upsertValueAccessorDispatchHelper(
+        const Type *argument_type,
+        const AggregationID agg_id,
+        ArgTypes &&...args) {
+  switch (agg_id) {
+     case AggregationID::kCount:
+       upsertValueAccessorCountHelper<
+           use_two_accessors, is_key_nullable, is_argument_nullable, KeyT>(
+               std::forward<ArgTypes>(args)...);
+       return;
+     case AggregationID::kSum:
+       upsertValueAccessorSumHelper<
+           use_two_accessors, is_key_nullable, is_argument_nullable, KeyT>(
+               argument_type, std::forward<ArgTypes>(args)...);
+       return;
+     default:
+       LOG(FATAL) << "Not supported";
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeAggregationStateHashTable
+    ::upsertValueAccessorCountHelper(
+        const attribute_id key_attr_id,
+        const attribute_id argument_id,
+        void *vec_table,
+        KeyValueAccessorT *key_accessor,
+        ArgumentValueAccessorT *argument_accessor) {
+  DCHECK_GE(key_attr_id, 0u);
+
+  if (is_argument_nullable && argument_id != kInvalidAttributeID) {
+    upsertValueAccessorCountUnary<use_two_accessors, is_key_nullable, KeyT>(
+        key_attr_id,
+        argument_id,
+        static_cast<std::atomic<std::size_t> *>(vec_table),
+        key_accessor,
+        argument_accessor);
+    return;
+  } else {
+    upsertValueAccessorCountNullary<is_key_nullable, KeyT>(
+        key_attr_id,
+        static_cast<std::atomic<std::size_t> *>(vec_table),
+        key_accessor);
+    return;
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeAggregationStateHashTable
+    ::upsertValueAccessorSumHelper(
+        const Type *argument_type,
+        const attribute_id key_attr_id,
+        const attribute_id argument_id,
+        void *vec_table,
+        KeyValueAccessorT *key_accessor,
+        ArgumentValueAccessorT *argument_accessor) {
+  DCHECK_GE(key_attr_id, 0u);
+  DCHECK_GE(argument_id, 0u);
+  DCHECK(argument_type != nullptr);
+
+  switch (argument_type->getTypeID()) {
+    case TypeID::kInt:
+      upsertValueAccessorIntegerSum<
+          use_two_accessors, is_key_nullable, is_argument_nullable, KeyT, int>(
+              key_attr_id,
+              argument_id,
+              static_cast<std::atomic<std::int64_t> *>(vec_table),
+              key_accessor,
+              argument_accessor);
+      return;
+    case TypeID::kLong:
+      upsertValueAccessorIntegerSum<
+          use_two_accessors, is_key_nullable, is_argument_nullable, KeyT, std::int64_t>(
+              key_attr_id,
+              argument_id,
+              static_cast<std::atomic<std::int64_t> *>(vec_table),
+              key_accessor,
+              argument_accessor);
+      return;
+    case TypeID::kFloat:
+      upsertValueAccessorGenericSum<
+          use_two_accessors, is_key_nullable, is_argument_nullable, KeyT, float>(
+              key_attr_id,
+              argument_id,
+              static_cast<std::atomic<double> *>(vec_table),
+              key_accessor,
+              argument_accessor);
+      return;
+    case TypeID::kDouble:
+      upsertValueAccessorGenericSum<
+          use_two_accessors, is_key_nullable, is_argument_nullable, KeyT, double>(
+              key_attr_id,
+              argument_id,
+              static_cast<std::atomic<double> *>(vec_table),
+              key_accessor,
+              argument_accessor);
+      return;
+    default:
+      LOG(FATAL) << "Not supported";
+  }
+}
+
+template <bool is_key_nullable, typename KeyT, typename ValueAccessorT>
+inline void CollisionFreeAggregationStateHashTable
+    ::upsertValueAccessorCountNullary(
+        const attribute_id key_attr_id,
+        std::atomic<std::size_t> *vec_table,
+        ValueAccessorT *accessor) {
+  accessor->beginIteration();
+  while (accessor->next()) {
+    const KeyT *key = static_cast<const KeyT *>(
+        accessor->template getUntypedValue<is_key_nullable>(key_attr_id));
+    if (is_key_nullable && key == nullptr) {
+      continue;
+    }
+    const std::size_t loc = *key;
+    vec_table[loc].fetch_add(1u, std::memory_order_relaxed);
+    existence_map_->setBit(loc);
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, typename KeyT,
+          typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeAggregationStateHashTable
+    ::upsertValueAccessorCountUnary(
+        const attribute_id key_attr_id,
+        const attribute_id argument_id,
+        std::atomic<std::size_t> *vec_table,
+        KeyValueAccessorT *key_accessor,
+        ArgumentValueAccessorT *argument_accessor) {
+  key_accessor->beginIteration();
+  while (key_accessor->next()) {
+    if (use_two_accessors) {
+      argument_accessor->next();
+    }
+    const KeyT *key = static_cast<const KeyT *>(
+        key_accessor->template getUntypedValue<is_key_nullable>(key_attr_id));
+    if (is_key_nullable && key == nullptr) {
+      continue;
+    }
+    const std::size_t loc = *key;
+    existence_map_->setBit(loc);
+    if (argument_accessor->getUntypedValue(argument_id) == nullptr) {
+      continue;
+    }
+    vec_table[loc].fetch_add(1u, std::memory_order_relaxed);
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename ArgumentT, typename StateT,
+          typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeAggregationStateHashTable
+    ::upsertValueAccessorIntegerSum(
+        const attribute_id key_attr_id,
+        const attribute_id argument_id,
+        std::atomic<StateT> *vec_table,
+        KeyValueAccessorT *key_accessor,
+        ArgumentValueAccessorT *argument_accessor) {
+  key_accessor->beginIteration();
+  while (key_accessor->next()) {
+    if (use_two_accessors) {
+      argument_accessor->next();
+    }
+    const KeyT *key = static_cast<const KeyT *>(
+        key_accessor->template getUntypedValue<is_key_nullable>(key_attr_id));
+    if (is_key_nullable && key == nullptr) {
+      continue;
+    }
+    const std::size_t loc = *key;
+    existence_map_->setBit(loc);
+    const ArgumentT *argument = static_cast<const ArgumentT *>(
+        argument_accessor->template getUntypedValue<is_argument_nullable>(argument_id));
+    if (is_argument_nullable && argument == nullptr) {
+      continue;
+    }
+    vec_table[loc].fetch_add(*argument, std::memory_order_relaxed);
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename ArgumentT, typename StateT,
+          typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeAggregationStateHashTable
+    ::upsertValueAccessorGenericSum(
+        const attribute_id key_attr_id,
+        const attribute_id argument_id,
+        std::atomic<StateT> *vec_table,
+        KeyValueAccessorT *key_accessor,
+        ArgumentValueAccessorT *argument_accessor) {
+  key_accessor->beginIteration();
+  while (key_accessor->next()) {
+    if (use_two_accessors) {
+      argument_accessor->next();
+    }
+    const KeyT *key = static_cast<const KeyT *>(
+        key_accessor->template getUntypedValue<is_key_nullable>(key_attr_id));
+    if (is_key_nullable && key == nullptr) {
+      continue;
+    }
+    const std::size_t loc = *key;
+    existence_map_->setBit(loc);
+    const ArgumentT *argument = static_cast<const ArgumentT *>(
+        argument_accessor->template getUntypedValue<is_argument_nullable>(argument_id));
+    if (is_argument_nullable && argument == nullptr) {
+      continue;
+    }
+    const ArgumentT arg_val = *argument;
+    std::atomic<StateT> &state = vec_table[loc];
+    StateT state_val = state.load(std::memory_order_relaxed);
+    while(!state.compare_exchange_weak(state_val, state_val + arg_val)) {}
+  }
+}
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_COLLISION_FREE_AGGREGATION_STATE_HASH_TABLE_HPP_


[05/13] incubator-quickstep git commit: Fixed the linking issue for the distributed cli.

Posted by ji...@apache.org.
Fixed the linking issue for the distributed cli.


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

Branch: refs/heads/collision-free-agg
Commit: dff4a145e2c2d3d7b84fb259e48e425310a52a8a
Parents: 259cd5e
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 31 12:19:00 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jan 31 12:19:00 2017 -0800

----------------------------------------------------------------------
 cli/distributed/CMakeLists.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dff4a145/cli/distributed/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/distributed/CMakeLists.txt b/cli/distributed/CMakeLists.txt
index a00ffda..1069abd 100644
--- a/cli/distributed/CMakeLists.txt
+++ b/cli/distributed/CMakeLists.txt
@@ -28,6 +28,7 @@ target_link_libraries(quickstep_cli_distributed_Cli
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_cli_Flags
+                      quickstep_cli_LineReader
                       quickstep_cli_PrintToScreen
                       quickstep_cli_distributed_Role
                       quickstep_parser_ParseStatement


[07/13] incubator-quickstep git commit: Initial commit.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/PackedPayloadAggregationStateHashTable.cpp
----------------------------------------------------------------------
diff --git a/storage/PackedPayloadAggregationStateHashTable.cpp b/storage/PackedPayloadAggregationStateHashTable.cpp
new file mode 100644
index 0000000..34c4177
--- /dev/null
+++ b/storage/PackedPayloadAggregationStateHashTable.cpp
@@ -0,0 +1,434 @@
+/**
+ * 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 "storage/PackedPayloadAggregationStateHashTable.hpp"
+
+namespace quickstep {
+
+PackedPayloadSeparateChainingAggregationStateHashTable
+    ::PackedPayloadSeparateChainingAggregationStateHashTable(
+        const std::vector<const Type *> &key_types,
+        const std::size_t num_entries,
+        const std::vector<AggregationHandle *> &handles,
+        StorageManager *storage_manager)
+        : key_types_(key_types),
+          num_handles_(handles.size()),
+          handles_(handles),
+          total_payload_size_(ComputeTotalPayloadSize(handles)),
+          storage_manager_(storage_manager),
+          kBucketAlignment(alignof(std::atomic<std::size_t>)),
+          kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
+          key_manager_(key_types_, kValueOffset + total_payload_size_),
+          bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
+  std::size_t payload_offset_running_sum = sizeof(SpinMutex);
+  for (const auto *handle : handles) {
+    payload_offsets_.emplace_back(payload_offset_running_sum);
+    payload_offset_running_sum += handle->getPayloadSize();
+  }
+
+  // NOTE(jianqiao): Potential memory leak / double freeing by copying from
+  // init_payload to buckets if payload contains out of line data.
+  init_payload_ =
+      static_cast<std::uint8_t *>(calloc(this->total_payload_size_, 1));
+  DCHECK(init_payload_ != nullptr);
+
+  for (std::size_t i = 0; i < num_handles_; ++i) {
+    handles_[i]->initPayload(init_payload_ + payload_offsets_[i]);
+  }
+
+  // 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_'.
+  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));
+}
+
+PackedPayloadSeparateChainingAggregationStateHashTable
+    ::~PackedPayloadSeparateChainingAggregationStateHashTable() {
+  if (blob_.valid()) {
+    const block_id blob_id = blob_->getID();
+    blob_.release();
+    storage_manager_->deleteBlockOrBlobFile(blob_id);
+  }
+  std::free(init_payload_);
+}
+
+void PackedPayloadSeparateChainingAggregationStateHashTable::clear() {
+  const std::size_t used_buckets =
+      header_->buckets_allocated.load(std::memory_order_relaxed);
+  // Destroy existing values, if necessary.
+  destroyPayload();
+
+  // 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();
+}
+
+void PackedPayloadSeparateChainingAggregationStateHashTable::destroyPayload() {
+  const std::size_t num_buckets =
+      header_->buckets_allocated.load(std::memory_order_relaxed);
+  void *bucket_ptr = static_cast<char *>(buckets_) + kValueOffset;
+  for (std::size_t bucket_num = 0; bucket_num < num_buckets; ++bucket_num) {
+    for (std::size_t handle_id = 0; handle_id < num_handles_; ++handle_id) {
+      void *value_internal_ptr =
+          static_cast<char *>(bucket_ptr) + this->payload_offsets_[handle_id];
+      handles_[handle_id]->destroyPayload(static_cast<std::uint8_t *>(value_internal_ptr));
+    }
+    bucket_ptr = static_cast<char *>(bucket_ptr) + bucket_size_;
+  }
+}
+
+bool PackedPayloadSeparateChainingAggregationStateHashTable::upsertValueAccessor(
+    const std::vector<std::vector<attribute_id>> &argument_ids,
+    const std::vector<attribute_id> &key_attr_ids,
+    ValueAccessor *accessor,
+    ColumnVectorsValueAccessor *aux_accessor) {
+  if (aux_accessor == nullptr) {
+    return upsertValueAccessorCompositeKeyInternal<false>(argument_ids,
+                                                          key_attr_ids,
+                                                          accessor,
+                                                          aux_accessor);
+  } else {
+    return upsertValueAccessorCompositeKeyInternal<true>(argument_ids,
+                                                         key_attr_ids,
+                                                         accessor,
+                                                         aux_accessor);
+  }
+}
+
+void PackedPayloadSeparateChainingAggregationStateHashTable
+    ::resize(const std::size_t extra_buckets,
+             const std::size_t extra_variable_storage,
+             const std::size_t retry_num) {
+  // 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);
+  }
+
+  destroyPayload();
+
+  // 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_;
+  }
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/PackedPayloadAggregationStateHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedPayloadAggregationStateHashTable.hpp b/storage/PackedPayloadAggregationStateHashTable.hpp
new file mode 100644
index 0000000..70152e7
--- /dev/null
+++ b/storage/PackedPayloadAggregationStateHashTable.hpp
@@ -0,0 +1,721 @@
+/**
+ * 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_PACKED_PAYLOAD_AGGREGATION_STATE_HASH_TABLE_HPP_
+#define QUICKSTEP_STORAGE_PACKED_PAYLOAD_AGGREGATION_STATE_HASH_TABLE_HPP_
+
+#include <algorithm>
+#include <atomic>
+#include <cstddef>
+#include <cstdlib>
+#include <limits>
+#include <memory>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationHandle.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 "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 "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/Alignment.hpp"
+#include "utility/HashPair.hpp"
+#include "utility/Macros.hpp"
+#include "utility/PrimeNumber.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+class PackedPayloadSeparateChainingAggregationStateHashTable
+    : public AggregationStateHashTableBase {
+ public:
+  PackedPayloadSeparateChainingAggregationStateHashTable(
+      const std::vector<const Type *> &key_types,
+      const std::size_t num_entries,
+      const std::vector<AggregationHandle *> &handles,
+      StorageManager *storage_manager);
+
+  ~PackedPayloadSeparateChainingAggregationStateHashTable() override;
+
+  void clear();
+
+  void destroyPayload() override;
+
+  bool upsertValueAccessor(
+      const std::vector<std::vector<attribute_id>> &argument_ids,
+      const std::vector<attribute_id> &key_attr_ids,
+      ValueAccessor *accessor,
+      ColumnVectorsValueAccessor *aux_accessor = nullptr) override;
+
+  inline block_id getBlobId() const {
+    return blob_->getID();
+  }
+
+  inline std::size_t numEntries() const {
+    return header_->buckets_allocated.load(std::memory_order_relaxed);
+  }
+
+  inline bool upsertCompositeKey(const std::vector<TypedValue> &key,
+                                 const std::uint8_t *source_state);
+
+  inline const std::uint8_t* getSingleCompositeKey(
+      const std::vector<TypedValue> &key) const;
+
+  inline const std::uint8_t* getSingleCompositeKey(
+      const std::vector<TypedValue> &key,
+      const int index) const;
+
+  template <typename FunctorT>
+  inline std::size_t forEach(FunctorT *functor) const;
+
+  template <typename FunctorT>
+  inline std::size_t forEach(FunctorT *functor, const int index) const;
+
+ private:
+  void resize(const std::size_t extra_buckets,
+              const std::size_t extra_variable_storage,
+              const std::size_t retry_num = 0);
+
+  inline std::size_t calculateVariableLengthCompositeKeyCopySize(
+      const std::vector<TypedValue> &key) const {
+    std::size_t total = 0;
+    for (std::vector<TypedValue>::size_type idx = 0; idx < key.size(); ++idx) {
+      if (!(*key_inline_)[idx]) {
+        total += key[idx].getDataSize();
+      }
+    }
+    return total;
+  }
+
+  inline bool getNextEntryCompositeKey(std::vector<TypedValue> *key,
+                                       const std::uint8_t **value,
+                                       std::size_t *entry_num) const;
+
+  inline std::uint8_t* upsertCompositeKeyInternal(
+      const std::vector<TypedValue> &key,
+      const std::size_t variable_key_size);
+
+  template <bool has_aux_accessor>
+  inline bool upsertValueAccessorCompositeKeyInternal(
+      const std::vector<std::vector<attribute_id>> &argument_ids,
+      const std::vector<attribute_id> &key_attr_ids,
+      ValueAccessor *accessor,
+      ColumnVectorsValueAccessor *aux_accessor);
+
+  // 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;
+
+  // 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;
+  }
+
+  inline static std::size_t ComputeTotalPayloadSize(
+      const std::vector<AggregationHandle *> &handles) {
+    std::size_t total_payload_size = sizeof(SpinMutex);
+    for (const auto *handle : handles) {
+      total_payload_size += handle->getPayloadSize();
+    }
+    return total_payload_size;
+  }
+
+  // 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;
+  }
+
+  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;
+  };
+
+  // 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_;
+
+  const std::size_t num_handles_;
+  const std::vector<AggregationHandle *> handles_;
+
+  std::size_t total_payload_size_;
+  std::vector<std::size_t> payload_offsets_;
+  std::uint8_t *init_payload_;
+
+  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_;
+
+  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;
+  }
+
+  // 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);
+
+  // 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);
+
+  // 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);
+
+  // 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.
+  inline bool isFull(const std::size_t extra_variable_storage) const;
+
+  // Helper object to manage key storage.
+  HashTableKeyManager<false, true> 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(PackedPayloadSeparateChainingAggregationStateHashTable);
+};
+
+/** @} */
+
+// ----------------------------------------------------------------------------
+// Implementations of template class methods follow.
+
+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<PackedPayloadSeparateChainingAggregationStateHashTable *>(
+                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) {
+    destination_hash_table_->upsertCompositeKey(group_by_key, source_state);
+  }
+
+ private:
+  PackedPayloadSeparateChainingAggregationStateHashTable *destination_hash_table_;
+
+  DISALLOW_COPY_AND_ASSIGN(HashTableMergerFast);
+};
+
+inline std::size_t PackedPayloadSeparateChainingAggregationStateHashTable
+    ::hashCompositeKey(const std::vector<TypedValue> &key) const {
+  DEBUG_ASSERT(!key.empty());
+  DEBUG_ASSERT(key.size() == key_types_.size());
+  std::size_t hash = key.front().getHash();
+  for (std::vector<TypedValue>::const_iterator key_it = key.begin() + 1;
+       key_it != key.end();
+       ++key_it) {
+    hash = CombineHashes(hash, key_it->getHash());
+  }
+  return hash;
+}
+
+inline bool PackedPayloadSeparateChainingAggregationStateHashTable
+    ::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;
+  }
+}
+
+
+inline bool PackedPayloadSeparateChainingAggregationStateHashTable
+    ::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) {
+  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 (!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 =
+          header_->buckets_allocated.fetch_add(1, std::memory_order_relaxed);
+      if (allocated_bucket_num >= header_->num_buckets) {
+        // Ran out of buckets.
+        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);
+    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;
+    }
+  }
+}
+
+inline const std::uint8_t* PackedPayloadSeparateChainingAggregationStateHashTable
+    ::getSingleCompositeKey(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)) {
+      // 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;
+}
+
+inline const std::uint8_t* PackedPayloadSeparateChainingAggregationStateHashTable
+    ::getSingleCompositeKey(const std::vector<TypedValue> &key,
+                            const int index) 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.
+      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;
+}
+
+inline bool PackedPayloadSeparateChainingAggregationStateHashTable
+    ::upsertCompositeKey(const std::vector<TypedValue> &key,
+                         const std::uint8_t *source_state) {
+  const std::size_t variable_size =
+      calculateVariableLengthCompositeKeyCopySize(key);
+  for (;;) {
+    {
+      SpinSharedMutexSharedLock<true> resize_lock(resize_shared_mutex_);
+      std::uint8_t *value =
+          upsertCompositeKeyInternal(key, variable_size);
+      if (value != nullptr) {
+        SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
+        for (unsigned int k = 0; k < num_handles_; ++k) {
+          handles_[k]->mergeStates(source_state + payload_offsets_[k],
+                                   value + payload_offsets_[k]);
+        }
+        return true;
+      }
+    }
+    resize(0, variable_size);
+  }
+}
+
+inline std::uint8_t* PackedPayloadSeparateChainingAggregationStateHashTable
+    ::upsertCompositeKeyInternal(const std::vector<TypedValue> &key,
+                                 const std::size_t variable_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)) {
+      // 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);
+
+  std::uint8_t *value = static_cast<unsigned char *>(bucket) + kValueOffset;
+  std::memcpy(value, init_payload_, 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 has_aux_accessor>
+inline bool PackedPayloadSeparateChainingAggregationStateHashTable
+    ::upsertValueAccessorCompositeKeyInternal(
+        const std::vector<std::vector<attribute_id>> &argument_ids,
+        const std::vector<attribute_id> &key_attr_ids,
+        ValueAccessor *accessor,
+        ColumnVectorsValueAccessor *aux_accessor) {
+  std::size_t variable_size;
+  std::vector<TypedValue> key_vector;
+  key_vector.resize(key_attr_ids.size());
+
+  // TODO(jianqiao): determine this bool value
+  const bool check_for_null_keys = true;
+
+  return InvokeOnAnyValueAccessor(
+      accessor,
+      [&](auto *accessor) -> bool {  // NOLINT(build/c++11)
+    bool continuing = true;
+    while (continuing) {
+      {
+        continuing = false;
+        SpinSharedMutexSharedLock<true> lock(resize_shared_mutex_);
+        while (accessor->next()) {
+          if (has_aux_accessor) {
+            aux_accessor->next();
+          }
+          // TODO(jianqiao): templatize to involve aux_accessor
+          if (this->GetCompositeKeyFromValueAccessor(*accessor,
+                                                     key_attr_ids,
+                                                     check_for_null_keys,
+                                                     &key_vector)) {
+            continue;
+          }
+          variable_size = this->calculateVariableLengthCompositeKeyCopySize(key_vector);
+          std::uint8_t *value = this->upsertCompositeKeyInternal(
+              key_vector, variable_size);
+          if (value == nullptr) {
+            continuing = true;
+            break;
+          } else {
+            SpinMutexLock lock(*(reinterpret_cast<SpinMutex *>(value)));
+            for (unsigned int k = 0; k < num_handles_; ++k) {
+              const auto &ids = argument_ids[k];
+              if (ids.empty()) {
+                handles_[k]->updateStateNullary(
+                    value + payload_offsets_[k]);
+              } else {
+                const attribute_id argument_id = ids.front();
+                if (has_aux_accessor && argument_id < 0) {
+                  DCHECK_NE(argument_id, kInvalidAttributeID);
+                  handles_[k]->updateStateUnary(aux_accessor->getTypedValue(-(argument_id+2)),
+                                                value + payload_offsets_[k]);
+                } else {
+                  handles_[k]->updateStateUnary(accessor->getTypedValue(argument_id),
+                                                value + payload_offsets_[k]);
+                }
+              }
+            }
+          }
+        }
+      }
+      if (continuing) {
+        this->resize(0, variable_size);
+        accessor->previous();
+        if (has_aux_accessor) {
+          aux_accessor->previous();
+        }
+      }
+    }
+    return true;
+  });
+}
+
+inline void PackedPayloadSeparateChainingAggregationStateHashTable
+    ::writeScalarKeyToBucket(const TypedValue &key,
+                             const std::size_t hash_code,
+                             void *bucket) {
+  *reinterpret_cast<std::size_t *>(static_cast<char *>(bucket) +
+                                   sizeof(std::atomic<std::size_t>)) =
+      hash_code;
+  key_manager_.writeKeyComponentToBucket(key, 0, bucket, nullptr);
+}
+
+inline void PackedPayloadSeparateChainingAggregationStateHashTable
+    ::writeCompositeKeyToBucket(const std::vector<TypedValue> &key,
+                                const std::size_t hash_code,
+                                void *bucket) {
+  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, nullptr);
+  }
+}
+
+inline bool PackedPayloadSeparateChainingAggregationStateHashTable::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;
+}
+
+template <typename FunctorT>
+inline std::size_t PackedPayloadSeparateChainingAggregationStateHashTable
+    ::forEach(FunctorT *functor) const {
+  std::size_t entries_visited = 0;
+  std::size_t entry_num = 0;
+  std::vector<TypedValue> key;
+  const std::uint8_t *value_ptr;
+  while (getNextEntryCompositeKey(&key, &value_ptr, &entry_num)) {
+    ++entries_visited;
+    (*functor)(key, value_ptr);
+    key.clear();
+  }
+  return entries_visited;
+}
+
+template <typename FunctorT>
+inline std::size_t PackedPayloadSeparateChainingAggregationStateHashTable
+    ::forEach(FunctorT *functor, const int index) const {
+  std::size_t entries_visited = 0;
+  std::size_t entry_num = 0;
+  std::vector<TypedValue> key;
+  const std::uint8_t *value_ptr;
+  while (getNextEntryCompositeKey(&key, &value_ptr, &entry_num)) {
+    ++entries_visited;
+    (*functor)(key, value_ptr + payload_offsets_[index]);
+    key.clear();
+  }
+  return entries_visited;
+}
+
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_PACKED_PAYLOAD_AGGREGATION_STATE_HASH_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/PartitionedHashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/PartitionedHashTablePool.hpp b/storage/PartitionedHashTablePool.hpp
index 95d1810..e9ca022 100644
--- a/storage/PartitionedHashTablePool.hpp
+++ b/storage/PartitionedHashTablePool.hpp
@@ -28,8 +28,7 @@
 
 #include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/HashTableBase.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/HashTableFactory.hpp"
 #include "utility/Macros.hpp"
 #include "utility/StringUtil.hpp"
 
@@ -54,33 +53,6 @@ class PartitionedHashTablePool {
   /**
    * @brief Constructor.
    *
-   * @param estimated_num_entries The maximum number of entries in a hash table.
-   * @param num_partitions The number of partitions (i.e. number of HashTables)
-   * @param hash_table_impl_type The type of hash table implementation.
-   * @param group_by_types A vector of pointer of types which form the group by
-   *        key.
-   * @param agg_handle The aggregation handle.
-   * @param storage_manager A pointer to the storage manager.
-   **/
-  PartitionedHashTablePool(const std::size_t estimated_num_entries,
-                           const std::size_t num_partitions,
-                           const HashTableImplType hash_table_impl_type,
-                           const std::vector<const Type *> &group_by_types,
-                           AggregationHandle *agg_handle,
-                           StorageManager *storage_manager)
-      : estimated_num_entries_(
-            setHashTableSize(estimated_num_entries, num_partitions)),
-        num_partitions_(num_partitions),
-        hash_table_impl_type_(hash_table_impl_type),
-        group_by_types_(group_by_types),
-        agg_handle_(DCHECK_NOTNULL(agg_handle)),
-        storage_manager_(DCHECK_NOTNULL(storage_manager)) {
-    initializeAllHashTables();
-  }
-
-  /**
-   * @brief Constructor.
-   *
    * @note This constructor is relevant for the HashTable specialized for
    *       aggregation.
    *
@@ -89,8 +61,6 @@ class PartitionedHashTablePool {
    * @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 of the payload elements (i.e.
-   *        AggregationStates).
    * @param handles The aggregation handles.
    * @param storage_manager A pointer to the storage manager.
    **/
@@ -98,7 +68,6 @@ class PartitionedHashTablePool {
                            const std::size_t num_partitions,
                            const HashTableImplType hash_table_impl_type,
                            const std::vector<const Type *> &group_by_types,
-                           const std::vector<std::size_t> &payload_sizes,
                            const std::vector<AggregationHandle *> &handles,
                            StorageManager *storage_manager)
       : estimated_num_entries_(
@@ -106,7 +75,6 @@ class PartitionedHashTablePool {
         num_partitions_(num_partitions),
         hash_table_impl_type_(hash_table_impl_type),
         group_by_types_(group_by_types),
-        payload_sizes_(payload_sizes),
         handles_(handles),
         storage_manager_(DCHECK_NOTNULL(storage_manager)) {
     initializeAllHashTables();
@@ -150,25 +118,17 @@ class PartitionedHashTablePool {
  private:
   void initializeAllHashTables() {
     for (std::size_t part_num = 0; part_num < num_partitions_; ++part_num) {
-      AggregationStateHashTableBase *part_hash_table = createNewHashTableFast();
+      AggregationStateHashTableBase *part_hash_table = createNewHashTable();
       hash_tables_.push_back(
           std::unique_ptr<AggregationStateHashTableBase>(part_hash_table));
     }
   }
 
   AggregationStateHashTableBase* createNewHashTable() {
-    return agg_handle_->createGroupByHashTable(hash_table_impl_type_,
-                                               group_by_types_,
-                                               estimated_num_entries_,
-                                               storage_manager_);
-  }
-
-  AggregationStateHashTableBase* createNewHashTableFast() {
-    return AggregationStateFastHashTableFactory::CreateResizable(
+    return AggregationStateHashTableFactory::CreateResizable(
                 hash_table_impl_type_,
                 group_by_types_,
                 estimated_num_entries_,
-                payload_sizes_,
                 handles_,
                 storage_manager_);
   }
@@ -189,10 +149,6 @@ class PartitionedHashTablePool {
   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/b46bc73c/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index de2d25b..ba9ccb8 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -28,7 +28,6 @@
 
 #include "catalog/CatalogRelationSchema.hpp"
 #include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationHandle.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/BasicColumnStoreTupleStorageSubBlock.hpp"
@@ -37,7 +36,6 @@
 #include "storage/CompressedColumnStoreTupleStorageSubBlock.hpp"
 #include "storage/CompressedPackedRowStoreTupleStorageSubBlock.hpp"
 #include "storage/CountedReference.hpp"
-#include "storage/HashTableBase.hpp"
 #include "storage/IndexSubBlock.hpp"
 #include "storage/InsertDestinationInterface.hpp"
 #include "storage/SMAIndexSubBlock.hpp"
@@ -396,166 +394,6 @@ void StorageBlock::selectSimple(const std::vector<attribute_id> &selection,
                                                       accessor.get());
 }
 
-AggregationState* StorageBlock::aggregate(
-    const AggregationHandle &handle,
-    const std::vector<std::unique_ptr<const Scalar>> &arguments,
-    const std::vector<attribute_id> *arguments_as_attributes,
-    const TupleIdSequence *filter) const {
-#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";
-    return aggregateHelperValueAccessor(handle, *arguments_as_attributes, filter);
-  }
-  // TODO(shoban): We may want to optimize for ScalarLiteral here.
-#endif
-
-  // Call aggregateHelperColumnVector() to materialize each argument as a
-  // ColumnVector, then aggregate over those.
-  return aggregateHelperColumnVector(handle, arguments, filter);
-}
-
-void StorageBlock::aggregateGroupBy(
-    const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
-    const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const TupleIdSequence *filter,
-    AggregationStateHashTableBase *hash_table,
-    std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
-  DCHECK_GT(group_by.size(), 0u)
-      << "Called aggregateGroupBy() with zero GROUP BY expressions";
-
-  SubBlocksReference sub_blocks_ref(*tuple_store_,
-                                    indices_,
-                                    indices_consistent_);
-
-  // IDs of 'arguments' as attributes in the ValueAccessor we create below.
-  std::vector<attribute_id> argument_ids;
-
-  // IDs of GROUP BY key element(s) in the ValueAccessor we create below.
-  std::vector<attribute_id> key_ids;
-
-  // An intermediate ValueAccessor that stores the materialized 'arguments' for
-  // this aggregate, as well as the GROUP BY expression values.
-  ColumnVectorsValueAccessor temp_result;
-  {
-    std::unique_ptr<ValueAccessor> accessor(tuple_store_->createValueAccessor(filter));
-    attribute_id attr_id = 0;
-
-    // First, put GROUP BY keys into 'temp_result'.
-    if (reuse_group_by_vectors->empty()) {
-      // Compute GROUP BY values from group_by Scalars, and store them in
-      // reuse_group_by_vectors for reuse by other aggregates on this same
-      // block.
-      reuse_group_by_vectors->reserve(group_by.size());
-      for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
-        reuse_group_by_vectors->emplace_back(
-            group_by_element->getAllValues(accessor.get(), &sub_blocks_ref));
-        temp_result.addColumn(reuse_group_by_vectors->back().get(), false);
-        key_ids.push_back(attr_id++);
-      }
-    } else {
-      // Reuse precomputed GROUP BY values from reuse_group_by_vectors.
-      DCHECK_EQ(group_by.size(), reuse_group_by_vectors->size())
-          << "Wrong number of reuse_group_by_vectors";
-      for (const std::unique_ptr<ColumnVector> &reuse_cv : *reuse_group_by_vectors) {
-        temp_result.addColumn(reuse_cv.get(), false);
-        key_ids.push_back(attr_id++);
-      }
-    }
-
-    // Compute argument vectors and add them to 'temp_result'.
-    for (const std::vector<std::unique_ptr<const Scalar>> &argument : arguments) {
-        for (const std::unique_ptr<const Scalar> &args : argument) {
-          temp_result.addColumn(args->getAllValues(accessor.get(), &sub_blocks_ref));
-          argument_ids.push_back(attr_id++);
-        }
-        if (argument.empty()) {
-          argument_ids.push_back(kInvalidAttributeID);
-        }
-     }
-  }
-
-  hash_table->upsertValueAccessorCompositeKeyFast(argument_ids,
-                                                  &temp_result,
-                                                  key_ids,
-                                                  true);
-}
-
-
-void StorageBlock::aggregateDistinct(
-    const AggregationHandle &handle,
-    const std::vector<std::unique_ptr<const Scalar>> &arguments,
-    const std::vector<attribute_id> *arguments_as_attributes,
-    const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const TupleIdSequence *filter,
-    AggregationStateHashTableBase *distinctify_hash_table,
-    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(tuple_store_->createValueAccessor(filter));
-
-#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);
-}
-
 // TODO(chasseur): Vectorization for updates.
 StorageBlock::UpdateResult StorageBlock::update(
     const unordered_map<attribute_id, unique_ptr<const Scalar>> &assignments,
@@ -1262,61 +1100,6 @@ std::unordered_map<attribute_id, TypedValue>* StorageBlock::generateUpdatedValue
   return update_map;
 }
 
-AggregationState* StorageBlock::aggregateHelperColumnVector(
-    const AggregationHandle &handle,
-    const std::vector<std::unique_ptr<const Scalar>> &arguments,
-    const TupleIdSequence *matches) const {
-  if (arguments.empty()) {
-    // Special case. This is a nullary aggregate (i.e. COUNT(*)).
-    return handle.accumulateNullary(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
-    // predicate).
-    std::unique_ptr<ValueAccessor> accessor;
-    if (matches == nullptr) {
-      accessor.reset(tuple_store_->createValueAccessor());
-    } else {
-      accessor.reset(tuple_store_->createValueAccessor(matches));
-    }
-
-    SubBlocksReference sub_blocks_ref(*tuple_store_,
-                                      indices_,
-                                      indices_consistent_);
-
-    // Materialize each argument's values for this block as a ColumnVector.
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    for (const std::unique_ptr<const Scalar> &argument : arguments) {
-      column_vectors.emplace_back(argument->getAllValues(accessor.get(), &sub_blocks_ref));
-    }
-
-    // Have the AggregationHandle actually do the aggregation.
-    return handle.accumulateColumnVectors(column_vectors);
-  }
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* StorageBlock::aggregateHelperValueAccessor(
-    const AggregationHandle &handle,
-    const std::vector<attribute_id> &argument_ids,
-    const TupleIdSequence *matches) const {
-  // Set up a ValueAccessor to aggregate over (possibly filtered based on the
-  // '*matches' to a filter predicate).
-  std::unique_ptr<ValueAccessor> accessor;
-  if (matches == nullptr) {
-    accessor.reset(tuple_store_->createValueAccessor());
-  } else {
-    accessor.reset(tuple_store_->createValueAccessor(matches));
-  }
-
-  // Have the AggregationHandle actually do the aggregation.
-  return handle.accumulateValueAccessor(
-      accessor.get(),
-      argument_ids);
-}
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
 void StorageBlock::updateHeader() {
   DEBUG_ASSERT(*static_cast<const int*>(block_memory_) == block_header_.ByteSize());
 
@@ -1346,59 +1129,4 @@ const std::size_t StorageBlock::getNumTuples() const {
   return tuple_store_->numTuples();
 }
 
-void StorageBlock::aggregateGroupByPartitioned(
-    const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
-    const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const TupleIdSequence *filter,
-    const std::size_t num_partitions,
-    ColumnVectorsValueAccessor *temp_result,
-    std::vector<attribute_id> *argument_ids,
-    std::vector<attribute_id> *key_ids,
-    std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
-  DCHECK(!group_by.empty())
-      << "Called aggregateGroupByPartitioned() with zero GROUP BY expressions";
-
-  SubBlocksReference sub_blocks_ref(*tuple_store_,
-                                    indices_,
-                                    indices_consistent_);
-
-  std::unique_ptr<ValueAccessor> accessor(
-      tuple_store_->createValueAccessor(filter));
-
-  attribute_id attr_id = 0;
-
-  // First, put GROUP BY keys into 'temp_result'.
-  if (reuse_group_by_vectors->empty()) {
-    // Compute GROUP BY values from group_by Scalars, and store them in
-    // reuse_group_by_vectors for reuse by other aggregates on this same
-    // block.
-    reuse_group_by_vectors->reserve(group_by.size());
-    for (const std::unique_ptr<const Scalar> &group_by_element : group_by) {
-      reuse_group_by_vectors->emplace_back(
-          group_by_element->getAllValues(accessor.get(), &sub_blocks_ref));
-      temp_result->addColumn(reuse_group_by_vectors->back().get(), false);
-      key_ids->push_back(attr_id++);
-    }
-  } else {
-    // Reuse precomputed GROUP BY values from reuse_group_by_vectors.
-    DCHECK_EQ(group_by.size(), reuse_group_by_vectors->size())
-        << "Wrong number of reuse_group_by_vectors";
-    for (const std::unique_ptr<ColumnVector> &reuse_cv : *reuse_group_by_vectors) {
-      temp_result->addColumn(reuse_cv.get(), false);
-      key_ids->push_back(attr_id++);
-    }
-  }
-
-  // Compute argument vectors and add them to 'temp_result'.
-  for (const std::vector<std::unique_ptr<const Scalar>> &argument : arguments) {
-    for (const std::unique_ptr<const Scalar> &args : argument) {
-      temp_result->addColumn(args->getAllValues(accessor.get(), &sub_blocks_ref));
-      argument_ids->push_back(attr_id++);
-    }
-    if (argument.empty()) {
-      argument_ids->push_back(kInvalidAttributeID);
-    }
-  }
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 16ea50f..d09ed3c 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -27,7 +27,6 @@
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "storage/CountedReference.hpp"
-#include "storage/HashTableBase.hpp"
 #include "storage/IndexSubBlock.hpp"
 #include "storage/StorageBlockBase.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -39,8 +38,6 @@
 
 namespace quickstep {
 
-class AggregationHandle;
-class AggregationState;
 class CatalogRelationSchema;
 class ColumnVector;
 class ColumnVectorsValueAccessor;
@@ -431,156 +428,6 @@ class StorageBlock : public StorageBlockBase {
                     InsertDestinationInterface *destination) const;
 
   /**
-   * @brief Perform non GROUP BY aggregation on the tuples in the this storage
-   *        block, returning the aggregated result (for this block) in an
-   *        AggregationState.
-   *
-   * @param handle Aggregation handle that will be used to compute aggregate.
-   * @param arguments The arguments of the aggregate function as expressions.
-   * @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 filter If non-NULL, then only tuple IDs which are set in the
-   *        filter will be checked (all others will be assumed to be false).
-   *
-   * @return Aggregated state for this block in the form of an
-   *         AggregationState. AggregationHandle::mergeStates() can be called
-   *         to merge with states from other blocks, and
-   *         AggregationHandle::finalize() can be used to generate a final
-   *         result.
-   **/
-  AggregationState* aggregate(
-      const AggregationHandle &handle,
-      const std::vector<std::unique_ptr<const Scalar>> &arguments,
-      const std::vector<attribute_id> *arguments_as_attributes,
-      const TupleIdSequence *filter) const;
-
-  /**
-   * @brief Perform GROUP BY aggregation on the tuples in the this storage
-   *        block.
-   *
-   * @param arguments The arguments to the aggregation function as Scalars.
-   * @param group_by The list of GROUP BY attributes/expressions. The tuples in
-   *        this storage block are grouped by these attributes before
-   *        aggregation.
-   * @param filter If non-NULL, then only tuple IDs which are set in the
-   *        filter will be checked (all others will be assumed to be false).
-   * @param hash_table Hash table to store aggregation state mapped based on
-   *        GROUP BY value list (defined by \c group_by).
-   * @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.
-   *
-   * For sample usage of aggregateGroupBy, see this relevant pseudo-C++ code:
-   * \code
-   * std::vector<std::unique_ptr<ColumnVector>> group_by_vectors;
-   * for each aggregate {
-   *   block.aggregateGroupBy(..., &group_by_vectors);
-   * }
-   * \endcode
-   **/
-  /*
-   * TODO(shoban): Currently, we use ColumnVectorsValueAccessor to compute
-   * temporary result for Scalars of aggregation attributes and GROUP BY
-   * attributes.  We will have to support specifying aggregation and GROUP BY
-   * attributes as std::vector<attribute_id> (like in selectSimple()) for fast
-   * path when there are no expressions specified in the query.
-   */
-  void aggregateGroupBy(
-      const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
-      const std::vector<std::unique_ptr<const Scalar>> &group_by,
-      const TupleIdSequence *filter,
-      AggregationStateHashTableBase *hash_table,
-      std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
-
-
-  /**
-   * @brief Perform the GROUP BY aggregation for the case when aggregation is
-   *        partitioned.
-   *
-   * TODO(harshad) - Refactor this class to use only one function
-   *       aggregateGroupBy.
-   * @note The difference between this method and the aggregateGroupBy method
-   *       is that in this method, the tuples are routed to different HashTables
-   *       based on the partition to which they belong to. The partition is
-   *       determined by the GROUP BY attributes. Right now hash based
-   *       partitioning is performed.
-   *
-   * @note This function only creates the ColumnVectorsValueAccessor needed for
-   *       the insertion in the hash table. The actual insertion in respective
-   *       hash tables should be handled by the caller. See
-   *       AggregationOperationState::aggregateHashTable() for one such
-   *       implementation.
-   *
-   * @param arguments The arguments to the aggregation function as Scalars.
-   * @param group_by The list of GROUP BY attributes/expressions. The tuples in
-   *        this storage block are grouped by these attributes before
-   *        aggregation.
-   * @param filter If non-NULL, then only tuple IDs which are set in the
-   *        filter will be checked (all others will be assumed to be false).
-   * @param num_partitions The number of partitions used for the aggregation.
-   * @param temp_result The ColumnVectorsValueAccessor used for collecting
-   *        the attribute values from this StorageBlock.
-   * @param arguments_ids The attribute IDs used for the aggregation, which
-   *        come from the arguments vector. If arguments is empty, this vector
-   *        is filled with invalid attribute IDs.
-   * @param key_ids The attribute IDs of the group by attributes.
-   * @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 aggregateGroupByPartitioned(
-      const std::vector<std::vector<std::unique_ptr<const Scalar>>> &arguments,
-      const std::vector<std::unique_ptr<const Scalar>> &group_by,
-      const TupleIdSequence *filter,
-      const std::size_t num_partitions,
-      ColumnVectorsValueAccessor *temp_result,
-      std::vector<attribute_id> *argument_ids,
-      std::vector<attribute_id> *key_ids,
-      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 filter If non-NULL, then only tuple IDs which are set in the
-   *        filter will be checked (all others will be assumed to be false).
-   * @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_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 TupleIdSequence *filter,
-                         AggregationStateHashTableBase *distinctify_hash_table,
-                         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
@@ -702,18 +549,6 @@ class StorageBlock : public StorageBlockBase {
       const tuple_id tuple,
       const std::unordered_map<attribute_id, std::unique_ptr<const Scalar>> &assignments) const;
 
-  AggregationState* 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(
-      const AggregationHandle &handle,
-      const std::vector<attribute_id> &argument_ids,
-      const TupleIdSequence *matches) const;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
   // Sort the tuples in storage block based on `sort_attribute'. If
   // `use_input_sequence' is set, we assume a pre-existing order of tuple-id
   // sequence specified by `sorted_sequence' and use stable sort to maintain

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 8571149..3a4eff5 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -172,6 +172,7 @@ add_library(quickstep_utility_CalculateInstalledMemory CalculateInstalledMemory.
 add_library(quickstep_utility_Cast ../empty_src.cpp Cast.hpp)
 add_library(quickstep_utility_CheckSnprintf ../empty_src.cpp CheckSnprintf.hpp)
 add_library(quickstep_utility_CompositeHash ../empty_src.cpp CompositeHash.hpp)
+add_library(quickstep_utility_ConcurrentBitVector ../empty_src.cpp ConcurrentBitVector.hpp)
 add_library(quickstep_utility_DAG ../empty_src.cpp DAG.hpp)
 add_library(quickstep_utility_DisjointTreeForest ../empty_src.cpp DisjointTreeForest.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)
@@ -238,6 +239,9 @@ target_link_libraries(quickstep_utility_CompositeHash
                       quickstep_types_TypedValue
                       quickstep_utility_HashPair
                       glog)
+target_link_libraries(quickstep_utility_ConcurrentBitVector
+                      quickstep_utility_BitManipulation
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_DAG
                       glog
                       quickstep_utility_Macros)
@@ -337,6 +341,7 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_Cast
                       quickstep_utility_CheckSnprintf
                       quickstep_utility_CompositeHash
+                      quickstep_utility_ConcurrentBitVector
                       quickstep_utility_DAG
                       quickstep_utility_DisjointTreeForest
                       quickstep_utility_EqualsAnyConstant

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/utility/ConcurrentBitVector.hpp
----------------------------------------------------------------------
diff --git a/utility/ConcurrentBitVector.hpp b/utility/ConcurrentBitVector.hpp
new file mode 100644
index 0000000..e36f03c
--- /dev/null
+++ b/utility/ConcurrentBitVector.hpp
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_CONCURRENT_BIT_VECTOR_HPP_
+#define QUICKSTEP_UTILITY_CONCURRENT_BIT_VECTOR_HPP_
+
+#include <cstddef>
+#include <cstdint>
+#include <cstdlib>
+#include <cstring>
+#include <limits>
+
+#include "utility/BitManipulation.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+class ConcurrentBitVector {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param memory_location The location of memory to use for the ConcurrentBitVector.
+   * @param num_bits The length of the ConcurrentBitVector in bits.
+   * @param initialize If true, initialize all the bytes of the memory to 0.
+   */
+  ConcurrentBitVector(void *memory_location,
+                      const std::size_t num_bits,
+                      const bool initialize)
+      : owned_(false),
+        num_bits_(num_bits),
+        data_array_(static_cast<DataType *>(memory_location)),
+        data_array_size_((num_bits >> kHigherOrderShift) + (num_bits & kLowerOrderMask ? 1 : 0)) {
+    DCHECK_GT(num_bits, 0);
+    DCHECK(data_array_ != nullptr);
+
+    if (initialize) {
+      clear();
+    }
+  }
+
+  explicit ConcurrentBitVector(const std::size_t num_bits)
+      : owned_(true),
+        num_bits_(num_bits),
+        data_array_(static_cast<DataType *>(std::malloc(BytesNeeded(num_bits)))),
+        data_array_size_((num_bits >> kHigherOrderShift) + (num_bits & kLowerOrderMask ? 1 : 0)) {
+    DCHECK_GT(num_bits, 0);
+    clear();
+  }
+
+  ~ConcurrentBitVector() {
+    if (owned_ && (num_bits_ != 0)) {
+      std::free(data_array_);
+    }
+  }
+
+  inline static std::size_t BytesNeeded(const std::size_t num_bits) {
+    if (num_bits & kLowerOrderMask) {
+      return ((num_bits >> kHigherOrderShift) + 1) * kDataSize;
+    } else {
+      return (num_bits >> kHigherOrderShift) * kDataSize;
+    }
+  }
+
+  inline std::size_t size() const {
+    return num_bits_;
+  }
+
+  inline void clear() {
+    std::memset(data_array_, 0, BytesNeeded(num_bits_));
+  }
+
+  inline bool getBit(const std::size_t bit_num) const {
+    const std::size_t data_value =
+        data_array_[bit_num >> kHigherOrderShift].load(std::memory_order_relaxed);
+    return (data_value << (bit_num & kLowerOrderMask)) & kTopBit;
+  }
+
+  inline void setBit(const std::size_t bit_num) const {
+    data_array_[bit_num >> kHigherOrderShift].fetch_or(
+        kTopBit >> (bit_num & kLowerOrderMask), std::memory_order_relaxed);
+  }
+
+  inline std::size_t firstOne(std::size_t position = 0) const {
+    DCHECK_LT(position, num_bits_);
+
+    const std::size_t position_index = position >> kHigherOrderShift;
+    const std::size_t data_value =
+        data_array_[position_index].load(std::memory_order_relaxed)
+            & (std::numeric_limits<std::size_t>::max() >> (position & kLowerOrderMask));
+    if (data_value) {
+      return (position & ~kLowerOrderMask) | leading_zero_count<std::size_t>(data_value);
+    }
+
+    for (std::size_t array_idx = position_index + 1;
+         array_idx < data_array_size_;
+         ++array_idx) {
+      const std::size_t data_value =
+          data_array_[array_idx].load(std::memory_order_relaxed);
+      if (data_value) {
+        return (array_idx << kHigherOrderShift) | leading_zero_count<std::size_t>(data_value);
+      }
+    }
+
+    return num_bits_;
+  }
+
+  inline std::size_t nextOne(const std::size_t position) const {
+    const std::size_t search_pos = position + 1;
+    return search_pos >= num_bits_ ? num_bits_ : firstOne(search_pos);
+  }
+
+  inline std::size_t onesCount() const {
+    std::size_t count = 0;
+    for (std::size_t array_idx = 0;
+         array_idx < data_array_size_;
+         ++array_idx) {
+      count += population_count<std::size_t>(
+          data_array_[array_idx].load(std::memory_order_relaxed));
+    }
+    return count;
+  }
+
+  inline std::size_t onesCount(const std::size_t start_position,
+                               const std::size_t end_position) const {
+    DCHECK_LE(start_position, end_position);
+    DCHECK_LT(start_position, num_bits_);
+    DCHECK_LE(end_position, num_bits_);
+
+    const std::size_t start_index = start_position >> kHigherOrderShift;
+    const std::size_t end_index = end_position >> kHigherOrderShift;
+    if (start_index == end_index) {
+      const std::size_t data_value =
+          data_array_[start_index].load(std::memory_order_relaxed)
+              & (std::numeric_limits<std::size_t>::max() >> (start_position & kLowerOrderMask))
+              &  ~(std::numeric_limits<std::size_t>::max() >> (end_position & kLowerOrderMask));
+      return population_count<std::size_t>(data_value);
+    } else {
+      const std::size_t first_data =
+          data_array_[start_index].load(std::memory_order_relaxed)
+              & (std::numeric_limits<std::size_t>::max() >> (start_position & kLowerOrderMask));
+      std::size_t count = population_count<std::size_t>(first_data);
+
+      for (std::size_t array_idx = start_index + 1;
+           array_idx < end_index;
+           ++array_idx) {
+        count += population_count<std::size_t>(
+            data_array_[array_idx].load(std::memory_order_relaxed));
+      }
+
+      const std::size_t last_offset = end_position & kLowerOrderMask;
+      if (last_offset != 0) {
+        const std::size_t last_data =
+            data_array_[end_index].load(std::memory_order_relaxed)
+                &  ~(std::numeric_limits<std::size_t>::max() >> last_offset);
+        count += population_count<std::size_t>(last_data);
+      }
+
+      return count;
+    }
+  }
+
+ private:
+  typedef std::atomic<std::size_t> DataType;
+  static constexpr std::size_t kDataSize = sizeof(DataType);
+
+  // This works as long as the bit-width of size_t is power of 2:
+  static constexpr std::size_t kLowerOrderMask = (sizeof(std::size_t) << 3) - 1;
+  // This works for 32-bit or 64-bit size_t:
+  static constexpr std::size_t kHigherOrderShift = sizeof(std::size_t) == 4 ? 5 : 6;
+
+  static constexpr std::size_t kOne = static_cast<std::size_t>(1);
+  static constexpr std::size_t kTopBit = kOne << kLowerOrderMask;
+
+  const bool owned_;
+  const std::size_t num_bits_;
+  DataType *data_array_;
+  const std::size_t data_array_size_;
+
+  DISALLOW_COPY_AND_ASSIGN(ConcurrentBitVector);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_CONCURRENT_BIT_VECTOR_HPP_


[13/13] incubator-quickstep git commit: Initial commit.

Posted by ji...@apache.org.
Initial commit.


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

Branch: refs/heads/collision-free-agg
Commit: b46bc73c81adb4c36a076e79c5782d38c15a9a7a
Parents: 5ffdaaf
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Jan 30 14:46:39 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Tue Jan 31 17:00:16 2017 -0600

----------------------------------------------------------------------
 .../aggregation/AggregateFunctionCount.cpp      |    6 +-
 .../aggregation/AggregationConcreteHandle.cpp   |   44 -
 .../aggregation/AggregationConcreteHandle.hpp   |  131 +-
 expressions/aggregation/AggregationHandle.hpp   |  202 +-
 .../aggregation/AggregationHandleAvg.cpp        |   84 +-
 .../aggregation/AggregationHandleAvg.hpp        |  116 +-
 .../aggregation/AggregationHandleCount.cpp      |  127 +-
 .../aggregation/AggregationHandleCount.hpp      |  136 +-
 .../aggregation/AggregationHandleDistinct.cpp   |   34 +-
 .../aggregation/AggregationHandleDistinct.hpp   |   56 +-
 .../aggregation/AggregationHandleMax.cpp        |   84 +-
 .../aggregation/AggregationHandleMax.hpp        |  101 +-
 .../aggregation/AggregationHandleMin.cpp        |   84 +-
 .../aggregation/AggregationHandleMin.hpp        |  111 +-
 .../aggregation/AggregationHandleSum.cpp        |   81 +-
 .../aggregation/AggregationHandleSum.hpp        |  118 +-
 expressions/aggregation/AggregationID.hpp       |    4 +-
 expressions/aggregation/CMakeLists.txt          |   34 +-
 query_execution/QueryContext.hpp                |   14 -
 query_optimizer/CMakeLists.txt                  |    3 +
 query_optimizer/ExecutionGenerator.cpp          |  137 +-
 query_optimizer/ExecutionGenerator.hpp          |    8 +-
 query_optimizer/cost_model/CMakeLists.txt       |    3 +
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  126 +-
 .../cost_model/StarSchemaSimpleCostModel.hpp    |   79 +
 query_optimizer/expressions/ExpressionUtil.hpp  |    8 +-
 relational_operators/CMakeLists.txt             |   15 +
 .../DestroyAggregationStateOperator.cpp         |    7 -
 .../FinalizeAggregationOperator.cpp             |   16 +-
 .../FinalizeAggregationOperator.hpp             |   14 +-
 .../InitializeAggregationStateOperator.cpp      |   68 +
 .../InitializeAggregationStateOperator.hpp      |  103 +
 storage/AggregationOperationState.cpp           |  661 ++---
 storage/AggregationOperationState.hpp           |  142 +-
 storage/CMakeLists.txt                          |  114 +-
 .../CollisionFreeAggregationStateHashTable.cpp  |  254 ++
 .../CollisionFreeAggregationStateHashTable.hpp  |  568 +++++
 storage/FastHashTable.hpp                       | 2403 ------------------
 storage/FastHashTableFactory.hpp                |  224 --
 storage/FastSeparateChainingHashTable.hpp       | 1551 -----------
 storage/HashTable.proto                         |    7 +-
 storage/HashTableBase.hpp                       |   44 +-
 storage/HashTableFactory.hpp                    |   44 +-
 storage/HashTablePool.hpp                       |   74 +-
 .../PackedPayloadAggregationStateHashTable.cpp  |  434 ++++
 .../PackedPayloadAggregationStateHashTable.hpp  |  721 ++++++
 storage/PartitionedHashTablePool.hpp            |   50 +-
 storage/StorageBlock.cpp                        |  272 --
 storage/StorageBlock.hpp                        |  165 --
 utility/CMakeLists.txt                          |    5 +
 utility/ConcurrentBitVector.hpp                 |  209 ++
 51 files changed, 3648 insertions(+), 6448 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregateFunctionCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionCount.cpp b/expressions/aggregation/AggregateFunctionCount.cpp
index 466ff2f..9795b4a 100644
--- a/expressions/aggregation/AggregateFunctionCount.cpp
+++ b/expressions/aggregation/AggregateFunctionCount.cpp
@@ -53,16 +53,16 @@ AggregationHandle* AggregateFunctionCount::createHandle(
 
   if (argument_types.empty()) {
     // COUNT(*)
-    return new AggregationHandleCount<true, false>();
+    return new AggregationHandleCount<true, false>(nullptr);
   } else if (argument_types.front()->isNullable()) {
     // COUNT(some_nullable_argument)
-    return new AggregationHandleCount<false, true>();
+    return new AggregationHandleCount<false, true>(argument_types.front());
   } else {
     // COUNT(non_nullable_argument)
     //
     // TODO(chasseur): Modify query optimizer to optimize-away COUNT with a
     // non-nullable argument and convert it to COUNT(*).
-    return new AggregationHandleCount<false, false>();
+    return new AggregationHandleCount<false, false>(argument_types.front());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationConcreteHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.cpp b/expressions/aggregation/AggregationConcreteHandle.cpp
index e3fb520..3151a91 100644
--- a/expressions/aggregation/AggregationConcreteHandle.cpp
+++ b/expressions/aggregation/AggregationConcreteHandle.cpp
@@ -19,50 +19,6 @@
 
 #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/b46bc73c/expressions/aggregation/AggregationConcreteHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.hpp b/expressions/aggregation/AggregationConcreteHandle.hpp
index 398a032..93e9bd0 100644
--- a/expressions/aggregation/AggregationConcreteHandle.hpp
+++ b/expressions/aggregation/AggregationConcreteHandle.hpp
@@ -26,8 +26,7 @@
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTable.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
 #include "storage/HashTableBase.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/TypedValue.hpp"
@@ -51,7 +50,7 @@ class ValueAccessor;
  *        merging two group by hash tables.
  **/
 template <typename HandleT>
-class HashTableStateUpserterFast {
+class HashTableStateUpserter {
  public:
   /**
    * @brief Constructor.
@@ -61,7 +60,7 @@ class HashTableStateUpserterFast {
    *        table. The corresponding state (for the same key) in the destination
    *        hash table will be upserted.
    **/
-  HashTableStateUpserterFast(const HandleT &handle,
+  HashTableStateUpserter(const HandleT &handle,
                              const std::uint8_t *source_state)
       : handle_(handle), source_state_(source_state) {}
 
@@ -72,14 +71,14 @@ class HashTableStateUpserterFast {
    *        table that is being upserted.
    **/
   void operator()(std::uint8_t *destination_state) {
-    handle_.mergeStatesFast(source_state_, destination_state);
+    handle_.mergeStates(source_state_, destination_state);
   }
 
  private:
   const HandleT &handle_;
   const std::uint8_t *source_state_;
 
-  DISALLOW_COPY_AND_ASSIGN(HashTableStateUpserterFast);
+  DISALLOW_COPY_AND_ASSIGN(HashTableStateUpserter);
 };
 
 /**
@@ -102,50 +101,19 @@ class AggregationConcreteHandle : public AggregationHandle {
                << "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;
+  AggregationConcreteHandle(const AggregationID agg_id)
+      : AggregationHandle(agg_id) {}
 
   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(
+  ColumnVector* finalizeHashTableHelper(
       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(
+  inline TypedValue finalizeGroupInHashTable(
       const AggregationStateHashTableBase &hash_table,
       const std::vector<TypedValue> &group_key,
       int index) const {
@@ -153,15 +121,10 @@ class AggregationConcreteHandle : public AggregationHandle {
         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(
+    return static_cast<const HandleT *>(this)->finalizeHashTableEntry(
         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);
 };
@@ -195,7 +158,7 @@ class HashTableAggregateFinalizer {
                          const unsigned char *byte_ptr) {
     group_by_keys_->emplace_back(group_by_key);
     output_column_vector_->appendTypedValue(
-        handle_.finalizeHashTableEntryFast(byte_ptr));
+        handle_.finalizeHashTableEntry(byte_ptr));
   }
 
  private:
@@ -209,70 +172,8 @@ class HashTableAggregateFinalizer {
 // ----------------------------------------------------------------------------
 // Implementations of templated methods follow:
 
-template <typename HandleT, typename StateT>
-StateT* AggregationConcreteHandle::
-    aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
-        const AggregationStateHashTableBase &distinctify_hash_table) const {
-  const HandleT &handle = static_cast<const HandleT &>(*this);
-  StateT *state = static_cast<StateT *>(createInitialState());
-
-  // 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(
+ColumnVector* AggregationConcreteHandle::finalizeHashTableHelper(
     const Type &result_type,
     const AggregationStateHashTableBase &hash_table,
     std::vector<std::vector<TypedValue>> *group_by_keys,
@@ -287,14 +188,14 @@ ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
           new NativeColumnVector(result_type, hash_table_concrete.numEntries());
       HashTableAggregateFinalizer<HandleT, NativeColumnVector> finalizer(
           handle, group_by_keys, result);
-      hash_table_concrete.forEachCompositeKeyFast(&finalizer, index);
+      hash_table_concrete.forEach(&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);
+      hash_table_concrete.forEach(&finalizer, index);
       return result;
     }
   } else {
@@ -303,7 +204,7 @@ ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
           new NativeColumnVector(result_type, group_by_keys->size());
       for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
         result->appendTypedValue(
-            finalizeGroupInHashTableFast<HandleT, HashTableT>(
+            finalizeGroupInHashTable<HandleT, HashTableT>(
                 hash_table, group_by_key, index));
       }
       return result;
@@ -312,7 +213,7 @@ ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
           result_type, hash_table_concrete.numEntries());
       for (const std::vector<TypedValue> &group_by_key : *group_by_keys) {
         result->appendTypedValue(
-            finalizeGroupInHashTableFast<HandleT, HashTableT>(
+            finalizeGroupInHashTable<HandleT, HashTableT>(
                 hash_table, group_by_key, index));
       }
       return result;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index 4b51179..8e2aea6 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -25,6 +25,7 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
 #include "storage/HashTableBase.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
@@ -32,6 +33,7 @@
 namespace quickstep {
 
 class ColumnVector;
+class ColumnVectorsValueAccessor;
 class StorageManager;
 class Type;
 class ValueAccessor;
@@ -108,6 +110,14 @@ class AggregationHandle {
    **/
   virtual ~AggregationHandle() {}
 
+  AggregationID getAggregationID() const {
+    return agg_id_;
+  }
+
+  virtual std::vector<const Type *> getArgumentTypes() const = 0;
+
+  virtual const Type* getResultType() const = 0;
+
   /**
    * @brief Create an initial "blank" state for this aggregation.
    *
@@ -116,29 +126,6 @@ class AggregationHandle {
   virtual AggregationState* createInitialState() const = 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(*)).
    *
@@ -153,63 +140,16 @@ class AggregationHandle {
       const std::size_t num_tuples) const = 0;
 
   /**
-   * @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.
+   * @brief TODO
    *
-   * @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;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  /**
-   * @brief Accumulate (iterate over) all values in columns accessible through
-   *        a ValueAccessor and return a new AggregationState which can be
-   *        merged with other states or finalized.
-   *
-   * @param accessor A ValueAccessor that the columns to be aggregated can be
-   *        accessed through.
-   * @param accessor_ids The attribute_ids that correspond to the columns in
-   *        accessor to aggeregate. These correspond to the aggregate
-   *        function's arguments, in order.
-   * @return A new AggregationState which contains the accumulated results from
-   *         applying the aggregate to the specified columns in accessor.
-   *         Caller is responsible for deleting the returned AggregationState.
-   **/
-  virtual AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_ids) const = 0;
-#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(
+  virtual AggregationState* accumulate(
       ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const = 0;
+      ColumnVectorsValueAccessor *aux_accessor,
+      const std::vector<attribute_id> &argument_ids) const = 0;
 
   /**
    * @brief Merge two AggregationStates, updating one in-place. This computes a
@@ -269,99 +209,12 @@ class AggregationHandle {
       int index) const = 0;
 
   /**
-   * @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;
-
-  /**
-   * @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;
-
-  /**
-   * @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;
-
-  /**
-   * @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;
-
-  /**
    * @brief Get the number of bytes needed to store the aggregation handle's
    *        state.
    **/
-  virtual std::size_t getPayloadSize() const { return 1; }
+  virtual std::size_t getPayloadSize() const {
+    return 1u;
+  }
 
   /**
    * @brief Update the aggregation state for nullary aggregation function e.g.
@@ -394,8 +247,8 @@ class AggregationHandle {
    * @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 {}
+  virtual void mergeStates(const std::uint8_t *src,
+                           std::uint8_t *dst) const {}
 
   /**
    * @brief Initialize the payload (in the aggregation hash table) for the given
@@ -413,20 +266,11 @@ class AggregationHandle {
    **/
   virtual void destroyPayload(std::uint8_t *byte_ptr) const {}
 
-  /**
-   * @brief Inform the aggregation handle to block (prohibit) updates on the
-   *        aggregation state.
-   **/
-  virtual void blockUpdate() {}
-
-  /**
-   * @brief Inform the aggregation handle to allow updates on the
-   *        aggregation state.
-   **/
-  virtual void allowUpdate() {}
-
  protected:
-  AggregationHandle() {}
+  AggregationHandle(const AggregationID agg_id)
+      : agg_id_(agg_id) {}
+
+  const AggregationID agg_id_;
 
  private:
   DISALLOW_COPY_AND_ASSIGN(AggregationHandle);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index 2481092..d81c179 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.cpp
@@ -24,8 +24,8 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/PackedPayloadAggregationStateHashTable.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
@@ -42,7 +42,8 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleAvg::AggregationHandleAvg(const Type &type)
-    : argument_type_(type), block_update_(false) {
+    : AggregationConcreteHandle(AggregationID::kAvg),
+      argument_type_(type) {
   // We sum Int as Long and Float as Double so that we have more headroom when
   // adding many values.
   TypeID type_precision_id;
@@ -87,52 +88,28 @@ AggregationHandleAvg::AggregationHandleAvg(const Type &type)
             ->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();
+AggregationState* AggregationHandleAvg::accumulate(
+    ValueAccessor *accessor,
+    ColumnVectorsValueAccessor *aux_accessor,
+    const std::vector<attribute_id> &argument_ids) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for AVG: " << argument_ids.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;
-}
+  const attribute_id argument_id = argument_ids.front();
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
-#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();
+  ValueAccessor *target_accessor =
+      argument_id >= 0 ? accessor : aux_accessor;
+  const attribute_id target_argument_id =
+      argument_id >= 0 ? argument_id : -(argument_id+2);
 
   AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
   std::size_t count = 0;
   state->sum_ = fast_add_operator_->accumulateValueAccessor(
-      state->sum_, accessor, accessor_ids.front(), &count);
+      state->sum_, target_accessor, target_argument_id, &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 {
@@ -147,8 +124,8 @@ void AggregationHandleAvg::mergeStates(const AggregationState &source,
       avg_destination->sum_, avg_source.sum_);
 }
 
-void AggregationHandleAvg::mergeStatesFast(const std::uint8_t *source,
-                                           std::uint8_t *destination) const {
+void AggregationHandleAvg::mergeStates(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 *>(
@@ -179,27 +156,10 @@ 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<
+  return finalizeHashTableHelper<
       AggregationHandleAvg,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+      PackedPayloadSeparateChainingAggregationStateHashTable>(
+          *result_type_, hash_table, group_by_keys, index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index 47132c6..aa5f427 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -28,7 +28,6 @@
 #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"
@@ -106,16 +105,18 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
  public:
   ~AggregationHandleAvg() override {}
 
+  std::vector<const Type *> getArgumentTypes() const override {
+    return {&argument_type_};
+  }
+
+  const Type* getResultType() const override {
+    return result_type_;
+  }
+
   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.
    **/
@@ -129,28 +130,19 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     ++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);
-  }
+  AggregationState* accumulate(
+      ValueAccessor *accessor,
+      ColumnVectorsValueAccessor *aux_accessor,
+      const std::vector<attribute_id> &argument_ids) const override;
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  TypedValue finalize(const AggregationState &state) const override;
 
-  void allowUpdate() override { block_update_ = false; }
+  std::size_t getPayloadSize() const override {
+    return blank_state_.getPayloadSize();
+  }
 
   void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *sum_ptr =
@@ -169,43 +161,22 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     }
   }
 
-  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;
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    DCHECK(argument.isPlausibleInstanceOf(argument_type_.getSignature()));
+    if (argument.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, argument);
+    ++(*count_ptr);
+  }
 
-  TypedValue finalize(const AggregationState &state) const override;
+  void mergeStates(const std::uint8_t *source,
+                   std::uint8_t *destination) 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 =
@@ -221,29 +192,6 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
       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;
 
@@ -261,8 +209,6 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
   std::unique_ptr<UncheckedBinaryOperator> merge_add_operator_;
   std::unique_ptr<UncheckedBinaryOperator> divide_operator_;
 
-  bool block_update_;
-
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleAvg);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.cpp b/expressions/aggregation/AggregationHandleCount.cpp
index 034c942..a5c9fd8 100644
--- a/expressions/aggregation/AggregationHandleCount.cpp
+++ b/expressions/aggregation/AggregationHandleCount.cpp
@@ -25,14 +25,9 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+#include "storage/PackedPayloadAggregationStateHashTable.hpp"
 #include "storage/ValueAccessor.hpp"
 #include "storage/ValueAccessorUtil.hpp"
-#endif
-
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
@@ -48,73 +43,32 @@ 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);
-}
-
-template <bool count_star, bool nullable_type>
-AggregationState*
-AggregationHandleCount<count_star, nullable_type>::accumulateColumnVectors(
-    const std::vector<std::unique_ptr<ColumnVector>> &column_vectors) const {
+AggregationState* AggregationHandleCount<count_star, nullable_type>::accumulate(
+    ValueAccessor *accessor,
+    ColumnVectorsValueAccessor *aux_accessor,
+    const std::vector<attribute_id> &argument_ids) const {
   DCHECK(!count_star)
       << "Called non-nullary accumulation method on an AggregationHandleCount "
       << "set up for nullary COUNT(*)";
 
-  DCHECK_EQ(1u, column_vectors.size())
-      << "Got wrong number of ColumnVectors for COUNT: "
-      << column_vectors.size();
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for COUNT: " << argument_ids.size();
 
-  std::size_t count = 0;
-  InvokeOnColumnVector(
-      *column_vectors.front(),
-      [&](const auto &column_vector) -> void {  // NOLINT(build/c++11)
-        if (nullable_type) {
-          // TODO(shoban): Iterating over the ColumnVector is a rather slow way
-          // to do this. We should look at extending the ColumnVector interface
-          // to do a quick count of the non-null values (i.e. the length minus
-          // the population count of the null bitmap). We should do something
-          // similar for ValueAccessor too.
-          for (std::size_t pos = 0; pos < column_vector.size(); ++pos) {
-            count += !column_vector.getTypedValue(pos).isNull();
-          }
-        } else {
-          count = column_vector.size();
-        }
-      });
+  const attribute_id argument_id = argument_ids.front();
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
-  return new AggregationStateCount(count);
-}
+  ValueAccessor *target_accessor =
+      argument_id >= 0 ? accessor : aux_accessor;
+  const attribute_id target_argument_id =
+      argument_id >= 0 ? argument_id : -(argument_id+2);
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-template <bool count_star, bool nullable_type>
-AggregationState*
-AggregationHandleCount<count_star, nullable_type>::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  DCHECK(!count_star)
-      << "Called non-nullary accumulation method on an AggregationHandleCount "
-      << "set up for nullary COUNT(*)";
-
-  DCHECK_EQ(1u, accessor_ids.size())
-      << "Got wrong number of attributes for COUNT: " << accessor_ids.size();
-
-  const attribute_id accessor_id = accessor_ids.front();
   std::size_t count = 0;
   InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-      accessor,
-      [&accessor_id, &count](auto *accessor) -> void {  // NOLINT(build/c++11)
+      target_accessor,
+      [&target_argument_id, &count](auto *accessor) -> void {  // NOLINT(build/c++11)
         if (nullable_type) {
           while (accessor->next()) {
-            count += !accessor->getTypedValue(accessor_id).isNull();
+            count += !accessor->getTypedValue(target_argument_id).isNull();
           }
         } else {
           count = accessor->getNumTuples();
@@ -123,24 +77,6 @@ AggregationHandleCount<count_star, nullable_type>::accumulateValueAccessor(
 
   return new AggregationStateCount(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(
@@ -156,7 +92,7 @@ void AggregationHandleCount<count_star, nullable_type>::mergeStates(
 }
 
 template <bool count_star, bool nullable_type>
-void AggregationHandleCount<count_star, nullable_type>::mergeStatesFast(
+void AggregationHandleCount<count_star, nullable_type>::mergeStates(
     const std::uint8_t *source, std::uint8_t *destination) const {
   const std::int64_t *src_count_ptr =
       reinterpret_cast<const std::int64_t *>(source);
@@ -170,33 +106,10 @@ 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<
+  return finalizeHashTableHelper<
       AggregationHandleCount<count_star, nullable_type>,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+      PackedPayloadSeparateChainingAggregationStateHashTable>(
+          TypeFactory::GetType(kLong), hash_table, group_by_keys, index);
 }
 
 // Explicitly instantiate and compile in the different versions of

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index 6aab0cd..bf9450f 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.hpp
@@ -29,8 +29,9 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
 #include "storage/HashTableBase.hpp"
+#include "types/LongType.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
@@ -98,25 +99,26 @@ class AggregationHandleCount : public AggregationConcreteHandle {
  public:
   ~AggregationHandleCount() override {}
 
+  std::vector<const Type *> getArgumentTypes() const override {
+    if (argument_type_ == nullptr) {
+      return {};
+    } else {
+      return {argument_type_};
+    }
+  }
+
+  const Type* getResultType() const override {
+    return &LongType::InstanceNonNullable();
+  }
+
   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;
-
   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.
    */
@@ -127,81 +129,50 @@ class AggregationHandleCount : public AggregationConcreteHandle {
     }
   }
 
-  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;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_id) const override;
-#endif
-
-  void aggregateValueAccessorIntoHashTable(
+  AggregationState* accumulate(
       ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
+      ColumnVectorsValueAccessor *aux_accessor,
+      const std::vector<attribute_id> &argument_ids) 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));
+  std::size_t getPayloadSize() const override {
+    return sizeof(std::int64_t);
+  }
+
+  void initPayload(std::uint8_t *byte_ptr) const override {
+    std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
+    *count_ptr = 0;
+  }
+
+  inline void updateStateNullary(std::uint8_t *byte_ptr) const override {
+    ++(*reinterpret_cast<std::int64_t *>(byte_ptr));
   }
 
-  inline TypedValue finalizeHashTableEntryFast(
-      const std::uint8_t *byte_ptr) const {
-    const std::int64_t *count_ptr =
-        reinterpret_cast<const std::int64_t *>(byte_ptr);
-    return TypedValue(*count_ptr);
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    if ((!nullable_type) || (!argument.isNull())) {
+      ++(*reinterpret_cast<std::int64_t *>(byte_ptr));
+    }
+  }
+
+  void mergeStates(const std::uint8_t *source,
+                   std::uint8_t *destination) const override;
+
+  inline TypedValue finalizeHashTableEntry(const std::uint8_t *byte_ptr) const {
+    return TypedValue(*reinterpret_cast<const std::int64_t *>(byte_ptr));
   }
 
   ColumnVector* finalizeHashTable(
@@ -209,36 +180,17 @@ class AggregationHandleCount : public AggregationConcreteHandle {
       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;
 
   /**
    * @brief Constructor.
    **/
-  AggregationHandleCount() : block_update_(false) {}
+  AggregationHandleCount(const Type *argument_type)
+      : AggregationConcreteHandle(AggregationID::kCount),
+        argument_type_(argument_type) {}
 
-  bool block_update_;
+  const Type *argument_type_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleCount);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleDistinct.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.cpp b/expressions/aggregation/AggregationHandleDistinct.cpp
index 0dc8b56..c6c47c7 100644
--- a/expressions/aggregation/AggregationHandleDistinct.cpp
+++ b/expressions/aggregation/AggregationHandleDistinct.cpp
@@ -22,10 +22,9 @@
 #include <cstddef>
 #include <memory>
 #include <vector>
-#include <utility>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
+#include "storage/PackedPayloadAggregationStateHashTable.hpp"
 
 #include "types/TypedValue.hpp"
 
@@ -34,34 +33,6 @@
 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,
@@ -73,7 +44,8 @@ ColumnVector* AggregationHandleDistinct::finalizeHashTable(
                                                const bool &dumb_placeholder) -> void {
     group_by_keys->emplace_back(std::move(group_by_key));
   };
-  static_cast<const AggregationStateFastHashTable&>(hash_table).forEachCompositeKeyFast(&keys_retriever);
+  static_cast<const PackedPayloadSeparateChainingAggregationStateHashTable &>(
+      hash_table).forEach(&keys_retriever);
 
   return nullptr;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleDistinct.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.hpp b/expressions/aggregation/AggregationHandleDistinct.hpp
index 838bfdd..0d8905b 100644
--- a/expressions/aggregation/AggregationHandleDistinct.hpp
+++ b/expressions/aggregation/AggregationHandleDistinct.hpp
@@ -26,6 +26,7 @@
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
 #include "storage/HashTableBase.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
@@ -49,7 +50,17 @@ class AggregationHandleDistinct : public AggregationConcreteHandle {
   /**
    * @brief Constructor.
    **/
-  AggregationHandleDistinct() {}
+  AggregationHandleDistinct()
+      : AggregationConcreteHandle(AggregationID::kDistinct) {}
+
+  std::vector<const Type *> getArgumentTypes() const override {
+    return {};
+  }
+
+  const Type* getResultType() const override {
+    LOG(FATAL)
+        << "AggregationHandleDistinct does not support getResultType().";
+  }
 
   AggregationState* createInitialState() const override {
     LOG(FATAL)
@@ -62,21 +73,13 @@ class AggregationHandleDistinct : public AggregationConcreteHandle {
         << "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(
+  AggregationState* accumulate(
       ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_ids) const override {
+      ColumnVectorsValueAccessor *aux_accessor,
+      const std::vector<attribute_id> &argument_ids) const override {
     LOG(FATAL) << "AggregationHandleDistinct does not support "
-                  "accumulateValueAccessor().";
+                  "accumulate().";
   }
-#endif
 
   void mergeStates(const AggregationState &source,
                    AggregationState *destination) const override {
@@ -87,33 +90,6 @@ class AggregationHandleDistinct : public AggregationConcreteHandle {
     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,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.cpp b/expressions/aggregation/AggregationHandleMax.cpp
index c2d571b..327b2b2 100644
--- a/expressions/aggregation/AggregationHandleMax.cpp
+++ b/expressions/aggregation/AggregationHandleMax.cpp
@@ -23,8 +23,8 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/PackedPayloadAggregationStateHashTable.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
@@ -39,51 +39,32 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleMax::AggregationHandleMax(const Type &type)
-    : type_(type), block_update_(false) {
+    : AggregationConcreteHandle(AggregationID::kMax),
+      type_(type) {
   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();
+AggregationState* AggregationHandleMax::accumulate(
+    ValueAccessor *accessor,
+    ColumnVectorsValueAccessor *aux_accessor,
+    const std::vector<attribute_id> &argument_ids) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for MAX: " << argument_ids.size();
 
-  return new AggregationStateMax(fast_comparator_->accumulateColumnVector(
-      type_.getNullableVersion().makeNullValue(), *column_vectors.front()));
-}
+  const attribute_id argument_id = argument_ids.front();
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
-#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();
+  ValueAccessor *target_accessor =
+      argument_id >= 0 ? accessor : aux_accessor;
+  const attribute_id target_argument_id =
+      argument_id >= 0 ? argument_id : -(argument_id+2);
 
   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();
+      target_accessor,
+      target_argument_id));
 }
 
 void AggregationHandleMax::mergeStates(const AggregationState &source,
@@ -98,12 +79,12 @@ void AggregationHandleMax::mergeStates(const AggregationState &source,
   }
 }
 
-void AggregationHandleMax::mergeStatesFast(const std::uint8_t *source,
-                                           std::uint8_t *destination) const {
+void AggregationHandleMax::mergeStates(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);
+    compareAndUpdate(dst_max_ptr, *src_max_ptr);
   }
 }
 
@@ -111,27 +92,10 @@ 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<
+  return finalizeHashTableHelper<
       AggregationHandleMax,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+      PackedPayloadSeparateChainingAggregationStateHashTable>(
+          type_.getNullableVersion(), hash_table, group_by_keys, index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index d851a0c..635c7d8 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -28,7 +28,6 @@
 #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"
@@ -86,16 +85,18 @@ class AggregationHandleMax : public AggregationConcreteHandle {
  public:
   ~AggregationHandleMax() override {}
 
+  std::vector<const Type *> getArgumentTypes() const override {
+    return {&type_};
+  }
+
+  const Type* getResultType() const override {
+    return &type_;
+  }
+
   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.
    */
@@ -105,23 +106,17 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     compareAndUpdate(static_cast<AggregationStateMax *>(state), value);
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value,
-                                  std::uint8_t *byte_ptr) const {
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
-    TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
-    compareAndUpdateFast(max_ptr, value);
-  }
-
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  AggregationState* accumulate(
+      ValueAccessor *accessor,
+      ColumnVectorsValueAccessor *aux_accessor,
+      const std::vector<attribute_id> &argument_ids) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void allowUpdate() override { block_update_ = false; }
+  std::size_t getPayloadSize() const override {
+    return sizeof(TypedValue);
+  }
 
   void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
@@ -136,38 +131,21 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     }
   }
 
-  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;
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    DCHECK(argument.isPlausibleInstanceOf(type_.getSignature()));
+    TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
+    compareAndUpdate(max_ptr, argument);
+  }
 
   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_);
-  }
+  void mergeStates(const std::uint8_t *source,
+                   std::uint8_t *destination) const override;
 
-  inline TypedValue finalizeHashTableEntryFast(
+  inline TypedValue finalizeHashTableEntry(
       const std::uint8_t *byte_ptr) const {
     const TypedValue *max_ptr = reinterpret_cast<const TypedValue *>(byte_ptr);
     return TypedValue(*max_ptr);
@@ -178,27 +156,6 @@ class AggregationHandleMax : public AggregationConcreteHandle {
       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;
 
@@ -227,8 +184,8 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     }
   }
 
-  inline void compareAndUpdateFast(TypedValue *max_ptr,
-                                   const TypedValue &value) const {
+  inline void compareAndUpdate(TypedValue *max_ptr,
+                               const TypedValue &value) const {
     if (value.isNull()) return;
     if (max_ptr->isNull() ||
         fast_comparator_->compareTypedValues(value, *max_ptr)) {
@@ -239,8 +196,6 @@ class AggregationHandleMax : public AggregationConcreteHandle {
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 
-  bool block_update_;
-
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMax);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.cpp b/expressions/aggregation/AggregationHandleMin.cpp
index a07f299..fe4a61b 100644
--- a/expressions/aggregation/AggregationHandleMin.cpp
+++ b/expressions/aggregation/AggregationHandleMin.cpp
@@ -23,8 +23,8 @@
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/PackedPayloadAggregationStateHashTable.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
@@ -39,51 +39,32 @@ namespace quickstep {
 class StorageManager;
 
 AggregationHandleMin::AggregationHandleMin(const Type &type)
-    : type_(type), block_update_(false) {
+    : AggregationConcreteHandle(AggregationID::kMin),
+      type_(type) {
   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();
+AggregationState* AggregationHandleMin::accumulate(
+    ValueAccessor *accessor,
+    ColumnVectorsValueAccessor *aux_accessor,
+    const std::vector<attribute_id> &argument_ids) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for MIN: " << argument_ids.size();
 
-  return new AggregationStateMin(fast_comparator_->accumulateColumnVector(
-      type_.getNullableVersion().makeNullValue(), *column_vectors.front()));
-}
+  const attribute_id argument_id = argument_ids.front();
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
-#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();
+  ValueAccessor *target_accessor =
+      argument_id >= 0 ? accessor : aux_accessor;
+  const attribute_id target_argument_id =
+      argument_id >= 0 ? argument_id : -(argument_id+2);
 
   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();
+      target_accessor,
+      target_argument_id));
 }
 
 void AggregationHandleMin::mergeStates(const AggregationState &source,
@@ -98,13 +79,13 @@ void AggregationHandleMin::mergeStates(const AggregationState &source,
   }
 }
 
-void AggregationHandleMin::mergeStatesFast(const std::uint8_t *source,
-                                           std::uint8_t *destination) const {
+void AggregationHandleMin::mergeStates(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);
+    compareAndUpdate(dst_min_ptr, *src_min_ptr);
   }
 }
 
@@ -112,27 +93,10 @@ 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<
+  return finalizeHashTableHelper<
       AggregationHandleMin,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+      PackedPayloadSeparateChainingAggregationStateHashTable>(
+          type_.getNonNullableVersion(), hash_table, group_by_keys, index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b46bc73c/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index e3472ec..3571f02 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -28,7 +28,6 @@
 #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"
@@ -88,42 +87,39 @@ class AggregationHandleMin : public AggregationConcreteHandle {
  public:
   ~AggregationHandleMin() override {}
 
+  std::vector<const Type *> getArgumentTypes() const override {
+    return {&type_};
+  }
+
+  const Type* getResultType() const override {
+    return &type_;
+  }
+
   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);
-  }
+  AggregationState* accumulate(
+      ValueAccessor *accessor,
+      ColumnVectorsValueAccessor *aux_accessor,
+      const std::vector<attribute_id> &argument_ids) const override;
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  TypedValue finalize(const AggregationState &state) const override {
+    return static_cast<const AggregationStateMin &>(state).min_;
+  }
 
-  void allowUpdate() override { block_update_ = false; }
+  std::size_t getPayloadSize() const override {
+    return sizeof(TypedValue);
+  }
 
   void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
@@ -138,41 +134,19 @@ class AggregationHandleMin : public AggregationConcreteHandle {
     }
   }
 
-  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 void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    DCHECK(argument.isPlausibleInstanceOf(type_.getSignature()));
+    TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
+    compareAndUpdate(min_ptr, argument);
   }
 
-  inline TypedValue finalizeHashTableEntry(
-      const AggregationState &state) const {
-    return static_cast<const AggregationStateMin &>(state).min_;
-  }
+  void mergeStates(const std::uint8_t *source,
+                   std::uint8_t *destination) const override;
 
-  inline TypedValue finalizeHashTableEntryFast(
+  inline TypedValue finalizeHashTableEntry(
       const std::uint8_t *byte_ptr) const {
-    const TypedValue *min_ptr = reinterpret_cast<const TypedValue *>(byte_ptr);
-    return TypedValue(*min_ptr);
+    return *reinterpret_cast<const TypedValue *>(byte_ptr);
   }
 
   ColumnVector* finalizeHashTable(
@@ -180,27 +154,6 @@ class AggregationHandleMin : public AggregationConcreteHandle {
       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;
 
@@ -228,8 +181,8 @@ class AggregationHandleMin : public AggregationConcreteHandle {
     }
   }
 
-  inline void compareAndUpdateFast(TypedValue *min_ptr,
-                                   const TypedValue &value) const {
+  inline void compareAndUpdate(TypedValue *min_ptr,
+                               const TypedValue &value) const {
     if (value.isNull()) return;
     if (min_ptr->isNull() ||
         fast_comparator_->compareTypedValues(value, *min_ptr)) {
@@ -240,8 +193,6 @@ class AggregationHandleMin : public AggregationConcreteHandle {
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 
-  bool block_update_;
-
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMin);
 };