You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by zu...@apache.org on 2017/02/08 08:53:44 UTC

[01/18] incubator-quickstep git commit: A workaround to remove query result relation in the distributed version. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/tmb_poll_interval d5579bf07 -> 0724f4237 (forced update)


A workaround to remove query result relation in the distributed version.


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

Branch: refs/heads/tmb_poll_interval
Commit: aef1c3586580cfa72eb031fafe08700f6d5d9a86
Parents: 27a8055
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Feb 7 00:41:45 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Feb 7 00:41:45 2017 -0800

----------------------------------------------------------------------
 query_execution/ForemanDistributed.cpp | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aef1c358/query_execution/ForemanDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanDistributed.cpp b/query_execution/ForemanDistributed.cpp
index 4d95f16..e6f22ec 100644
--- a/query_execution/ForemanDistributed.cpp
+++ b/query_execution/ForemanDistributed.cpp
@@ -201,8 +201,12 @@ void ForemanDistributed::run() {
 
         // TODO(quickstep-team): Dynamically scale-up/down Shiftbosses.
         if (query_result_saved_shiftbosses_[query_id].size() == shiftboss_directory_.size()) {
-          processSaveQueryResultResponseMessage(proto.cli_id(), proto.relation_id());
+          const relation_id result_relation_id = proto.relation_id();
+          processSaveQueryResultResponseMessage(proto.cli_id(), result_relation_id);
           query_result_saved_shiftbosses_.erase(query_id);
+
+          // TODO(zuyu): Refactor to clean-up blocks in Shiftbosses.
+          catalog_database_->dropRelationById(result_relation_id);
         }
         break;
       }


[09/18] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index b942c1b..0b34908 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -20,7 +20,7 @@
 #include "storage/AggregationOperationState.hpp"
 
 #include <cstddef>
-#include <cstdio>
+#include <cstdint>
 #include <memory>
 #include <string>
 #include <utility>
@@ -34,29 +34,32 @@
 #include "expressions/aggregation/AggregateFunction.hpp"
 #include "expressions/aggregation/AggregateFunctionFactory.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "expressions/aggregation/AggregationHandleDistinct.hpp"
-#include "expressions/aggregation/AggregationID.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/AggregationOperationState.pb.h"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "storage/CollisionFreeVectorTable.hpp"
 #include "storage/HashTableFactory.hpp"
+#include "storage/HashTableBase.hpp"
 #include "storage/InsertDestination.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/SubBlocksReference.hpp"
 #include "storage/TupleIdSequence.hpp"
+#include "storage/TupleStorageSubBlock.hpp"
 #include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "storage/ValueAccessorUtil.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
 #include "types/containers/Tuple.hpp"
 #include "utility/lip_filter/LIPFilterAdaptiveProber.hpp"
 
-#include "glog/logging.h"
+#include "gflags/gflags.h"
 
-using std::unique_ptr;
+#include "glog/logging.h"
 
 namespace quickstep {
 
@@ -80,148 +83,145 @@ 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());
+  }
+
+  // Prepare group-by key 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) {
+      group_by_key_ids_.emplace_back(ValueAccessorSource::kBase, attr_id);
+    } else {
+      group_by_key_ids_.emplace_back(ValueAccessorSource::kDerived,
+                                     non_trivial_expressions_.size());
+      non_trivial_expressions_.emplace_back(group_by_element.release());
+    }
   }
 
   std::vector<AggregationHandle *> group_by_handles;
-  group_by_handles.clear();
-
-  if (aggregate_functions.size() == 0) {
-    // If there is no aggregation function, then it is a distinctify operation
-    // on the group-by expressions.
-    DCHECK_GT(group_by_list_.size(), 0u);
-
-    handles_.emplace_back(new AggregationHandleDistinct());
-    arguments_.push_back({});
-    is_distinct_.emplace_back(false);
-    group_by_hashtable_pool_.reset(new HashTablePool(estimated_num_entries,
-                                                     hash_table_impl_type,
-                                                     group_by_types,
-                                                     {1},
-                                                     handles_,
-                                                     storage_manager));
-  } else {
-    // Set up each individual aggregate in this operation.
-    std::vector<const AggregateFunction *>::const_iterator agg_func_it =
-        aggregate_functions.begin();
-    std::vector<std::vector<std::unique_ptr<const Scalar>>>::const_iterator
-        args_it = arguments_.begin();
-    std::vector<bool>::const_iterator is_distinct_it = is_distinct_.begin();
-    std::vector<HashTableImplType>::const_iterator
-        distinctify_hash_table_impl_types_it =
-            distinctify_hash_table_impl_types.begin();
-    std::vector<std::size_t> payload_sizes;
-    for (; agg_func_it != aggregate_functions.end();
-         ++agg_func_it, ++args_it, ++is_distinct_it) {
-      // Get the Types of this aggregate's arguments so that we can create an
-      // AggregationHandle.
-      std::vector<const Type *> argument_types;
-      for (const std::unique_ptr<const Scalar> &argument : *args_it) {
-        argument_types.emplace_back(&argument->getType());
-      }
 
-      // Sanity checks: aggregate function exists and can apply to the specified
-      // arguments.
-      DCHECK(*agg_func_it != nullptr);
-      DCHECK((*agg_func_it)->canApplyToTypes(argument_types));
-
-      // Have the AggregateFunction create an AggregationHandle that we can use
-      // to do actual aggregate computation.
-      handles_.emplace_back((*agg_func_it)->createHandle(argument_types));
-
-      if (!group_by_list_.empty()) {
-        // Aggregation with GROUP BY: combined payload is partially updated in
-        // the presence of DISTINCT.
-        if (*is_distinct_it) {
-          handles_.back()->blockUpdate();
-        }
-        group_by_handles.emplace_back(handles_.back());
-        payload_sizes.emplace_back(group_by_handles.back()->getPayloadSize());
+  // 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>>>::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();
+  for (; agg_func_it != aggregate_functions.end();
+       ++agg_func_it, ++args_it, ++is_distinct_it) {
+    // Get the Types of this aggregate's arguments so that we can create an
+    // AggregationHandle.
+    std::vector<const Type *> argument_types;
+    for (const std::unique_ptr<const Scalar> &argument : *args_it) {
+      argument_types.emplace_back(&argument->getType());
+    }
+
+    // Prepare argument attribute ids and non-trivial expressions.
+    std::vector<MultiSourceAttributeId> argument_ids;
+    for (std::unique_ptr<const Scalar> &argument : *args_it) {
+      const attribute_id attr_id =
+          argument->getAttributeIdForValueAccessor();
+      if (attr_id != kInvalidAttributeID) {
+        argument_ids.emplace_back(ValueAccessorSource::kBase, attr_id);
       } 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
+        argument_ids.emplace_back(ValueAccessorSource::kDerived,
+                                  non_trivial_expressions_.size());
+        non_trivial_expressions_.emplace_back(argument.release());
       }
+    }
+    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);
+    DCHECK((*agg_func_it)->canApplyToTypes(argument_types));
 
-      // Initialize the corresponding distinctify hash table if this is a
-      // DISTINCT aggregation.
+    // Have the AggregateFunction create an AggregationHandle that we can use
+    // to do actual aggregate computation.
+    handles_.emplace_back((*agg_func_it)->createHandle(argument_types));
+
+    if (!group_by_key_ids_.empty()) {
+      // Aggregation with GROUP BY: combined payload is partially updated in
+      // the presence of DISTINCT.
       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,
-                estimated_num_entries,
-                {0},
-                {},
-                storage_manager));
-        ++distinctify_hash_table_impl_types_it;
-      } else {
-        distinctify_hashtables_.emplace_back(nullptr);
+        handles_.back()->blockUpdate();
       }
+      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());
     }
 
-    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 {
-        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_handles,
-                                         storage_manager));
-      }
+    // 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 need to estimate for each distinct aggregation an
+      // estimated_num_distinct_keys value during query optimization.
+      distinctify_hashtables_.emplace_back(
+          AggregationStateHashTableFactory::CreateResizable(
+              *distinctify_hash_table_impl_types_it,
+              key_types,
+              estimated_num_entries,
+              {} /* handles */,
+              storage_manager));
+      ++distinctify_hash_table_impl_types_it;
+    } else {
+      distinctify_hashtables_.emplace_back(nullptr);
+    }
+  }
+
+  if (!group_by_key_ids_.empty()) {
+    // Aggregation with GROUP BY: create the hash table (pool).
+    if (is_aggregate_collision_free_) {
+      collision_free_hashtable_.reset(
+          AggregationStateHashTableFactory::CreateResizable(
+              hash_table_impl_type,
+              group_by_types_,
+              estimated_num_entries,
+              group_by_handles,
+              storage_manager));
+    } 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_,
+                                       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 +269,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,153 +353,210 @@ 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);
-  } else {
-    aggregateBlockHashTable(input_block, lip_filter_adaptive_prober);
+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;
+  }
+
+  // Currently we require that all the group-by keys are ScalarAttributes for
+  // the convenient of implementing copy elision.
+  // TODO(jianqiao): relax this requirement.
+  for (const auto &group_by_element : group_by) {
+    if (group_by_element->getAttributeIdForValueAccessor() == kInvalidAttributeID) {
+      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::finalizeAggregate(
-    InsertDestination *output_destination) {
-  if (group_by_list_.empty()) {
-    finalizeSingleState(output_destination);
+std::size_t AggregationOperationState::getNumInitializationPartitions() const {
+  if (is_aggregate_collision_free_) {
+    return static_cast<CollisionFreeVectorTable *>(
+        collision_free_hashtable_.get())->getNumInitializationPartitions();
   } else {
-    finalizeHashTable(output_destination);
+    return 0u;
   }
 }
 
-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());
-    }
+std::size_t AggregationOperationState::getNumFinalizationPartitions() const {
+  if (is_aggregate_collision_free_) {
+    return static_cast<CollisionFreeVectorTable *>(
+        collision_free_hashtable_.get())->getNumFinalizationPartitions();
+  } else if (is_aggregate_partitioned_) {
+    return partitioned_group_by_hashtable_pool_->getNumPartitions();
+  } else  {
+    return 1u;
   }
 }
 
-void AggregationOperationState::aggregateBlockSingleState(
-    const block_id input_block) {
-  // Aggregate per-block state for each aggregate.
-  std::vector<std::unique_ptr<AggregationState>> local_state;
+void AggregationOperationState::initialize(const std::size_t partition_id) {
+  if (is_aggregate_collision_free_) {
+    static_cast<CollisionFreeVectorTable *>(
+        collision_free_hashtable_.get())->initialize(partition_id);
+  } else {
+    LOG(FATAL) << "AggregationOperationState::initializeState() "
+               << "is not supported by this aggregation";
+  }
+}
 
+void AggregationOperationState::aggregateBlock(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) {
-    std::unique_ptr<ValueAccessor> accessor(
-        block->getTupleStorageSubBlock().createValueAccessor());
-    matches.reset(block->getMatchesForPredicate(predicate_.get(), matches.get()));
+    matches.reset(block->getMatchesForPredicate(predicate_.get()));
+    shared_accessor.reset(
+        base_accessor->createSharedTupleIdSequenceAdapterVirtual(*matches));
+    accessor = shared_accessor.get();
+  }
+  if (lip_filter_adaptive_prober != nullptr) {
+    matches.reset(lip_filter_adaptive_prober->filterValueAccessor(accessor));
+    shared_accessor.reset(
+        base_accessor->createSharedTupleIdSequenceAdapterVirtual(*matches));
+    accessor = shared_accessor.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]);
+  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));
     }
-#endif
+  }
+
+  accessor->beginIterationVirtual();
+
+  ValueAccessorMultiplexer accessor_mux(accessor, non_trivial_results.get());
+  if (group_by_key_ids_.empty()) {
+    aggregateBlockSingleState(accessor_mux);
+  } else {
+    aggregateBlockHashTable(accessor_mux);
+  }
+}
+
+void AggregationOperationState::aggregateBlockSingleState(
+    const ValueAccessorMultiplexer &accessor_mux) {
+  // Aggregate per-block state for each aggregate.
+  std::vector<std::unique_ptr<AggregationState>> local_state;
+
+  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 = nullptr;
     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);
+      handle->insertValueAccessorIntoDistinctifyHashTable(
+          argument_ids,
+          {},
+          accessor_mux,
+          distinctify_hashtables_[agg_idx].get());
     } 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()));
+      if (argument_ids.empty()) {
+        // Special case. This is a nullary aggregate (i.e. COUNT(*)).
+        ValueAccessor *base_accessor = accessor_mux.getBaseAccessor();
+        DCHECK(base_accessor != nullptr);
+        state = handle->accumulateNullary(base_accessor->getNumTuplesVirtual());
+      } else {
+        // Have the AggregationHandle actually do the aggregation.
+        state = handle->accumulateValueAccessor(argument_ids, accessor_mux);
+      }
     }
+    local_state.emplace_back(state);
   }
 
   // Merge per-block aggregation states back with global state.
   mergeSingleState(local_state);
 }
 
-void AggregationOperationState::aggregateBlockHashTable(
-    const block_id input_block,
-    LIPFilterAdaptiveProber *lip_filter_adaptive_prober) {
-  BlockReference block(
-      storage_manager_->getBlock(input_block, input_relation_));
-
-  // 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()));
-  }
-  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()));
-  }
-
-  // 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;
-
+void AggregationOperationState::mergeSingleState(
+    const std::vector<std::unique_ptr<AggregationState>> &local_state) {
+  DCHECK_EQ(local_state.size(), single_states_.size());
   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);
+    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 {
+  HashTableMerger merger(static_cast<PackedPayloadHashTable *>(dst));
+  static_cast<PackedPayloadHashTable *>(src)->forEachCompositeKey(&merger);
+}
 
-  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);
+void AggregationOperationState::aggregateBlockHashTable(
+    const ValueAccessorMultiplexer &accessor_mux) {
+  if (is_aggregate_collision_free_) {
+    aggregateBlockHashTableImplCollisionFree(accessor_mux);
+  } else if (is_aggregate_partitioned_) {
+    aggregateBlockHashTableImplPartitioned(accessor_mux);
   } else {
-    ColumnVectorsValueAccessor temp_result;
-    // IDs of 'arguments' as attributes in the ValueAccessor we create below.
-    std::vector<attribute_id> argument_ids;
+    aggregateBlockHashTableImplThreadPrivate(accessor_mux);
+  }
+}
+
+void AggregationOperationState::aggregateBlockHashTableImplCollisionFree(
+    const ValueAccessorMultiplexer &accessor_mux) {
+  DCHECK(collision_free_hashtable_ != nullptr);
+
+  collision_free_hashtable_->upsertValueAccessorCompositeKey(argument_ids_,
+                                                             group_by_key_ids_,
+                                                             accessor_mux);
+}
+
+void AggregationOperationState::aggregateBlockHashTableImplPartitioned(
+    const ValueAccessorMultiplexer &accessor_mux) {
+  DCHECK(partitioned_group_by_hashtable_pool_ != nullptr);
+
+  std::vector<attribute_id> group_by_key_ids;
+  for (const MultiSourceAttributeId &key_id : group_by_key_ids_) {
+    DCHECK(key_id.source == ValueAccessorSource::kBase);
+    group_by_key_ids.emplace_back(key_id.attr_id);
+  }
 
-    // IDs of GROUP BY key element(s) in the ValueAccessor we create below.
-    std::vector<attribute_id> key_ids;
+  InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+      accessor_mux.getBaseAccessor(),
+      [&](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 +564,74 @@ 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> base_adapter(
+          accessor->createSharedTupleIdSequenceAdapter(
+              *partition_membership[partition]));
+
+      std::unique_ptr<ValueAccessor> derived_adapter;
+      if (accessor_mux.getDerivedAccessor() != nullptr) {
+        derived_adapter.reset(
+            accessor_mux.getDerivedAccessor()->createSharedTupleIdSequenceAdapterVirtual(
+                *partition_membership[partition]));
+      }
+
+      ValueAccessorMultiplexer local_mux(base_adapter.get(), derived_adapter.get());
       partitioned_group_by_hashtable_pool_->getHashTable(partition)
-          ->upsertValueAccessorCompositeKeyFast(
-              argument_ids, adapter[partition].get(), key_ids, true);
+          ->upsertValueAccessorCompositeKey(argument_ids_,
+                                            group_by_key_ids_,
+                                            local_mux);
     }
+  });
+}
+
+void AggregationOperationState::aggregateBlockHashTableImplThreadPrivate(
+    const ValueAccessorMultiplexer &accessor_mux) {
+  DCHECK(group_by_hashtable_pool_ != nullptr);
+
+  for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
+    if (is_distinct_[agg_idx]) {
+      handles_[agg_idx]->insertValueAccessorIntoDistinctifyHashTable(
+          argument_ids_[agg_idx],
+          group_by_key_ids_,
+          accessor_mux,
+          distinctify_hashtables_[agg_idx].get());
+    }
+  }
+
+  AggregationStateHashTableBase *agg_hash_table =
+      group_by_hashtable_pool_->getHashTable();
+
+  agg_hash_table->upsertValueAccessorCompositeKey(argument_ids_,
+                                                  group_by_key_ids_,
+                                                  accessor_mux);
+  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);
   }
 }
 
@@ -556,80 +655,83 @@ 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) {
+  std::vector<std::unique_ptr<ColumnVector>> final_values;
+  CollisionFreeVectorTable *hash_table =
+      static_cast<CollisionFreeVectorTable *>(collision_free_hashtable_.get());
+
+  const std::size_t max_length =
+      hash_table->getNumTuplesInFinalizationPartition(partition_id);
+  ColumnVectorsValueAccessor complete_result;
+
+  DCHECK_EQ(1u, group_by_types_.size());
+  const Type *key_type = group_by_types_.front();
+  DCHECK(NativeColumnVector::UsableForType(*key_type));
+
+  std::unique_ptr<NativeColumnVector> key_cv(
+      std::make_unique<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) {
+    const Type *result_type = handles_[i]->getResultType();
+    DCHECK(NativeColumnVector::UsableForType(*result_type));
+
+    std::unique_ptr<NativeColumnVector> result_cv(
+        std::make_unique<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) {
+  PackedPayloadHashTable *hash_table =
+      static_cast<PackedPayloadHashTable *>(
+          partitioned_group_by_hashtable_pool_->getHashTable(partition_id));
+
   // 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.
+  if (handles_.empty()) {
+    const auto keys_retriever = [&group_by_keys](std::vector<TypedValue> &group_by_key,
+                                                 const std::uint8_t *dumb_placeholder) -> void {
+      group_by_keys.emplace_back(std::move(group_by_key));
+    };
 
-  // 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();
-  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());
-    }
+    hash_table->forEachCompositeKey(&keys_retriever);
   }
 
   // Collect per-aggregate finalized values.
   std::vector<std::unique_ptr<ColumnVector>> final_values;
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    if (is_distinct_[agg_idx]) {
-      DCHECK(group_by_hashtable_pool_ != nullptr);
-      auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
-      DCHECK(hash_tables != nullptr);
-      if (hash_tables->empty()) {
-        // We may have a case where hash_tables is empty, e.g. no input blocks.
-        // However for aggregateOnDistinctifyHashTableForGroupBy to work
-        // correctly, we should create an empty group by hash table.
-        AggregationStateHashTableBase *new_hash_table =
-            group_by_hashtable_pool_->getHashTableFast();
-        group_by_hashtable_pool_->returnHashTable(new_hash_table);
-        hash_tables = group_by_hashtable_pool_->getAllHashTables();
-      }
-      DCHECK(hash_tables->back() != nullptr);
-      AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
-      DCHECK(agg_hash_table != nullptr);
-      handles_[agg_idx]->allowUpdate();
-      handles_[agg_idx]->aggregateOnDistinctifyHashTableForGroupBy(
-          *distinctify_hashtables_[agg_idx], agg_hash_table, agg_idx);
-    }
-
-    auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
-    DCHECK(hash_tables != nullptr);
-    if (hash_tables->empty()) {
-      // We may have a case where hash_tables is empty, e.g. no input blocks.
-      // However for aggregateOnDistinctifyHashTableForGroupBy to work
-      // correctly, we should create an empty group by hash table.
-      AggregationStateHashTableBase *new_hash_table =
-          group_by_hashtable_pool_->getHashTableFast();
-      group_by_hashtable_pool_->returnHashTable(new_hash_table);
-      hash_tables = group_by_hashtable_pool_->getAllHashTables();
-    }
-    AggregationStateHashTableBase *agg_hash_table = hash_tables->back().get();
-    DCHECK(agg_hash_table != nullptr);
     ColumnVector *agg_result_col = handles_[agg_idx]->finalizeHashTable(
-        *agg_hash_table, &group_by_keys, agg_idx);
+        *hash_table, agg_idx, &group_by_keys);
     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 +742,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 +775,64 @@ 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();
-    }
+void AggregationOperationState::finalizeHashTableImplThreadPrivate(
+    InsertDestination *output_destination) {
+  // 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;
   }
-  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();
-    }
+
+  std::unique_ptr<AggregationStateHashTableBase> final_hash_table_ptr(
+      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_ptr.get());
+    hash_table->destroyPayload();
   }
-}
 
