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

incubator-quickstep git commit: Fix the memory leak in FastSeparateChaining hash table.

Repository: incubator-quickstep
Updated Branches:
  refs/heads/master d3a092059 -> 5b2479178


Fix the memory leak in FastSeparateChaining hash table.

- Destruction of payload in hash table.
- Separate phases of payload destruction of aggregation hash tables.
- First the payloads will be destructed by respective aggregation
  handles.
- Second the hash table itself will be destructed.
- Remove FastSeparateChaining::DestroyValues function.


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

Branch: refs/heads/master
Commit: 5b247917823e2fe1aa110bd44ab5347d84cfbc3b
Parents: d3a0920
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Fri Sep 23 16:35:48 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Thu Oct 20 14:45:01 2016 -0500

----------------------------------------------------------------------
 expressions/aggregation/AggregationHandle.hpp   |   8 +
 .../aggregation/AggregationHandleAvg.hpp        |   8 +
 .../aggregation/AggregationHandleMax.hpp        |   7 +
 .../aggregation/AggregationHandleMin.hpp        |   7 +
 .../aggregation/AggregationHandleSum.hpp        |   8 +
 query_execution/QueryContext.hpp                |  14 ++
 .../DestroyAggregationStateOperator.cpp         |   7 +
 storage/AggregationOperationState.cpp           |  13 +-
 storage/AggregationOperationState.hpp           |  12 +-
 storage/CMakeLists.txt                          |   1 +
 storage/FastSeparateChainingHashTable.hpp       | 238 +++----------------
 storage/HashTableBase.hpp                       |   6 +
 12 files changed, 113 insertions(+), 216 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b247917/expressions/aggregation/AggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandle.hpp b/expressions/aggregation/AggregationHandle.hpp