-void AggregationOperationState::finalizeAggregatePartitioned(
-    const std::size_t partition_id, InsertDestination *output_destination) {
+  PackedPayloadHashTable *final_hash_table =
+      static_cast<PackedPayloadHashTable *>(final_hash_table_ptr.get());
+
   // 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;
 
+  if (handles_.empty()) {
+    const auto keys_retriever = [&group_by_keys](std::vector<TypedValue> &group_by_key,
+                                                 const std::uint8_t *dumb_placeholder) -> void {
+      group_by_keys.emplace_back(std::move(group_by_key));
+    };
+
+    final_hash_table->forEachCompositeKey(&keys_retriever);
+  }
+
+
   // Collect per-aggregate finalized values.
   std::vector<std::unique_ptr<ColumnVector>> final_values;
   for (std::size_t agg_idx = 0; agg_idx < handles_.size(); ++agg_idx) {
-    AggregationStateHashTableBase *hash_table =
-        partitioned_group_by_hashtable_pool_->getHashTable(partition_id);
-    ColumnVector *agg_result_col = handles_[agg_idx]->finalizeHashTable(
-        *hash_table, &group_by_keys, agg_idx);
-    if (agg_result_col != nullptr) {
-      final_values.emplace_back(agg_result_col);
+    if (is_distinct_[agg_idx]) {
+      handles_[agg_idx]->allowUpdate();
+      handles_[agg_idx]->aggregateOnDistinctifyHashTableForGroupBy(
+          *distinctify_hashtables_[agg_idx], agg_idx, final_hash_table);
     }
+
+    ColumnVector *agg_result_col =
+        handles_[agg_idx]->finalizeHashTable(
+            *final_hash_table, agg_idx, &group_by_keys);
+    DCHECK(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 +843,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/2d89e4fb/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index 591e3a1..13ee377 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -24,31 +24,27 @@
 #include <memory>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "expressions/aggregation/AggregationID.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
-#include "storage/AggregationOperationState.pb.h"
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTablePool.hpp"
 #include "storage/PartitionedHashTablePool.hpp"
 #include "storage/StorageBlockInfo.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "utility/Macros.hpp"
 
-#include "gflags/gflags.h"
-
 namespace quickstep {
 
+namespace serialization { class AggregationOperationState; }
+
 class AggregateFunction;
 class CatalogDatabaseLite;
 class CatalogRelationSchema;
 class InsertDestination;
 class LIPFilterAdaptiveProber;
 class StorageManager;
-
-DECLARE_int32(num_aggregation_partitions);
-DECLARE_int32(partition_aggregation_num_groups_threshold);
+class Type;
 
 /** \addtogroup Storage
  *  @{
@@ -156,6 +152,29 @@ class AggregationOperationState {
       const CatalogDatabaseLite &database);
 
   /**
+   * @brief Get the number of partitions to be used for initializing the
+   *        aggregation.
+   *
+   * @return The number of partitions to be used for initializing the aggregation.
+   **/
+  std::size_t getNumInitializationPartitions() const;
+
+  /**
+   * @brief Get the number of partitions to be used for finalizing the
+   *        aggregation.
+   *
+   * @return The number of partitions to be used for finalizing the aggregation.
+   **/
+  std::size_t getNumFinalizationPartitions() const;
+
+  /**
+   * @brief Initialize the specified partition of this aggregation.
+   *
+   * @param partition_id ID of the partition to be initialized.
+   */
+  void initialize(const std::size_t partition_id);
+
+  /**
    * @brief Compute aggregates on the tuples of the given storage block,
    *        updating the running state maintained by this
    *        AggregationOperationState.
@@ -166,127 +185,95 @@ 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);
+  void finalizeAggregate(const std::size_t partition_id,
+                         InsertDestination *output_destination);
 
-  bool isAggregatePartitioned() const {
-    return is_aggregate_partitioned_;
-  }
+ private:
+  // Check whether partitioned aggregation can be applied.
+  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;
 
-  /**
-   * @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;
-  }
+  // Aggregate on input block.
+  void aggregateBlockSingleState(const ValueAccessorMultiplexer &accessor_mux);
 
-  int dflag;
+  void aggregateBlockHashTable(const ValueAccessorMultiplexer &accessor_mux);
 
- private:
   // Merge locally (per storage block) aggregated states with global aggregation
   // states.
   void mergeSingleState(
       const std::vector<std::unique_ptr<AggregationState>> &local_state);
 
-  // Aggregate on input block.
-  void aggregateBlockSingleState(const block_id input_block);
-  void aggregateBlockHashTable(const block_id input_block,
-                               LIPFilterAdaptiveProber *lip_filter_adaptive_prober);
+  void mergeGroupByHashTables(AggregationStateHashTableBase *src,
+                              AggregationStateHashTableBase *dst) const;
 
+  // Finalize the aggregation results into output_destination.
   void finalizeSingleState(InsertDestination *output_destination);
-  void finalizeHashTable(InsertDestination *output_destination);
 
-  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);
-  }
+  void finalizeHashTable(const std::size_t partition_id,
+                         InsertDestination *output_destination);
+
+  // Specialized implementations for aggregateBlockHashTable.
+  void aggregateBlockHashTableImplCollisionFree(
+      const ValueAccessorMultiplexer &accessor_mux);
+
+  void aggregateBlockHashTableImplPartitioned(
+      const ValueAccessorMultiplexer &accessor_mux);
+
+  void aggregateBlockHashTableImplThreadPrivate(
+      const ValueAccessorMultiplexer &accessor_mux);
+
+  // 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_;
+
+  std::vector<MultiSourceAttributeId> group_by_key_ids_;
+  std::vector<std::vector<MultiSourceAttributeId>> argument_ids_;
 
-#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<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_;
@@ -295,14 +282,15 @@ class AggregationOperationState {
   //
   // TODO(shoban): We should ideally store the aggregation state together in one
   // hash table to prevent multiple lookups.
-  std::vector<std::unique_ptr<AggregationStateHashTableBase>>
-      group_by_hashtables_;
+  std::vector<std::unique_ptr<AggregationStateHashTableBase>> group_by_hashtables_;
 
   // A vector of group by hash table pools.
   std::unique_ptr<HashTablePool> group_by_hashtable_pool_;
 
   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/2d89e4fb/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index a44c3a7..293be17 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_CollisionFreeVectorTable
+            CollisionFreeVectorTable.cpp
+            CollisionFreeVectorTable.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)
@@ -226,6 +226,7 @@ add_library(quickstep_storage_InsertDestination_proto
 add_library(quickstep_storage_LinearOpenAddressingHashTable
             ../empty_src.cpp
             LinearOpenAddressingHashTable.hpp)
+add_library(quickstep_storage_PackedPayloadHashTable PackedPayloadHashTable.cpp PackedPayloadHashTable.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)
@@ -253,6 +254,7 @@ add_library(quickstep_storage_TupleIdSequence ../empty_src.cpp TupleIdSequence.h
 add_library(quickstep_storage_TupleReference ../empty_src.cpp TupleReference.hpp)
 add_library(quickstep_storage_TupleStorageSubBlock TupleStorageSubBlock.cpp TupleStorageSubBlock.hpp)
 add_library(quickstep_storage_ValueAccessor ../empty_src.cpp ValueAccessor.hpp)
+add_library(quickstep_storage_ValueAccessorMultiplexer ../empty_src.cpp ValueAccessorMultiplexer.hpp)
 add_library(quickstep_storage_ValueAccessorUtil ../empty_src.cpp ValueAccessorUtil.hpp)
 add_library(quickstep_storage_WindowAggregationOperationState
             WindowAggregationOperationState.hpp
@@ -272,22 +274,25 @@ target_link_libraries(quickstep_storage_AggregationOperationState
                       quickstep_expressions_aggregation_AggregateFunction
                       quickstep_expressions_aggregation_AggregateFunctionFactory
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_expressions_aggregation_AggregationHandleDistinct
-                      quickstep_expressions_aggregation_AggregationID
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_storage_AggregationOperationState_proto
-                      quickstep_storage_HashTable
+                      quickstep_storage_CollisionFreeVectorTable
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
                       quickstep_storage_HashTablePool
                       quickstep_storage_InsertDestination
                       quickstep_storage_PartitionedHashTablePool
+                      quickstep_storage_PackedPayloadHashTable
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
+                      quickstep_storage_SubBlocksReference
                       quickstep_storage_TupleIdSequence
+                      quickstep_storage_TupleStorageSubBlock
                       quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorMultiplexer
+                      quickstep_storage_ValueAccessorUtil
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorsValueAccessor
@@ -430,6 +435,24 @@ if(QUICKSTEP_HAVE_BITWEAVING)
                         quickstep_utility_Macros)
 endif()
 # CMAKE_VALIDATE_IGNORE_END
+target_link_libraries(quickstep_storage_CollisionFreeVectorTable
+                      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_ValueAccessorMultiplexer
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_BarrieredReadWriteConcurrentBitVector
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_ColumnStoreUtil
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogRelationSchema
@@ -627,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
@@ -731,16 +708,19 @@ target_link_libraries(quickstep_storage_HashTable
                       quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_HashTableBase
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_HashTable_proto
                       quickstep_types_Type_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_storage_HashTableFactory
                       glog
+                      quickstep_storage_CollisionFreeVectorTable
                       quickstep_storage_HashTable
                       quickstep_storage_HashTable_proto
                       quickstep_storage_HashTableBase
                       quickstep_storage_LinearOpenAddressingHashTable
+                      quickstep_storage_PackedPayloadHashTable
                       quickstep_storage_SeparateChainingHashTable
                       quickstep_storage_SimpleScalarSeparateChainingHashTable
                       quickstep_storage_TupleReference
@@ -759,13 +739,10 @@ target_link_libraries(quickstep_storage_HashTableKeyManager
                       quickstep_utility_Macros)
 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)
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_IndexSubBlock
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_predicate_PredicateCost
@@ -820,14 +797,32 @@ target_link_libraries(quickstep_storage_LinearOpenAddressingHashTable
                       quickstep_utility_Alignment
                       quickstep_utility_Macros
                       quickstep_utility_PrimeNumber)
-target_link_libraries(quickstep_storage_PartitionedHashTablePool
-                      glog
+target_link_libraries(quickstep_storage_PackedPayloadHashTable
+                      quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_FastHashTableFactory
                       quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableKeyManager
+                      quickstep_storage_StorageBlob
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_StorageManager
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorMultiplexer
+                      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_StringUtil)
+                      quickstep_utility_PrimeNumber)
+target_link_libraries(quickstep_storage_PartitionedHashTablePool
+                      glog
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_HashTableFactory
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_PreloaderThread
                       glog
                       quickstep_catalog_CatalogDatabase
@@ -936,7 +931,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
@@ -945,7 +939,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
@@ -1068,6 +1061,10 @@ target_link_libraries(quickstep_storage_ValueAccessor
                       quickstep_types_TypedValue
                       quickstep_types_containers_Tuple
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_storage_ValueAccessorMultiplexer
+                      glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_ValueAccessorUtil
                       glog
                       quickstep_storage_BasicColumnStoreValueAccessor
@@ -1115,6 +1112,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_BasicColumnStoreValueAccessor
                       quickstep_storage_BloomFilterIndexSubBlock
                       quickstep_storage_CSBTreeIndexSubBlock
+                      quickstep_storage_CollisionFreeVectorTable
                       quickstep_storage_ColumnStoreUtil
                       quickstep_storage_CompressedBlockBuilder
                       quickstep_storage_CompressedColumnStoreTupleStorageSubBlock
@@ -1125,9 +1123,6 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_CompressedTupleStorageSubBlock
                       quickstep_storage_CountedReference
                       quickstep_storage_EvictionPolicy
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_FastHashTableFactory
-                      quickstep_storage_FastSeparateChainingHashTable
                       quickstep_storage_FileManager
                       quickstep_storage_FileManagerLocal
                       quickstep_storage_Flags
@@ -1144,6 +1139,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_InsertDestination_proto
                       quickstep_storage_LinearOpenAddressingHashTable
                       quickstep_storage_PartitionedHashTablePool
+                      quickstep_storage_PackedPayloadHashTable
                       quickstep_storage_PreloaderThread
                       quickstep_storage_SMAIndexSubBlock
                       quickstep_storage_SeparateChainingHashTable
@@ -1166,6 +1162,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_TupleReference
                       quickstep_storage_TupleStorageSubBlock
                       quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_storage_ValueAccessorUtil
                       quickstep_storage_WindowAggregationOperationState
                       quickstep_storage_WindowAggregationOperationState_proto)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/CollisionFreeVectorTable.cpp
----------------------------------------------------------------------
diff --git a/storage/CollisionFreeVectorTable.cpp b/storage/CollisionFreeVectorTable.cpp
new file mode 100644
index 0000000..d836014
--- /dev/null
+++ b/storage/CollisionFreeVectorTable.cpp
@@ -0,0 +1,285 @@
+/**
+ * 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/CollisionFreeVectorTable.hpp"
+
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <cstdlib>
+#include <memory>
+#include <vector>
+
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/BarrieredReadWriteConcurrentBitVector.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+CollisionFreeVectorTable::CollisionFreeVectorTable(
+    const Type *key_type,
+    const std::size_t num_entries,
+    const std::vector<AggregationHandle *> &handles,
+    StorageManager *storage_manager)
+    : key_type_(key_type),
+      num_entries_(num_entries),
+      num_handles_(handles.size()),
+      handles_(handles),
+      num_finalize_partitions_(CalculateNumFinalizationPartitions(num_entries_)),
+      storage_manager_(storage_manager) {
+  DCHECK_GT(num_entries, 0u);
+
+  std::size_t required_memory = 0;
+  const std::size_t existence_map_offset = 0;
+  std::vector<std::size_t> state_offsets;
+
+  required_memory += CacheLineAlignedBytes(
+      BarrieredReadWriteConcurrentBitVector::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: {
+        DCHECK_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";
+    }
+
+    state_offsets.emplace_back(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 BarrieredReadWriteConcurrentBitVector(
+      reinterpret_cast<char *>(memory_start) + existence_map_offset,
+      num_entries,
+      false /* initialize */));
+
+  for (std::size_t i = 0; i < num_handles_; ++i) {
+    // Columnwise layout.
+    vec_tables_.emplace_back(
+        reinterpret_cast<char *>(memory_start) + state_offsets.at(i));
+  }
+
+  memory_size_ = required_memory;
+  num_init_partitions_ = CalculateNumInitializationPartitions(memory_size_);
+}
+
+CollisionFreeVectorTable::~CollisionFreeVectorTable() {
+  const block_id blob_id = blob_->getID();
+  blob_.release();
+  storage_manager_->deleteBlockOrBlobFile(blob_id);
+}
+
+void CollisionFreeVectorTable::destroyPayload() {
+}
+
+bool CollisionFreeVectorTable::upsertValueAccessorCompositeKey(
+    const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+    const std::vector<MultiSourceAttributeId> &key_ids,
+    const ValueAccessorMultiplexer &accessor_mux) {
+  DCHECK_EQ(1u, key_ids.size());
+
+  if (handles_.empty()) {
+    InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        accessor_mux.getValueAccessorBySource(key_ids.front().source),
+        [&key_ids, this](auto *accessor) -> void {  // NOLINT(build/c++11)
+      this->upsertValueAccessorKeyOnlyHelper(key_type_->isNullable(),
+                                             key_type_,
+                                             key_ids.front().attr_id,
+                                             accessor);
+    });
+    return true;
+  }
+
+  DCHECK(accessor_mux.getDerivedAccessor() == nullptr ||
+         accessor_mux.getDerivedAccessor()->getImplementationType()
+             == ValueAccessor::Implementation::kColumnVectors);
+
+  ValueAccessor *base_accessor = accessor_mux.getBaseAccessor();
+  ColumnVectorsValueAccessor *derived_accesor =
+      static_cast<ColumnVectorsValueAccessor *>(accessor_mux.getDerivedAccessor());
+
+  // Dispatch to specialized implementations to achieve maximum performance.
+  InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+      base_accessor,
+      [&argument_ids, &key_ids, &derived_accesor, this](auto *accessor) -> void {  // NOLINT(build/c++11)
+    const ValueAccessorSource key_source = key_ids.front().source;
+    const attribute_id key_id = key_ids.front().attr_id;
+    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 AggregationHandle *handle = handles_[i];
+      const auto &argument_types = handle->getArgumentTypes();
+      const auto &argument_ids_i = argument_ids[i];
+
+      ValueAccessorSource argument_source;
+      attribute_id argument_id;
+      const Type *argument_type;
+      bool is_argument_nullable;
+
+      if (argument_ids_i.empty()) {
+        argument_source = ValueAccessorSource::kInvalid;
+        argument_id = kInvalidAttributeID;
+
+        DCHECK(argument_types.empty());
+        argument_type = nullptr;
+        is_argument_nullable = false;
+      } else {
+        DCHECK_EQ(1u, argument_ids_i.size());
+        argument_source = argument_ids_i.front().source;
+        argument_id = argument_ids_i.front().attr_id;
+
+        DCHECK_EQ(1u, argument_types.size());
+        argument_type = argument_types.front();
+        is_argument_nullable = argument_type->isNullable();
+      }
+
+      if (key_source == ValueAccessorSource::kBase) {
+        if (argument_source == ValueAccessorSource::kBase) {
+          this->upsertValueAccessorDispatchHelper<false>(is_key_nullable,
+                                                         is_argument_nullable,
+                                                         key_type_,
+                                                         argument_type,
+                                                         handle->getAggregationID(),
+                                                         key_id,
+                                                         argument_id,
+                                                         vec_tables_[i],
+                                                         accessor,
+                                                         accessor);
+        } else {
+          this->upsertValueAccessorDispatchHelper<true>(is_key_nullable,
+                                                        is_argument_nullable,
+                                                        key_type_,
+                                                        argument_type,
+                                                        handle->getAggregationID(),
+                                                        key_id,
+                                                        argument_id,
+                                                        vec_tables_[i],
+                                                        accessor,
+                                                        derived_accesor);
+        }
+      } else {
+        if (argument_source == ValueAccessorSource::kBase) {
+          this->upsertValueAccessorDispatchHelper<true>(is_key_nullable,
+                                                        is_argument_nullable,
+                                                        key_type_,
+                                                        argument_type,
+                                                        handle->getAggregationID(),
+                                                        key_id,
+                                                        argument_id,
+                                                        vec_tables_[i],
+                                                        derived_accesor,
+                                                        accessor);
+        } else {
+          this->upsertValueAccessorDispatchHelper<false>(is_key_nullable,
+                                                         is_argument_nullable,
+                                                         key_type_,
+                                                         argument_type,
+                                                         handle->getAggregationID(),
+                                                         key_id,
+                                                         argument_id,
+                                                         vec_tables_[i],
+                                                         derived_accesor,
+                                                         derived_accesor);
+        }
+      }
+    }
+  });
+  return true;
+}
+
+void CollisionFreeVectorTable::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 CollisionFreeVectorTable::finalizeState(const std::size_t partition_id,
+                                             const 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


[17/18] incubator-quickstep git commit: Removed the temp query result relation in the distributed version.

Posted by zu...@apache.org.
Removed the temp query result relation in the distributed version.


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

Branch: refs/heads/tmb_poll_interval
Commit: aa7f6fe4e07804524aca0f1574935ae3f73c985d
Parents: dd8747f
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Feb 8 00:33:31 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Feb 8 00:35:53 2017 -0800

----------------------------------------------------------------------
 cli/distributed/CMakeLists.txt               |  1 +
 cli/distributed/Cli.cpp                      | 20 +++++++++++++++++++-
 cli/distributed/Conductor.cpp                | 13 ++++++++++++-
 cli/distributed/Conductor.hpp                |  4 ++++
 query_execution/QueryExecutionMessages.proto |  4 ++++
 query_execution/QueryExecutionTypedefs.hpp   |  2 ++
 6 files changed, 42 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/cli/distributed/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/distributed/CMakeLists.txt b/cli/distributed/CMakeLists.txt
index b46082f..5804321 100644
--- a/cli/distributed/CMakeLists.txt
+++ b/cli/distributed/CMakeLists.txt
@@ -25,6 +25,7 @@ add_library(quickstep_cli_distributed_Role Role.cpp Role.hpp)
 # Link dependencies:
 target_link_libraries(quickstep_cli_distributed_Conductor
                       glog
+                      quickstep_catalog_CatalogDatabase
                       quickstep_cli_DefaultsConfigurator
                       quickstep_cli_Flags
                       quickstep_cli_distributed_Role

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/cli/distributed/Cli.cpp
----------------------------------------------------------------------
diff --git a/cli/distributed/Cli.cpp b/cli/distributed/Cli.cpp
index 5af70e6..386654d 100644
--- a/cli/distributed/Cli.cpp
+++ b/cli/distributed/Cli.cpp
@@ -122,7 +122,10 @@ void Cli::init() {
 
   // Prepare for submitting a query.
   bus_.RegisterClientAsSender(cli_id_, kSqlQueryMessage);
+
   bus_.RegisterClientAsReceiver(cli_id_, kQueryExecutionSuccessMessage);
+  bus_.RegisterClientAsSender(cli_id_, kQueryResultTeardownMessage);
+
   bus_.RegisterClientAsReceiver(cli_id_, kQueryExecutionErrorMessage);
 }
 
@@ -191,7 +194,7 @@ void Cli::run() {
             CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
 
             if (proto.has_result_relation()) {
-              CatalogRelation result_relation(proto.result_relation());
+              const CatalogRelation result_relation(proto.result_relation());
 
               PrintToScreen::PrintRelation(result_relation, storage_manager_.get(), stdout);
 
@@ -199,6 +202,21 @@ void Cli::run() {
               for (const block_id block : blocks) {
                 storage_manager_->deleteBlockOrBlobFile(block);
               }
+
+              // Notify Conductor to remove the temp query result relation in the Catalog.
+              S::QueryResultTeardownMessage proto_response;
+              proto_response.set_relation_id(result_relation.getID());
+
+              const size_t proto_response_length = proto_response.ByteSize();
+              char *proto_response_bytes = static_cast<char*>(malloc(proto_response_length));
+              CHECK(proto_response.SerializeToArray(proto_response_bytes, proto_response_length));
+
+              TaggedMessage response_message(static_cast<const void*>(proto_response_bytes),
+                                             proto_response_length,
+                                             kQueryResultTeardownMessage);
+              free(proto_response_bytes);
+
+              QueryExecutionUtil::SendTMBMessage(&bus_, cli_id_, conductor_client_id_, move(response_message));
             }
 
             std::chrono::duration<double, std::milli> time_in_ms = end - start;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/cli/distributed/Conductor.cpp
----------------------------------------------------------------------
diff --git a/cli/distributed/Conductor.cpp b/cli/distributed/Conductor.cpp
index 13d4d57..cf2eb4b 100644
--- a/cli/distributed/Conductor.cpp
+++ b/cli/distributed/Conductor.cpp
@@ -28,6 +28,7 @@
 #include <string>
 #include <utility>
 
+#include "catalog/CatalogDatabase.hpp"
 #include "cli/DefaultsConfigurator.hpp"
 #include "cli/Flags.hpp"
 #include "parser/ParseStatement.hpp"
@@ -73,6 +74,7 @@ void Conductor::init() {
     }
 
     query_processor_ = make_unique<QueryProcessor>(move(catalog_path));
+    catalog_database_ = query_processor_->getDefaultDatabase();
   } catch (const std::exception &e) {
     LOG(FATAL) << "FATAL ERROR DURING STARTUP: " << e.what()
                << "\nIf you intended to create a new database, "
@@ -93,12 +95,14 @@ void Conductor::init() {
   bus_.RegisterClientAsSender(conductor_client_id_, kQueryExecutionErrorMessage);
   bus_.RegisterClientAsSender(conductor_client_id_, kAdmitRequestMessage);
 
+  bus_.RegisterClientAsReceiver(conductor_client_id_, kQueryResultTeardownMessage);
+
   block_locator_ = make_unique<BlockLocator>(&bus_);
   block_locator_->start();
 
   foreman_ = make_unique<ForemanDistributed>(*block_locator_,
                                              std::bind(&QueryProcessor::saveCatalog, query_processor_.get()), &bus_,
-                                             query_processor_->getDefaultDatabase());
+                                             catalog_database_);
   foreman_->start();
 }
 
@@ -129,6 +133,13 @@ void Conductor::run() {
         processSqlQueryMessage(sender, new string(move(proto.sql_query())));
         break;
       }
+      case kQueryResultTeardownMessage: {
+        S::QueryResultTeardownMessage proto;
+        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+        catalog_database_->dropRelationById(proto.relation_id());
+        break;
+      }
       default:
         LOG(FATAL) << "Unknown TMB message type";
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/cli/distributed/Conductor.hpp
----------------------------------------------------------------------
diff --git a/cli/distributed/Conductor.hpp b/cli/distributed/Conductor.hpp
index e8c9582..09bf2b9 100644
--- a/cli/distributed/Conductor.hpp
+++ b/cli/distributed/Conductor.hpp
@@ -34,6 +34,8 @@
 
 namespace quickstep {
 
+class CatalogDatabase;
+
 /** \addtogroup CliDistributed
  *  @{
  */
@@ -63,6 +65,8 @@ class Conductor final : public Role {
   SqlParserWrapper parser_wrapper_;
 
   std::unique_ptr<QueryProcessor> query_processor_;
+  // Not owned.
+  CatalogDatabase *catalog_database_;
 
   tmb::client_id conductor_client_id_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 115a9a3..68f286d 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -138,6 +138,10 @@ message QueryExecutionSuccessMessage {
   optional CatalogRelationSchema result_relation = 1;
 }
 
+message QueryResultTeardownMessage {
+  required int32 relation_id = 1;
+}
+
 message QueryExecutionErrorMessage {
   required string error_message = 1;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/aa7f6fe4/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index 9f78302..994bd60 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -105,6 +105,8 @@ enum QueryExecutionMessageType : message_type_id {
   kQueryExecutionSuccessMessage,
   kQueryExecutionErrorMessage,
 
+  kQueryResultTeardownMessage,  // From CLI to Conductor.
+
   // BlockLocator related messages, sorted in a life cycle of StorageManager
   // with a unique block domain.
   kBlockDomainRegistrationMessage,  // From Worker to BlockLocator.


[08/18] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/CollisionFreeVectorTable.hpp
----------------------------------------------------------------------
diff --git a/storage/CollisionFreeVectorTable.hpp b/storage/CollisionFreeVectorTable.hpp
new file mode 100644
index 0000000..4f3e238
--- /dev/null
+++ b/storage/CollisionFreeVectorTable.hpp
@@ -0,0 +1,730 @@
+/**
+ * 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 <algorithm>
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <cstring>
+#include <memory>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "utility/BarrieredReadWriteConcurrentBitVector.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class AggregationHandle;
+class StorageManager;
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+class CollisionFreeVectorTable : public AggregationStateHashTableBase {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param key_type The group-by key type.
+   * @param num_entries The estimated number of entries this table will hold.
+   * @param handles The aggregation handles.
+   * @param storage_manager The StorageManager to use (a StorageBlob will be
+   *        allocated to hold this table's contents).
+   **/
+  CollisionFreeVectorTable(
+      const Type *key_type,
+      const std::size_t num_entries,
+      const std::vector<AggregationHandle *> &handles,
+      StorageManager *storage_manager);
+
+  ~CollisionFreeVectorTable() override;
+
+  void destroyPayload() override;
+
+  /**
+   * @brief Get the number of partitions to be used for initializing the table.
+   *
+   * @return The number of partitions to be used for initializing the table.
+   */
+  inline std::size_t getNumInitializationPartitions() const {
+    return num_init_partitions_;
+  }
+
+  /**
+   * @brief Get the number of partitions to be used for finalizing the aggregation.
+   *
+   * @return The number of partitions to be used for finalizing the aggregation.
+   */
+  inline std::size_t getNumFinalizationPartitions() const {
+    return num_finalize_partitions_;
+  }
+
+  /**
+   * @brief Get the exact number of tuples in the specified finalization partition.
+   *
+   * @return The exact number of tuples in the specified finalization partition.
+   */
+  inline std::size_t getNumTuplesInFinalizationPartition(
+      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_->onesCountInRange(start_position, end_position);
+  }
+
+  /**
+   * @brief Initialize the specified partition of this aggregation table.
+   *
+   * @param partition_id ID of the partition to be initialized.
+   */
+  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));
+  }
+
+  /**
+   * @brief Use aggregation handles to update (multiple) aggregation states in
+   *        this vector table, with group-by keys and arguments drawn from the
+   *        given ValueAccessors.
+   *
+   * @param argument_ids The multi-source attribute IDs of each argument
+   *        component to be read from \p accessor_mux.
+   * @param key_ids The multi-source attribute IDs of each group-by key
+   *        component to be read from \p accessor_mux.
+   * @param accessor_mux A ValueAccessorMultiplexer object that contains the
+   *        ValueAccessors which will be used to access keys. beginIteration()
+   *        should be called on the accessors before calling this method.
+   * @return Always return true.
+   **/
+  bool upsertValueAccessorCompositeKey(
+      const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      const ValueAccessorMultiplexer &accessor_mux) override;
+
+  /**
+   * @brief Copy the keys from this table to a NativeColumnVector, for the
+   *        specified partition.
+   *
+   * @param partition_id ID of the partition to copy keys from.
+   * @param output_cv The NativeColumnVector to copy keys to.
+   */
+  void finalizeKey(const std::size_t partition_id,
+                   NativeColumnVector *output_cv) const;
+
+
+  /**
+   * @brief Finalize the aggregation states to a NativeColumnVector, for the
+   *        specified partition and aggregation handle.
+   *
+   * @param partition_id ID of the partition to finalize.
+   * @param handle_id ID of the aggregation handle to finalize.
+   * @param output_cv The NativeColumnVector to write finalized values to.
+   */
+  void finalizeState(const std::size_t partition_id,
+                     const 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 static std::size_t CalculateNumInitializationPartitions(
+      const std::size_t memory_size) {
+    // Set initialization memory block size as 4MB.
+    constexpr std::size_t kInitBlockSize = 4uL * 1024u * 1024u;
+
+    // At least 1 partition, at most 80 partitions.
+    // TODO(jianqiao): set the upbound as (# of workers * 2) instead of the
+    // hardcoded 80.
+    return std::max(1uL, std::min(memory_size / kInitBlockSize, 80uL));
+  }
+
+  inline static std::size_t CalculateNumFinalizationPartitions(
+      const std::size_t num_entries) {
+    // Set finalization segment size as 4096 entries.
+    constexpr std::size_t kFinalizeSegmentSize = 4uL * 1024L;
+
+    // At least 1 partition, at most 80 partitions.
+    // TODO(jianqiao): set the upbound as (# of workers * 2) instead of the
+    // hardcoded 80.
+    return std::max(1uL, std::min(num_entries / kFinalizeSegmentSize, 80uL));
+  }
+
+  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 <typename ...ArgTypes>
+  inline void upsertValueAccessorKeyOnlyHelper(
+      const bool is_key_nullable,
+      const Type *key_type,
+      ArgTypes &&...args);
+
+  template <bool is_key_nullable, typename KeyT, typename KeyValueAccessorT>
+  inline void upsertValueAccessorKeyOnly(
+      const attribute_id key_attr_id,
+      KeyValueAccessorT *key_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;
+
+  template <typename ...ArgTypes>
+  inline void finalizeStateDispatchHelper(const AggregationID agg_id,
+                                          const Type *argument_type,
+                                          const void *vec_table,
+                                          ArgTypes &&...args) const;
+
+  template <typename ...ArgTypes>
+  inline void finalizeStateSumHelper(const Type *argument_type,
+                                     const void *vec_table,
+                                     ArgTypes &&...args) const;
+
+  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;
+
+  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;
+
+  const Type *key_type_;
+  const std::size_t num_entries_;
+
+  const std::size_t num_handles_;
+  const std::vector<AggregationHandle *> handles_;
+
+  std::unique_ptr<BarrieredReadWriteConcurrentBitVector> 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(CollisionFreeVectorTable);
+};
+
+// ----------------------------------------------------------------------------
+// Implementations of template methods follow.
+
+template <bool use_two_accessors, typename ...ArgTypes>
+inline void CollisionFreeVectorTable
+    ::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 CollisionFreeVectorTable
+    ::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 CollisionFreeVectorTable
+    ::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 <typename ...ArgTypes>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorKeyOnlyHelper(const bool is_key_nullable,
+                                       const Type *key_type,
+                                       ArgTypes &&...args) {
+  switch (key_type->getTypeID()) {
+    case TypeID::kInt: {
+      if (is_key_nullable) {
+        upsertValueAccessorKeyOnly<true, int>(std::forward<ArgTypes>(args)...);
+      } else {
+        upsertValueAccessorKeyOnly<false, int>(std::forward<ArgTypes>(args)...);
+      }
+      return;
+    }
+    case TypeID::kLong: {
+      if (is_key_nullable) {
+        upsertValueAccessorKeyOnly<true, std::int64_t>(std::forward<ArgTypes>(args)...);
+      } else {
+        upsertValueAccessorKeyOnly<false, std::int64_t>(std::forward<ArgTypes>(args)...);
+      }
+      return;
+    }
+    default:
+      LOG(FATAL) << "Not supported";
+  }
+}
+
+template <bool is_key_nullable, typename KeyT, typename ValueAccessorT>
+inline void CollisionFreeVectorTable
+    ::upsertValueAccessorKeyOnly(const attribute_id key_attr_id,
+                                 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;
+    }
+    existence_map_->setBit(*key);
+  }
+}
+
+template <bool use_two_accessors, bool is_key_nullable, bool is_argument_nullable,
+          typename KeyT, typename KeyValueAccessorT, typename ArgumentValueAccessorT>
+inline void CollisionFreeVectorTable
+    ::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, 0);
+
+  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 CollisionFreeVectorTable
+    ::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, 0);
+  DCHECK_GE(argument_id, 0);
+  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 CollisionFreeVectorTable
+    ::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 CollisionFreeVectorTable
+    ::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();
+  if (use_two_accessors) {
+    argument_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 CollisionFreeVectorTable
+    ::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();
+  if (use_two_accessors) {
+    argument_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 CollisionFreeVectorTable
+    ::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();
+  if (use_two_accessors) {
+    argument_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)) {}
+  }
+}
+
+template <typename KeyT>
+inline void CollisionFreeVectorTable
+    ::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 CollisionFreeVectorTable
+    ::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 CollisionFreeVectorTable
+    ::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 CollisionFreeVectorTable
+    ::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 CollisionFreeVectorTable
+    ::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);
+  }
+}
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_COLLISION_FREE_AGGREGATION_STATE_HASH_TABLE_HPP_


[07/18] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/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>


[13/18] incubator-quickstep git commit: Reduce the number of group-by attributes by pulling tables up aggregations

Posted by zu...@apache.org.
Reduce the number of group-by attributes by pulling tables up aggregations


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

Branch: refs/heads/tmb_poll_interval
Commit: f46ae1512e5b4665b855d9b3201d42773fa899e8
Parents: 2d89e4f
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sun Jan 29 18:36:14 2017 -0600
Committer: Jignesh Patel <jm...@hotmail.com>
Committed: Tue Feb 7 17:45:58 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/Optimizer.cpp                   |   3 +-
 query_optimizer/Optimizer.hpp                   |   2 -
 query_optimizer/PhysicalGenerator.cpp           |   3 +
 query_optimizer/PhysicalGenerator.hpp           |  11 +-
 query_optimizer/rules/CMakeLists.txt            |  23 ++
 .../rules/ReduceGroupByAttributes.cpp           | 217 +++++++++++++++++++
 .../rules/ReduceGroupByAttributes.hpp           | 143 ++++++++++++
 query_optimizer/tests/OptimizerTest.cpp         |   2 +-
 .../tests/OptimizerTextTestRunner.cpp           |   7 +-
 .../tests/OptimizerTextTestRunner.hpp           |   3 +-
 11 files changed, 405 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 8b8fa3c..e750a1e 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -216,6 +216,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_rules_InjectJoinFilters
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
+                      quickstep_queryoptimizer_rules_ReduceGroupByAttributes
                       quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/Optimizer.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/Optimizer.cpp b/query_optimizer/Optimizer.cpp