index 48ce7fe..4b51179 100644
--- a/expressions/aggregation/AggregationHandle.hpp
+++ b/expressions/aggregation/AggregationHandle.hpp
@@ -406,6 +406,14 @@ class AggregationHandle {
   virtual void initPayload(std::uint8_t *byte_ptr) const {}
 
   /**
+   * @brief Destroy the payload (in the aggregation hash table) for the given
+   *        aggregation handle.
+   *
+   * @param byte_ptr The pointer to the aggregation state in the hash table.
+   **/
+  virtual void destroyPayload(std::uint8_t *byte_ptr) const {}
+
+  /**
    * @brief Inform the aggregation handle to block (prohibit) updates on the
    *        aggregation state.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b247917/expressions/aggregation/AggregationHandleAvg.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.hpp b/expressions/aggregation/AggregationHandleAvg.hpp
index 3c6e0c2..47132c6 100644
--- a/expressions/aggregation/AggregationHandleAvg.hpp
+++ b/expressions/aggregation/AggregationHandleAvg.hpp
@@ -161,6 +161,14 @@ class AggregationHandleAvg : public AggregationConcreteHandle {
     *count_ptr = blank_state_.count_;
   }
 
+  void destroyPayload(std::uint8_t *byte_ptr) const override {
+    TypedValue *sum_ptr =
+        reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset_);
+    if (sum_ptr != nullptr) {
+      sum_ptr->~TypedValue();
+    }
+  }
+
   AggregationState* accumulateColumnVectors(
       const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
       const override;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b247917/expressions/aggregation/AggregationHandleMax.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMax.hpp b/expressions/aggregation/AggregationHandleMax.hpp
index 5fb9f44..d851a0c 100644
--- a/expressions/aggregation/AggregationHandleMax.hpp
+++ b/expressions/aggregation/AggregationHandleMax.hpp
@@ -129,6 +129,13 @@ class AggregationHandleMax : public AggregationConcreteHandle {
     *max_ptr = t1;
   }
 
+  void destroyPayload(std::uint8_t *byte_ptr) const override {
+    TypedValue *max_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
+    if (max_ptr != nullptr) {
+      max_ptr->~TypedValue();
+    }
+  }
+
   AggregationState* accumulateColumnVectors(
       const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
       const override;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b247917/expressions/aggregation/AggregationHandleMin.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleMin.hpp b/expressions/aggregation/AggregationHandleMin.hpp
index 173911d..e3472ec 100644
--- a/expressions/aggregation/AggregationHandleMin.hpp
+++ b/expressions/aggregation/AggregationHandleMin.hpp
@@ -131,6 +131,13 @@ class AggregationHandleMin : public AggregationConcreteHandle {
     *min_ptr = t1;
   }
 
+  void destroyPayload(std::uint8_t *byte_ptr) const override {
+    TypedValue *min_ptr = reinterpret_cast<TypedValue *>(byte_ptr);
+    if (min_ptr != nullptr) {
+      min_ptr->~TypedValue();
+    }
+  }
+
   AggregationState* accumulateColumnVectors(
       const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
       const override;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b247917/expressions/aggregation/AggregationHandleSum.hpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.hpp b/expressions/aggregation/AggregationHandleSum.hpp
index 6c334a6..f0d23e1 100644
--- a/expressions/aggregation/AggregationHandleSum.hpp
+++ b/expressions/aggregation/AggregationHandleSum.hpp
@@ -153,6 +153,14 @@ class AggregationHandleSum : public AggregationConcreteHandle {
     *null_ptr = true;
   }
 
+  void destroyPayload(std::uint8_t *byte_ptr) const override {
+    TypedValue *sum_ptr =
+        reinterpret_cast<TypedValue *>(byte_ptr + blank_state_.sum_offset_);
+    if (sum_ptr != nullptr) {
+      sum_ptr->~TypedValue();
+    }
+  }
+
   AggregationState* accumulateColumnVectors(
       const std::vector<std::unique_ptr<ColumnVector>> &column_vectors)
       const override;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b247917/query_execution/QueryContext.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.hpp b/query_execution/QueryContext.hpp
index 78794f1..7cc44ad 100644
--- a/query_execution/QueryContext.hpp
+++ b/query_execution/QueryContext.hpp
@@ -187,6 +187,20 @@ 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/5b247917/relational_operators/DestroyAggregationStateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyAggregationStateOperator.cpp b/relational_operators/DestroyAggregationStateOperator.cpp
index 62ca9e7..49be43d 100644
--- a/relational_operators/DestroyAggregationStateOperator.cpp
+++ b/relational_operators/DestroyAggregationStateOperator.cpp
@@ -58,6 +58,13 @@ 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/5b247917/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 7908db1..249026d 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -166,8 +166,7 @@ AggregationOperationState::AggregationOperationState(
       }
 
       // Initialize the corresponding distinctify hash table if this is a
-      // DISTINCT
-      // aggregation.
+      // DISTINCT aggregation.
       if (*is_distinct_it) {
         std::vector<const Type *> key_types(group_by_types);
         key_types.insert(
@@ -595,4 +594,14 @@ void AggregationOperationState::finalizeHashTable(
   output_destination->bulkInsertTuples(&complete_result);
 }
 
+void AggregationOperationState::destroyAggregationHashTablePayload() {
+  if (group_by_hashtable_pool_ != nullptr) {
+    auto all_hash_tables = group_by_hashtable_pool_->getAllHashTables();
+    DCHECK(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();
+    }
+  }
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b247917/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index cbbfc22..3b0f286 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -167,6 +167,11 @@ class AggregationOperationState {
    **/
   void finalizeAggregate(InsertDestination *output_destination);
 
+  /**
+   * @brief Destroy the payloads in the aggregation hash tables.
+   **/
+  void destroyAggregationHashTablePayload();
+
   static void mergeGroupByHashTables(AggregationStateHashTableBase *src,
                                      AggregationStateHashTableBase *dst);
 
@@ -185,6 +190,9 @@ class AggregationOperationState {
   void finalizeSingleState(InsertDestination *output_destination);
   void finalizeHashTable(InsertDestination *output_destination);
 
+  // A vector of group by hash table pools.
+  std::unique_ptr<HashTablePool> group_by_hashtable_pool_;
+
   // 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_;
@@ -193,7 +201,6 @@ class AggregationOperationState {
 
   // Each individual aggregate in this operation has an AggregationHandle and
   // some number of Scalar arguments.
-  //  std::vector<std::unique_ptr<AggregationHandle>> handles_;
   std::vector<AggregationHandle *> handles_;
   std::vector<std::vector<std::unique_ptr<const Scalar>>> arguments_;
 
@@ -221,9 +228,6 @@ class AggregationOperationState {
   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_;
-
   StorageManager *storage_manager_;
 
   DISALLOW_COPY_AND_ASSIGN(AggregationOperationState);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b247917/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index e85e005..325a7cb 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -660,6 +660,7 @@ target_link_libraries(quickstep_storage_FastHashTableFactory
                       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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b247917/storage/FastSeparateChainingHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastSeparateChainingHashTable.hpp b/storage/FastSeparateChainingHashTable.hpp
index a41535c..17ee1d2 100644
--- a/storage/FastSeparateChainingHashTable.hpp
+++ b/storage/FastSeparateChainingHashTable.hpp
@@ -27,6 +27,7 @@
 #include <utility>
 #include <vector>
 
+#include "expressions/aggregation/AggregationHandle.hpp"
 #include "storage/FastHashTable.hpp"
 #include "storage/HashTable.hpp"
 #include "storage/HashTableBase.hpp"
@@ -67,35 +68,7 @@ class FastSeparateChainingHashTable
                                 const std::vector<AggregationHandle *> &handles,
                                 StorageManager *storage_manager);
 
-  FastSeparateChainingHashTable(const std::vector<const Type *> &key_types,
-                                void *hash_table_memory,
-                                const std::size_t hash_table_memory_size,
-                                const bool new_hash_table,
-                                const bool hash_table_memory_zeroed);
-
-  // Delegating constructors for single scalar keys.
-  FastSeparateChainingHashTable(const Type &key_type,
-                                const std::size_t num_entries,
-                                StorageManager *storage_manager)
-      : FastSeparateChainingHashTable(std::vector<const Type *>(1, &key_type),
-                                      num_entries,
-                                      storage_manager) {}
-
-  FastSeparateChainingHashTable(const Type &key_type,
-                                void *hash_table_memory,
-                                const std::size_t hash_table_memory_size,
-                                const bool new_hash_table,
-                                const bool hash_table_memory_zeroed)
-      : FastSeparateChainingHashTable(std::vector<const Type *>(1, &key_type),
-                                      hash_table_memory,
-                                      hash_table_memory_size,
-                                      new_hash_table,
-                                      hash_table_memory_zeroed) {}
-
   ~FastSeparateChainingHashTable() override {
-    DestroyValues(buckets_,
-                  header_->buckets_allocated.load(std::memory_order_relaxed),
-                  bucket_size_);
     std::free(init_payload_);
   }
 
@@ -167,6 +140,23 @@ class FastSeparateChainingHashTable
       const std::size_t total_variable_key_size,
       HashTablePreallocationState *prealloc_state) override;
 
+  void destroyPayload() override {
+    void *hash_buckets = buckets_;
+    const std::size_t num_buckets =
+        header_->buckets_allocated.load(std::memory_order_relaxed);
+    const std::size_t bucket_size = bucket_size_;
+    void *bucket_ptr = static_cast<char *>(hash_buckets) + kValueOffset;
+    for (std::size_t bucket_num = 0; bucket_num < num_buckets; ++bucket_num) {
+      void *value_internal_ptr = bucket_ptr;
+      for (std::size_t handle_num = 0; handle_num < handles_.size(); ++handle_num) {
+        value_internal_ptr =
+            static_cast<char *>(value_internal_ptr) + this->payload_offsets_[handle_num];
+        handles_[handle_num]->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;
@@ -190,13 +180,6 @@ class FastSeparateChainingHashTable
             1) *
            kBucketAlignment;
   }
-  // If ValueT is not trivially destructible, invoke its destructor for all
-  // values held in the specified buckets (including those in "empty" buckets
-  // that were default constructed). If ValueT is trivially destructible, this
-  // is a no-op.
-  void DestroyValues(void *buckets,
-                     const std::size_t num_buckets,
-                     const std::size_t bucket_size);
 
   // Attempt to find an empty bucket to insert 'hash_code' into, starting after
   // '*bucket' in the chain (or, if '*bucket' is NULL, starting from the slot
@@ -242,6 +225,8 @@ class FastSeparateChainingHashTable
   // 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_;
+
   // Helper object to manage key storage.
   HashTableKeyManager<serializable, force_key_copy> key_manager_;
 
@@ -309,12 +294,14 @@ FastSeparateChainingHashTable<resizable,
                                           true),
       kBucketAlignment(alignof(std::atomic<std::size_t>)),
       kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
+      handles_(handles),
       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 : handles) {
+  for (auto handle : this->handles_) {
     handle->initPayload(init_payload_ + this->payload_offsets_[k]);
     k++;
   }
@@ -433,155 +420,6 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-FastSeparateChainingHashTable<resizable,
-                              serializable,
-                              force_key_copy,
-                              allow_duplicate_keys>::
-    FastSeparateChainingHashTable(const std::vector<const Type *> &key_types,
-                                  void *hash_table_memory,
-                                  const std::size_t hash_table_memory_size,
-                                  const bool new_hash_table,
-                                  const bool hash_table_memory_zeroed)
-    : FastHashTable<resizable,
-                    serializable,
-                    force_key_copy,
-                    allow_duplicate_keys>(key_types,
-                                          hash_table_memory,
-                                          hash_table_memory_size,
-                                          new_hash_table,
-                                          hash_table_memory_zeroed,
-                                          false,
-                                          false,
-                                          true),
-      kBucketAlignment(alignof(std::atomic<std::size_t>) < alignof(std::uint8_t)
-                           ? alignof(std::uint8_t)
-                           : alignof(std::atomic<std::size_t>)),
-      kValueOffset(sizeof(std::atomic<std::size_t>) + sizeof(std::size_t)),
-      key_manager_(this->key_types_, kValueOffset + sizeof(std::uint8_t)),
-      bucket_size_(ComputeBucketSize(key_manager_.getFixedKeySize())) {
-  // Bucket size always rounds up to the alignment requirement of the atomic
-  // size_t "next" pointer at the front or a ValueT, whichever is larger.
-  //
-  // Make sure that the larger of the two alignment requirements also satisfies
-  // the smaller.
-  static_assert(
-      alignof(std::atomic<std::size_t>) < alignof(std::uint8_t)
-          ? alignof(std::uint8_t) % alignof(std::atomic<std::size_t>) == 0
-          : alignof(std::atomic<std::size_t>) % alignof(std::uint8_t) == 0,
-      "Alignment requirement of std::atomic<std::size_t> does not "
-      "evenly divide with alignment requirement of ValueT.");
-
-  // Give base HashTable information about what key components are stored
-  // inline from 'key_manager_'.
-  this->setKeyInline(key_manager_.getKeyInline());
-
-  // FIXME(chasseur): If we are reconstituting a HashTable using a block of
-  // memory whose start was aligned differently than the memory block that was
-  // originally used (modulo alignof(Header)), we could wind up with all of our
-  // data structures misaligned. If memory is inside a
-  // StorageBlock/StorageBlob, this will never occur, since the StorageManager
-  // always allocates slots aligned to kCacheLineBytes. Similarly, this isn't
-  // a problem for memory inside any other allocation aligned to at least
-  // alignof(Header) == kCacheLineBytes.
-
-  void *aligned_memory_start = this->hash_table_memory_;
-  std::size_t available_memory = this->hash_table_memory_size_;
-
-  if (align(alignof(Header),
-            sizeof(Header),
-            aligned_memory_start,
-            available_memory) == nullptr) {
-    FATAL_ERROR("Attempted to create a non-resizable "
-                << "SeparateChainingHashTable with "
-                << available_memory
-                << " bytes of memory at "
-                << aligned_memory_start
-                << " which either can not fit a "
-                << "SeparateChainingHashTable::Header or meet its alignement "
-                << "requirement.");
-  } else if (aligned_memory_start != this->hash_table_memory_) {
-    // In general, we could get memory of any alignment, although at least
-    // cache-line aligned would be nice.
-    DEV_WARNING("StorageBlob memory adjusted by "
-                << (this->hash_table_memory_size_ - available_memory)
-                << " bytes to meet alignment requirement for "
-                << "SeparateChainingHashTable::Header.");
-  }
-
-  header_ = static_cast<Header *>(aligned_memory_start);
-  aligned_memory_start =
-      static_cast<char *>(aligned_memory_start) + sizeof(Header);
-  available_memory -= sizeof(Header);
-
-  if (new_hash_table) {
-    std::size_t estimated_bucket_capacity =
-        available_memory /
-        (kHashTableLoadFactor * sizeof(std::atomic<std::size_t>) +
-         bucket_size_ + key_manager_.getEstimatedVariableKeySize());
-    std::size_t num_slots = get_previous_prime_number(
-        estimated_bucket_capacity * kHashTableLoadFactor);
-
-    // Fill in the header.
-    header_->num_slots = num_slots;
-    header_->num_buckets = num_slots / kHashTableLoadFactor;
-    header_->buckets_allocated.store(0, std::memory_order_relaxed);
-    header_->variable_length_bytes_allocated.store(0,
-                                                   std::memory_order_relaxed);
-  }
-
-  // Locate the slot array.
-  slots_ = static_cast<std::atomic<std::size_t> *>(aligned_memory_start);
-  aligned_memory_start = static_cast<char *>(aligned_memory_start) +
-                         sizeof(std::atomic<std::size_t>) * header_->num_slots;
-  available_memory -= sizeof(std::atomic<std::size_t>) * header_->num_slots;
-
-  if (new_hash_table && !hash_table_memory_zeroed) {
-    std::memset(
-        slots_, 0x0, sizeof(std::atomic<std::size_t>) * header_->num_slots);
-  }
-
-  // Locate the buckets.
-  buckets_ = aligned_memory_start;
-  // Extra-paranoid: sizeof(Header) should almost certainly be a multiple of
-  // kBucketAlignment, unless ValueT has some members with seriously big
-  // (> kCacheLineBytes) alignment requirements specified using alignas().
-  if (align(kBucketAlignment, bucket_size_, buckets_, available_memory) ==
-      nullptr) {
-    FATAL_ERROR("Attempted to create a non-resizable "
-                << "SeparateChainingHashTable with "
-                << this->hash_table_memory_size_
-                << " bytes of memory at "
-                << this->hash_table_memory_
-                << ", which can hold an aligned "
-                << "SeparateChainingHashTable::Header but does not have "
-                << "enough remaining space for even a single hash bucket.");
-  } else if (buckets_ != aligned_memory_start) {
-    DEV_WARNING(
-        "Bucket array start position adjusted to meet alignment "
-        "requirement for SeparateChainingHashTable's value type.");
-    if (header_->num_buckets * bucket_size_ > available_memory) {
-      DEBUG_ASSERT(new_hash_table);
-      --(header_->num_buckets);
-    }
-  }
-  available_memory -= bucket_size_ * header_->num_buckets;
-
-  // Make sure "next" pointers in buckets are zeroed-out.
-  if (new_hash_table && !hash_table_memory_zeroed) {
-    std::memset(buckets_, 0x0, header_->num_buckets * bucket_size_);
-  }
-
-  // Locate variable-length key storage region.
-  key_manager_.setVariableLengthStorageInfo(
-      static_cast<char *>(buckets_) + header_->num_buckets * bucket_size_,
-      available_memory,
-      &(header_->variable_length_bytes_allocated));
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
 void FastSeparateChainingHashTable<resizable,
                                    serializable,
                                    force_key_copy,
@@ -589,7 +427,7 @@ void FastSeparateChainingHashTable<resizable,
   const std::size_t used_buckets =
       header_->buckets_allocated.load(std::memory_order_relaxed);
   // Destroy existing values, if necessary.
-  DestroyValues(buckets_, used_buckets, bucket_size_);
+  destroyPayload();
 
   // Zero-out slot array.
   std::memset(
@@ -995,10 +833,11 @@ std::uint8_t* FastSeparateChainingHashTable<resizable,
 
   // Copy the supplied 'initial_value' into place.
   std::uint8_t *value = static_cast<unsigned char *>(bucket) + kValueOffset;
-  if (init_value_ptr == nullptr)
+  if (init_value_ptr == nullptr) {
     memcpy(value, init_payload_, this->total_payload_size_);
-  else
+  } else {
     memcpy(value, init_value_ptr, this->total_payload_size_);
+  }
 
   // Update the previous chain pointer to point to the new bucket.
   pending_chain_ptr->store(pending_chain_ptr_finish_value,
@@ -1466,8 +1305,7 @@ void FastSeparateChainingHashTable<
                 original_variable_storage_used);
   }
 
-  // Destroy values in the original hash table, if neccesary,
-  DestroyValues(buckets_, original_buckets_used, bucket_size_);
+  destroyPayload();
 
   // Make resized structures active.
   std::swap(this->blob_, resized_blob);
@@ -1562,26 +1400,6 @@ template <bool resizable,
           bool serializable,
           bool force_key_copy,
           bool allow_duplicate_keys>
-void FastSeparateChainingHashTable<
-    resizable,
-    serializable,
-    force_key_copy,
-    allow_duplicate_keys>::DestroyValues(void *hash_buckets,
-                                         const std::size_t num_buckets,
-                                         const std::size_t bucket_size) {
-  if (!std::is_trivially_destructible<std::uint8_t>::value) {
-    void *value_ptr = static_cast<char *>(hash_buckets) + kValueOffset;
-    for (std::size_t bucket_num = 0; bucket_num < num_buckets; ++bucket_num) {
-      static_cast<std::uint8_t *>(value_ptr)->~uint8_t();
-      value_ptr = static_cast<char *>(value_ptr) + bucket_size;
-    }
-  }
-}
-
-template <bool resizable,
-          bool serializable,
-          bool force_key_copy,
-          bool allow_duplicate_keys>
 inline bool FastSeparateChainingHashTable<resizable,
                                           serializable,
                                           force_key_copy,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b247917/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index cd0a141..a3180bb 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -99,6 +99,12 @@ class HashTableBase {
     return false;
   }
 
+  /**
+   * @brief Destroy the payload stored in the hash table.
+   **/
+  virtual void destroyPayload() {
+  }
+
  protected:
   HashTableBase() {}