index b14c938..1b91574 100644
--- a/query_optimizer/Optimizer.cpp
+++ b/query_optimizer/Optimizer.cpp
@@ -30,10 +30,11 @@ void Optimizer::generateQueryHandle(const ParseStatement &parse_statement,
                                     OptimizerContext *optimizer_context,
                                     QueryHandle *query_handle) {
   LogicalGenerator logical_generator(optimizer_context);
+  PhysicalGenerator physical_generator(optimizer_context);
   ExecutionGenerator execution_generator(catalog_database, query_handle);
 
   execution_generator.generatePlan(
-      physical_generator_.generatePlan(
+      physical_generator.generatePlan(
           logical_generator.generatePlan(*catalog_database, parse_statement)));
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/Optimizer.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/Optimizer.hpp b/query_optimizer/Optimizer.hpp
index 36f956a..227dd04 100644
--- a/query_optimizer/Optimizer.hpp
+++ b/query_optimizer/Optimizer.hpp
@@ -70,8 +70,6 @@ class Optimizer {
                            QueryHandle *query_handle);
 
  private:
-  PhysicalGenerator physical_generator_;
-
   DISALLOW_COPY_AND_ASSIGN(Optimizer);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 5dc0ffb..1b68f49 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -30,6 +30,7 @@
 #include "query_optimizer/rules/InjectJoinFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
 #include "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp"
+#include "query_optimizer/rules/ReduceGroupByAttributes.hpp"
 #include "query_optimizer/rules/ReorderColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
@@ -127,6 +128,8 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   // general FusePhysical optimization) in the future.
   rules.emplace_back(new PushDownLowCostDisjunctivePredicate());
 
+  rules.emplace_back(new ReduceGroupByAttributes(optimizer_context_));
+
   if (FLAGS_reorder_hash_joins) {
     rules.emplace_back(new StarSchemaHashJoinOrderOptimization());
     rules.emplace_back(new PruneColumns());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/PhysicalGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.hpp b/query_optimizer/PhysicalGenerator.hpp
index 886a173..42fea86 100644
--- a/query_optimizer/PhysicalGenerator.hpp
+++ b/query_optimizer/PhysicalGenerator.hpp
@@ -33,6 +33,8 @@
 namespace quickstep {
 namespace optimizer {
 
+class OptimizerContext;
+
 /** \addtogroup QueryOptimizer
  *  @{
  */
@@ -43,9 +45,12 @@ namespace optimizer {
 class PhysicalGenerator : public LogicalToPhysicalMapper {
  public:
   /**
-   * @brief Constructor
+   * @brief Constructor.
+   *
+   * @param optimizer_context The optimizer context.
    */
-  PhysicalGenerator() {
+  explicit PhysicalGenerator(OptimizerContext *optimizer_context)
+      : optimizer_context_(optimizer_context) {
     createStrategies();
   }
 
@@ -125,6 +130,8 @@ class PhysicalGenerator : public LogicalToPhysicalMapper {
    */
   std::unordered_map<logical::LogicalPtr, physical::PhysicalPtr> logical_to_physical_map_;
 
+  OptimizerContext *optimizer_context_;
+
   /**
    * @brief The complete physical plan.
    */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 223c78c..029d816 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -29,6 +29,9 @@ 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_ReduceGroupByAttributes
+            ReduceGroupByAttributes.cpp
+            ReduceGroupByAttributes.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)
@@ -143,6 +146,25 @@ target_link_libraries(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_logical_PatternMatcher
                       quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_ReduceGroupByAttributes
+                      ${GFLAGS_LIB_NAME}
+                      quickstep_catalog_CatalogRelation
+                      quickstep_queryoptimizer_OptimizerContext
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_TableReference
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_PruneColumns
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ExprId
@@ -272,6 +294,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
+                      quickstep_queryoptimizer_rules_ReduceGroupByAttributes
                       quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/rules/ReduceGroupByAttributes.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReduceGroupByAttributes.cpp b/query_optimizer/rules/ReduceGroupByAttributes.cpp
new file mode 100644
index 0000000..dcdd27a
--- /dev/null
+++ b/query_optimizer/rules/ReduceGroupByAttributes.cpp
@@ -0,0 +1,217 @@
+/**
+ * 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/ReduceGroupByAttributes.hpp"
+
+#include <algorithm>
+#include <map>
+#include <vector>
+#include <unordered_set>
+#include <utility>
+
+#include "catalog/CatalogRelation.hpp"
+#include "query_optimizer/OptimizerContext.hpp"
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/physical/Aggregate.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/TableReference.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/rules/PruneColumns.hpp"
+
+#include "gflags/gflags.h"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+DEFINE_uint64(reduce_group_by_attributes_threshold, 3u,
+              "The threshold for a stored relation's number of attributes in a "
+              "group-by clause for the ReduceGroupByAttributes optimization "
+              "rule to pull the stored relation up the aggregation");
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr ReduceGroupByAttributes::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+  cost_model_.reset(new cost::StarSchemaSimpleCostModel(
+      std::static_pointer_cast<const P::TopLevelPlan>(input)->shared_subplans()));
+
+  P::PhysicalPtr output = applyInternal(input);
+  if (output != input) {
+    output = PruneColumns().apply(output);
+  }
+  return output;
+}
+
+P::PhysicalPtr ReduceGroupByAttributes::applyInternal(const P::PhysicalPtr &input) {
+  std::vector<P::PhysicalPtr> new_children;
+  for (const P::PhysicalPtr &child : input->children()) {
+    new_children.push_back(applyInternal(child));
+  }
+
+  if (new_children != input->children()) {
+    return applyToNode(input->copyWithNewChildren(new_children));
+  } else {
+    return applyToNode(input);
+  }
+}
+
+P::PhysicalPtr ReduceGroupByAttributes::applyToNode(const P::PhysicalPtr &input) {
+  P::TableReferencePtr table_reference;
+  if (P::SomeTableReference::MatchesWithConditionalCast(input, &table_reference)) {
+    // Collect the attributes-to-TableReference mapping info.
+    for (const auto &attr : table_reference->attribute_list()) {
+      source_.emplace(attr->id(), std::make_pair(table_reference, attr));
+    }
+    return input;
+  }
+
+  P::AggregatePtr aggregate;
+  if (!P::SomeAggregate::MatchesWithConditionalCast(input, &aggregate) ||
+      aggregate->grouping_expressions().size() <= 1u) {
+    return input;
+  }
+
+  // Divide the group-by attributes into groups based on their source table.
+  std::map<P::TableReferencePtr, std::vector<E::AttributeReferencePtr>> table_attributes;
+  for (const auto &expr : aggregate->grouping_expressions()) {
+    const auto source_it = source_.find(expr->id());
+    if (source_it != source_.end()) {
+      table_attributes[source_it->second.first].emplace_back(source_it->second.second);
+    }
+  }
+
+  std::unordered_set<E::ExprId> erased_grouping_attr_ids;
+  std::vector<std::pair<P::TableReferencePtr, E::AttributeReferencePtr>> hoisted_tables;
+
+  // For each group (i.e. each source table), if it is profitable then we pull
+  // the table up the aggregation.
+  for (const auto &pair : table_attributes) {
+    const P::TableReferencePtr table = pair.first;
+    const std::vector<E::AttributeReferencePtr> &attributes = pair.second;
+    // TODO(jianqiao): find a cost-based metic instead of hard-coding the threshold
+    // number of group-by attributes.
+    if (attributes.size() <= FLAGS_reduce_group_by_attributes_threshold) {
+      continue;
+    }
+
+    std::vector<AttributeInfo> attr_infos;
+    for (const auto &attr : attributes) {
+      attr_infos.emplace_back(attr,
+                              cost_model_->impliesUniqueAttributes(table, {attr}),
+                              !attr->getValueType().isVariableLength(),
+                              attr->getValueType().maximumByteLength());
+    }
+
+    std::vector<const AttributeInfo *> attr_info_refs;
+    for (const auto &info : attr_infos) {
+      attr_info_refs.emplace_back(&info);
+    }
+    std::sort(attr_info_refs.begin(),
+              attr_info_refs.end(),
+              AttributeInfo::IsBetterThan);
+
+    const AttributeInfo &best_candidate = *attr_info_refs.front();
+    if (!best_candidate.is_unique) {
+      // Cannot find a key attribute. Give up pulling this table up.
+      continue;
+    }
+
+    const E::AttributeReferencePtr key_attribute = best_candidate.attribute;
+    hoisted_tables.emplace_back(table, key_attribute);
+
+    for (const auto &attr : attributes) {
+      if (attr->id() != key_attribute->id()) {
+        erased_grouping_attr_ids.emplace(attr->id());
+      }
+    }
+  }
+
+  if (erased_grouping_attr_ids.empty()) {
+    return input;
+  }
+
+  // Reconstuct the Aggregate node with reduced group-by attributes and then
+  // construct HashJoin nodes on top of the Aggregate.
+  std::vector<E::NamedExpressionPtr> reduced_grouping_expressions;
+  for (const auto &expr : aggregate->grouping_expressions()) {
+    if (erased_grouping_attr_ids.find(expr->id()) == erased_grouping_attr_ids.end()) {
+      reduced_grouping_expressions.emplace_back(expr);
+    }
+  }
+
+  const P::AggregatePtr new_aggregate =
+      P::Aggregate::Create(aggregate->input(),
+                           reduced_grouping_expressions,
+                           aggregate->aggregate_expressions(),
+                           aggregate->filter_predicate());
+
+  P::PhysicalPtr output = new_aggregate;
+  std::vector<E::NamedExpressionPtr> project_expressions =
+      E::ToNamedExpressions(output->getOutputAttributes());
+  for (const auto &pair : hoisted_tables) {
+    const P::TableReferencePtr &source_table = pair.first;
+    const E::AttributeReferencePtr &probe_attribute = pair.second;
+
+    E::AttributeReferencePtr build_attribute;
+    std::vector<E::AttributeReferencePtr> new_attribute_list;
+    for (const auto &attr : source_table->attribute_list()) {
+      if (attr->id() == probe_attribute->id()) {
+        build_attribute =
+          E::AttributeReference::Create(optimizer_context_->nextExprId(),
+                                        attr->attribute_name(),
+                                        attr->attribute_alias(),
+                                        attr->relation_name(),
+                                        attr->getValueType(),
+                                        E::AttributeReferenceScope::kLocal);
+        new_attribute_list.emplace_back(build_attribute);
+      } else {
+        new_attribute_list.emplace_back(attr);
+        project_expressions.emplace_back(attr);
+      }
+    }
+
+    DCHECK(build_attribute != nullptr);
+    const P::TableReferencePtr build_side_table =
+        P::TableReference::Create(source_table->relation(),
+                                  source_table->relation()->getName(),
+                                  new_attribute_list);
+    output = P::HashJoin::Create(output,
+                                 build_side_table,
+                                 {probe_attribute},
+                                 {build_attribute},
+                                 nullptr,
+                                 project_expressions,
+                                 P::HashJoin::JoinType::kInnerJoin);
+  }
+
+  return output;
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/rules/ReduceGroupByAttributes.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReduceGroupByAttributes.hpp b/query_optimizer/rules/ReduceGroupByAttributes.hpp
new file mode 100644
index 0000000..5a1f295
--- /dev/null
+++ b/query_optimizer/rules/ReduceGroupByAttributes.hpp
@@ -0,0 +1,143 @@
+/**
+ * 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_REDUCE_GROUP_BY_ATTRIBUTES_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_REDUCE_GROUP_BY_ATTRIBUTES_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/TableReference.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+class OptimizerContext;
+
+/**
+ * @brief Rule that applies to a physical plan to reduce the number of group-by
+ *        attributes for Aggregate nodes (to improve performance) by pulling
+ *        joins up the aggregations.
+ *
+ * For example, let R be a relation with PRIMARY KEY x and attributes y, z. Let
+ * S be a relation with FOREIGN KEY u refering to R(x) and attribute v. Then the
+ * optimization rule will transform the physical plan:
+ *   Aggregate(
+ *     [input relation]: HashJoin(
+ *                         [probe relation]: S
+ *                         [build relation]: R
+ *                         [join expression]: S.u = R.x
+ *                         [project attributes]: v, x, y, z
+ *                       )
+ *     [aggregate expression]: SUM(v) AS sum_v
+ *     [group-by attributes]: x, y, z
+ *   )
+ *
+ * into:
+ *   HashJoin(
+ *     [probe relation]: Aggregate(
+ *                         [input relation]: S
+ *                         [aggregate expression]: SUM(v) AS sum_v
+ *                         [group-by attribute]: u
+ *                       ) AS T
+ *     [build relation]: R
+ *     [join expression]: T.u = R.x
+ *     [project attributes]: sum_v, x, y, z
+ *   )
+ */
+class ReduceGroupByAttributes : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param optimizer_context The optimizer context.
+   */
+  explicit ReduceGroupByAttributes(OptimizerContext *optimizer_context)
+      : optimizer_context_(optimizer_context) {}
+
+  ~ReduceGroupByAttributes() override {}
+
+  std::string getName() const override {
+    return "ReduceGroupByAttributes";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  struct AttributeInfo {
+    AttributeInfo(const expressions::AttributeReferencePtr &attribute_in,
+                  const bool is_unique_in,
+                  const bool is_fixed_length_in,
+                  const std::size_t maximum_size_in)
+        : attribute(attribute_in),
+          is_unique(is_unique_in),
+          is_fixed_length(is_fixed_length_in),
+          maximum_size(maximum_size_in) {}
+
+    // In the situation that there are multiple attributes that can serve as the
+    // group-by key, we define an ordering based on aggregation performance (e.g.
+    // it is faster to do aggregation with a fix-length attribute as the group-by
+    // key than with a variable-length attribute).
+    inline static bool IsBetterThan(const AttributeInfo *lhs,
+                                    const AttributeInfo *rhs) {
+      if (lhs->is_unique != rhs->is_unique) {
+        return lhs->is_unique;
+      }
+      if (lhs->is_fixed_length != rhs->is_fixed_length) {
+        return lhs->is_fixed_length;
+      }
+      if (lhs->maximum_size != rhs->maximum_size) {
+        return lhs->maximum_size < rhs->maximum_size;
+      }
+      return lhs->attribute->id() < rhs->attribute->id();
+    }
+
+    const expressions::AttributeReferencePtr attribute;
+    const bool is_unique;
+    const bool is_fixed_length;
+    const std::size_t maximum_size;
+  };
+
+  physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input);
+  physical::PhysicalPtr applyToNode(const physical::PhysicalPtr &input);
+
+  OptimizerContext *optimizer_context_;
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+
+  // Maps an attribute's id to the TableReference that generates the attribute.
+  std::unordered_map<expressions::ExprId,
+                     std::pair<physical::TableReferencePtr,
+                               expressions::AttributeReferencePtr>> source_;
+
+  DISALLOW_COPY_AND_ASSIGN(ReduceGroupByAttributes);
+};
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_REDUCE_GROUP_BY_ATTRIBUTES_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/tests/OptimizerTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTest.cpp b/query_optimizer/tests/OptimizerTest.cpp
index 3838638..7eb7a11 100644
--- a/query_optimizer/tests/OptimizerTest.cpp
+++ b/query_optimizer/tests/OptimizerTest.cpp
@@ -62,7 +62,7 @@ OptimizerTest::OptimizerTest()
       catalog_database_(
           new CatalogDatabase(catalog_.get(), "TestDatabase" /* name */, 0)),
       optimizer_context_(new OptimizerContext),
-      physical_generator_(new PhysicalGenerator()) {}
+      physical_generator_(new PhysicalGenerator(optimizer_context_.get())) {}
 
 E::AliasPtr OptimizerTest::createAlias(const E::ExpressionPtr &expression,
                                        const std::string &attribute_name,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/tests/OptimizerTextTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTestRunner.cpp b/query_optimizer/tests/OptimizerTextTestRunner.cpp
index b9238c9..cb8f153 100644
--- a/query_optimizer/tests/OptimizerTextTestRunner.cpp
+++ b/query_optimizer/tests/OptimizerTextTestRunner.cpp
@@ -80,7 +80,7 @@ void OptimizerTextTestRunner::runTestCase(const std::string &input,
       }
       if (output_physical_plan) {
         physical_plan =
-            generatePhysicalPlan(optimized_logical_plan);
+            generatePhysicalPlan(optimized_logical_plan, &optimizer_context);
         ++num_options;
       }
 
@@ -126,8 +126,9 @@ logical::LogicalPtr OptimizerTextTestRunner::generateLogicalPlan(
 }
 
 physical::PhysicalPtr OptimizerTextTestRunner::generatePhysicalPlan(
-    const logical::LogicalPtr &logical_plan) {
-  PhysicalGenerator physical_generator;
+    const logical::LogicalPtr &logical_plan,
+    OptimizerContext *optimizer_context) {
+  PhysicalGenerator physical_generator(optimizer_context);
   return physical_generator.generatePlan(logical_plan);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/tests/OptimizerTextTestRunner.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTextTestRunner.hpp b/query_optimizer/tests/OptimizerTextTestRunner.hpp
index 27fa14f..d8f604b 100644
--- a/query_optimizer/tests/OptimizerTextTestRunner.hpp
+++ b/query_optimizer/tests/OptimizerTextTestRunner.hpp
@@ -73,7 +73,8 @@ class OptimizerTextTestRunner : public TextBasedTestRunner {
                                           OptimizerContext *optimizer_context);
 
   physical::PhysicalPtr generatePhysicalPlan(
-      const logical::LogicalPtr &logical_plan);
+      const logical::LogicalPtr &logical_plan,
+      OptimizerContext *optimizer_context);
 
   SqlParserWrapper sql_parser_;
   TestDatabaseLoader test_database_loader_;


[03/18] incubator-quickstep git commit: Adds regex to specify 16.+ versions of Darwin

Posted by zu...@apache.org.
Adds regex to specify 16.+ versions of Darwin

Before this change, only one recent version of Darwin was correcly
having the compiler flags modified to allow for deprecated syscalls.
After this change, all versions of Darwin 16+ should correcly apply the
cxx flags. This is still a somewhat temporary fix because the real
issue (replacing the deprecated calls) has not been addressed.


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

Branch: refs/heads/tmb_poll_interval
Commit: ec76096bc766c1a2c0f21929b3a998c51e417208
Parents: 669a74c
Author: cramja <ma...@gmail.com>
Authored: Tue Feb 7 11:28:39 2017 -0600
Committer: cramja <ma...@gmail.com>
Committed: Tue Feb 7 14:33:41 2017 -0600

----------------------------------------------------------------------
 CMakeLists.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ec76096b/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index e74a274..918069c 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -299,7 +299,7 @@ else()
   # builds to fail. As a short-term workaround for now, we turn off deprecated
   # warnings so that they do not cause build failures anymore.
   # TODO: Remove this workaround by fixing the protobuf_cmake and glog_cmake.
-  if (${CMAKE_SYSTEM} MATCHES "Darwin-16.1.0")
+  if (${CMAKE_SYSTEM} MATCHES "Darwin-16.[0-9]*.[0-9]*")
     if (${CMAKE_CXX_COMPILER_ID} MATCHES "Clang")
       CHECK_CXX_COMPILER_FLAG("-Wno-error=deprecated-declarations" COMPILER_HAS_WNO_DEPRECATED)
       if (COMPILER_HAS_WNO_DEPRECATED)


[10/18] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp b/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
index 6e6d188..798ba76 100644
--- a/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleMin_unittest.cpp
@@ -32,8 +32,9 @@
 #include "expressions/aggregation/AggregationHandleMin.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "storage/AggregationOperationState.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/CharType.hpp"
 #include "types/DatetimeIntervalType.hpp"
 #include "types/DatetimeLit.hpp"
@@ -50,10 +51,7 @@
 #include "types/VarCharType.hpp"
 #include "types/YearMonthIntervalType.hpp"
 #include "types/containers/ColumnVector.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
-#endif
 
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
@@ -222,34 +220,6 @@ class AggregationHandleMinTest : public ::testing::Test {
   }
 
   template <typename GenericType>
-  void checkAggregationMinGenericColumnVector() {
-    const GenericType &type = GenericType::Instance(true);
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_min_->finalize(*aggregation_handle_min_state_)
-            .isNull());
-
-    typename GenericType::cpptype min;
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorGeneric<GenericType>(type, &min));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_min_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckMinValue<typename GenericType::cpptype>(
-        min, *aggregation_handle_min_, *cv_state);
-
-    aggregation_handle_min_->mergeStates(*cv_state,
-                                         aggregation_handle_min_state_.get());
-    CheckMinValue<typename GenericType::cpptype>(
-        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename GenericType>
   void checkAggregationMinGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
@@ -265,7 +235,8 @@ class AggregationHandleMinTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_min_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -277,7 +248,6 @@ class AggregationHandleMinTest : public ::testing::Test {
     CheckMinValue<typename GenericType::cpptype>(
         min, *aggregation_handle_min_, *aggregation_handle_min_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   template <typename StringType>
   void checkAggregationMinString() {
@@ -382,33 +352,6 @@ class AggregationHandleMinTest : public ::testing::Test {
   }
 
   template <typename StringType, typename ColumnVectorType>
-  void checkAggregationMinStringColumnVector() {
-    const StringType &type = StringType::Instance(10, true);
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_min_->finalize(*aggregation_handle_min_state_)
-            .isNull());
-
-    std::string min;
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorString<ColumnVectorType>(type, &min));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_min_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckMinString(min, *aggregation_handle_min_, *cv_state);
-
-    aggregation_handle_min_->mergeStates(*cv_state,
-                                         aggregation_handle_min_state_.get());
-    CheckMinString(
-        min, *aggregation_handle_min_, *aggregation_handle_min_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename StringType, typename ColumnVectorType>
   void checkAggregationMinStringValueAccessor() {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(type);
@@ -423,7 +366,8 @@ class AggregationHandleMinTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_min_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -434,7 +378,6 @@ class AggregationHandleMinTest : public ::testing::Test {
     CheckMinString(
         min, *aggregation_handle_min_, *aggregation_handle_min_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   std::unique_ptr<AggregationHandle> aggregation_handle_min_;
   std::unique_ptr<AggregationState> aggregation_handle_min_state_;
@@ -511,43 +454,6 @@ TEST_F(AggregationHandleMinTest, VarCharTypeTest) {
   checkAggregationMinString<VarCharType>();
 }
 
-TEST_F(AggregationHandleMinTest, IntTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<IntType>();
-}
-
-TEST_F(AggregationHandleMinTest, LongTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<LongType>();
-}
-
-TEST_F(AggregationHandleMinTest, FloatTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<FloatType>();
-}
-
-TEST_F(AggregationHandleMinTest, DoubleTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<DoubleType>();
-}
-
-TEST_F(AggregationHandleMinTest, DatetimeTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<DatetimeType>();
-}
-
-TEST_F(AggregationHandleMinTest, DatetimeIntervalTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<DatetimeIntervalType>();
-}
-
-TEST_F(AggregationHandleMinTest, YearMonthIntervalTypeColumnVectorTest) {
-  checkAggregationMinGenericColumnVector<YearMonthIntervalType>();
-}
-
-TEST_F(AggregationHandleMinTest, CharTypeColumnVectorTest) {
-  checkAggregationMinStringColumnVector<CharType, NativeColumnVector>();
-}
-
-TEST_F(AggregationHandleMinTest, VarCharTypeColumnVectorTest) {
-  checkAggregationMinStringColumnVector<VarCharType, IndirectColumnVector>();
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 TEST_F(AggregationHandleMinTest, IntTypeValueAccessorTest) {
   checkAggregationMinGenericValueAccessor<IntType>();
 }
@@ -583,7 +489,6 @@ TEST_F(AggregationHandleMinTest, CharTypeValueAccessorTest) {
 TEST_F(AggregationHandleMinTest, VarCharTypeValueAccessorTest) {
   checkAggregationMinStringValueAccessor<VarCharType, IndirectColumnVector>();
 }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
 #ifdef QUICKSTEP_DEBUG
 TEST_F(AggregationHandleMinDeathTest, WrongTypeTest) {
@@ -685,28 +590,25 @@ TEST_F(AggregationHandleMinTest, GroupByTableMergeTest) {
   initializeHandle(int_non_null_type);
   storage_manager_.reset(new StorageManager("./test_min_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
-          {aggregation_handle_min_.get()->getPayloadSize()},
           {aggregation_handle_min_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
-          {aggregation_handle_min_.get()->getPayloadSize()},
           {aggregation_handle_min_.get()},
           storage_manager_.get()));
 
-  AggregationStateFastHashTable *destination_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(
-          destination_hash_table.get());
+  PackedPayloadHashTable *destination_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(destination_hash_table.get());
 
-  AggregationStateFastHashTable *source_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
+  PackedPayloadHashTable *source_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(source_hash_table.get());
 
   AggregationHandleMin *aggregation_handle_min_derived =
       static_cast<AggregationHandleMin *>(aggregation_handle_min_.get());
@@ -776,47 +678,47 @@ TEST_F(AggregationHandleMinTest, GroupByTableMergeTest) {
   memcpy(buffer + 1,
          common_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_min_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(common_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          common_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_min_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+  destination_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_min_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(exclusive_source_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_min_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
-                                                      buffer);
+  destination_hash_table_derived->upsertCompositeKey(exclusive_destination_key,
+                                                     buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  AggregationOperationState::mergeGroupByHashTables(
-      source_hash_table.get(), destination_hash_table.get());
+  HashTableMerger merger(destination_hash_table_derived);
+  source_hash_table_derived->forEachCompositeKey(&merger);
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckMinValue<int>(
       common_key_source_min_val.getLiteral<int>(),
-      aggregation_handle_min_derived->finalizeHashTableEntryFast(
+      aggregation_handle_min_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(common_key) +
           1));
-  CheckMinValue<int>(exclusive_key_destination_min_val.getLiteral<int>(),
-                     aggregation_handle_min_derived->finalizeHashTableEntryFast(
-                         destination_hash_table_derived->getSingleCompositeKey(
-                             exclusive_destination_key) +
-                         1));
-  CheckMinValue<int>(exclusive_key_source_min_val.getLiteral<int>(),
-                     aggregation_handle_min_derived->finalizeHashTableEntryFast(
-                         source_hash_table_derived->getSingleCompositeKey(
-                             exclusive_source_key) +
-                         1));
+  CheckMinValue<int>(
+      exclusive_key_destination_min_val.getLiteral<int>(),
+      aggregation_handle_min_derived->finalizeHashTableEntry(
+          destination_hash_table_derived->getSingleCompositeKey(
+              exclusive_destination_key) + 1));
+  CheckMinValue<int>(
+      exclusive_key_source_min_val.getLiteral<int>(),
+      aggregation_handle_min_derived->finalizeHashTableEntry(
+          source_hash_table_derived->getSingleCompositeKey(
+              exclusive_source_key) + 1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp b/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
index 1d1c084..31a35a0 100644
--- a/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleSum_unittest.cpp
@@ -29,8 +29,9 @@
 #include "expressions/aggregation/AggregationHandleSum.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "storage/AggregationOperationState.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/CharType.hpp"
 #include "types/DatetimeIntervalType.hpp"
 #include "types/DoubleType.hpp"
@@ -45,10 +46,7 @@
 #include "types/VarCharType.hpp"
 #include "types/YearMonthIntervalType.hpp"
 #include "types/containers/ColumnVector.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
-#endif
 
 #include "gtest/gtest.h"
 
@@ -186,36 +184,6 @@ class AggregationHandleSumTest : public ::testing::Test {
   }
 
   template <typename GenericType, typename PrecisionType>
-  void checkAggregationSumGenericColumnVector() {
-    const GenericType &type = GenericType::Instance(true);
-
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_sum_->finalize(*aggregation_handle_sum_state_)
-            .isNull());
-
-    typename PrecisionType::cpptype sum;
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorGeneric<GenericType, typename PrecisionType::cpptype>(
-            type, &sum));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_sum_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckSumValue<typename PrecisionType::cpptype>(
-        sum, *aggregation_handle_sum_, *cv_state);
-
-    aggregation_handle_sum_->mergeStates(*cv_state,
-                                         aggregation_handle_sum_state_.get());
-    CheckSumValue<typename PrecisionType::cpptype>(
-        sum, *aggregation_handle_sum_, *aggregation_handle_sum_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename GenericType, typename PrecisionType>
   void checkAggregationSumGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
 
@@ -233,7 +201,8 @@ class AggregationHandleSumTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_sum_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -245,7 +214,6 @@ class AggregationHandleSumTest : public ::testing::Test {
     CheckSumValue<typename PrecisionType::cpptype>(
         sum, *aggregation_handle_sum_, *aggregation_handle_sum_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   std::unique_ptr<AggregationHandle> aggregation_handle_sum_;
   std::unique_ptr<AggregationState> aggregation_handle_sum_state_;
@@ -306,33 +274,6 @@ TEST_F(AggregationHandleSumTest, YearMonthIntervalTypeTest) {
   checkAggregationSumGeneric<YearMonthIntervalType, YearMonthIntervalType>();
 }
 
-TEST_F(AggregationHandleSumTest, IntTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<IntType, LongType>();
-}
-
-TEST_F(AggregationHandleSumTest, LongTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<LongType, LongType>();
-}
-
-TEST_F(AggregationHandleSumTest, FloatTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<FloatType, DoubleType>();
-}
-
-TEST_F(AggregationHandleSumTest, DoubleTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<DoubleType, DoubleType>();
-}
-
-TEST_F(AggregationHandleSumTest, DatetimeIntervalTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<DatetimeIntervalType,
-                                         DatetimeIntervalType>();
-}
-
-TEST_F(AggregationHandleSumTest, YearMonthIntervalTypeColumnVectorTest) {
-  checkAggregationSumGenericColumnVector<YearMonthIntervalType,
-                                         YearMonthIntervalType>();
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 TEST_F(AggregationHandleSumTest, IntTypeValueAccessorTest) {
   checkAggregationSumGenericValueAccessor<IntType, LongType>();
 }
@@ -358,7 +299,6 @@ TEST_F(AggregationHandleSumTest, YearMonthIntervalTypeValueAccessorTest) {
   checkAggregationSumGenericValueAccessor<YearMonthIntervalType,
                                           YearMonthIntervalType>();
 }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
 #ifdef QUICKSTEP_DEBUG
 TEST_F(AggregationHandleSumDeathTest, CharTypeTest) {
@@ -464,28 +404,25 @@ TEST_F(AggregationHandleSumTest, GroupByTableMergeTest) {
   initializeHandle(long_non_null_type);
   storage_manager_.reset(new StorageManager("./test_sum_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_sum_.get()->getPayloadSize()},
           {aggregation_handle_sum_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_sum_.get()->getPayloadSize()},
           {aggregation_handle_sum_.get()},
           storage_manager_.get()));
 
-  AggregationStateFastHashTable *destination_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(
-          destination_hash_table.get());
+  PackedPayloadHashTable *destination_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(destination_hash_table.get());
 
-  AggregationStateFastHashTable *source_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
+  PackedPayloadHashTable *source_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(source_hash_table.get());
 
   AggregationHandleSum *aggregation_handle_sum_derived =
       static_cast<AggregationHandleSum *>(aggregation_handle_sum_.get());
@@ -563,49 +500,47 @@ TEST_F(AggregationHandleSumTest, GroupByTableMergeTest) {
   memcpy(buffer + 1,
          common_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_sum_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(common_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          common_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_sum_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+  destination_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_sum_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(exclusive_source_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_sum_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
-                                                      buffer);
+  destination_hash_table_derived->upsertCompositeKey(exclusive_destination_key,
+                                                     buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  AggregationOperationState::mergeGroupByHashTables(
-      source_hash_table.get(), destination_hash_table.get());
+  HashTableMerger merger(destination_hash_table_derived);
+  source_hash_table_derived->forEachCompositeKey(&merger);
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckSumValue<std::int64_t>(
       common_key_merged_val.getLiteral<std::int64_t>(),
-      aggregation_handle_sum_derived->finalizeHashTableEntryFast(
+      aggregation_handle_sum_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(common_key) +
           1));
   CheckSumValue<std::int64_t>(
       exclusive_key_destination_sum_val.getLiteral<std::int64_t>(),
-      aggregation_handle_sum_derived->finalizeHashTableEntryFast(
+      aggregation_handle_sum_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(
-              exclusive_destination_key) +
-          1));
+              exclusive_destination_key) + 1));
   CheckSumValue<std::int64_t>(
       exclusive_key_source_sum_val.getLiteral<std::int64_t>(),
-      aggregation_handle_sum_derived->finalizeHashTableEntryFast(
+      aggregation_handle_sum_derived->finalizeHashTableEntry(
           source_hash_table_derived->getSingleCompositeKey(
-              exclusive_source_key) +
-          1));
+              exclusive_source_key) + 1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/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/2d89e4fb/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 7f90e11..8b8fa3c 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
@@ -125,6 +126,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_relationaloperators_DropTableOperator
                       quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_relationaloperators_HashJoinOperator
+                      quickstep_relationaloperators_InitializeAggregationOperator
                       quickstep_relationaloperators_InsertOperator
                       quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_relationaloperators_RelationalOperator
@@ -145,10 +147,12 @@ 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
                       quickstep_types_containers_Tuple_proto
+                      quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros
                       quickstep_utility_SqlError)
 if (ENABLE_DISTRIBUTED)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 6918313..1b50caa 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"
@@ -105,6 +106,7 @@
 #include "relational_operators/DropTableOperator.hpp"
 #include "relational_operators/FinalizeAggregationOperator.hpp"
 #include "relational_operators/HashJoinOperator.hpp"
+#include "relational_operators/InitializeAggregationOperator.hpp"
 #include "relational_operators/InsertOperator.hpp"
 #include "relational_operators/NestedLoopsJoinOperator.hpp"
 #include "relational_operators/RelationalOperator.hpp"
@@ -126,9 +128,11 @@
 #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"
+#include "utility/EqualsAnyConstant.hpp"
 #include "utility/SqlError.hpp"
 
 #include "gflags/gflags.h"
@@ -159,6 +163,10 @@ static const volatile bool aggregate_hashtable_type_dummy
 
 DEFINE_bool(parallelize_load, true, "Parallelize loading data files.");
 
+DEFINE_int64(collision_free_vector_table_max_size, 1000000000,
+              "The maximum allowed key range (number of entries) for using a "
+              "CollisionFreeVectorTable.");
+
 namespace E = ::quickstep::optimizer::expressions;
 namespace P = ::quickstep::optimizer::physical;
 namespace S = ::quickstep::serialization;
@@ -371,6 +379,105 @@ void ExecutionGenerator::dropAllTemporaryRelations() {
   }
 }
 
+bool ExecutionGenerator::canUseCollisionFreeAggregation(
+    const P::AggregatePtr &aggregate,
+    const std::size_t estimated_num_groups,
+    std::size_t *max_num_groups) const {
+#ifdef QUICKSTEP_DISTRIBUTED
+  // Currently we cannot do this fast path with the distributed setting. See
+  // the TODOs at InitializeAggregationOperator::getAllWorkOrderProtos() and
+  // FinalizeAggregationOperator::getAllWorkOrderProtos().
+  return false;
+#endif
+
+  // Supports only single group-by key.
+  if (aggregate->grouping_expressions().size() != 1) {
+    return false;
+  }
+
+  // We need to know the exact min/max stats of the group-by key.
+  // So it must be a CatalogAttribute (but not an expression).
+  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(jianqiao):
+  // 1. Handle the case where min_cpp_value is below 0 or far greater than 0.
+  // 2. Reason about the table size bound (e.g. by checking memory size) instead
+  //    of hardcoding it as a gflag.
+  if (min_cpp_value < 0 ||
+      max_cpp_value >= FLAGS_collision_free_vector_table_max_size ||
+      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());
+
+    if (agg_func->is_distinct()) {
+      return false;
+    }
+
+    // TODO(jianqiao): Support AggregationID::AVG.
+    if (!QUICKSTEP_EQUALS_ANY_CONSTANT(agg_func->getAggregate().getAggregationID(),
+                                       AggregationID::kCount,
+                                       AggregationID::kSum)) {
+      return false;
+    }
+
+    const auto &arguments = agg_func->getArguments();
+    if (arguments.size() > 1u) {
+      return false;
+    }
+
+    if (arguments.size() == 1u) {
+      if (!QUICKSTEP_EQUALS_ANY_CONSTANT(arguments.front()->getValueType().getTypeID(),
+                                         TypeID::kInt,
+                                         TypeID::kLong,
+                                         TypeID::kFloat,
+                                         TypeID::kDouble)) {
+        return false;
+      }
+    }
+  }
+
+  *max_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) {
@@ -1475,6 +1582,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;
@@ -1495,9 +1604,25 @@ 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 max_num_groups;
+    if (canUseCollisionFreeAggregation(physical_plan,
+                                       estimated_num_groups,
+                                       &max_num_groups)) {
+      aggr_state_proto->set_hash_table_impl_type(
+          serialization::HashTableImplType::COLLISION_FREE_VECTOR);
+      aggr_state_proto->set_estimated_num_entries(max_num_groups);
+      use_parallel_initialization = true;
+    } else {
+      // Otherwise, use SeparateChaining.
+      aggr_state_proto->set_hash_table_impl_type(
+          serialization::HashTableImplType::SEPARATE_CHAINING);
+      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()) {
@@ -1535,10 +1660,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(
@@ -1553,6 +1674,18 @@ void ExecutionGenerator::convertAggregate(
                                          false /* is_pipeline_breaker */);
   }
 
+  if (use_parallel_initialization) {
+    const QueryPlan::DAGNodeIndex initialize_aggregation_operator_index =
+        execution_plan_->addRelationalOperator(
+            new InitializeAggregationOperator(
+                query_handle_->query_id(),
+                aggr_state_index));
+
+    execution_plan_->addDirectDependency(aggregation_operator_index,
+                                         initialize_aggregation_operator_index,
+                                         true /* is_pipeline_breaker */);
+  }
+
   // 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/2d89e4fb/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index eba6eee..987f11a 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"
@@ -204,6 +206,22 @@ class ExecutionGenerator {
   std::string getNewRelationName();
 
   /**
+   * @brief Checks whether an aggregate node can be efficiently evaluated with
+   *        the collision-free aggregation fast path.
+   *
+   * @param aggregate The physical aggregate node to be checked.
+   * @param estimated_num_groups The estimated number of groups for the aggregate.
+   * @param exact_num_groups If collision-free aggregation is applicable, the
+   *        pointed content of this pointer will be set as the maximum possible
+   *        number of groups that the collision-free hash table need to hold.
+   * @return A bool value indicating whether collision-free aggregation can be
+   *         used to evaluate \p aggregate.
+   */
+  bool canUseCollisionFreeAggregation(const physical::AggregatePtr &aggregate,
+                                      const std::size_t estimated_num_groups,
+                                      std::size_t *max_num_groups) const;
+
+  /**
    * @brief Sets up the info of the CatalogRelation represented by TableReference.
    *        TableReference is not converted to any operator.
    *
@@ -427,7 +445,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/2d89e4fb/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index c18dc77..df4114d 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -47,6 +47,9 @@ add_library(quickstep_relationaloperators_FinalizeAggregationOperator
             FinalizeAggregationOperator.cpp
             FinalizeAggregationOperator.hpp)
 add_library(quickstep_relationaloperators_HashJoinOperator HashJoinOperator.cpp HashJoinOperator.hpp)
+add_library(quickstep_relationaloperators_InitializeAggregationOperator
+            InitializeAggregationOperator.cpp
+            InitializeAggregationOperator.hpp)
 add_library(quickstep_relationaloperators_InsertOperator InsertOperator.cpp InsertOperator.hpp)
 add_library(quickstep_relationaloperators_NestedLoopsJoinOperator
             NestedLoopsJoinOperator.cpp
@@ -254,6 +257,17 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_utility_lipfilter_LIPFilterAdaptiveProber
                       quickstep_utility_lipfilter_LIPFilterUtil
                       tmb)
+target_link_libraries(quickstep_relationaloperators_InitializeAggregationOperator
+                      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
@@ -548,6 +562,7 @@ target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_DropTableOperator
                       quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_relationaloperators_HashJoinOperator
+                      quickstep_relationaloperators_InitializeAggregationOperator
                       quickstep_relationaloperators_InsertOperator
                       quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_relationaloperators_RebuildWorkOrder

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/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/2d89e4fb/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 0cbf635..77b4b97 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());
+    for (std::size_t part_id = 0;
+         part_id < agg_state->getNumFinalizationPartitions();
          ++part_id) {
       container->addNormalWorkOrder(
           new FinalizeAggregationWorkOrder(
               query_id_,
+              part_id,
               agg_state,
-              query_context->getInsertDestination(output_destination_index_),
-              part_id),
+              query_context->getInsertDestination(output_destination_index_)),
           op_index_);
     }
   }
@@ -61,7 +61,7 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
 
 // TODO(quickstep-team) : Think about how the number of partitions could be
 // accessed in this function. Until then, we can't use partitioned aggregation
-// with the distributed version.
+// finalization with the distributed version.
 bool FinalizeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
   if (blocking_dependencies_met_ && !started_) {
     started_ = true;
@@ -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/2d89e4fb/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/2d89e4fb/relational_operators/InitializeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeAggregationOperator.cpp b/relational_operators/InitializeAggregationOperator.cpp
new file mode 100644
index 0000000..b1063ad
--- /dev/null
+++ b/relational_operators/InitializeAggregationOperator.cpp
@@ -0,0 +1,72 @@
+/**
+ * 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/InitializeAggregationOperator.hpp"
+
+#include <cstddef>
+
+#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 "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace quickstep {
+
+bool InitializeAggregationOperator::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 InitializeAggregationWorkOrder(query_id_,
+                                             part_id,
+                                             agg_state),
+          op_index_);
+    }
+    started_ = true;
+  }
+  return true;
+}
+
+// TODO(quickstep-team) : Think about how the number of partitions could be
+// accessed in this function. Until then, we can't use partitioned aggregation
+// initialization with the distributed version.
+bool InitializeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  LOG(FATAL) << "Not supported";
+}
+
+void InitializeAggregationWorkOrder::execute() {
+  state_->initialize(partition_id_);
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/relational_operators/InitializeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InitializeAggregationOperator.hpp b/relational_operators/InitializeAggregationOperator.hpp
new file mode 100644
index 0000000..58d848b
--- /dev/null
+++ b/relational_operators/InitializeAggregationOperator.hpp
@@ -0,0 +1,122 @@
+/**
+ * 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_OPERATOR_HPP_
+#define QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_AGGREGATION_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
+ *  @{
+ */
+
+/**
+ * @brief An operator which initializes an AggregationOperationState.
+ **/
+class InitializeAggregationOperator : public RelationalOperator {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param query_id The ID of this query.
+   * @param aggr_state_index The index of the AggregationOperationState in QueryContext.
+   **/
+  InitializeAggregationOperator(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) {}
+
+  ~InitializeAggregationOperator() override {}
+
+  std::string getName() const override {
+    return "InitializeAggregationOperator";
+  }
+
+  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(InitializeAggregationOperator);
+};
+
+/**
+ * @brief A WorkOrder produced by InitializeAggregationOperator.
+ **/
+class InitializeAggregationWorkOrder : public WorkOrder {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param query_id The ID of the query to which this operator belongs.
+   * @param partition_id The partition ID for which the work order is issued.
+   * @param state The AggregationOperationState to be initialized.
+   */
+  InitializeAggregationWorkOrder(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)) {}
+
+  ~InitializeAggregationWorkOrder() override {}
+
+  void execute() override;
+
+ private:
+  const std::size_t partition_id_;
+
+  AggregationOperationState *state_;
+
+  DISALLOW_COPY_AND_ASSIGN(InitializeAggregationWorkOrder);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_RELATIONAL_OPERATORS_INITIALIZE_AGGREGATION_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 5e8d03d..bd2a0f8 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -184,8 +184,11 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     }
     case serialization::FINALIZE_AGGREGATION: {
       LOG(INFO) << "Creating FinalizeAggregationWorkOrder in Shiftboss " << shiftboss_index;
+      // TODO(quickstep-team): Handle inner-table partitioning in the distributed
+      // setting.
       return new FinalizeAggregationWorkOrder(
           proto.query_id(),
+          0uL /* partition_id */,
           query_context->getAggregationState(proto.GetExtension(
               serialization::FinalizeAggregationWorkOrder::aggr_state_index)),
           query_context->getInsertDestination(


[02/18] incubator-quickstep git commit: Fix the bug with SingleIdentityHashFilter when filter_cardinality is 0.

Posted by zu...@apache.org.
Fix the bug with SingleIdentityHashFilter when filter_cardinality is 0.


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

Branch: refs/heads/tmb_poll_interval
Commit: 669a74c71fde158106e72fba936de2104998aa61
Parents: aef1c35
Author: jianqiao <ji...@node-2.jianqiao.quickstep-pg0.wisc.cloudlab.us>
Authored: Sun Feb 5 15:46:48 2017 -0600
Committer: jianqiao <ji...@node-2.jianqiao.quickstep-pg0.wisc.cloudlab.us>
Committed: Tue Feb 7 10:16:01 2017 -0600

----------------------------------------------------------------------
 query_optimizer/rules/AttachLIPFilters.cpp      | 3 ++-
 utility/lip_filter/SingleIdentityHashFilter.hpp | 2 +-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/669a74c7/query_optimizer/rules/AttachLIPFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachLIPFilters.cpp b/query_optimizer/rules/AttachLIPFilters.cpp
index 48b68bc..4b6ac59 100644
--- a/query_optimizer/rules/AttachLIPFilters.cpp
+++ b/query_optimizer/rules/AttachLIPFilters.cpp
@@ -19,6 +19,7 @@
 
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
 
+#include <algorithm>
 #include <map>
 #include <set>
 #include <unordered_set>
@@ -128,7 +129,7 @@ void AttachLIPFilters::attachLIPFilters(
           lip_filter_configuration_->addBuildInfo(
               P::SingleIdentityHashFilterBuildInfo::Create(
                   pair.second->source_attribute,
-                  pair.second->estimated_cardinality * 8),
+                  std::max(64uL, pair.second->estimated_cardinality * 8u)),
               pair.second->source);
           lip_filter_configuration_->addProbeInfo(
               P::LIPFilterProbeInfo::Create(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/669a74c7/utility/lip_filter/SingleIdentityHashFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/SingleIdentityHashFilter.hpp b/utility/lip_filter/SingleIdentityHashFilter.hpp
index 2823818..5c0e8a2 100644
--- a/utility/lip_filter/SingleIdentityHashFilter.hpp
+++ b/utility/lip_filter/SingleIdentityHashFilter.hpp
@@ -66,7 +66,7 @@ class SingleIdentityHashFilter : public LIPFilter {
       : LIPFilter(LIPFilterType::kSingleIdentityHashFilter),
         filter_cardinality_(filter_cardinality),
         bit_array_(GetByteSize(filter_cardinality)) {
-    DCHECK_GE(filter_cardinality, 0u);
+    DCHECK_GE(filter_cardinality, 1u);
     std::memset(bit_array_.data(),
                 0x0,
                 sizeof(std::atomic<std::uint8_t>) * GetByteSize(filter_cardinality));


[04/18] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 16ea50f..25d675c 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,11 +38,7 @@
 
 namespace quickstep {
 
-class AggregationHandle;
-class AggregationState;
 class CatalogRelationSchema;
-class ColumnVector;
-class ColumnVectorsValueAccessor;
 class InsertDestinationInterface;
 class Predicate;
 class Scalar;
@@ -431,156 +426,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 +547,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/2d89e4fb/storage/ValueAccessorMultiplexer.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessorMultiplexer.hpp b/storage/ValueAccessorMultiplexer.hpp
new file mode 100644
index 0000000..fe2fa8e
--- /dev/null
+++ b/storage/ValueAccessorMultiplexer.hpp
@@ -0,0 +1,145 @@
+/**
+ * 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_VALUE_ACCESSOR_MULTIPLEXER_HPP_
+#define QUICKSTEP_STORAGE_VALUE_ACCESSOR_MULTIPLEXER_HPP_
+
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class ValueAccessor;
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+enum class ValueAccessorSource {
+  kBase = 0,
+  kDerived,
+  kInvalid
+};
+
+/**
+ * @brief A data structure for representing attribute ids referring multiple
+ *        ValueAccessors.
+ */
+struct MultiSourceAttributeId {
+  MultiSourceAttributeId(const ValueAccessorSource in_source,
+                         const attribute_id in_attr_id)
+      : source(in_source),
+        attr_id(in_attr_id) {}
+
+  MultiSourceAttributeId(const MultiSourceAttributeId &other)
+      : source(other.source),
+        attr_id(other.attr_id) {}
+
+  const ValueAccessorSource source;
+  const attribute_id attr_id;
+};
+
+/**
+ * @brief A class that encapsulates multiple ValueAccessors and provides helper
+ *        methods for accessing the ValueAccessors with MultiSourceAttributeId.
+ *
+ * This class is in its very initial form that serves a small set of essential
+ * functionalities for the purpose of aggregation copy elision. That is, given a
+ * storage block to be aggregated on, we may have aggregations on a storage
+ * attribute (e.g. SUM(x)) or on a non-trivial expression (e.g. SUM(x * y)).
+ * For the former case, copy elision is applicable that the attribute gets accessed
+ * directly from the storage block. In the later case, we have to create a
+ * temporary data structure (i.e. ColumnVectorsValueAccessor) that stores the
+ * intermediate results. Thus, we refer to the ValueAccessor created directly
+ * from the storage block as the BASE accessor and the intermediate result
+ * ColumnVectorsValueAccessor as the DERIVED accessor. And we utilize this class
+ * (ValueAccessorMultiplexer) to pass both accessors around to enable copy elision.
+ *
+ * This class (together with ValueAccessorSource and MultiSourceAttributeId)
+ * may be rewritten or exteneded later to more generally support copy elisions
+ * in various scenarios.
+ */
+class ValueAccessorMultiplexer {
+ public:
+  /**
+   * @brief Constructor for base accessor only.
+   *
+   * @param base_accessor The base accessor.
+   */
+  explicit ValueAccessorMultiplexer(ValueAccessor *base_accessor)
+      : base_accessor_(base_accessor),
+        derived_accessor_(nullptr) {}
+
+  /**
+   * @brief Constructor.
+   *
+   * @param base_accessor The base accessor.
+   * @param derived_accessor The derived accessor.
+   */
+  ValueAccessorMultiplexer(ValueAccessor *base_accessor,
+                           ValueAccessor *derived_accessor)
+      : base_accessor_(base_accessor),
+        derived_accessor_(derived_accessor) {}
+
+  /**
+   * @return The base accessor.
+   */
+  inline ValueAccessor* getBaseAccessor() const {
+    return base_accessor_;
+  }
+
+  /**
+   * @return The derived accessor.
+   */
+  inline ValueAccessor* getDerivedAccessor() const {
+    return derived_accessor_;
+  }
+
+  /**
+   * @brief Get the value accessor that corresponds to the specified source.
+   *
+   * @param source The value accessor source.
+   * @return The value accessor that corresponds to \p source.
+   */
+  inline ValueAccessor* getValueAccessorBySource(
+      const ValueAccessorSource &source) const {
+    switch (source) {
+      case ValueAccessorSource::kBase:
+        return base_accessor_;
+      case ValueAccessorSource::kDerived:
+        return derived_accessor_;
+      default:
+        return nullptr;
+    }
+  }
+
+ private:
+  ValueAccessor *base_accessor_;
+  ValueAccessor *derived_accessor_;
+
+  DISALLOW_COPY_AND_ASSIGN(ValueAccessorMultiplexer);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_VALUE_ACCESSOR_MULTIPLEXER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/utility/BarrieredReadWriteConcurrentBitVector.hpp
----------------------------------------------------------------------
diff --git a/utility/BarrieredReadWriteConcurrentBitVector.hpp b/utility/BarrieredReadWriteConcurrentBitVector.hpp
new file mode 100644
index 0000000..0086c7f
--- /dev/null
+++ b/utility/BarrieredReadWriteConcurrentBitVector.hpp
@@ -0,0 +1,282 @@
+/**
+ * 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_BARRIERED_READ_WRITE_CONCURRENT_BIT_VECTOR_HPP_
+#define QUICKSTEP_UTILITY_BARRIERED_READ_WRITE_CONCURRENT_BIT_VECTOR_HPP_
+
+#include <atomic>
+#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
+ *  @{
+ */
+
+/**
+ * @brief A bit vector that supports concurrent read/write operations, with a
+ *        RESTRICTED CONCURRENCY LEVEL that the read operations and the write
+ *        operations must be isolated with a (mostly implicit) barrier.
+ * 
+ * In other words, when using this bit vector, the read operations and write
+ * operations must be grouped into phases. Within a phase there can be either
+ * concurrent read operations or concurrent write operations, but not both (or
+ * the bit vector's behavior will be undefined).
+ **/
+class BarrieredReadWriteConcurrentBitVector {
+ public:
+  /**
+   * @brief Constructor for a bit vector with external storage.
+   *
+   * @param memory_location The location of memory to use for the bit vector.
+   * @param num_bits The length of the bit vector in bits.
+   * @param initialize If true, initialize all the bytes of the memory to 0.
+   */
+  BarrieredReadWriteConcurrentBitVector(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, 0u);
+    DCHECK(data_array_ != nullptr);
+
+    if (initialize) {
+      clear();
+    }
+  }
+
+  /**
+   * @brief Constructor for a bit vector which owns its own storage.
+   *
+   * @param num_bits The length of the bit vector in bits.
+   **/
+  explicit BarrieredReadWriteConcurrentBitVector(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, 0u);
+    clear();
+  }
+
+  /**
+   * @brief Destructor. Frees any owned memory.
+   */
+  ~BarrieredReadWriteConcurrentBitVector() {
+    if (owned_) {
+      std::free(data_array_);
+    }
+  }
+
+  /**
+   * @brief Calculate the number of bytes needed to store a bit vector of the
+   *        given number of bits.
+   *
+   * @param num_bits The desired length of a BitVector in bits.
+   * @return The number of bytes needed for the BitVector.
+   **/
+  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;
+    }
+  }
+
+  /**
+   * @return The length of this bit vector in bits.
+   **/
+  inline std::size_t size() const {
+    return num_bits_;
+  }
+
+  /**
+   * @brief Clear this bit vector, setting all bits to zero.
+   **/
+  inline void clear() {
+    std::memset(data_array_, 0, BytesNeeded(num_bits_));
+  }
+
+  /**
+   * @brief Get the value of a single bit.
+   *
+   * @param bit_num The position of the desired bit.
+   * @return The value of the bit at bit_num.
+   **/
+  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;
+  }
+
+  /**
+   * @brief Set the value of a single bit.
+   *
+   * @param bit_num The position of the desired bit.
+   * @param value The new value to set for the bit at bit_num.
+   **/
+  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);
+  }
+
+  /**
+   * @brief Find the first 1-bit AT or AFTER the specified position in this bit
+   *        vector.
+   *
+   * @param position The first bit to search for a one.
+   * @return The position of the first one AT or AFTER \p position in this bit
+   *         vector.
+   **/
+  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_;
+  }
+
+  /**
+   * @brief Find the first 1-bit AFTER the specified position in this bit vector.
+   *
+   * @param position The first bit to search for a one.
+   * @return The position of the first one AFTER \p position in this bit vector.
+   **/
+  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);
+  }
+
+  /**
+   * @brief Count the total number of 1-bits in this bit vector.
+   *
+   * @return The number of ones in this bit vector.
+   **/
+  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;
+  }
+
+  /**
+   * @brief Count the total number of 1-bits in this bit vector within the
+   *        specified range (start point INCLUSIVE but end point EXCLUSIVE).
+   *
+   * @param The start position of the range.
+   * @param The end position of the range.
+   *
+   * @return The number of ones within the range, INCLUDING the 1-bit at
+   *         \p start_position, but EXCLUDING the 1-bit at \p end_position.
+   **/
+  inline std::size_t onesCountInRange(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(BarrieredReadWriteConcurrentBitVector);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_BARRIERED_READ_WRITE_CONCURRENT_BIT_VECTOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index aeff388..ca04462 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -172,6 +172,9 @@ 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_BarrieredReadWriteConcurrentBitVector
+            ../empty_src.cpp
+            BarrieredReadWriteConcurrentBitVector.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 +241,9 @@ target_link_libraries(quickstep_utility_CompositeHash
                       quickstep_types_TypedValue
                       quickstep_utility_HashPair
                       glog)
+target_link_libraries(quickstep_utility_BarrieredReadWriteConcurrentBitVector
+                      quickstep_utility_BitManipulation
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_DAG
                       glog
                       quickstep_utility_Macros)
@@ -330,6 +336,7 @@ target_link_libraries(quickstep_utility_TreeStringSerializable
 add_library(quickstep_utility ../empty_src.cpp UtilityModule.hpp)
 target_link_libraries(quickstep_utility
                       quickstep_utility_Alignment
+                      quickstep_utility_BarrieredReadWriteConcurrentBitVector
                       quickstep_utility_BitManipulation
                       quickstep_utility_BitVector
                       quickstep_utility_BloomFilter


[05/18] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/PackedPayloadHashTable.cpp
----------------------------------------------------------------------
diff --git a/storage/PackedPayloadHashTable.cpp b/storage/PackedPayloadHashTable.cpp
new file mode 100644
index 0000000..bf5eaee
--- /dev/null
+++ b/storage/PackedPayloadHashTable.cpp
@@ -0,0 +1,463 @@
+/**
+ * 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/PackedPayloadHashTable.hpp"
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <cstdlib>
+#include <vector>
+
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "storage/HashTableKeyManager.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "threading/SpinMutex.hpp"
+#include "threading/SpinSharedMutex.hpp"
+#include "types/Type.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/Alignment.hpp"
+#include "utility/Macros.hpp"
+#include "utility/PrimeNumber.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+PackedPayloadHashTable::PackedPayloadHashTable(
+    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));
+}
+
+PackedPayloadHashTable::~PackedPayloadHashTable() {
+  if (blob_.valid()) {
+    const block_id blob_id = blob_->getID();
+    blob_.release();
+    storage_manager_->deleteBlockOrBlobFile(blob_id);
+  }
+  std::free(init_payload_);
+}
+
+void PackedPayloadHashTable::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 PackedPayloadHashTable::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 PackedPayloadHashTable::upsertValueAccessorCompositeKey(
+    const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+    const std::vector<MultiSourceAttributeId> &key_attr_ids,
+    const ValueAccessorMultiplexer &accessor_mux) {
+  ValueAccessor *base_accessor = accessor_mux.getBaseAccessor();
+  ValueAccessor *derived_accessor = accessor_mux.getDerivedAccessor();
+
+  base_accessor->beginIterationVirtual();
+  if (derived_accessor == nullptr) {
+    return upsertValueAccessorCompositeKeyInternal<false>(
+        argument_ids,
+        key_attr_ids,
+        base_accessor,
+        nullptr);
+  } else {
+    DCHECK(derived_accessor->getImplementationType()
+               == ValueAccessor::Implementation::kColumnVectors);
+    derived_accessor->beginIterationVirtual();
+    return upsertValueAccessorCompositeKeyInternal<true>(
+        argument_ids,
+        key_attr_ids,
+        base_accessor,
+        static_cast<ColumnVectorsValueAccessor *>(derived_accessor));
+  }
+}
+
+void PackedPayloadHashTable::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/2d89e4fb/storage/PackedPayloadHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedPayloadHashTable.hpp b/storage/PackedPayloadHashTable.hpp
new file mode 100644
index 0000000..f87a1de
--- /dev/null
+++ b/storage/PackedPayloadHashTable.hpp
@@ -0,0 +1,995 @@
+/**
+ * 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_HASH_TABLE_HPP_
+#define QUICKSTEP_STORAGE_PACKED_PAYLOAD_HASH_TABLE_HPP_
+
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <cstring>
+#include <limits>
+#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/ValueAccessorMultiplexer.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "threading/SpinMutex.hpp"
+#include "threading/SpinSharedMutex.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/HashPair.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class StorageManager;
+class Type;
+class ValueAccessor;
+
+/** \addtogroup Storage
+ *  @{
+ */
+
+/**
+ * @brief Aggregation hash table implementation in which the payload can be just
+ *        a bunch of bytes. This implementation is suitable for aggregation with
+ *        multiple aggregation handles (e.g. SUM, MAX, MIN etc).
+ *
+ * At present the hash table uses separate chaining to resolve collisions, i.e.
+ * 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.
+ **/
+class PackedPayloadHashTable : public AggregationStateHashTableBase {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @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 handles The aggregation handles.
+   * @param storage_manager The StorageManager to use (a StorageBlob will be
+   *        allocated to hold this hash table's contents).
+   **/
+  PackedPayloadHashTable(
+      const std::vector<const Type *> &key_types,
+      const std::size_t num_entries,
+      const std::vector<AggregationHandle *> &handles,
+      StorageManager *storage_manager);
+
+  ~PackedPayloadHashTable() override;
+
+  /**
+   * @brief Erase all entries in this hash table.
+   *
+   * @warning This method is not guaranteed to be threadsafe.
+   **/
+  void clear();
+
+  void destroyPayload() override;
+
+  /**
+   * @brief Use aggregation handles to update (multiple) aggregation states in
+   *        this hash table, with group-by keys and arguments drawn from the
+   *        given ValueAccessors. New states are first inserted if not already
+   *        present.
+   *
+   * @note This method is threadsafe with regard to other calls to
+   *       upsertCompositeKey() and upsertValueAccessorCompositeKey().
+   *
+   * @param argument_ids The multi-source attribute IDs of each argument
+   *        component to be read from \p accessor_mux.
+   * @param key_ids The multi-source attribute IDs of each group-by key
+   *        component to be read from \p accessor_mux.
+   * @param accessor_mux A ValueAccessorMultiplexer object that contains the
+   *        ValueAccessors which will be used to access keys. beginIteration()
+   *        should be called on the accessors before calling this method.
+   * @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
+   *         accessors' iterations will be left on the first tuple which could
+   *         not be inserted).
+   **/
+  bool upsertValueAccessorCompositeKey(
+      const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      const ValueAccessorMultiplexer &accessor_mux) override;
+
+  /**
+   * @return The ID of the StorageBlob used to store this hash table.
+   **/
+  inline block_id getBlobId() const {
+    return blob_->getID();
+  }
+
+  /**
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() or upsertValueAccessorCompositeKey() are
+   *          taking place (i.e. that this HashTable is immutable for the
+   *          duration of the call).
+   *          Concurrent calls to getSingleCompositeKey(), forEach(), and
+   *          forEachCompositeKey() are safe.
+   *
+   * @return The number of entries in this HashTable.
+   **/
+  inline std::size_t numEntries() const {
+    return header_->buckets_allocated.load(std::memory_order_relaxed);
+  }
+
+  /**
+   * @brief Use aggregation handles to merge the given aggregation states into
+   *        the aggregation states mapped to the given key. New states are first
+   *        inserted if not already present.
+   *
+   * @warning The key must not be null.
+   * @note This method is threadsafe with regard to other calls to
+   *       upsertCompositeKey() and upsertValueAccessorCompositeKey().
+   *
+   * @param key The key.
+   * @param source_state The source aggregation states to be merged into this
+   *        hash table.
+   * @return True on success, false if upsert failed because there was not
+   *         enough space to insert a new entry in this hash table.
+   **/
+  inline bool upsertCompositeKey(const std::vector<TypedValue> &key,
+                                 const std::uint8_t *source_state);
+
+  /**
+   * @brief Apply a functor to an aggregation state mapped to the given key.
+   *        First inserting a new state if one is not already present.
+   *
+   * @warning The key must not be null.
+   * @note This method is threadsafe with regard to other calls to
+   *       upsertCompositeKey() and upsertValueAccessorCompositeKey().
+   *
+   * @param key The key.
+   * @param functor A pointer to a functor, which should provide a call
+   *        operator which takes an aggregation state (of type std::uint8_t *)
+   *        as an argument.
+   * @param index The index of the target aggregation state among those states
+   *        mapped to \p key.
+   * @return True on success, false if upsert failed because there was not
+   *         enough space to insert a new entry in this hash table.
+   **/
+  template <typename FunctorT>
+  inline bool upsertCompositeKey(const std::vector<TypedValue> &key,
+                                 FunctorT *functor,
+                                 const std::size_t index);
+
+  /**
+   * @brief Lookup a composite key against this hash table to find a matching
+   *        entry.
+   *
+   * @warning The key must not be null.
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() 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 getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param key The key to look up.
+   * @return The value of a matched entry if a matching key is found.
+   *         Otherwise, return NULL.
+   **/
+  inline const std::uint8_t* getSingleCompositeKey(
+      const std::vector<TypedValue> &key) const;
+
+  /**
+   * @brief Lookup a composite key against this hash table to find a matching
+   *        entry. Then return the aggregation state component with the
+   *        specified index.
+   *
+   * @warning The key must not be null.
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() 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 getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param key The key to look up.
+   * @param index The index of the target aggregation state among those states
+   *        mapped to \p key.
+   * @return The aggregation state of the specified index if a matching key is
+   *         found. Otherwise, return NULL.
+   **/
+  inline const std::uint8_t* getSingleCompositeKey(
+      const std::vector<TypedValue> &key,
+      const std::size_t index) const;
+
+  /**
+   * @brief Apply a functor to each (key, value) pair in this hash table.
+   *
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() 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 getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param functor A pointer to a functor, which should provide a call operator
+   *        which takes 2 arguments: const TypedValue&, const std::uint8_t*.
+   *        The call operator will be invoked once on each key, value pair in
+   *        this hash table.
+   * @return The number of key-value pairs visited.
+   **/
+  template <typename FunctorT>
+  inline std::size_t forEach(FunctorT *functor) const;
+
+  /**
+   * @brief Apply a functor to each (key, aggregation state) pair in this hash
+   *        table, where the aggregation state is retrieved from the value
+   *        that maps to the corresponding key with the specified index.
+   *
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() 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 getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param functor A pointer to a functor, which should provide a call operator
+   *        which takes 2 arguments: const TypedValue&, const std::uint8_t*.
+   *        The call operator will be invoked once on each (key, aggregation state)
+   *        pair in this hash table.
+   * @param index The index of the target aggregation state among those states
+   *        mapped to \p key.
+   * @return The number of key-value pairs visited.
+   **/
+  template <typename FunctorT>
+  inline std::size_t forEach(FunctorT *functor, const int index) const;
+
+  /**
+   * @brief Apply a functor to each key, value pair in this hash table.
+   *        Composite key version.
+   *
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() 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 getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param functor A pointer to a functor, which should provide a call operator
+   *        which takes 2 arguments: const TypedValue&, const std::uint8_t*.
+   *        The call operator will be invoked once on each key, value pair in
+   *        this hash table.
+   * @return The number of key-value pairs visited.
+   **/
+  template <typename FunctorT>
+  inline std::size_t forEachCompositeKey(FunctorT *functor) const;
+
+  /**
+   * @brief Apply a functor to each (key, aggregation state) pair in this hash
+   *        table, where the aggregation state is retrieved from the value
+   *        that maps to the corresponding key with the specified index.
+   *        Composite key version.
+   *
+   * @warning This method assumes that no concurrent calls to
+   *          upsertCompositeKey() 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 getSingleCompositeKey(),
+   *          forEach(), and forEachCompositeKey() are safe.
+   *
+   * @param functor A pointer to a functor, which should provide a call operator
+   *        which takes 2 arguments: const TypedValue&, const std::uint8_t*.
+   *        The call operator will be invoked once on each (key, aggregation state)
+   *        pair in this hash table.
+   * @param index The index of the target aggregation state among those states
+   *        mapped to \p key.
+   * @return The number of key-value pairs visited.
+   **/
+  template <typename FunctorT>
+  inline std::size_t forEachCompositeKey(FunctorT *functor,
+                                         const std::size_t 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 getNextEntry(TypedValue *key,
+                           const std::uint8_t **value,
+                           std::size_t *entry_num) const;
+
+  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 use_two_accessors>
+  inline bool upsertValueAccessorCompositeKeyInternal(
+      const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      ValueAccessor *base_accessor,
+      ColumnVectorsValueAccessor *derived_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_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 <bool use_two_accessors,
+            bool check_for_null_keys,
+            typename ValueAccessorT>
+  inline static bool GetCompositeKeyFromValueAccessor(
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      const ValueAccessorT *accessor,
+      const ColumnVectorsValueAccessor *derived_accessor,
+      std::vector<TypedValue> *key_vector) {
+    for (std::size_t key_idx = 0; key_idx < key_ids.size(); ++key_idx) {
+      const MultiSourceAttributeId &key_id = key_ids[key_idx];
+      if (use_two_accessors && key_id.source == ValueAccessorSource::kDerived) {
+        (*key_vector)[key_idx] = derived_accessor->getTypedValue(key_id.attr_id);
+      } else {
+        (*key_vector)[key_idx] = accessor->getTypedValue(key_id.attr_id);
+      }
+      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.
+  inline 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(PackedPayloadHashTable);
+};
+
+/** @} */
+
+// ----------------------------------------------------------------------------
+// Implementations of template class methods follow.
+
+class HashTableMerger {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param handle The Aggregation handle being used.
+   * @param destination_hash_table The destination hash table to which other
+   *        hash tables will be merged.
+   **/
+  explicit HashTableMerger(PackedPayloadHashTable *destination_hash_table)
+      : destination_hash_table_(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:
+  PackedPayloadHashTable *destination_hash_table_;
+
+  DISALLOW_COPY_AND_ASSIGN(HashTableMerger);
+};
+
+inline std::size_t PackedPayloadHashTable::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 PackedPayloadHashTable::getNextEntry(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_;
+    *key = key_manager_.getKeyComponentTyped(bucket, 0);
+    *value = reinterpret_cast<const std::uint8_t *>(bucket + kValueOffset);
+    ++(*entry_num);
+    return true;
+  } else {
+    return false;
+  }
+}
+
+inline bool PackedPayloadHashTable::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 PackedPayloadHashTable::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* PackedPayloadHashTable::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* PackedPayloadHashTable::getSingleCompositeKey(
+    const std::vector<TypedValue> &key,
+    const std::size_t 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 PackedPayloadHashTable::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);
+  }
+}
+
+template <typename FunctorT>
+inline bool PackedPayloadHashTable::upsertCompositeKey(
+    const std::vector<TypedValue> &key,
+    FunctorT *functor,
+    const std::size_t index) {
+  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) {
+        (*functor)(value + payload_offsets_[index]);
+        return true;
+      }
+    }
+    resize(0, variable_size);
+  }
+}
+
+
+inline std::uint8_t* PackedPayloadHashTable::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 use_two_accessors>
+inline bool PackedPayloadHashTable::upsertValueAccessorCompositeKeyInternal(
+    const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+    const std::vector<MultiSourceAttributeId> &key_ids,
+    ValueAccessor *base_accessor,
+    ColumnVectorsValueAccessor *derived_accessor) {
+  std::size_t variable_size;
+  std::vector<TypedValue> key_vector;
+  key_vector.resize(key_ids.size());
+
+  return InvokeOnAnyValueAccessor(
+      base_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 (use_two_accessors) {
+            derived_accessor->next();
+          }
+          if (this->GetCompositeKeyFromValueAccessor<use_two_accessors, true>(
+                  key_ids,
+                  accessor,
+                  derived_accessor,
+                  &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 MultiSourceAttributeId &arg_id = ids.front();
+                if (use_two_accessors && arg_id.source == ValueAccessorSource::kDerived) {
+                  DCHECK_NE(arg_id.attr_id, kInvalidAttributeID);
+                  handles_[k]->updateStateUnary(derived_accessor->getTypedValue(arg_id.attr_id),
+                                                value + payload_offsets_[k]);
+                } else {
+                  handles_[k]->updateStateUnary(accessor->getTypedValue(arg_id.attr_id),
+                                                value + payload_offsets_[k]);
+                }
+              }
+            }
+          }
+        }
+      }
+      if (continuing) {
+        this->resize(0, variable_size);
+        accessor->previous();
+        if (use_two_accessors) {
+          derived_accessor->previous();
+        }
+      }
+    }
+    return true;
+  });
+}
+
+inline void PackedPayloadHashTable::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 PackedPayloadHashTable::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 PackedPayloadHashTable::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 PackedPayloadHashTable::forEach(FunctorT *functor) const {
+  std::size_t entries_visited = 0;
+  std::size_t entry_num = 0;
+  TypedValue key;
+  const std::uint8_t *value_ptr;
+  while (getNextEntry(&key, &value_ptr, &entry_num)) {
+    ++entries_visited;
+    (*functor)(key, value_ptr);
+  }
+  return entries_visited;
+}
+
+template <typename FunctorT>
+inline std::size_t PackedPayloadHashTable::forEach(
+    FunctorT *functor, const int index) const {
+  std::size_t entries_visited = 0;
+  std::size_t entry_num = 0;
+  TypedValue key;
+  const std::uint8_t *value_ptr;
+  while (getNextEntry(&key, &value_ptr, &entry_num)) {
+    ++entries_visited;
+    (*functor)(key, value_ptr + payload_offsets_[index]);
+    key.clear();
+  }
+  return entries_visited;
+}
+
+template <typename FunctorT>
+inline std::size_t PackedPayloadHashTable::forEachCompositeKey(
+    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 PackedPayloadHashTable::forEachCompositeKey(
+    FunctorT *functor,
+    const std::size_t 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_HASH_TABLE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/storage/PartitionedHashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/PartitionedHashTablePool.hpp b/storage/PartitionedHashTablePool.hpp
index 95d1810..0e62511 100644
--- a/storage/PartitionedHashTablePool.hpp
+++ b/storage/PartitionedHashTablePool.hpp
@@ -21,22 +21,19 @@
 #define QUICKSTEP_STORAGE_PARTITIONED_HASH_TABLE_POOL_HPP_
 
 #include <algorithm>
-#include <chrono>
+#include <cstddef>
 #include <memory>
-#include <utility>
 #include <vector>
 
-#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"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
+class AggregationHandle;
 class StorageManager;
 class Type;
 
@@ -54,33 +51,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 +59,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 +66,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 +73,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 +116,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 +147,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/2d89e4fb/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index de2d25b..0cc7735 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -19,8 +19,8 @@
 
 #include "storage/StorageBlock.hpp"
 
-#include <climits>
 #include <memory>
+#include <random>
 #include <type_traits>
 #include <unordered_map>
 #include <utility>
@@ -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


[11/18] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index d851a0c..8f8c0d8 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -21,15 +21,14 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_MAX_HPP_
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <utility>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -40,9 +39,8 @@
 
 namespace quickstep {
 
+class AggregationStateHashTableBase;
 class ColumnVector;
-class StorageManager;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -86,42 +84,41 @@ 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.
-   */
   inline void iterateUnaryInl(AggregationStateMax *state,
                               const TypedValue &value) const {
     DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
     compareAndUpdate(static_cast<AggregationStateMax *>(state), value);
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value,
-                                  std::uint8_t *byte_ptr) const {
+  inline void iterateUnaryInl(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);
+    compareAndUpdate(max_ptr, value);
   }
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  AggregationState* accumulateValueAccessor(
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) 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 +133,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 {
+    if (!block_update_) {
+      iterateUnaryInl(argument, byte_ptr);
+    }
+  }
 
   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);
@@ -175,29 +155,16 @@ class AggregationHandleMax : public AggregationConcreteHandle {
 
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for MAX aggregation.
-   */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        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); }
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
 
  private:
   friend class AggregateFunctionMax;
@@ -227,8 +194,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 +206,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/2d89e4fb/expressions/aggregation/AggregationHandleMin.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.cpp b/expressions/aggregation/AggregationHandleMin.cpp
index a07f299..08fb141 100644
--- a/expressions/aggregation/AggregationHandleMin.cpp
+++ b/expressions/aggregation/AggregationHandleMin.cpp
@@ -19,15 +19,16 @@
 
 #include "expressions/aggregation/AggregationHandleMin.hpp"
 
+#include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
@@ -36,54 +37,32 @@
 
 namespace quickstep {
 
-class StorageManager;
+class ColumnVector;
 
 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::accumulateValueAccessor(
+    const std::vector<MultiSourceAttributeId> &argument_ids,
+    const ValueAccessorMultiplexer &accessor_mux) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for MIN: " << argument_ids.size();
 
-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();
+  const ValueAccessorSource argument_source = argument_ids.front().source;
+  const attribute_id argument_id = argument_ids.front().attr_id;
 
-  return new AggregationStateMin(fast_comparator_->accumulateColumnVector(
-      type_.getNullableVersion().makeNullValue(), *column_vectors.front()));
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleMin::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  DCHECK_EQ(1u, accessor_ids.size())
-      << "Got wrong number of attributes for MIN: " << accessor_ids.size();
+  DCHECK(argument_source != ValueAccessorSource::kInvalid);
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
   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();
+      accessor_mux.getValueAccessorBySource(argument_source),
+      argument_id));
 }
 
 void AggregationHandleMin::mergeStates(const AggregationState &source,
@@ -98,41 +77,37 @@ 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);
   }
 }
 
 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);
+    const std::size_t index,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  return finalizeHashTableHelper<AggregationHandleMin>(
+      type_, hash_table, index, group_by_keys);
 }
 
-AggregationState*
-AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleMin::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleMin,
-      AggregationStateMin>(distinctify_hash_table);
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleMin, AggregationStateMin>(
+          distinctify_hash_table);
 }
 
 void AggregationHandleMin::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleMin,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+    const std::size_t index,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<AggregationHandleMin>(
+      distinctify_hash_table, index, aggregation_hash_table);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index e3472ec..0e62be5 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -21,15 +21,14 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_MIN_HPP_
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <utility>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -40,9 +39,8 @@
 
 namespace quickstep {
 
+class AggregationStateHashTableBase;
 class ColumnVector;
-class StorageManager;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -88,42 +86,45 @@ 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 {
+  inline void iterateUnaryInl(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);
+    compareAndUpdate(min_ptr, value);
   }
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  AggregationState* accumulateValueAccessor(
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void allowUpdate() override { block_update_ = false; }
+  TypedValue finalize(const AggregationState &state) const override {
+    return static_cast<const AggregationStateMin &>(state).min_;
+  }
+
+  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,68 +139,33 @@ 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 {
+    if (!block_update_) {
+      iterateUnaryInl(argument, byte_ptr);
+    }
   }
 
-  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(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
-  /**
-   * @brief Implementation of
-   * AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for MIN aggregation.
-   */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        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); }
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
 
  private:
   friend class AggregateFunctionMin;
@@ -228,8 +194,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 +206,6 @@ class AggregationHandleMin : public AggregationConcreteHandle {
   const Type &type_;
   std::unique_ptr<UncheckedComparator> fast_comparator_;
 
-  bool block_update_;
-
   DISALLOW_COPY_AND_ASSIGN(AggregationHandleMin);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 642d88d..9f5f220 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -20,13 +20,13 @@
 #include "expressions/aggregation/AggregationHandleSum.hpp"
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
-#include <utility>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
@@ -40,10 +40,11 @@
 
 namespace quickstep {
 
-class StorageManager;
+class ColumnVector;
 
 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,27 @@ 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::accumulateValueAccessor(
+    const std::vector<MultiSourceAttributeId> &argument_ids,
+    const ValueAccessorMultiplexer &accessor_mux) 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 ValueAccessorSource argument_source = argument_ids.front().source;
+  const attribute_id argument_id = argument_ids.front().attr_id;
 
-#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();
+  DCHECK(argument_source != ValueAccessorSource::kInvalid);
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
   std::size_t num_tuples = 0;
-  TypedValue va_sum = fast_operator_->accumulateValueAccessor(
-      blank_state_.sum_, accessor, accessor_ids.front(), &num_tuples);
+  TypedValue va_sum =
+      fast_operator_->accumulateValueAccessor(
+          blank_state_.sum_,
+          accessor_mux.getValueAccessorBySource(argument_source),
+          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 +115,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 =
@@ -162,29 +143,25 @@ TypedValue AggregationHandleSum::finalize(const AggregationState &state) const {
 
 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);
+    const std::size_t index,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  return finalizeHashTableHelper<AggregationHandleSum>(
+      *result_type_, hash_table, index, group_by_keys);
 }
 
-AggregationState*
-AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleSum::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleSum,
-      AggregationStateSum>(distinctify_hash_table);
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleSum, AggregationStateSum>(
+          distinctify_hash_table);
 }
 
 void AggregationHandleSum::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleSum,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+    const std::size_t index,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<AggregationHandleSum>(
+      distinctify_hash_table, index, aggregation_hash_table);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index f0d23e1..ba4fa9b 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -21,15 +21,14 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_SUM_HPP_
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <utility>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -40,9 +39,8 @@
 
 namespace quickstep {
 
+class AggregationStateHashTableBase;
 class ColumnVector;
-class StorageManager;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -101,16 +99,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,8 +121,8 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     state->null_ = false;
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value,
-                                  std::uint8_t *byte_ptr) const {
+  inline void iterateUnaryInl(const TypedValue &value,
+                              std::uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
     if (value.isNull()) return;
     TypedValue *sum_ptr =
@@ -133,16 +133,18 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     *null_ptr = false;
   }
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  AggregationState* accumulateValueAccessor(
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void allowUpdate() override { block_update_ = false; }
+  TypedValue finalize(const AggregationState &state) const override;
+
+  std::size_t getPayloadSize() const override {
+    return blank_state_.getPayloadSize();
+  }
 
   void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *sum_ptr =
@@ -161,70 +163,32 @@ 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 {
+    if (!block_update_) {
+      iterateUnaryInl(argument, byte_ptr);
+    }
   }
 
-  inline TypedValue finalizeHashTableEntryFast(
-      const std::uint8_t *byte_ptr) const {
-    std::uint8_t *value_ptr = const_cast<std::uint8_t *>(byte_ptr);
-    TypedValue *sum_ptr =
-        reinterpret_cast<TypedValue *>(value_ptr + blank_state_.sum_offset_);
-    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(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for SUM aggregation.
-   */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        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();
-  }
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
 
  private:
   friend class AggregateFunctionSum;
@@ -242,8 +206,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/2d89e4fb/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index e9503f7..4220a8d 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -55,9 +55,6 @@ add_library(quickstep_expressions_aggregation_AggregationHandleAvg
 add_library(quickstep_expressions_aggregation_AggregationHandleCount
             AggregationHandleCount.cpp
             AggregationHandleCount.hpp)
-add_library(quickstep_expressions_aggregation_AggregationHandleDistinct
-            AggregationHandleDistinct.cpp
-            AggregationHandleDistinct.hpp)
 add_library(quickstep_expressions_aggregation_AggregationHandleMax
             AggregationHandleMax.cpp
             AggregationHandleMax.hpp)
@@ -144,20 +141,21 @@ target_link_libraries(quickstep_expressions_aggregation_AggregateFunctionSum
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationConcreteHandle
                       glog
-                      quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
                       quickstep_storage_HashTableFactory
+                      quickstep_storage_PackedPayloadHashTable
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_threading_SpinMutex
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandle
                       glog
-                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_HashTableBase
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_types_TypedValue
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleAvg
@@ -165,10 +163,8 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleAvg
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypeFactory
@@ -183,39 +179,25 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleCount
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_storage_ValueAccessorUtil
+                      quickstep_types_LongType
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
-                      quickstep_types_containers_ColumnVectorUtil
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_expressions_aggregation_AggregationHandleDistinct
-                      glog
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_aggregation_AggregationConcreteHandle
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_types_TypedValue
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMax
                       glog
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
@@ -225,14 +207,11 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleMin
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
@@ -242,10 +221,8 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleSum
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_aggregation_AggregationConcreteHandle
                       quickstep_expressions_aggregation_AggregationHandle
-                      quickstep_storage_FastHashTable
-                      quickstep_storage_HashTable
-                      quickstep_storage_HashTableBase
-                      quickstep_storage_HashTableFactory
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_threading_SpinMutex
                       quickstep_types_Type
                       quickstep_types_TypeFactory
@@ -271,7 +248,6 @@ target_link_libraries(quickstep_expressions_aggregation
                       quickstep_expressions_aggregation_AggregationHandle
                       quickstep_expressions_aggregation_AggregationHandleAvg
                       quickstep_expressions_aggregation_AggregationHandleCount
-                      quickstep_expressions_aggregation_AggregationHandleDistinct
                       quickstep_expressions_aggregation_AggregationHandleMax
                       quickstep_expressions_aggregation_AggregationHandleMin
                       quickstep_expressions_aggregation_AggregationHandleSum
@@ -301,7 +277,9 @@ target_link_libraries(AggregationHandle_tests
                       quickstep_expressions_aggregation_AggregationID
                       quickstep_storage_AggregationOperationState
                       quickstep_storage_HashTableBase
+                      quickstep_storage_PackedPayloadHashTable
                       quickstep_storage_StorageManager
+                      quickstep_storage_ValueAccessorMultiplexer
                       quickstep_types_CharType
                       quickstep_types_DateOperatorOverloads
                       quickstep_types_DatetimeIntervalType

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp b/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
index 79d4448..0ad50d5 100644
--- a/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleAvg_unittest.cpp
@@ -29,8 +29,9 @@
 #include "expressions/aggregation/AggregationHandleAvg.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "storage/AggregationOperationState.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/CharType.hpp"
 #include "types/DateOperatorOverloads.hpp"
 #include "types/DatetimeIntervalType.hpp"
@@ -46,10 +47,7 @@
 #include "types/VarCharType.hpp"
 #include "types/YearMonthIntervalType.hpp"
 #include "types/containers/ColumnVector.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
-#endif
 
 #include "gtest/gtest.h"
 
@@ -192,39 +190,6 @@ class AggregationHandleAvgTest : public ::testing::Test {
   }
 
   template <typename GenericType, typename OutputType = DoubleType>
-  void checkAggregationAvgGenericColumnVector() {
-    const GenericType &type = GenericType::Instance(true);
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_avg_->finalize(*aggregation_handle_avg_state_)
-            .isNull());
-
-    typename GenericType::cpptype sum;
-    SetDataType(0, &sum);
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorGeneric<GenericType>(type, &sum));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_avg_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckAvgValue<typename OutputType::cpptype>(
-        static_cast<typename OutputType::cpptype>(sum) / kNumSamples,
-        *aggregation_handle_avg_,
-        *cv_state);
-
-    aggregation_handle_avg_->mergeStates(*cv_state,
-                                         aggregation_handle_avg_state_.get());
-    CheckAvgValue<typename OutputType::cpptype>(
-        static_cast<typename OutputType::cpptype>(sum) / kNumSamples,
-        *aggregation_handle_avg_,
-        *aggregation_handle_avg_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename GenericType, typename OutputType = DoubleType>
   void checkAggregationAvgGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
@@ -240,7 +205,8 @@ class AggregationHandleAvgTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_avg_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -256,7 +222,6 @@ class AggregationHandleAvgTest : public ::testing::Test {
         *aggregation_handle_avg_,
         *aggregation_handle_avg_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   std::unique_ptr<AggregationHandle> aggregation_handle_avg_;
   std::unique_ptr<AggregationState> aggregation_handle_avg_state_;
@@ -311,33 +276,6 @@ TEST_F(AggregationHandleAvgTest, YearMonthIntervalTypeTest) {
   checkAggregationAvgGeneric<YearMonthIntervalType, YearMonthIntervalType>();
 }
 
-TEST_F(AggregationHandleAvgTest, IntTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<IntType>();
-}
-
-TEST_F(AggregationHandleAvgTest, LongTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<LongType>();
-}
-
-TEST_F(AggregationHandleAvgTest, FloatTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<FloatType>();
-}
-
-TEST_F(AggregationHandleAvgTest, DoubleTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<DoubleType>();
-}
-
-TEST_F(AggregationHandleAvgTest, DatetimeIntervalTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<DatetimeIntervalType,
-                                         DatetimeIntervalType>();
-}
-
-TEST_F(AggregationHandleAvgTest, YearMonthIntervalTypeColumnVectorTest) {
-  checkAggregationAvgGenericColumnVector<YearMonthIntervalType,
-                                         YearMonthIntervalType>();
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 TEST_F(AggregationHandleAvgTest, IntTypeValueAccessorTest) {
   checkAggregationAvgGenericValueAccessor<IntType>();
 }
@@ -363,7 +301,6 @@ TEST_F(AggregationHandleAvgTest, YearMonthIntervalTypeValueAccessorTest) {
   checkAggregationAvgGenericValueAccessor<YearMonthIntervalType,
                                           YearMonthIntervalType>();
 }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
 #ifdef QUICKSTEP_DEBUG
 TEST_F(AggregationHandleAvgDeathTest, CharTypeTest) {
@@ -468,28 +405,25 @@ TEST_F(AggregationHandleAvgTest, GroupByTableMergeTestAvg) {
   initializeHandle(long_non_null_type);
   storage_manager_.reset(new StorageManager("./test_avg_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_avg_.get()->getPayloadSize()},
           {aggregation_handle_avg_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_avg_.get()->getPayloadSize()},
           {aggregation_handle_avg_.get()},
           storage_manager_.get()));
 
-  AggregationStateFastHashTable *destination_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(
-          destination_hash_table.get());
+  PackedPayloadHashTable *destination_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(destination_hash_table.get());
 
-  AggregationStateFastHashTable *source_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
+  PackedPayloadHashTable *source_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(source_hash_table.get());
 
   AggregationHandleAvg *aggregation_handle_avg_derived =
       static_cast<AggregationHandleAvg *>(aggregation_handle_avg_.get());
@@ -546,29 +480,29 @@ TEST_F(AggregationHandleAvgTest, GroupByTableMergeTestAvg) {
   memcpy(buffer + 1,
          common_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_avg_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(common_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          common_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_avg_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+  destination_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_avg_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(exclusive_source_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_avg_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
+  destination_hash_table_derived->upsertCompositeKey(exclusive_destination_key,
                                                       buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  AggregationOperationState::mergeGroupByHashTables(
-      source_hash_table.get(), destination_hash_table.get());
+  HashTableMerger merger(destination_hash_table_derived);
+  source_hash_table_derived->forEachCompositeKey(&merger);
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
@@ -576,21 +510,19 @@ TEST_F(AggregationHandleAvgTest, GroupByTableMergeTestAvg) {
       (common_key_destination_avg_val.getLiteral<std::int64_t>() +
        common_key_source_avg_val.getLiteral<std::int64_t>()) /
           static_cast<double>(2),
-      aggregation_handle_avg_derived->finalizeHashTableEntryFast(
+      aggregation_handle_avg_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(common_key) +
           1));
   CheckAvgValue<double>(
       exclusive_key_destination_avg_val.getLiteral<std::int64_t>(),
-      aggregation_handle_avg_derived->finalizeHashTableEntryFast(
+      aggregation_handle_avg_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(
-              exclusive_destination_key) +
-          1));
+              exclusive_destination_key) + 1));
   CheckAvgValue<double>(
       exclusive_key_source_avg_val.getLiteral<std::int64_t>(),
-      aggregation_handle_avg_derived->finalizeHashTableEntryFast(
+      aggregation_handle_avg_derived->finalizeHashTableEntry(
           source_hash_table_derived->getSingleCompositeKey(
-              exclusive_source_key) +
-          1));
+              exclusive_source_key) + 1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp b/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
index 78bd249..86a014b 100644
--- a/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleCount_unittest.cpp
@@ -30,8 +30,9 @@
 #include "expressions/aggregation/AggregationHandleCount.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "storage/AggregationOperationState.hpp"
-#include "storage/FastHashTableFactory.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/CharType.hpp"
 #include "types/DoubleType.hpp"
 #include "types/FloatType.hpp"
@@ -43,10 +44,7 @@
 #include "types/TypedValue.hpp"
 #include "types/VarCharType.hpp"
 #include "types/containers/ColumnVector.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
-#endif
 
 #include "gtest/gtest.h"
 
@@ -216,32 +214,6 @@ class AggregationHandleCountTest : public ::testing::Test {
   }
 
   template <typename NumericType>
-  void checkAggregationCountNumericColumnVector(int test_count) {
-    const NumericType &type = NumericType::Instance(true);
-    initializeHandle(&type);
-    CheckCountValue(
-        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
-
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorNumeric<NumericType>(type, test_count));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_count_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckCountValue(test_count, *aggregation_handle_count_, *cv_state);
-
-    aggregation_handle_count_->mergeStates(
-        *cv_state, aggregation_handle_count_state_.get());
-    CheckCountValue(test_count,
-                    *aggregation_handle_count_,
-                    *aggregation_handle_count_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename NumericType>
   void checkAggregationCountNumericValueAccessor(int test_count) {
     const NumericType &type = NumericType::Instance(true);
     initializeHandle(&type);
@@ -255,7 +227,8 @@ class AggregationHandleCountTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_count_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -267,7 +240,6 @@ class AggregationHandleCountTest : public ::testing::Test {
                     *aggregation_handle_count_,
                     *aggregation_handle_count_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   template <typename StringType>
   void checkAggregationCountString(int test_count) {
@@ -326,32 +298,6 @@ class AggregationHandleCountTest : public ::testing::Test {
   }
 
   template <typename StringType, typename ColumnVectorType>
-  void checkAggregationCountStringColumnVector(int test_count) {
-    const StringType &type = StringType::Instance(10, true);
-    initializeHandle(&type);
-    CheckCountValue(
-        0, *aggregation_handle_count_, *aggregation_handle_count_state_);
-
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorString<ColumnVectorType>(type, test_count));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_count_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckCountValue(test_count, *aggregation_handle_count_, *cv_state);
-
-    aggregation_handle_count_->mergeStates(
-        *cv_state, aggregation_handle_count_state_.get());
-    CheckCountValue(test_count,
-                    *aggregation_handle_count_,
-                    *aggregation_handle_count_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename StringType, typename ColumnVectorType>
   void checkAggregationCountStringValueAccessor(int test_count) {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(&type);
@@ -365,7 +311,8 @@ class AggregationHandleCountTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_count_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -377,7 +324,6 @@ class AggregationHandleCountTest : public ::testing::Test {
                     *aggregation_handle_count_,
                     *aggregation_handle_count_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   std::unique_ptr<AggregationHandle> aggregation_handle_count_;
   std::unique_ptr<AggregationState> aggregation_handle_count_state_;
@@ -425,38 +371,6 @@ TEST_F(AggregationHandleCountTest, VarCharTypeTest) {
   checkAggregationCountString<VarCharType>(10000);
 }
 
-TEST_F(AggregationHandleCountTest, IntTypeColumnVectorTest) {
-  checkAggregationCountNumericColumnVector<IntType>(0);
-  checkAggregationCountNumericColumnVector<IntType>(10000);
-}
-
-TEST_F(AggregationHandleCountTest, LongTypeColumnVectorTest) {
-  checkAggregationCountNumericColumnVector<LongType>(0);
-  checkAggregationCountNumericColumnVector<LongType>(10000);
-}
-
-TEST_F(AggregationHandleCountTest, FloatTypeColumnVectorTest) {
-  checkAggregationCountNumericColumnVector<FloatType>(0);
-  checkAggregationCountNumericColumnVector<FloatType>(10000);
-}
-
-TEST_F(AggregationHandleCountTest, DoubleTypeColumnVectorTest) {
-  checkAggregationCountNumericColumnVector<DoubleType>(0);
-  checkAggregationCountNumericColumnVector<DoubleType>(10000);
-}
-
-TEST_F(AggregationHandleCountTest, CharTypeColumnVectorTest) {
-  checkAggregationCountStringColumnVector<CharType, NativeColumnVector>(0);
-  checkAggregationCountStringColumnVector<CharType, NativeColumnVector>(10000);
-}
-
-TEST_F(AggregationHandleCountTest, VarCharTypeColumnVectorTest) {
-  checkAggregationCountStringColumnVector<VarCharType, IndirectColumnVector>(0);
-  checkAggregationCountStringColumnVector<VarCharType, IndirectColumnVector>(
-      10000);
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 TEST_F(AggregationHandleCountTest, IntTypeValueAccessorTest) {
   checkAggregationCountNumericValueAccessor<IntType>(0);
   checkAggregationCountNumericValueAccessor<IntType>(10000);
@@ -488,7 +402,6 @@ TEST_F(AggregationHandleCountTest, VarCharTypeValueAccessorTest) {
   checkAggregationCountStringValueAccessor<VarCharType, IndirectColumnVector>(
       10000);
 }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
 TEST_F(AggregationHandleCountTest, canApplyToTypeTest) {
   EXPECT_TRUE(ApplyToTypesTest(kInt));
@@ -511,28 +424,25 @@ TEST_F(AggregationHandleCountTest, GroupByTableMergeTestCount) {
   initializeHandle(&long_non_null_type);
   storage_manager_.reset(new StorageManager("./test_count_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_count_.get()->getPayloadSize()},
           {aggregation_handle_count_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &long_non_null_type),
           10,
-          {aggregation_handle_count_.get()->getPayloadSize()},
           {aggregation_handle_count_.get()},
           storage_manager_.get()));
 
-  AggregationStateFastHashTable *destination_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(
-          destination_hash_table.get());
+  PackedPayloadHashTable *destination_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(destination_hash_table.get());
 
-  AggregationStateFastHashTable *source_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
+  PackedPayloadHashTable *source_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(source_hash_table.get());
 
   // TODO(harshad) - Use TemplateUtil::CreateBoolInstantiatedInstance to
   // generate all the combinations of the bool template arguments and test them.
@@ -612,49 +522,48 @@ TEST_F(AggregationHandleCountTest, GroupByTableMergeTestCount) {
   memcpy(buffer + 1,
          common_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_count_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(common_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          common_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_count_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+  destination_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_count_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(exclusive_source_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_count_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
-                                                      buffer);
+  destination_hash_table_derived->upsertCompositeKey(exclusive_destination_key,
+                                                     buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  AggregationOperationState::mergeGroupByHashTables(
-      source_hash_table.get(), destination_hash_table.get());
+  HashTableMerger merger(destination_hash_table_derived);
+  source_hash_table_derived->forEachCompositeKey(&merger);
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckCountValue(
       common_key_destination_count_val.getLiteral<std::int64_t>() +
           common_key_source_count_val.getLiteral<std::int64_t>(),
-      aggregation_handle_count_derived->finalizeHashTableEntryFast(
+      aggregation_handle_count_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(common_key) +
           1));
   CheckCountValue(
       exclusive_key_destination_count_val.getLiteral<std::int64_t>(),
-      aggregation_handle_count_derived->finalizeHashTableEntryFast(
+      aggregation_handle_count_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(
-              exclusive_destination_key) +
-          1));
-  CheckCountValue(exclusive_key_source_count_val.getLiteral<std::int64_t>(),
-                  aggregation_handle_count_derived->finalizeHashTableEntryFast(
-                      source_hash_table_derived->getSingleCompositeKey(
-                          exclusive_source_key) +
-                      1));
+              exclusive_destination_key) + 1));
+  CheckCountValue(
+      exclusive_key_source_count_val.getLiteral<std::int64_t>(),
+      aggregation_handle_count_derived->finalizeHashTableEntry(
+          source_hash_table_derived->getSingleCompositeKey(
+              exclusive_source_key) + 1));
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp b/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
index 026bd1d..d5e8d18 100644
--- a/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
+++ b/expressions/aggregation/tests/AggregationHandleMax_unittest.cpp
@@ -32,9 +32,10 @@
 #include "expressions/aggregation/AggregationHandleMax.hpp"
 #include "expressions/aggregation/AggregationID.hpp"
 #include "storage/AggregationOperationState.hpp"
-#include "storage/FastHashTableFactory.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/CharType.hpp"
 #include "types/DatetimeIntervalType.hpp"
 #include "types/DatetimeLit.hpp"
@@ -51,10 +52,7 @@
 #include "types/VarCharType.hpp"
 #include "types/YearMonthIntervalType.hpp"
 #include "types/containers/ColumnVector.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
-#endif
 
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
@@ -223,34 +221,6 @@ class AggregationHandleMaxTest : public ::testing::Test {
   }
 
   template <typename GenericType>
-  void checkAggregationMaxGenericColumnVector() {
-    const GenericType &type = GenericType::Instance(true);
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_max_->finalize(*aggregation_handle_max_state_)
-            .isNull());
-
-    typename GenericType::cpptype max;
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorGeneric<GenericType>(type, &max));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_max_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckMaxValue<typename GenericType::cpptype>(
-        max, *aggregation_handle_max_, *cv_state);
-
-    aggregation_handle_max_->mergeStates(*cv_state,
-                                         aggregation_handle_max_state_.get());
-    CheckMaxValue<typename GenericType::cpptype>(
-        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename GenericType>
   void checkAggregationMaxGenericValueAccessor() {
     const GenericType &type = GenericType::Instance(true);
     initializeHandle(type);
@@ -266,7 +236,8 @@ class AggregationHandleMaxTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_max_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -278,7 +249,6 @@ class AggregationHandleMaxTest : public ::testing::Test {
     CheckMaxValue<typename GenericType::cpptype>(
         max, *aggregation_handle_max_, *aggregation_handle_max_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   template <typename StringType>
   void checkAggregationMaxString() {
@@ -385,33 +355,6 @@ class AggregationHandleMaxTest : public ::testing::Test {
   }
 
   template <typename StringType, typename ColumnVectorType>
-  void checkAggregationMaxStringColumnVector() {
-    const StringType &type = StringType::Instance(10, true);
-    initializeHandle(type);
-    EXPECT_TRUE(
-        aggregation_handle_max_->finalize(*aggregation_handle_max_state_)
-            .isNull());
-
-    std::string max;
-    std::vector<std::unique_ptr<ColumnVector>> column_vectors;
-    column_vectors.emplace_back(
-        createColumnVectorString<ColumnVectorType>(type, &max));
-
-    std::unique_ptr<AggregationState> cv_state(
-        aggregation_handle_max_->accumulateColumnVectors(column_vectors));
-
-    // Test the state generated directly by accumulateColumnVectors(), and also
-    // test after merging back.
-    CheckMaxString(max, *aggregation_handle_max_, *cv_state);
-
-    aggregation_handle_max_->mergeStates(*cv_state,
-                                         aggregation_handle_max_state_.get());
-    CheckMaxString(
-        max, *aggregation_handle_max_, *aggregation_handle_max_state_);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <typename StringType, typename ColumnVectorType>
   void checkAggregationMaxStringValueAccessor() {
     const StringType &type = StringType::Instance(10, true);
     initializeHandle(type);
@@ -426,7 +369,8 @@ class AggregationHandleMaxTest : public ::testing::Test {
 
     std::unique_ptr<AggregationState> va_state(
         aggregation_handle_max_->accumulateValueAccessor(
-            accessor.get(), std::vector<attribute_id>(1, 0)));
+            {MultiSourceAttributeId(ValueAccessorSource::kBase, 0)},
+            ValueAccessorMultiplexer(accessor.get())));
 
     // Test the state generated directly by accumulateValueAccessor(), and also
     // test after merging back.
@@ -437,7 +381,6 @@ class AggregationHandleMaxTest : public ::testing::Test {
     CheckMaxString(
         max, *aggregation_handle_max_, *aggregation_handle_max_state_);
   }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
   std::unique_ptr<AggregationHandle> aggregation_handle_max_;
   std::unique_ptr<AggregationState> aggregation_handle_max_state_;
@@ -514,43 +457,6 @@ TEST_F(AggregationHandleMaxTest, VarCharTypeTest) {
   checkAggregationMaxString<VarCharType>();
 }
 
-TEST_F(AggregationHandleMaxTest, IntTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<IntType>();
-}
-
-TEST_F(AggregationHandleMaxTest, LongTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<LongType>();
-}
-
-TEST_F(AggregationHandleMaxTest, FloatTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<FloatType>();
-}
-
-TEST_F(AggregationHandleMaxTest, DoubleTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<DoubleType>();
-}
-
-TEST_F(AggregationHandleMaxTest, DatetimeTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<DatetimeType>();
-}
-
-TEST_F(AggregationHandleMaxTest, DatetimeIntervalTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<DatetimeIntervalType>();
-}
-
-TEST_F(AggregationHandleMaxTest, YearMonthIntervalTypeColumnVectorTest) {
-  checkAggregationMaxGenericColumnVector<YearMonthIntervalType>();
-}
-
-TEST_F(AggregationHandleMaxTest, CharTypeColumnVectorTest) {
-  checkAggregationMaxStringColumnVector<CharType, NativeColumnVector>();
-}
-
-TEST_F(AggregationHandleMaxTest, VarCharColumnVectorTypeTest) {
-  checkAggregationMaxStringColumnVector<VarCharType, IndirectColumnVector>();
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 TEST_F(AggregationHandleMaxTest, IntTypeValueAccessorTest) {
   checkAggregationMaxGenericValueAccessor<IntType>();
 }
@@ -586,7 +492,6 @@ TEST_F(AggregationHandleMaxTest, CharTypeValueAccessorTest) {
 TEST_F(AggregationHandleMaxTest, VarCharValueAccessorTypeTest) {
   checkAggregationMaxStringValueAccessor<VarCharType, IndirectColumnVector>();
 }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
 #ifdef QUICKSTEP_DEBUG
 TEST_F(AggregationHandleMaxDeathTest, WrongTypeTest) {
@@ -689,28 +594,25 @@ TEST_F(AggregationHandleMaxTest, GroupByTableMergeTest) {
   initializeHandle(int_non_null_type);
   storage_manager_.reset(new StorageManager("./test_max_data"));
   std::unique_ptr<AggregationStateHashTableBase> source_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
-          {aggregation_handle_max_.get()->getPayloadSize()},
           {aggregation_handle_max_.get()},
           storage_manager_.get()));
   std::unique_ptr<AggregationStateHashTableBase> destination_hash_table(
-      AggregationStateFastHashTableFactory::CreateResizable(
+      AggregationStateHashTableFactory::CreateResizable(
           HashTableImplType::kSeparateChaining,
           std::vector<const Type *>(1, &int_non_null_type),
           10,
-          {aggregation_handle_max_.get()->getPayloadSize()},
           {aggregation_handle_max_.get()},
           storage_manager_.get()));
 
-  AggregationStateFastHashTable *destination_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(
-          destination_hash_table.get());
+  PackedPayloadHashTable *destination_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(destination_hash_table.get());
 
-  AggregationStateFastHashTable *source_hash_table_derived =
-      static_cast<AggregationStateFastHashTable *>(source_hash_table.get());
+  PackedPayloadHashTable *source_hash_table_derived =
+      static_cast<PackedPayloadHashTable *>(source_hash_table.get());
 
   AggregationHandleMax *aggregation_handle_max_derived =
       static_cast<AggregationHandleMax *>(aggregation_handle_max_.get());
@@ -780,47 +682,47 @@ TEST_F(AggregationHandleMaxTest, GroupByTableMergeTest) {
   memcpy(buffer + 1,
          common_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_max_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(common_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          common_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_max_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(common_key, buffer);
+  destination_hash_table_derived->upsertCompositeKey(common_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_source_state.get()->getPayloadAddress(),
          aggregation_handle_max_.get()->getPayloadSize());
-  source_hash_table_derived->putCompositeKey(exclusive_source_key, buffer);
+  source_hash_table_derived->upsertCompositeKey(exclusive_source_key, buffer);
 
   memcpy(buffer + 1,
          exclusive_key_destination_state.get()->getPayloadAddress(),
          aggregation_handle_max_.get()->getPayloadSize());
-  destination_hash_table_derived->putCompositeKey(exclusive_destination_key,
-                                                      buffer);
+  destination_hash_table_derived->upsertCompositeKey(exclusive_destination_key,
+                                                     buffer);
 
   EXPECT_EQ(2u, destination_hash_table_derived->numEntries());
   EXPECT_EQ(2u, source_hash_table_derived->numEntries());
 
-  AggregationOperationState::mergeGroupByHashTables(
-      source_hash_table.get(), destination_hash_table.get());
+  HashTableMerger merger(destination_hash_table_derived);
+  source_hash_table_derived->forEachCompositeKey(&merger);
 
   EXPECT_EQ(3u, destination_hash_table_derived->numEntries());
 
   CheckMaxValue<int>(
       common_key_destination_max_val.getLiteral<int>(),
-      aggregation_handle_max_derived->finalizeHashTableEntryFast(
+      aggregation_handle_max_derived->finalizeHashTableEntry(
           destination_hash_table_derived->getSingleCompositeKey(common_key) +
           1));
-  CheckMaxValue<int>(exclusive_key_destination_max_val.getLiteral<int>(),
-                     aggregation_handle_max_derived->finalizeHashTableEntryFast(
-                         destination_hash_table_derived->getSingleCompositeKey(
-                             exclusive_destination_key) +
-                         1));
-  CheckMaxValue<int>(exclusive_key_source_max_val.getLiteral<int>(),
-                     aggregation_handle_max_derived->finalizeHashTableEntryFast(
-                         source_hash_table_derived->getSingleCompositeKey(
-                             exclusive_source_key) +
-                         1));
+  CheckMaxValue<int>(
+      exclusive_key_destination_max_val.getLiteral<int>(),
+      aggregation_handle_max_derived->finalizeHashTableEntry(
+          destination_hash_table_derived->getSingleCompositeKey(
+              exclusive_destination_key) + 1));
+  CheckMaxValue<int>(
+      exclusive_key_source_max_val.getLiteral<int>(),
+      aggregation_handle_max_derived->finalizeHashTableEntry(
+          source_hash_table_derived->getSingleCompositeKey(
+              exclusive_source_key) + 1));
 }
 
 }  // namespace quickstep


[06/18] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/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/2d89e4fb/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/2d89e4fb/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/2d89e4fb/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index a3180bb..b4b6918 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -23,11 +23,14 @@
 #include <cstddef>
 #include <vector>
 
-#include "ValueAccessor.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
+class ColumnVectorsValueAccessor;
+class ValueAccessor;
+
 /** \addtogroup Storage
  *  @{
  */
@@ -38,6 +41,7 @@ namespace quickstep {
  *        HashTableFactory to create a HashTable.
  **/
 enum class HashTableImplType {
+  kCollisionFreeVector,
   kLinearOpenAddressing,
   kSeparateChaining,
   kSimpleScalarSeparateChaining
@@ -74,6 +78,17 @@ class HashTableBase {
  public:
   virtual ~HashTableBase() {}
 
+ 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.
@@ -90,30 +105,23 @@ class HashTableBase {
    *
    * Optionally, we can also remove the AggregationStateHashTableBase
    * specialization from this file.
+   *
+   * TODO(jianqiao): Refractor the interface design for aggregation hash table.
    **/
-  virtual bool upsertValueAccessorCompositeKeyFast(
-      const std::vector<attribute_id> &argument,
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_attr_ids,
-      const bool check_for_null_keys) {
-    return false;
-  }
+  virtual bool upsertValueAccessorCompositeKey(
+      const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_attr_ids,
+      const ValueAccessorMultiplexer &accessor_mux) = 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/2d89e4fb/storage/HashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.hpp b/storage/HashTableFactory.hpp
index d690557..9686429 100644
--- a/storage/HashTableFactory.hpp
+++ b/storage/HashTableFactory.hpp
@@ -24,10 +24,12 @@
 #include <string>
 #include <vector>
 
+#include "storage/CollisionFreeVectorTable.hpp"
 #include "storage/HashTable.hpp"
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTable.pb.h"
 #include "storage/LinearOpenAddressingHashTable.hpp"
+#include "storage/PackedPayloadHashTable.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,62 @@ 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 Factory class that makes it easier to instantiate aggregation state
+ *        hash tables.
+ **/
+class AggregationStateHashTableFactory {
+ public:
+  /**
+   * @brief Create a new aggregation state hash table, with the type selected by
+   *        hash_table_type. Other parameters are forwarded to the hash table's
+   *        constructor.
+   *
+   * @param hash_table_type The specific hash table 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 hash table's constructor.
+   * @param num_entries The estimated number of entries the hash table will
+   *        hold. Forwarded as-is to the hash table's constructor.
+   * @param storage_manager The StorageManager to use (a StorageBlob will be
+   *        allocated to hold the hash table's contents). Forwarded as-is to the
+   *        hash table constructor.
+   * @return A new aggregation state hash table.
+   **/
+
+  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 PackedPayloadHashTable(
+            key_types, num_entries, handles, storage_manager);
+      case HashTableImplType::kCollisionFreeVector:
+        DCHECK_EQ(1u, key_types.size());
+        return new CollisionFreeVectorTable(
+            key_types.front(), num_entries, handles, storage_manager);
+      default: {
+        LOG(FATAL) << "Unrecognized HashTableImplType in "
+                   << "AggregationStateHashTableFactory::createResizable()";
+      }
+    }
+  }
+
+ 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/2d89e4fb/storage/HashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTablePool.hpp b/storage/HashTablePool.hpp
index 96cf849..df527b8 100644
--- a/storage/HashTablePool.hpp
+++ b/storage/HashTablePool.hpp
@@ -20,23 +20,21 @@
 #ifndef QUICKSTEP_STORAGE_HASH_TABLE_POOL_HPP_
 #define QUICKSTEP_STORAGE_HASH_TABLE_POOL_HPP_
 
-#include <chrono>
+#include <cstddef>
 #include <memory>
 #include <utility>
 #include <vector>
 
-#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"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
+class AggregationHandle;
 class StorageManager;
 class Type;
 
@@ -56,36 +54,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 +61,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 +94,7 @@ class HashTablePool {
         return ret_hash_table.release();
       }
     }
-    return createNewHashTableFast();
+    return createNewHashTable();
   }
 
   /**
@@ -180,18 +125,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 +151,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_;
 


[16/18] incubator-quickstep git commit: Revert "A workaround to remove query result relation in the distributed version."

Posted by zu...@apache.org.
Revert "A workaround to remove query result relation in the distributed version."

This reverts commit aef1c3586580cfa72eb031fafe08700f6d5d9a86.


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

Branch: refs/heads/tmb_poll_interval
Commit: dd8747fd7aba95ce2f0f325297e108bbac5d958d
Parents: 3b65b0f
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Feb 8 00:02:27 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Feb 8 00:35:53 2017 -0800

----------------------------------------------------------------------
 query_execution/ForemanDistributed.cpp | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dd8747fd/query_execution/ForemanDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanDistributed.cpp b/query_execution/ForemanDistributed.cpp
index e6f22ec..4d95f16 100644
--- a/query_execution/ForemanDistributed.cpp
+++ b/query_execution/ForemanDistributed.cpp
@@ -201,12 +201,8 @@ void ForemanDistributed::run() {
 
         // TODO(quickstep-team): Dynamically scale-up/down Shiftbosses.
         if (query_result_saved_shiftbosses_[query_id].size() == shiftboss_directory_.size()) {
-          const relation_id result_relation_id = proto.relation_id();
-          processSaveQueryResultResponseMessage(proto.cli_id(), result_relation_id);
+          processSaveQueryResultResponseMessage(proto.cli_id(), proto.relation_id());
           query_result_saved_shiftbosses_.erase(query_id);
-
-          // TODO(zuyu): Refactor to clean-up blocks in Shiftbosses.
-          catalog_database_->dropRelationById(result_relation_id);
         }
         break;
       }


[14/18] incubator-quickstep git commit: Avoid crash due to an error in deleting a file in HDFS.

Posted by zu...@apache.org.
Avoid crash due to an error in deleting a file in HDFS.


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

Branch: refs/heads/tmb_poll_interval
Commit: 9a95c23bdfebe877288f325b3844b3a01c77c08a
Parents: f46ae15
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Feb 7 23:23:50 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Feb 7 23:23:50 2017 -0800

----------------------------------------------------------------------
 storage/FileManagerHdfs.cpp | 15 ++++++++++-----
 1 file changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9a95c23b/storage/FileManagerHdfs.cpp
----------------------------------------------------------------------
diff --git a/storage/FileManagerHdfs.cpp b/storage/FileManagerHdfs.cpp
index 937750a..3a5167c 100644
--- a/storage/FileManagerHdfs.cpp
+++ b/storage/FileManagerHdfs.cpp
@@ -117,12 +117,17 @@ size_t FileManagerHdfs::numSlots(const block_id block) const {
 bool FileManagerHdfs::deleteBlockOrBlob(const block_id block) {
   const string filename(blockFilename(block));
 
-  if ((hdfsDelete(hdfs_, filename.c_str(), 0) == 0) || (errno == ENOENT)) {
-    return true;
-  } else {
-    LOG(ERROR) << "Failed to delete file " << filename << " with error: " << strerror(errno);
-    return false;
+  if (hdfsDelete(hdfs_, filename.c_str(), 0)) {
+    switch (errno) {
+      case EINPROGRESS:
+      case ENOENT:
+        break;
+      default:
+        LOG(ERROR) << "Failed to delete file " << filename << " with error: " << strerror(errno);
+    }
   }
+
+  return true;
 }
 
 bool FileManagerHdfs::readBlockOrBlob(const block_id block,


[15/18] incubator-quickstep git commit: Fixed a memory leak in the distributed version.

Posted by zu...@apache.org.
Fixed a memory leak in the distributed version.


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

Branch: refs/heads/tmb_poll_interval
Commit: 3b65b0fdf40efa47862323834917cc19f2478ba3
Parents: 9a95c23
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Feb 8 00:35:43 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Feb 8 00:35:52 2017 -0800

----------------------------------------------------------------------
 query_execution/PolicyEnforcerDistributed.cpp | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3b65b0fd/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
index 38b8a34..49a1d9a 100644
--- a/query_execution/PolicyEnforcerDistributed.cpp
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -240,6 +240,7 @@ void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manage
     TaggedMessage message(static_cast<const void*>(proto_bytes),
                           proto_length,
                           kQueryTeardownMessage);
+    free(proto_bytes);
 
     DLOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
                << "') to all Shiftbosses";


[18/18] incubator-quickstep git commit: Defined TMB Message Poll Interval as a gflag.

Posted by zu...@apache.org.
Defined TMB Message Poll Interval as a gflag.


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

Branch: refs/heads/tmb_poll_interval
Commit: 0724f42376c88e107e7baa28387791a23e927bb5
Parents: aa7f6fe
Author: Zuyu Zhang <zu...@apache.org>
Authored: Mon Feb 6 12:26:56 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Feb 8 00:53:36 2017 -0800

----------------------------------------------------------------------
 third_party/src/tmb/CMakeLists.txt            | 41 ++++++++++++----------
 third_party/src/tmb/include/tmb/message_bus.h |  5 ---
 third_party/src/tmb/src/message_bus.cc        | 20 +++++++++--
 3 files changed, 40 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0724f423/third_party/src/tmb/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/third_party/src/tmb/CMakeLists.txt b/third_party/src/tmb/CMakeLists.txt
index 14e467b..a280a93 100644
--- a/third_party/src/tmb/CMakeLists.txt
+++ b/third_party/src/tmb/CMakeLists.txt
@@ -379,6 +379,26 @@ if (ENABLE_ZOOKEEPER)
   include_directories(${ZOOKEEPER_INCLUDE_DIRS})
 endif()
 
+set_gflags_lib_name ()
+
+# NOTE(chasseur): We only add gflags and gtest to the build if those targets
+# don't already exist, so that TMB can be brought in to a build that already
+# uses one or both of those libraries with add_subdirectory() and not cause
+# name collisions.
+
+# Build GFlags command-line processing library if needed.
+if ((NOT TARGET ${GFLAGS_LIB_NAME}) AND (BUILD_BENCHMARKS OR ENABLE_NATIVENET))
+  add_subdirectory(third_party/gflags)
+  include_directories(${CMAKE_CURRENT_BINARY_DIR}/third_party/gflags/include)
+endif()
+
+# Googletest Framework For Unit Testing
+if (NOT TARGET gtest)
+  add_subdirectory(third_party/gtest)
+  include_directories(third_party/gtest/include)
+  enable_testing()
+endif()
+
 # Include path for TMB.
 include_directories(${PROJECT_SOURCE_DIR}/include)
 set(TMB_INCLUDE_DIRS ${TMB_INCLUDE_DIRS} CACHE STRING
@@ -391,7 +411,8 @@ link_directories(${tmb_BINARY_DIR}/src)
 add_library(tmb
             ${TMB_SRCS})
 target_link_libraries(tmb
-                      ${CMAKE_THREAD_LIBS_INIT})
+                      ${CMAKE_THREAD_LIBS_INIT}
+                      ${GFLAGS_LIB_NAME})
 
 if (ENABLE_LEVELDB)
   target_link_libraries(tmb
@@ -418,24 +439,6 @@ if (ENABLE_ZOOKEEPER)
                         ${ZOOKEEPER_LIBRARIES})
 endif()
 
-# NOTE(chasseur): We only add gflags and gtest to the build if those targets
-# don't already exist, so that TMB can be brought in to a build that already
-# uses one or both of those libraries with add_subdirectory() and not cause
-# name collisions.
-
-# Build GFlags command-line processing library if needed.
-if ((NOT TARGET gflags_nothreads-static) AND (BUILD_BENCHMARKS OR ENABLE_NATIVENET))
-  add_subdirectory(third_party/gflags)
-  include_directories(${CMAKE_CURRENT_BINARY_DIR}/third_party/gflags/include)
-endif()
-
-# Googletest Framework For Unit Testing
-if (NOT TARGET gtest)
-  add_subdirectory(third_party/gtest)
-  include_directories(third_party/gtest/include)
-  enable_testing()
-endif()
-
 # Build the tmb_net_server executable if enabled.
 if (ENABLE_NATIVENET)
   add_executable(tmb_net_server src/tmb_net_server.cc)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0724f423/third_party/src/tmb/include/tmb/message_bus.h
----------------------------------------------------------------------
diff --git a/third_party/src/tmb/include/tmb/message_bus.h b/third_party/src/tmb/include/tmb/message_bus.h
index a4ca525..74e298d 100644
--- a/third_party/src/tmb/include/tmb/message_bus.h
+++ b/third_party/src/tmb/include/tmb/message_bus.h
@@ -496,11 +496,6 @@ class MessageBus {
       internal::IteratorAdapter<const AnnotatedMessage> *adapter) = 0;
 
  private:
-  // The number of milliseconds to sleep between calls to
-  // ReceiveIfAvailableImpl() in the default active-polling implementation of
-  // ReceiveImpl().
-  static const unsigned int kReceivePollIntervalMS = 100;
-
   // Disallow copy and assign:
   MessageBus(const MessageBus &orig) = delete;
   MessageBus& operator=(const MessageBus &rhs) = delete;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0724f423/third_party/src/tmb/src/message_bus.cc
----------------------------------------------------------------------
diff --git a/third_party/src/tmb/src/message_bus.cc b/third_party/src/tmb/src/message_bus.cc
index 44324ec..7fd0efb 100644
--- a/third_party/src/tmb/src/message_bus.cc
+++ b/third_party/src/tmb/src/message_bus.cc
@@ -24,9 +24,25 @@
 #include <cstdlib>
 #include <thread>  // NOLINT(build/c++11)
 
+#include "gflags/gflags.h"
+
 namespace tmb {
 
-const unsigned int MessageBus::kReceivePollIntervalMS;
+static bool ValidateTmbReceivePollInterval(const char *flagname,
+                                           std::int32_t value) {
+  if (value > 0) {
+    return true;
+  } else {
+    std::fprintf(stderr, "--%s must be at least 1\n", flagname);
+    return false;
+  }
+}
+DEFINE_int32(tmb_receive_poll_interval, 50,
+             "The number of milliseconds to sleep between calls to ReceiveIfAvailableImpl() "
+             "in the default active-polling implementation of ReceiveImpl().");
+static const bool tmb_receive_poll_interval_dummy = gflags::RegisterFlagValidator(
+    &FLAGS_tmb_receive_poll_interval,
+    &ValidateTmbReceivePollInterval);
 
 internal::NetMessageRemovalInterface*
     MessageBus::GetNetMessageRemovalInterface() {
@@ -49,7 +65,7 @@ std::size_t MessageBus::ReceiveImpl(const client_id receiver_id,
                                                 pusher);
   while (received == 0) {
     std::this_thread::sleep_for(
-        std::chrono::milliseconds(kReceivePollIntervalMS));
+        std::chrono::milliseconds(FLAGS_tmb_receive_poll_interval));
     received = ReceiveIfAvailableImpl(receiver_id,
                                       minimum_priority,
                                       max_messages,


[12/18] incubator-quickstep git commit: - Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key. - Supports copy elision for aggregation.

Posted by zu...@apache.org.
- Adds CollisionFreeVectorTable to support specialized fast path aggregation for range-bounded single integer group-by key.
- Supports copy elision for aggregation.


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

Branch: refs/heads/tmb_poll_interval
Commit: 2d89e4fbf3b51d7768d928d56b25ec0ab52faabb
Parents: ec76096
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 Feb 7 15:29:19 2017 -0600

----------------------------------------------------------------------
 .../aggregation/AggregateFunctionCount.cpp      |    6 +-
 .../aggregation/AggregationConcreteHandle.cpp   |   32 +-
 .../aggregation/AggregationConcreteHandle.hpp   |  160 +-
 expressions/aggregation/AggregationHandle.hpp   |  193 +-
 .../aggregation/AggregationHandleAvg.cpp        |   95 +-
 .../aggregation/AggregationHandleAvg.hpp        |  115 +-
 .../aggregation/AggregationHandleCount.cpp      |  147 +-
 .../aggregation/AggregationHandleCount.hpp      |  154 +-
 .../aggregation/AggregationHandleDistinct.cpp   |   81 -
 .../aggregation/AggregationHandleDistinct.hpp   |  130 -
 .../aggregation/AggregationHandleMax.cpp        |   91 +-
 .../aggregation/AggregationHandleMax.hpp        |  111 +-
 .../aggregation/AggregationHandleMin.cpp        |   91 +-
 .../aggregation/AggregationHandleMin.hpp        |  120 +-
 .../aggregation/AggregationHandleSum.cpp        |   93 +-
 .../aggregation/AggregationHandleSum.hpp        |  114 +-
 expressions/aggregation/CMakeLists.txt          |   58 +-
 .../tests/AggregationHandleAvg_unittest.cpp     |  110 +-
 .../tests/AggregationHandleCount_unittest.cpp   |  145 +-
 .../tests/AggregationHandleMax_unittest.cpp     |  158 +-
 .../tests/AggregationHandleMin_unittest.cpp     |  158 +-
 .../tests/AggregationHandleSum_unittest.cpp     |  109 +-
 query_execution/QueryContext.hpp                |   14 -
 query_optimizer/CMakeLists.txt                  |    4 +
 query_optimizer/ExecutionGenerator.cpp          |  147 +-
 query_optimizer/ExecutionGenerator.hpp          |   20 +-
 relational_operators/CMakeLists.txt             |   15 +
 .../DestroyAggregationStateOperator.cpp         |    7 -
 .../FinalizeAggregationOperator.cpp             |   16 +-
 .../FinalizeAggregationOperator.hpp             |   14 +-
 .../InitializeAggregationOperator.cpp           |   72 +
 .../InitializeAggregationOperator.hpp           |  122 +
 relational_operators/WorkOrderFactory.cpp       |    3 +
 storage/AggregationOperationState.cpp           |  834 +++---
 storage/AggregationOperationState.hpp           |  178 +-
 storage/CMakeLists.txt                          |  131 +-
 storage/CollisionFreeVectorTable.cpp            |  285 +++
 storage/CollisionFreeVectorTable.hpp            |  730 ++++++
 storage/FastHashTable.hpp                       | 2403 ------------------
 storage/FastHashTableFactory.hpp                |  224 --
 storage/FastSeparateChainingHashTable.hpp       | 1551 -----------
 storage/HashTable.proto                         |    7 +-
 storage/HashTableBase.hpp                       |   42 +-
 storage/HashTableFactory.hpp                    |   63 +-
 storage/HashTablePool.hpp                       |   79 +-
 storage/PackedPayloadHashTable.cpp              |  463 ++++
 storage/PackedPayloadHashTable.hpp              |  995 ++++++++
 storage/PartitionedHashTablePool.hpp            |   56 +-
 storage/StorageBlock.cpp                        |  274 +-
 storage/StorageBlock.hpp                        |  167 --
 storage/ValueAccessorMultiplexer.hpp            |  145 ++
 .../BarrieredReadWriteConcurrentBitVector.hpp   |  282 ++
 utility/CMakeLists.txt                          |    7 +
 53 files changed, 4722 insertions(+), 7099 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/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/2d89e4fb/expressions/aggregation/AggregationConcreteHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.cpp b/expressions/aggregation/AggregationConcreteHandle.cpp
index e3fb520..fa21056 100644
--- a/expressions/aggregation/AggregationConcreteHandle.cpp
+++ b/expressions/aggregation/AggregationConcreteHandle.cpp
@@ -22,16 +22,14 @@
 #include <cstddef>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTable.hpp"
 #include "storage/HashTableFactory.hpp"
+#include "storage/PackedPayloadHashTable.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 
 namespace quickstep {
 
 class StorageManager;
 class Type;
-class ValueAccessor;
 
 AggregationStateHashTableBase* AggregationConcreteHandle::createDistinctifyHashTable(
     const HashTableImplType hash_table_impl,
@@ -39,30 +37,26 @@ AggregationStateHashTableBase* AggregationConcreteHandle::createDistinctifyHashT
     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(
+  return AggregationStateHashTableFactory::CreateResizable(
       hash_table_impl,
       key_types,
       estimated_num_distinct_keys,
+      {},
       storage_manager);
 }
 
 void AggregationConcreteHandle::insertValueAccessorIntoDistinctifyHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &key_ids,
+    const std::vector<MultiSourceAttributeId> &argument_ids,
+    const std::vector<MultiSourceAttributeId> &key_ids,
+    const ValueAccessorMultiplexer &accessor_mux,
     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 */);
+  std::vector<MultiSourceAttributeId> concatenated_ids(key_ids);
+  for (const MultiSourceAttributeId &arg_id : argument_ids) {
+    concatenated_ids.emplace_back(arg_id);
   }
+
+  static_cast<PackedPayloadHashTable *>(distinctify_hash_table)
+      ->upsertValueAccessorCompositeKey({}, concatenated_ids, accessor_mux);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationConcreteHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationConcreteHandle.hpp b/expressions/aggregation/AggregationConcreteHandle.hpp
index 398a032..2287ca1 100644
--- a/expressions/aggregation/AggregationConcreteHandle.hpp
+++ b/expressions/aggregation/AggregationConcreteHandle.hpp
@@ -21,14 +21,15 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_CONCRETE_HANDLE_HPP_
 
 #include <cstddef>
+#include <cstdint>
 #include <utility>
 #include <vector>
 
-#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 "storage/PackedPayloadHashTable.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
@@ -40,7 +41,6 @@ namespace quickstep {
 
 class StorageManager;
 class Type;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -51,7 +51,7 @@ class ValueAccessor;
  *        merging two group by hash tables.
  **/
 template <typename HandleT>
-class HashTableStateUpserterFast {
+class HashTableStateUpserter {
  public:
   /**
    * @brief Constructor.
@@ -61,8 +61,8 @@ class HashTableStateUpserterFast {
    *        table. The corresponding state (for the same key) in the destination
    *        hash table will be upserted.
    **/
-  HashTableStateUpserterFast(const HandleT &handle,
-                             const std::uint8_t *source_state)
+  HashTableStateUpserter(const HandleT &handle,
+                         const std::uint8_t *source_state)
       : handle_(handle), source_state_(source_state) {}
 
   /**
@@ -72,14 +72,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);
 };
 
 /**
@@ -93,74 +93,62 @@ class HashTableStateUpserterFast {
  **/
 class AggregationConcreteHandle : public AggregationHandle {
  public:
-  /**
-   * @brief Default implementaion for AggregationHandle::accumulateNullary().
-   */
-  AggregationState* accumulateNullary(
-      const std::size_t num_tuples) const override {
-    LOG(FATAL) << "Called accumulateNullary on an AggregationHandle that "
-               << "takes at least one argument.";
-  }
-
-  /**
-   * @brief Implementaion for AggregationHandle::createDistinctifyHashTable()
-   *        that creates a new HashTable for the distinctify step for
-   *        DISTINCT aggregation.
-   */
   AggregationStateHashTableBase* createDistinctifyHashTable(
       const HashTableImplType hash_table_impl,
       const std::vector<const Type *> &key_types,
       const std::size_t estimated_num_distinct_keys,
       StorageManager *storage_manager) const override;
 
-  /**
-   * @brief Implementaion for
-   * AggregationHandle::insertValueAccessorIntoDistinctifyHashTable()
-   * that inserts the GROUP BY expressions and aggregation arguments together
-   * as keys into the distinctify hash table.
-   */
   void insertValueAccessorIntoDistinctifyHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_ids,
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      const ValueAccessorMultiplexer &accessor_mux,
       AggregationStateHashTableBase *distinctify_hash_table) const override;
 
+  void blockUpdate() override {
+    block_update_ = true;
+  }
+
+  void allowUpdate() override {
+    block_update_ = false;
+  }
+
  protected:
-  AggregationConcreteHandle() {}
+  explicit AggregationConcreteHandle(const AggregationID agg_id)
+      : AggregationHandle(agg_id),
+        block_update_(false) {}
 
   template <typename HandleT, typename StateT>
-  StateT* aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
+  StateT* aggregateOnDistinctifyHashTableForSingleUnaryHelper(
       const AggregationStateHashTableBase &distinctify_hash_table) const;
 
-  template <typename HandleT, typename HashTableT>
-  void aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
+  template <typename HandleT>
+  void aggregateOnDistinctifyHashTableForGroupByUnaryHelper(
       const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *hash_table,
-      std::size_t index) const;
+      const std::size_t index,
+      AggregationStateHashTableBase *hash_table) const;
 
-  template <typename HandleT, typename HashTableT>
-  ColumnVector* finalizeHashTableHelperFast(
+  template <typename HandleT>
+  ColumnVector* finalizeHashTableHelper(
       const Type &result_type,
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const;
 
-  template <typename HandleT, typename HashTableT>
-  inline TypedValue finalizeGroupInHashTableFast(
+  template <typename HandleT>
+  inline TypedValue finalizeGroupInHashTable(
       const AggregationStateHashTableBase &hash_table,
-      const std::vector<TypedValue> &group_key,
-      int index) const {
+      const std::size_t index,
+      const std::vector<TypedValue> &group_key) const {
     const std::uint8_t *group_state =
-        static_cast<const HashTableT &>(hash_table).getSingleCompositeKey(group_key, index);
+        static_cast<const PackedPayloadHashTable &>(hash_table)
+            .getSingleCompositeKey(group_key, index);
     DCHECK(group_state != nullptr)
         << "Could not find entry for specified group_key in HashTable";
-    return static_cast<const HandleT *>(this)->finalizeHashTableEntryFast(
-        group_state);
+    return static_cast<const HandleT *>(this)->finalizeHashTableEntry(group_state);
   }
 
-  template <typename HandleT, typename HashTableT>
-  void mergeGroupByHashTablesHelperFast(
-      const AggregationStateHashTableBase &source_hash_table,
-      AggregationStateHashTableBase *destination_hash_table) const;
+  bool block_update_;
 
  private:
   DISALLOW_COPY_AND_ASSIGN(AggregationConcreteHandle);
@@ -185,17 +173,10 @@ class HashTableAggregateFinalizer {
         output_column_vector_(output_column_vector) {}
 
   inline void operator()(const std::vector<TypedValue> &group_by_key,
-                         const AggregationState &group_state) {
-    group_by_keys_->emplace_back(group_by_key);
-    output_column_vector_->appendTypedValue(
-        handle_.finalizeHashTableEntry(group_state));
-  }
-
-  inline void operator()(const std::vector<TypedValue> &group_by_key,
-                         const unsigned char *byte_ptr) {
+                         const std::uint8_t *byte_ptr) {
     group_by_keys_->emplace_back(group_by_key);
     output_column_vector_->appendTypedValue(
-        handle_.finalizeHashTableEntryFast(byte_ptr));
+        handle_.finalizeHashTableEntry(byte_ptr));
   }
 
  private:
@@ -211,7 +192,7 @@ class HashTableAggregateFinalizer {
 
 template <typename HandleT, typename StateT>
 StateT* AggregationConcreteHandle::
-    aggregateOnDistinctifyHashTableForSingleUnaryHelperFast(
+    aggregateOnDistinctifyHashTableForSingleUnaryHelper(
         const AggregationStateHashTableBase &distinctify_hash_table) const {
   const HandleT &handle = static_cast<const HandleT &>(*this);
   StateT *state = static_cast<StateT *>(createInitialState());
@@ -219,15 +200,14 @@ StateT* AggregationConcreteHandle::
   // A lambda function which will be called on each key from the distinctify
   // hash table.
   const auto aggregate_functor = [&handle, &state](
-      const TypedValue &key, const std::uint8_t &dumb_placeholder) {
+      const 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);
+  const auto &hash_table =
+      static_cast<const PackedPayloadHashTable &>(distinctify_hash_table);
   // Invoke the lambda function "aggregate_functor" on each key from the
   // distinctify hash table.
   hash_table.forEach(&aggregate_functor);
@@ -235,20 +215,20 @@ StateT* AggregationConcreteHandle::
   return state;
 }
 
-template <typename HandleT, typename HashTableT>
+template <typename HandleT>
 void AggregationConcreteHandle::
-    aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast(
+    aggregateOnDistinctifyHashTableForGroupByUnaryHelper(
         const AggregationStateHashTableBase &distinctify_hash_table,
-        AggregationStateHashTableBase *aggregation_hash_table,
-        std::size_t index) const {
+        const std::size_t index,
+        AggregationStateHashTableBase *aggregation_hash_table) const {
   const HandleT &handle = static_cast<const HandleT &>(*this);
-  HashTableT *target_hash_table =
-      static_cast<HashTableT *>(aggregation_hash_table);
+  PackedPayloadHashTable *target_hash_table =
+      static_cast<PackedPayloadHashTable *>(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) {
+      std::vector<TypedValue> &key, const std::uint8_t *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.
@@ -258,28 +238,28 @@ void AggregationConcreteHandle::
     // 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);
+      handle.iterateUnaryInl(argument, state);
     };
 
-    target_hash_table->upsertCompositeKeyFast(key, nullptr, &upserter, index);
+    target_hash_table->upsertCompositeKey(key, &upserter, index);
   };
 
-  const HashTableT &source_hash_table =
-      static_cast<const HashTableT &>(distinctify_hash_table);
+  const PackedPayloadHashTable &source_hash_table =
+      static_cast<const PackedPayloadHashTable &>(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);
+  source_hash_table.forEachCompositeKey(&aggregate_functor);
 }
 
-template <typename HandleT, typename HashTableT>
-ColumnVector* AggregationConcreteHandle::finalizeHashTableHelperFast(
+template <typename HandleT>
+ColumnVector* AggregationConcreteHandle::finalizeHashTableHelper(
     const Type &result_type,
     const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
+    const std::size_t index,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
   const HandleT &handle = static_cast<const HandleT &>(*this);
-  const HashTableT &hash_table_concrete =
-      static_cast<const HashTableT &>(hash_table);
+  const PackedPayloadHashTable &hash_table_concrete =
+      static_cast<const PackedPayloadHashTable &>(hash_table);
 
   if (group_by_keys->empty()) {
     if (NativeColumnVector::UsableForType(result_type)) {
@@ -287,14 +267,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.forEachCompositeKey(&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.forEachCompositeKey(&finalizer, index);
       return result;
     }
   } else {
@@ -303,8 +283,8 @@ 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>(
-                hash_table, group_by_key, index));
+            finalizeGroupInHashTable<HandleT>(
+                hash_table, index, group_by_key));
       }
       return result;
     } else {
@@ -312,8 +292,8 @@ 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>(
-                hash_table, group_by_key, index));
+            finalizeGroupInHashTable<HandleT>(
+                hash_table, index, group_by_key));
       }
       return result;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index 4b51179..9c7b166 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -21,20 +21,21 @@
 #define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_HPP_
 
 #include <cstddef>
-#include <memory>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
 #include "storage/HashTableBase.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 class ColumnVector;
 class StorageManager;
 class Type;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -109,34 +110,34 @@ class AggregationHandle {
   virtual ~AggregationHandle() {}
 
   /**
-   * @brief Create an initial "blank" state for this aggregation.
+   * @brief Get the ID of this aggregation.
    *
-   * @return An initial "blank" state for this particular aggregation.
+   * @return The AggregationID of this AggregationHandle.
    **/
-  virtual AggregationState* createInitialState() const = 0;
+  inline AggregationID getAggregationID() const {
+    return agg_id_;
+  }
 
   /**
-   * @brief Create a new HashTable for aggregation with GROUP BY.
+   * @brief Get the list of Types (in order) for arguments to this aggregation.
    *
-   * @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.
+   * @return The list of Types for arguments to this aggregation.
+   */
+  virtual std::vector<const Type *> getArgumentTypes() const = 0;
+
+  /**
+   * @brief Get the result Type of this aggregation.
+   *
+   * @return The result Type of this aggregation.
+   */
+  virtual const Type* getResultType() const = 0;
+
+  /**
+   * @brief Create an initial "blank" state for this aggregation.
+   *
+   * @return An initial "blank" state for this particular aggregation.
    **/
-  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;
+  virtual AggregationState* createInitialState() const = 0;
 
   /**
    * @brief Accumulate over tuples for a nullary aggregate function (one that
@@ -146,70 +147,31 @@ class AggregationHandle {
    *        data is accessed, the only thing that a nullary aggeregate can know
    *        about input is its cardinality.
    * @return A new AggregationState which contains the accumulated results from
-   *         applying the (nullary) aggregate to the specified number of
-   *         tuples.
+   *         applying the (nullary) aggregate to the specified number of tuples.
    **/
   virtual AggregationState* accumulateNullary(
-      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.
-   *
-   * @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;
+      const std::size_t num_tuples) const {
+    LOG(FATAL) << "Called accumulateNullary on an AggregationHandle that "
+               << "takes at least one argument.";
+  }
 
-#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.
+   *        the ValueAccessors from a ValueAccessorMultiplexer 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.
+   * @param argument_ids The multi-source attribute ids that correspond to the
+   *        columns in \p accessor_mux to aggeregate. These correspond to the
+   *        aggregate function's arguments, in order.
+   * @param accessor_mux A ValueAccessorMultiplexer object that contains the
+   *        ValueAccessors.
    * @return A new AggregationState which contains the accumulated results from
-   *         applying the aggregate to the specified columns in accessor.
+   *         applying the aggregate to the specified columns.
    *         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(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const = 0;
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) const = 0;
 
   /**
    * @brief Merge two AggregationStates, updating one in-place. This computes a
@@ -253,24 +215,24 @@ class AggregationHandle {
    * @param hash_table The HashTable to finalize states from. This should have
    *        have been created by calling createGroupByHashTable() on this same
    *        AggregationHandle.
+   * @param index The index of the AggregationHandle to be finalized.
    * @param group_by_keys A pointer to a vector of vectors of GROUP BY keys. If
    *        this is initially empty, it will be filled in with the GROUP BY
    *        keys visited by this method in the same order as the finalized
    *        values returned in the ColumnVector. If this is already filled in,
    *        then this method will visit the GROUP BY keys in the exact order
    *        specified.
-   * @param index The index of the AggregationHandle to be finalized.
    *
    * @return A ColumnVector containing each group's finalized aggregate value.
    **/
   virtual ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const = 0;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const = 0;
 
   /**
    * @brief Create a new HashTable for the distinctify step for DISTINCT
-   * aggregation.
+   *        aggregation.
    *
    * Distinctify is the first step for DISTINCT aggregation. This step inserts
    * the GROUP BY expression values and aggregation arguments together as keys
@@ -283,8 +245,8 @@ class AggregationHandle {
    * we simply treat it as a special GROUP BY case that the GROUP BY expression
    * vector is empty.
    *
-   * @param hash_table_impl The choice of which concrete HashTable
-   *        implementation to use.
+   * @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
@@ -307,13 +269,14 @@ class AggregationHandle {
 
   /**
    * @brief Inserts the GROUP BY expressions and aggregation arguments together
-   * as keys into the distinctify hash table.
+   *        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 argument_ids The argument ids that correspond to the columns in
+   *        \p accessor_mux.
+   * @param key_ids The group-by key ids that correspond to the columns in
+   *        \p accessor_mux.
+   * @param accessor_mux A ValueAccessorMultiplexer object that contains the
+   *        ValueAccessors to be iterated over to read tuples.
    * @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
@@ -321,13 +284,14 @@ class AggregationHandle {
    *        by calling createDistinctifyHashTable();
    */
   virtual void insertValueAccessorIntoDistinctifyHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &key_ids,
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_ids,
+      const ValueAccessorMultiplexer &accessor_mux,
       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.
+   *        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
@@ -346,25 +310,26 @@ class AggregationHandle {
    * @param distinctify_hash_table Hash table which stores the GROUP BY
    *        expression values and aggregation arguments together as hash table
    *        keys.
+   * @param index The index of the AggregationHandle to perform aggregation.
    * @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;
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) 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.
+   * @brief Update the aggregation state for nullary aggregation function, e.g.
    *        COUNT(*).
    *
    * @note This function should be overloaded by those aggregation function
@@ -372,7 +337,10 @@ class AggregationHandle {
    *
    * @param byte_ptr The pointer where the aggregation state is stored.
    **/
-  virtual void updateStateNullary(std::uint8_t *byte_ptr) const {}
+  virtual void updateStateNullary(std::uint8_t *byte_ptr) const {
+    LOG(FATAL) << "Called updateStateNullary on an AggregationHandle that "
+               << "takes at least one argument.";
+  }
 
   /**
    * @brief Update the aggregation state for unary aggregation function e.g.
@@ -383,7 +351,7 @@ class AggregationHandle {
    * @param byte_ptr The pointer where the aggregation state is stored.
    **/
   virtual void updateStateUnary(const TypedValue &argument,
-                                std::uint8_t *byte_ptr) const {}
+                                std::uint8_t *byte_ptr) const = 0;
 
   /**
    * @brief Merge two aggregation states for this aggregation handle.
@@ -394,8 +362,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 = 0;
 
   /**
    * @brief Initialize the payload (in the aggregation hash table) for the given
@@ -403,7 +371,7 @@ class AggregationHandle {
    *
    * @param byte_ptr The pointer to the aggregation state in the hash table.
    **/
-  virtual void initPayload(std::uint8_t *byte_ptr) const {}
+  virtual void initPayload(std::uint8_t *byte_ptr) const = 0;
 
   /**
    * @brief Destroy the payload (in the aggregation hash table) for the given
@@ -411,22 +379,25 @@ class AggregationHandle {
    *
    * @param byte_ptr The pointer to the aggregation state in the hash table.
    **/
-  virtual void destroyPayload(std::uint8_t *byte_ptr) const {}
+  virtual void destroyPayload(std::uint8_t *byte_ptr) const = 0;
 
   /**
    * @brief Inform the aggregation handle to block (prohibit) updates on the
    *        aggregation state.
    **/
-  virtual void blockUpdate() {}
+  virtual void blockUpdate() = 0;
 
   /**
-   * @brief Inform the aggregation handle to allow updates on the
-   *        aggregation state.
+   * @brief Inform the aggregation handle to allow updates on the aggregation
+   *        state.
    **/
-  virtual void allowUpdate() {}
+  virtual void allowUpdate() = 0;
 
  protected:
-  AggregationHandle() {}
+  explicit 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/2d89e4fb/expressions/aggregation/AggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index 2481092..46bec1e 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.cpp
@@ -20,12 +20,13 @@
 #include "expressions/aggregation/AggregationHandleAvg.hpp"
 
 #include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
@@ -39,10 +40,11 @@
 
 namespace quickstep {
 
-class StorageManager;
+class ColumnVector;
 
 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 +89,29 @@ 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::accumulateValueAccessor(
+    const std::vector<MultiSourceAttributeId> &argument_ids,
+    const ValueAccessorMultiplexer &accessor_mux) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for AVG: " << argument_ids.size();
 
-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();
+  const ValueAccessorSource argument_source = argument_ids.front().source;
+  const attribute_id argument_id = argument_ids.front().attr_id;
 
-  AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
-  std::size_t count = 0;
-  state->sum_ = fast_add_operator_->accumulateColumnVector(
-      state->sum_, *column_vectors.front(), &count);
-  state->count_ = count;
-  return state;
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleAvg::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  DCHECK_EQ(1u, accessor_ids.size())
-      << "Got wrong number of attributes for AVG: " << accessor_ids.size();
+  DCHECK(argument_source != ValueAccessorSource::kInvalid);
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
   AggregationStateAvg *state = new AggregationStateAvg(blank_state_);
   std::size_t count = 0;
-  state->sum_ = fast_add_operator_->accumulateValueAccessor(
-      state->sum_, accessor, accessor_ids.front(), &count);
+  state->sum_ =
+      fast_add_operator_->accumulateValueAccessor(
+          state->sum_,
+          accessor_mux.getValueAccessorBySource(argument_source),
+          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 +126,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 *>(
@@ -177,29 +156,25 @@ TypedValue AggregationHandleAvg::finalize(const AggregationState &state) const {
 
 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);
+    const std::size_t index,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  return finalizeHashTableHelper<AggregationHandleAvg>(
+      *result_type_, hash_table, index, group_by_keys);
 }
 
-AggregationState*
-AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleAvg::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleAvg,
-      AggregationStateAvg>(distinctify_hash_table);
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleAvg, AggregationStateAvg>(
+          distinctify_hash_table);
 }
 
 void AggregationHandleAvg::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleAvg,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+    const std::size_t index,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<AggregationHandleAvg>(
+      distinctify_hash_table, index, aggregation_hash_table);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index 47132c6..970561c 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -25,11 +25,9 @@
 #include <memory>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
@@ -40,9 +38,8 @@
 
 namespace quickstep {
 
+class AggregationStateHashTableBase;
 class ColumnVector;
-class StorageManager;
-class ValueAccessor;
 
 /** \addtogroup Expressions
  *  @{
@@ -106,19 +103,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.
-   **/
   inline void iterateUnaryInl(AggregationStateAvg *state,
                               const TypedValue &value) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
@@ -129,8 +125,8 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     ++state->count_;
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value,
-                                  std::uint8_t *byte_ptr) const {
+  inline void iterateUnaryInl(const TypedValue &value,
+                              std::uint8_t *byte_ptr) const {
     DCHECK(value.isPlausibleInstanceOf(argument_type_.getSignature()));
     if (value.isNull()) return;
     TypedValue *sum_ptr =
@@ -141,16 +137,18 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     ++(*count_ptr);
   }
 
-  inline void updateStateUnary(const TypedValue &argument,
-                               std::uint8_t *byte_ptr) const override {
-    if (!block_update_) {
-      iterateUnaryInlFast(argument, byte_ptr);
-    }
-  }
+  AggregationState* accumulateValueAccessor(
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) const override;
 
-  void blockUpdate() override { block_update_ = true; }
+  void mergeStates(const AggregationState &source,
+                   AggregationState *destination) const override;
 
-  void allowUpdate() override { block_update_ = false; }
+  TypedValue finalize(const AggregationState &state) const override;
+
+  std::size_t getPayloadSize() const override {
+    return blank_state_.getPayloadSize();
+  }
 
   void initPayload(std::uint8_t *byte_ptr) const override {
     TypedValue *sum_ptr =
@@ -169,43 +167,17 @@ 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 {
+    if (!block_update_) {
+      iterateUnaryInl(argument, byte_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 =
@@ -218,31 +190,16 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
 
   ColumnVector* finalizeHashTable(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for AVG aggregation.
-   */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        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();
-  }
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
 
  private:
   friend class AggregateFunctionAvg;
@@ -261,8 +218,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/2d89e4fb/expressions/aggregation/AggregationHandleCount.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.cpp b/expressions/aggregation/AggregationHandleCount.cpp
index 034c942..cf92ec7 100644
--- a/expressions/aggregation/AggregationHandleCount.cpp
+++ b/expressions/aggregation/AggregationHandleCount.cpp
@@ -21,100 +21,48 @@
 
 #include <atomic>
 #include <cstddef>
-#include <memory>
+#include <cstdint>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 #include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "storage/ValueAccessorUtil.hpp"
-#endif
-
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
-#include "types/containers/ColumnVectorUtil.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
-class StorageManager;
-class Type;
-class ValueAccessor;
+class ColumnVector;
 
 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>
+    ::accumulateValueAccessor(
+        const std::vector<MultiSourceAttributeId> &argument_ids,
+        const ValueAccessorMultiplexer &accessor_mux) 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 ValueAccessorSource argument_source = argument_ids.front().source;
+  const attribute_id argument_id = argument_ids.front().attr_id;
 
-  return new AggregationStateCount(count);
-}
+  DCHECK(argument_source != ValueAccessorSource::kInvalid);
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
-#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)
+      accessor_mux.getValueAccessorBySource(argument_source),
+      [&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(argument_id).isNull();
           }
         } else {
           count = accessor->getNumTuples();
@@ -123,24 +71,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 +86,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);
@@ -165,38 +95,35 @@ void AggregationHandleCount<count_star, nullable_type>::mergeStatesFast(
 }
 
 template <bool count_star, bool nullable_type>
-ColumnVector*
-AggregationHandleCount<count_star, nullable_type>::finalizeHashTable(
-    const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  return finalizeHashTableHelperFast<
-      AggregationHandleCount<count_star, nullable_type>,
-      AggregationStateFastHashTable>(
-      TypeFactory::GetType(kLong), hash_table, group_by_keys, index);
+ColumnVector* AggregationHandleCount<count_star, nullable_type>
+    ::finalizeHashTable(
+        const AggregationStateHashTableBase &hash_table,
+        const std::size_t index,
+        std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  return finalizeHashTableHelper<
+      AggregationHandleCount<count_star, nullable_type>>(
+          TypeFactory::GetType(kLong), hash_table, index, group_by_keys);
 }
 
 template <bool count_star, bool nullable_type>
-AggregationState* AggregationHandleCount<count_star, nullable_type>::
-    aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleCount<count_star, nullable_type>
+    ::aggregateOnDistinctifyHashTableForSingle(
         const AggregationStateHashTableBase &distinctify_hash_table) const {
-  DCHECK_EQ(count_star, false);
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
       AggregationHandleCount<count_star, nullable_type>,
-      AggregationStateCount>(distinctify_hash_table);
+      AggregationStateCount>(
+          distinctify_hash_table);
 }
 
 template <bool count_star, bool nullable_type>
-void AggregationHandleCount<count_star, nullable_type>::
-    aggregateOnDistinctifyHashTableForGroupBy(
+void AggregationHandleCount<count_star, nullable_type>
+    ::aggregateOnDistinctifyHashTableForGroupBy(
         const AggregationStateHashTableBase &distinctify_hash_table,
-        AggregationStateHashTableBase *aggregation_hash_table,
-        std::size_t index) const {
-  DCHECK_EQ(count_star, false);
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleCount<count_star, nullable_type>,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+        const std::size_t index,
+        AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<
+      AggregationHandleCount<count_star, nullable_type>>(
+          distinctify_hash_table, index, aggregation_hash_table);
 }
 
 // Explicitly instantiate and compile in the different versions of

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleCount.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleCount.hpp b/expressions/aggregation/AggregationHandleCount.hpp
index 6aab0cd..72ea923 100644
--- a/expressions/aggregation/AggregationHandleCount.hpp
+++ b/expressions/aggregation/AggregationHandleCount.hpp
@@ -23,23 +23,21 @@
 #include <atomic>
 #include <cstddef>
 #include <cstdint>
-#include <memory>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregationConcreteHandle.hpp"
 #include "expressions/aggregation/AggregationHandle.hpp"
-#include "storage/FastHashTable.hpp"
-#include "storage/HashTableBase.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "types/LongType.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
+class AggregationStateHashTableBase;
 class ColumnVector;
-class StorageManager;
 class Type;
-class ValueAccessor;
 
 template <bool, bool>
 class AggregationHandleCount;
@@ -98,28 +96,31 @@ 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 {
+  inline void iterateNullaryInl(std::uint8_t *byte_ptr) const {
     std::int64_t *count_ptr = reinterpret_cast<std::int64_t *>(byte_ptr);
-    (*count_ptr)++;
+    ++(*count_ptr);
   }
 
-  /**
-   * @brief Iterate with count aggregation state.
-   */
   inline void iterateUnaryInl(AggregationStateCount *state,
                               const TypedValue &value) const {
     if ((!nullable_type) || (!value.isNull())) {
@@ -127,118 +128,89 @@ class AggregationHandleCount : public AggregationConcreteHandle {
     }
   }
 
-  inline void iterateUnaryInlFast(const TypedValue &value,
-                                  std::uint8_t *byte_ptr) const {
+  inline void iterateUnaryInl(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);
+      ++(*reinterpret_cast<std::int64_t *>(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(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
+      const std::vector<MultiSourceAttributeId> &argument_ids,
+      const ValueAccessorMultiplexer &accessor_mux) 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 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);
+  void destroyPayload(std::uint8_t *byte_ptr) const override {}
+
+  inline void updateStateNullary(std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateNullaryInl(byte_ptr);
+    }
+  }
+
+  inline void updateStateUnary(const TypedValue &argument,
+                               std::uint8_t *byte_ptr) const override {
+    if (!block_update_) {
+      iterateUnaryInl(argument, 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(
       const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
+      const std::size_t index,
+      std::vector<std::vector<TypedValue>> *group_by_keys) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForSingle()
-   *        for SUM aggregation.
-   */
   AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override;
+      const AggregationStateHashTableBase &distinctify_hash_table) const override;
 
-  /**
-   * @brief Implementation of
-   *        AggregationHandle::aggregateOnDistinctifyHashTableForGroupBy()
-   *        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); }
+      const std::size_t index,
+      AggregationStateHashTableBase *aggregation_hash_table) const override;
 
  private:
   friend class AggregateFunctionCount;
 
   /**
-   * @brief Constructor.
+   * @brief Initialize handle for count.
+   *
+   * @param argument_type Type of the value to be counted. The parameter should
+   *        be nullptr for nullary aggregation (i.e. COUNT(*)).
    **/
-  AggregationHandleCount() : block_update_(false) {}
+  explicit 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/2d89e4fb/expressions/aggregation/AggregationHandleDistinct.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.cpp b/expressions/aggregation/AggregationHandleDistinct.cpp
deleted file mode 100644
index 0dc8b56..0000000
--- a/expressions/aggregation/AggregationHandleDistinct.cpp
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#include "expressions/aggregation/AggregationHandleDistinct.hpp"
-
-#include <cstddef>
-#include <memory>
-#include <vector>
-#include <utility>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-
-#include "types/TypedValue.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-class ColumnVector;
-class StorageManager;
-class Type;
-class ValueAccessor;
-
-AggregationStateHashTableBase* AggregationHandleDistinct::createGroupByHashTable(
-    const HashTableImplType hash_table_impl,
-    const std::vector<const Type*> &group_by_types,
-    const std::size_t estimated_num_groups,
-    StorageManager *storage_manager) const {
-  return createDistinctifyHashTable(
-      hash_table_impl,
-      group_by_types,
-      estimated_num_groups,
-      storage_manager);
-}
-
-void AggregationHandleDistinct::aggregateValueAccessorIntoHashTable(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &argument_ids,
-    const std::vector<attribute_id> &group_by_key_ids,
-    AggregationStateHashTableBase *hash_table) const {
-  DCHECK_EQ(argument_ids.size(), 0u);
-
-  insertValueAccessorIntoDistinctifyHashTable(
-      accessor,
-      group_by_key_ids,
-      hash_table);
-}
-
-ColumnVector* AggregationHandleDistinct::finalizeHashTable(
-    const AggregationStateHashTableBase &hash_table,
-    std::vector<std::vector<TypedValue>> *group_by_keys,
-    int index) const {
-  DCHECK(group_by_keys->empty());
-
-  const auto keys_retriever = [&group_by_keys](std::vector<TypedValue> &group_by_key,
-                                               const bool &dumb_placeholder) -> void {
-    group_by_keys->emplace_back(std::move(group_by_key));
-  };
-  static_cast<const AggregationStateFastHashTable&>(hash_table).forEachCompositeKeyFast(&keys_retriever);
-
-  return nullptr;
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleDistinct.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleDistinct.hpp b/expressions/aggregation/AggregationHandleDistinct.hpp
deleted file mode 100644
index 838bfdd..0000000
--- a/expressions/aggregation/AggregationHandleDistinct.hpp
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#ifndef QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_DISTINCT_HPP_
-#define QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_DISTINCT_HPP_
-
-#include <cstddef>
-#include <memory>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "expressions/aggregation/AggregationConcreteHandle.hpp"
-#include "storage/HashTableBase.hpp"
-#include "types/TypedValue.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-class AggregationState;
-class ColumnVector;
-class StorageManager;
-class Type;
-class ValueAccessor;
-
-/** \addtogroup Expressions
- *  @{
- */
-
-class AggregationHandleDistinct : public AggregationConcreteHandle {
- public:
-  /**
-   * @brief Constructor.
-   **/
-  AggregationHandleDistinct() {}
-
-  AggregationState* createInitialState() const override {
-    LOG(FATAL)
-        << "AggregationHandleDistinct does not support createInitialState().";
-  }
-
-  AggregationState* accumulateNullary(
-      const std::size_t num_tuples) const override {
-    LOG(FATAL)
-        << "AggregationHandleDistinct does not support accumulateNullary().";
-  }
-
-  AggregationState* accumulateColumnVectors(
-      const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
-      const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support "
-                  "accumulateColumnVectors().";
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  AggregationState* accumulateValueAccessor(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &accessor_ids) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support "
-                  "accumulateValueAccessor().";
-  }
-#endif
-
-  void mergeStates(const AggregationState &source,
-                   AggregationState *destination) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support mergeStates().";
-  }
-
-  TypedValue finalize(const AggregationState &state) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support finalize().";
-  }
-
-  AggregationState* aggregateOnDistinctifyHashTableForSingle(
-      const AggregationStateHashTableBase &distinctify_hash_table)
-      const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support "
-               << "aggregateOnDistinctifyHashTableForSingle().";
-  }
-
-  void aggregateOnDistinctifyHashTableForGroupBy(
-      const AggregationStateHashTableBase &distinctify_hash_table,
-      AggregationStateHashTableBase *groupby_hash_table,
-      std::size_t index) const override {
-    LOG(FATAL) << "AggregationHandleDistinct does not support "
-               << "aggregateOnDistinctifyHashTableForGroupBy().";
-  }
-
-  AggregationStateHashTableBase* createGroupByHashTable(
-      const HashTableImplType hash_table_impl,
-      const std::vector<const Type *> &group_by_types,
-      const std::size_t estimated_num_groups,
-      StorageManager *storage_manager) const override;
-
-  void aggregateValueAccessorIntoHashTable(
-      ValueAccessor *accessor,
-      const std::vector<attribute_id> &argument_ids,
-      const std::vector<attribute_id> &group_by_key_ids,
-      AggregationStateHashTableBase *hash_table) const override;
-
-  ColumnVector* finalizeHashTable(
-      const AggregationStateHashTableBase &hash_table,
-      std::vector<std::vector<TypedValue>> *group_by_keys,
-      int index) const override;
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(AggregationHandleDistinct);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_EXPRESSIONS_AGGREGATION_AGGREGATION_HANDLE_DISTINCT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2d89e4fb/expressions/aggregation/AggregationHandleMax.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.cpp b/expressions/aggregation/AggregationHandleMax.cpp
index c2d571b..fe1773f 100644
--- a/expressions/aggregation/AggregationHandleMax.cpp
+++ b/expressions/aggregation/AggregationHandleMax.cpp
@@ -19,15 +19,16 @@
 
 #include "expressions/aggregation/AggregationHandleMax.hpp"
 
+#include <cstddef>
+#include <cstdint>
 #include <memory>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
-#include "storage/HashTable.hpp"
-#include "storage/HashTableFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
@@ -36,54 +37,32 @@
 
 namespace quickstep {
 
-class StorageManager;
+class ColumnVector;
 
 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::accumulateValueAccessor(
+    const std::vector<MultiSourceAttributeId> &argument_ids,
+    const ValueAccessorMultiplexer &accessor_mux) const {
+  DCHECK_EQ(1u, argument_ids.size())
+      << "Got wrong number of attributes for MAX: " << argument_ids.size();
 
-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();
+  const ValueAccessorSource argument_source = argument_ids.front().source;
+  const attribute_id argument_id = argument_ids.front().attr_id;
 
-  return new AggregationStateMax(fast_comparator_->accumulateColumnVector(
-      type_.getNullableVersion().makeNullValue(), *column_vectors.front()));
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-AggregationState* AggregationHandleMax::accumulateValueAccessor(
-    ValueAccessor *accessor,
-    const std::vector<attribute_id> &accessor_ids) const {
-  DCHECK_EQ(1u, accessor_ids.size())
-      << "Got wrong number of attributes for MAX: " << accessor_ids.size();
+  DCHECK(argument_source != ValueAccessorSource::kInvalid);
+  DCHECK_NE(argument_id, kInvalidAttributeID);
 
   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();
+      accessor_mux.getValueAccessorBySource(argument_source),
+      argument_id));
 }
 
 void AggregationHandleMax::mergeStates(const AggregationState &source,
@@ -98,40 +77,36 @@ 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);
   }
 }
 
 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);
+    const std::size_t index,
+    std::vector<std::vector<TypedValue>> *group_by_keys) const {
+  return finalizeHashTableHelper<AggregationHandleMax>(
+      type_, hash_table, index, group_by_keys);
 }
 
-AggregationState*
-AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
+AggregationState* AggregationHandleMax::aggregateOnDistinctifyHashTableForSingle(
     const AggregationStateHashTableBase &distinctify_hash_table) const {
-  return aggregateOnDistinctifyHashTableForSingleUnaryHelperFast<
-      AggregationHandleMax,
-      AggregationStateMax>(distinctify_hash_table);
+  return aggregateOnDistinctifyHashTableForSingleUnaryHelper<
+      AggregationHandleMax, AggregationStateMax>(
+          distinctify_hash_table);
 }
 
 void AggregationHandleMax::aggregateOnDistinctifyHashTableForGroupBy(
     const AggregationStateHashTableBase &distinctify_hash_table,
-    AggregationStateHashTableBase *aggregation_hash_table,
-    std::size_t index) const {
-  aggregateOnDistinctifyHashTableForGroupByUnaryHelperFast<
-      AggregationHandleMax,
-      AggregationStateFastHashTable>(
-      distinctify_hash_table, aggregation_hash_table, index);
+    const std::size_t index,
+    AggregationStateHashTableBase *aggregation_hash_table) const {
+  aggregateOnDistinctifyHashTableForGroupByUnaryHelper<AggregationHandleMax>(
+      distinctify_hash_table, index, aggregation_hash_table);
 }
 
 }  // namespace quickstep