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 2016/10/21 03:13:30 UTC

[5/9] incubator-quickstep git commit: Clean up the old bloom-filter implementation which are attached to HashTables.

Clean up the old bloom-filter implementation which are attached to HashTables.


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

Branch: refs/heads/multiple_shiftboss
Commit: 55480d8d1719b5ff97b9562d53ddc63ba1c4d93d
Parents: 2ee5c1c
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Wed Sep 7 13:20:43 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Tue Oct 18 15:18:48 2016 -0500

----------------------------------------------------------------------
 query_execution/CMakeLists.txt   |   1 -
 query_execution/QueryContext.cpp |   8 +-
 query_execution/QueryContext.hpp |  53 ------------
 storage/CMakeLists.txt           |   2 -
 storage/FastHashTable.hpp        | 152 +++++-----------------------------
 storage/FastHashTableFactory.hpp |  35 +-------
 storage/HashTable.hpp            | 103 -----------------------
 storage/HashTable.proto          |   6 --
 storage/HashTableFactory.hpp     |  34 +-------
 9 files changed, 23 insertions(+), 371 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/55480d8d/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 6a84be1..dafdea4 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -189,7 +189,6 @@ target_link_libraries(quickstep_queryexecution_QueryContext
                       quickstep_storage_WindowAggregationOperationState
                       quickstep_types_TypedValue
                       quickstep_types_containers_Tuple
-                      quickstep_utility_BloomFilter
                       quickstep_utility_Macros
                       quickstep_utility_SortConfiguration)
 target_link_libraries(quickstep_queryexecution_QueryContext_proto

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/55480d8d/query_execution/QueryContext.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.cpp b/query_execution/QueryContext.cpp
index 2572e18..6612611 100644
--- a/query_execution/QueryContext.cpp
+++ b/query_execution/QueryContext.cpp
@@ -39,7 +39,6 @@
 #include "storage/InsertDestination.pb.h"
 #include "types/TypedValue.hpp"
 #include "types/containers/Tuple.hpp"
-#include "utility/BloomFilter.hpp"
 #include "utility/SortConfiguration.hpp"
 
 #include "glog/logging.h"
@@ -68,10 +67,6 @@ QueryContext::QueryContext(const serialization::QueryContext &proto,
                                                         storage_manager));
   }
 
-  for (int i = 0; i < proto.bloom_filters_size(); ++i) {
-    bloom_filters_.emplace_back(new BloomFilter(proto.bloom_filters(i)));
-  }
-
   for (int i = 0; i < proto.generator_functions_size(); ++i) {
     const GeneratorFunctionHandle *func_handle =
         GeneratorFunctionFactory::Instance().reconstructFromProto(proto.generator_functions(i));
@@ -83,8 +78,7 @@ QueryContext::QueryContext(const serialization::QueryContext &proto,
   for (int i = 0; i < proto.join_hash_tables_size(); ++i) {
     join_hash_tables_.emplace_back(
         JoinHashTableFactory::CreateResizableFromProto(proto.join_hash_tables(i),
-                                                       storage_manager,
-                                                       bloom_filters_));
+                                                       storage_manager));
   }
 
   for (int i = 0; i < proto.insert_destinations_size(); ++i) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/55480d8d/query_execution/QueryContext.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.hpp b/query_execution/QueryContext.hpp
index 393b55e..78794f1 100644
--- a/query_execution/QueryContext.hpp
+++ b/query_execution/QueryContext.hpp
@@ -35,7 +35,6 @@
 #include "storage/InsertDestination.hpp"
 #include "storage/WindowAggregationOperationState.hpp"
 #include "types/containers/Tuple.hpp"
-#include "utility/BloomFilter.hpp"
 #include "utility/Macros.hpp"
 #include "utility/SortConfiguration.hpp"
 
@@ -67,11 +66,6 @@ class QueryContext {
   typedef std::uint32_t aggregation_state_id;
 
   /**
-   * @brief A unique identifier for a BloomFilter per query.
-   **/
-  typedef std::uint32_t bloom_filter_id;
-
-  /**
    * @brief A unique identifier for a GeneratorFunctionHandle per query.
    **/
   typedef std::uint32_t generator_function_id;
@@ -193,52 +187,6 @@ class QueryContext {
   }
 
   /**
-   * @brief Whether the given BloomFilter id is valid.
-   *
-   * @param id The BloomFilter id.
-   *
-   * @return True if valid, otherwise false.
-   **/
-  bool isValidBloomFilterId(const bloom_filter_id id) const {
-    return id < bloom_filters_.size();
-  }
-
-  /**
-   * @brief Get a mutable reference to the BloomFilter.
-   *
-   * @param id The BloomFilter id.
-   *
-   * @return The BloomFilter, already created in the constructor.
-   **/
-  inline BloomFilter* getBloomFilterMutable(const bloom_filter_id id) {
-    DCHECK_LT(id, bloom_filters_.size());
-    return bloom_filters_[id].get();
-  }
-
-  /**
-   * @brief Get a constant pointer to the BloomFilter.
-   *
-   * @param id The BloomFilter id.
-   *
-   * @return The constant pointer to BloomFilter that is
-   *         already created in the constructor.
-   **/
-  inline const BloomFilter* getBloomFilter(const bloom_filter_id id) const {
-    DCHECK_LT(id, bloom_filters_.size());
-    return bloom_filters_[id].get();
-  }
-
-  /**
-   * @brief Destory the given BloomFilter.
-   *
-   * @param id The id of the BloomFilter to destroy.
-   **/
-  inline void destroyBloomFilter(const bloom_filter_id id) {
-    DCHECK_LT(id, bloom_filters_.size());
-    bloom_filters_[id].reset();
-  }
-
-  /**
    * @brief Whether the given GeneratorFunctionHandle id is valid.
    *
    * @param id The GeneratorFunctionHandle id.
@@ -507,7 +455,6 @@ class QueryContext {
 
  private:
   std::vector<std::unique_ptr<AggregationOperationState>> aggregation_states_;
-  std::vector<std::unique_ptr<BloomFilter>> bloom_filters_;
   std::vector<std::unique_ptr<const GeneratorFunctionHandle>> generator_functions_;
   std::vector<std::unique_ptr<InsertDestination>> insert_destinations_;
   std::vector<std::unique_ptr<JoinHashTable>> join_hash_tables_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/55480d8d/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index f05cc46..e85e005 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -643,7 +643,6 @@ target_link_libraries(quickstep_storage_FastHashTable
                       quickstep_threading_SpinSharedMutex
                       quickstep_types_Type
                       quickstep_types_TypedValue
-                      quickstep_utility_BloomFilter
                       quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_FastHashTableFactory
@@ -659,7 +658,6 @@ target_link_libraries(quickstep_storage_FastHashTableFactory
                       quickstep_storage_SimpleScalarSeparateChainingHashTable
                       quickstep_storage_TupleReference
                       quickstep_types_TypeFactory
-                      quickstep_utility_BloomFilter
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_FastSeparateChainingHashTable
                       quickstep_storage_FastHashTable

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/55480d8d/storage/FastHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTable.hpp b/storage/FastHashTable.hpp
index 4a95cd9..74d9ee3 100644
--- a/storage/FastHashTable.hpp
+++ b/storage/FastHashTable.hpp
@@ -39,7 +39,6 @@
 #include "threading/SpinSharedMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
-#include "utility/BloomFilter.hpp"
 #include "utility/HashPair.hpp"
 #include "utility/Macros.hpp"
 
@@ -958,62 +957,6 @@ class FastHashTable : public HashTableBase<resizable,
   template <typename FunctorT>
   std::size_t forEachCompositeKeyFast(FunctorT *functor, int index) const;
 
-  /**
-   * @brief A call to this function will cause a bloom filter to be built
-   *        during the build phase of this hash table.
-   **/
-  inline void enableBuildSideBloomFilter() {
-    has_build_side_bloom_filter_ = true;
-  }
-
-  /**
-   * @brief A call to this function will cause a set of bloom filters to be
-   *        probed during the probe phase of this hash table.
-   **/
-  inline void enableProbeSideBloomFilter() {
-    has_probe_side_bloom_filter_ = true;
-  }
-
-  /**
-   * @brief This function sets the pointer to the bloom filter to be
-   *        used during the build phase of this hash table.
-   * @warning Should call enable_build_side_bloom_filter() first to enable
-   *          bloom filter usage during build phase.
-   * @note The ownership of the bloom filter lies with the caller.
-   *
-   * @param bloom_filter The pointer to the bloom filter.
-   **/
-  inline void setBuildSideBloomFilter(BloomFilter *bloom_filter) {
-    build_bloom_filter_ = bloom_filter;
-  }
-
-  /**
-   * @brief This function adds a pointer to the list of bloom filters to be
-   *        used during the probe phase of this hash table.
-   * @warning Should call enable_probe_side_bloom_filter() first to enable
-   *          bloom filter usage during probe phase.
-   * @note The ownership of the bloom filter lies with the caller.
-   *
-   * @param bloom_filter The pointer to the bloom filter.
-   **/
-  inline void addProbeSideBloomFilter(const BloomFilter *bloom_filter) {
-    probe_bloom_filters_.emplace_back(bloom_filter);
-  }
-
-  /**
-   * @brief This function adds a vector of attribute ids corresponding to a
-   *        bloom filter used during the probe phase of this hash table.
-   * @warning Should call enable_probe_side_bloom_filter() first to enable
-   *          bloom filter usage during probe phase.
-   *
-   * @param probe_attribute_ids The vector of attribute ids to use for probing
-   *        the bloom filter.
-   **/
-  inline void addProbeSideAttributeIds(
-      std::vector<attribute_id> &&probe_attribute_ids) {
-    probe_attribute_ids_.push_back(probe_attribute_ids);
-  }
-
  protected:
   /**
    * @brief Constructor for new resizable hash table.
@@ -1318,12 +1261,6 @@ class FastHashTable : public HashTableBase<resizable,
                                    const attribute_id key_attr_id,
                                    FunctorT *functor) const;
 
-  // Data structures used for bloom filter optimized semi-joins.
-  bool has_build_side_bloom_filter_ = false;
-  bool has_probe_side_bloom_filter_ = false;
-  BloomFilter *build_bloom_filter_;
-  std::vector<const BloomFilter *> probe_bloom_filters_;
-  std::vector<std::vector<attribute_id>> probe_attribute_ids_;
   DISALLOW_COPY_AND_ASSIGN(FastHashTable);
 };
 
@@ -1449,13 +1386,6 @@ FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
                 total_entries, total_variable_key_size, &prealloc_state);
           }
         }
-        std::unique_ptr<BloomFilter> thread_local_bloom_filter;
-        if (has_build_side_bloom_filter_) {
-          thread_local_bloom_filter.reset(
-              new BloomFilter(build_bloom_filter_->getRandomSeed(),
-                              build_bloom_filter_->getNumberOfHashes(),
-                              build_bloom_filter_->getBitArraySize()));
-        }
         if (resizable) {
           while (result == HashTablePutResult::kOutOfSpace) {
             {
@@ -1474,12 +1404,6 @@ FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
                     variable_size,
                     (*functor)(*accessor),
                     using_prealloc ? &prealloc_state : nullptr);
-                // Insert into bloom filter, if enabled.
-                if (has_build_side_bloom_filter_) {
-                  thread_local_bloom_filter->insertUnSafe(
-                      static_cast<const std::uint8_t *>(key.getDataPtr()),
-                      key.getDataSize());
-                }
                 if (result == HashTablePutResult::kDuplicateKey) {
                   DEBUG_ASSERT(!using_prealloc);
                   return result;
@@ -1507,22 +1431,11 @@ FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>::
                                   variable_size,
                                   (*functor)(*accessor),
                                   using_prealloc ? &prealloc_state : nullptr);
-            // Insert into bloom filter, if enabled.
-            if (has_build_side_bloom_filter_) {
-              thread_local_bloom_filter->insertUnSafe(
-                  static_cast<const std::uint8_t *>(key.getDataPtr()),
-                  key.getDataSize());
-            }
             if (result != HashTablePutResult::kOK) {
               return result;
             }
           }
         }
-        // Update the build side bloom filter with thread local copy, if
-        // available.
-        if (has_build_side_bloom_filter_) {
-          build_bloom_filter_->bitwiseOr(thread_local_bloom_filter.get());
-        }
 
         return HashTablePutResult::kOK;
       });
@@ -2462,52 +2375,27 @@ void FastHashTable<resizable,
   InvokeOnAnyValueAccessor(
       accessor,
       [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-        while (accessor->next()) {
-          // Probe any bloom filters, if enabled.
-          if (has_probe_side_bloom_filter_) {
-            DCHECK_EQ(probe_bloom_filters_.size(), probe_attribute_ids_.size());
-            // Check if the key is contained in the BloomFilters or not.
-            bool bloom_miss = false;
-            for (std::size_t i = 0;
-                 i < probe_bloom_filters_.size() && !bloom_miss;
-                 ++i) {
-              const BloomFilter *bloom_filter = probe_bloom_filters_[i];
-              for (const attribute_id &attr_id : probe_attribute_ids_[i]) {
-                TypedValue bloom_key = accessor->getTypedValue(attr_id);
-                if (!bloom_filter->contains(static_cast<const std::uint8_t *>(
-                                                bloom_key.getDataPtr()),
-                                            bloom_key.getDataSize())) {
-                  bloom_miss = true;
-                  break;
-                }
-              }
-            }
-            if (bloom_miss) {
-              continue;  // On a bloom filter miss, probing the hash table can
-                         // be skipped.
-            }
-          }
-
-          TypedValue key = accessor->getTypedValue(key_attr_id);
-          if (check_for_null_keys && key.isNull()) {
-            continue;
-          }
-          const std::size_t true_hash = use_scalar_literal_hash_template
-                                            ? key.getHashScalarLiteral()
-                                            : key.getHash();
-          const std::size_t adjusted_hash =
-              adjust_hashes_template ? this->AdjustHash(true_hash) : true_hash;
-          std::size_t entry_num = 0;
-          const std::uint8_t *value;
-          while (this->getNextEntryForKey(
-              key, adjusted_hash, &value, &entry_num)) {
-            (*functor)(*accessor, *value);
-            if (!allow_duplicate_keys) {
-              break;
-            }
-          }
+    while (accessor->next()) {
+      TypedValue key = accessor->getTypedValue(key_attr_id);
+      if (check_for_null_keys && key.isNull()) {
+        continue;
+      }
+      const std::size_t true_hash = use_scalar_literal_hash_template
+                                        ? key.getHashScalarLiteral()
+                                        : key.getHash();
+      const std::size_t adjusted_hash =
+          adjust_hashes_template ? this->AdjustHash(true_hash) : true_hash;
+      std::size_t entry_num = 0;
+      const std::uint8_t *value;
+      while (this->getNextEntryForKey(
+          key, adjusted_hash, &value, &entry_num)) {
+        (*functor)(*accessor, *value);
+        if (!allow_duplicate_keys) {
+          break;
         }
-      });
+      }
+    }
+  });
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/55480d8d/storage/FastHashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/FastHashTableFactory.hpp b/storage/FastHashTableFactory.hpp
index 6d0b693..682cc2a 100644
--- a/storage/FastHashTableFactory.hpp
+++ b/storage/FastHashTableFactory.hpp
@@ -32,7 +32,6 @@
 #include "storage/SimpleScalarSeparateChainingHashTable.hpp"
 #include "storage/TupleReference.hpp"
 #include "types/TypeFactory.hpp"
-#include "utility/BloomFilter.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -183,14 +182,11 @@ class FastHashTableFactory {
    * @param proto A protobuf description of a resizable HashTable.
    * @param storage_manager The StorageManager to use (a StorageBlob will be
    *        allocated to hold the HashTable's contents).
-   * @param bloom_filters A vector of pointers to bloom filters that may be used
-   *        during hash table construction in build/probe phase.
    * @return A new resizable HashTable with parameters specified by proto.
    **/
   static FastHashTable<resizable, serializable, force_key_copy, allow_duplicate_keys>*
       CreateResizableFromProto(const serialization::HashTable &proto,
-                               StorageManager *storage_manager,
-                               const std::vector<std::unique_ptr<BloomFilter>> &bloom_filters) {
+                               StorageManager *storage_manager) {
     DCHECK(ProtoIsValid(proto))
         << "Attempted to create HashTable from invalid proto description:\n"
         << proto.DebugString();
@@ -204,35 +200,6 @@ class FastHashTableFactory {
                                       key_types,
                                       proto.estimated_num_entries(),
                                       storage_manager);
-
-    // TODO(ssaurabh): These lazy initializations can be moved from here and pushed to the
-    //                 individual implementations of the hash table constructors.
-
-    // Check if there are any build side bloom filter defined on the hash table.
-    if (proto.build_side_bloom_filter_id_size() > 0) {
-      hash_table->enableBuildSideBloomFilter();
-      hash_table->setBuildSideBloomFilter(bloom_filters[proto.build_side_bloom_filter_id(0)].get());
-    }
-
-    // Check if there are any probe side bloom filters defined on the hash table.
-    if (proto.probe_side_bloom_filters_size() > 0) {
-      hash_table->enableProbeSideBloomFilter();
-      // Add as many probe bloom filters as defined by the proto.
-      for (int j = 0; j < proto.probe_side_bloom_filters_size(); ++j) {
-        // Add the pointer to the probe bloom filter within the list of probe bloom filters to use.
-        const auto probe_side_bloom_filter = proto.probe_side_bloom_filters(j);
-        hash_table->addProbeSideBloomFilter(bloom_filters[probe_side_bloom_filter.probe_side_bloom_filter_id()].get());
-
-        // Add the attribute ids corresponding to this probe bloom filter.
-        std::vector<attribute_id> probe_attribute_ids;
-        for (int k = 0; k < probe_side_bloom_filter.probe_side_attr_ids_size(); ++k) {
-          const attribute_id probe_attribute_id = probe_side_bloom_filter.probe_side_attr_ids(k);
-          probe_attribute_ids.push_back(probe_attribute_id);
-        }
-        hash_table->addProbeSideAttributeIds(std::move(probe_attribute_ids));
-      }
-    }
-
     return hash_table;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/55480d8d/storage/HashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTable.hpp b/storage/HashTable.hpp
index f2dcb03..786a9bb 100644
--- a/storage/HashTable.hpp
+++ b/storage/HashTable.hpp
@@ -981,61 +981,6 @@ class HashTable : public HashTableBase<resizable,
   template <typename FunctorT>
   std::size_t forEachCompositeKey(FunctorT *functor) const;
 
-  /**
-   * @brief A call to this function will cause a bloom filter to be built
-   *        during the build phase of this hash table.
-   **/
-  inline void enableBuildSideBloomFilter() {
-    has_build_side_bloom_filter_ = true;
-  }
-
-  /**
-   * @brief A call to this function will cause a set of bloom filters to be
-   *        probed during the probe phase of this hash table.
-   **/
-  inline void enableProbeSideBloomFilter() {
-    has_probe_side_bloom_filter_ = true;
-  }
-
-  /**
-   * @brief This function sets the pointer to the bloom filter to be
-   *        used during the build phase of this hash table.
-   * @warning Should call enable_build_side_bloom_filter() first to enable
-   *          bloom filter usage during build phase.
-   * @note The ownership of the bloom filter lies with the caller.
-   *
-   * @param bloom_filter The pointer to the bloom filter.
-   **/
-  inline void setBuildSideBloomFilter(BloomFilter *bloom_filter) {
-    build_bloom_filter_ = bloom_filter;
-  }
-
-  /**
-   * @brief This function adds a pointer to the list of bloom filters to be
-   *        used during the probe phase of this hash table.
-   * @warning Should call enable_probe_side_bloom_filter() first to enable
-   *          bloom filter usage during probe phase.
-   * @note The ownership of the bloom filter lies with the caller.
-   *
-   * @param bloom_filter The pointer to the bloom filter.
-   **/
-  inline void addProbeSideBloomFilter(const BloomFilter *bloom_filter) {
-    probe_bloom_filters_.emplace_back(bloom_filter);
-  }
-
-  /**
-   * @brief This function adds a vector of attribute ids corresponding to a
-   *        bloom filter used during the probe phase of this hash table.
-   * @warning Should call enable_probe_side_bloom_filter() first to enable
-   *          bloom filter usage during probe phase.
-   *
-   * @param probe_attribute_ids The vector of attribute ids to use for probing
-   *        the bloom filter.
-   **/
-  inline void addProbeSideAttributeIds(std::vector<attribute_id> &&probe_attribute_ids) {
-    probe_attribute_ids_.push_back(probe_attribute_ids);
-  }
-
  protected:
   /**
    * @brief Constructor for new resizable hash table.
@@ -1316,13 +1261,6 @@ class HashTable : public HashTableBase<resizable,
                                    const attribute_id key_attr_id,
                                    FunctorT *functor) const;
 
-  // Data structures used for bloom filter optimized semi-joins.
-  bool has_build_side_bloom_filter_ = false;
-  bool has_probe_side_bloom_filter_ = false;
-  BloomFilter *build_bloom_filter_;
-  std::vector<const BloomFilter*> probe_bloom_filters_;
-  std::vector<std::vector<attribute_id>> probe_attribute_ids_;
-
   DISALLOW_COPY_AND_ASSIGN(HashTable);
 };
 
@@ -1467,12 +1405,6 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
                                                         &prealloc_state);
       }
     }
-    std::unique_ptr<BloomFilter> thread_local_bloom_filter;
-    if (has_build_side_bloom_filter_) {
-      thread_local_bloom_filter.reset(new BloomFilter(build_bloom_filter_->getRandomSeed(),
-                                                      build_bloom_filter_->getNumberOfHashes(),
-                                                      build_bloom_filter_->getBitArraySize()));
-    }
     if (resizable) {
       while (result == HashTablePutResult::kOutOfSpace) {
         {
@@ -1488,11 +1420,6 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
                                        variable_size,
                                        (*functor)(*accessor),
                                        using_prealloc ? &prealloc_state : nullptr);
-            // Insert into bloom filter, if enabled.
-            if (has_build_side_bloom_filter_) {
-              thread_local_bloom_filter->insertUnSafe(static_cast<const std::uint8_t *>(key.getDataPtr()),
-                                                      key.getDataSize());
-            }
             if (result == HashTablePutResult::kDuplicateKey) {
               DEBUG_ASSERT(!using_prealloc);
               return result;
@@ -1518,20 +1445,11 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
                                    variable_size,
                                    (*functor)(*accessor),
                                    using_prealloc ? &prealloc_state : nullptr);
-        // Insert into bloom filter, if enabled.
-        if (has_build_side_bloom_filter_) {
-          thread_local_bloom_filter->insertUnSafe(static_cast<const std::uint8_t *>(key.getDataPtr()),
-                                                  key.getDataSize());
-        }
         if (result != HashTablePutResult::kOK) {
           return result;
         }
       }
     }
-    // Update the build side bloom filter with thread local copy, if available.
-    if (has_build_side_bloom_filter_) {
-      build_bloom_filter_->bitwiseOr(thread_local_bloom_filter.get());
-    }
 
     return HashTablePutResult::kOK;
   });
@@ -2237,27 +2155,6 @@ void HashTable<ValueT, resizable, serializable, force_key_copy, allow_duplicate_
       accessor,
       [&](auto *accessor) -> void {  // NOLINT(build/c++11)
     while (accessor->next()) {
-      // Probe any bloom filters, if enabled.
-      if (has_probe_side_bloom_filter_) {
-        DCHECK_EQ(probe_bloom_filters_.size(), probe_attribute_ids_.size());
-        // Check if the key is contained in the BloomFilters or not.
-        bool bloom_miss = false;
-        for (std::size_t i = 0; i < probe_bloom_filters_.size() && !bloom_miss; ++i) {
-          const BloomFilter *bloom_filter = probe_bloom_filters_[i];
-          for (const attribute_id &attr_id : probe_attribute_ids_[i]) {
-            TypedValue bloom_key = accessor->getTypedValue(attr_id);
-            if (!bloom_filter->contains(static_cast<const std::uint8_t*>(bloom_key.getDataPtr()),
-                                        bloom_key.getDataSize())) {
-              bloom_miss = true;
-              break;
-            }
-          }
-        }
-        if (bloom_miss) {
-          continue;  // On a bloom filter miss, probing the hash table can be skipped.
-        }
-      }
-
       TypedValue key = accessor->getTypedValue(key_attr_id);
       if (check_for_null_keys && key.isNull()) {
         continue;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/55480d8d/storage/HashTable.proto
----------------------------------------------------------------------
diff --git a/storage/HashTable.proto b/storage/HashTable.proto
index ade30d8..1d4ccb0 100644
--- a/storage/HashTable.proto
+++ b/storage/HashTable.proto
@@ -34,10 +34,4 @@ message HashTable {
   required HashTableImplType hash_table_impl_type = 1;
   repeated Type key_types = 2;
   required uint64 estimated_num_entries = 3;
-  repeated uint32 build_side_bloom_filter_id = 4;
-  message ProbeSideBloomFilter {
-    required uint32 probe_side_bloom_filter_id = 1;
-    repeated uint32 probe_side_attr_ids = 2;
-  }
-  repeated ProbeSideBloomFilter probe_side_bloom_filters = 6;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/55480d8d/storage/HashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.hpp b/storage/HashTableFactory.hpp
index 40b39de..d690557 100644
--- a/storage/HashTableFactory.hpp
+++ b/storage/HashTableFactory.hpp
@@ -295,14 +295,11 @@ class HashTableFactory {
    * @param proto A protobuf description of a resizable HashTable.
    * @param storage_manager The StorageManager to use (a StorageBlob will be
    *        allocated to hold the HashTable's contents).
-   * @param bloom_filters A vector of pointers to bloom filters that may be used
-   *        during hash table construction in build/probe phase.
    * @return A new resizable HashTable with parameters specified by proto.
    **/
   static HashTable<ValueT, resizable, serializable, force_key_copy, allow_duplicate_keys>*
       CreateResizableFromProto(const serialization::HashTable &proto,
-                               StorageManager *storage_manager,
-                               const std::vector<std::unique_ptr<BloomFilter>> &bloom_filters) {
+                               StorageManager *storage_manager) {
     DCHECK(ProtoIsValid(proto))
         << "Attempted to create HashTable from invalid proto description:\n"
         << proto.DebugString();
@@ -316,35 +313,6 @@ class HashTableFactory {
                                       key_types,
                                       proto.estimated_num_entries(),
                                       storage_manager);
-
-    // TODO(ssaurabh): These lazy initializations can be moved from here and pushed to the
-    //                 individual implementations of the hash table constructors.
-
-    // Check if there are any build side bloom filter defined on the hash table.
-    if (proto.build_side_bloom_filter_id_size() > 0) {
-      hash_table->enableBuildSideBloomFilter();
-      hash_table->setBuildSideBloomFilter(bloom_filters[proto.build_side_bloom_filter_id(0)].get());
-    }
-
-    // Check if there are any probe side bloom filters defined on the hash table.
-    if (proto.probe_side_bloom_filters_size() > 0) {
-      hash_table->enableProbeSideBloomFilter();
-      // Add as many probe bloom filters as defined by the proto.
-      for (int j = 0; j < proto.probe_side_bloom_filters_size(); ++j) {
-        // Add the pointer to the probe bloom filter within the list of probe bloom filters to use.
-        const auto probe_side_bloom_filter = proto.probe_side_bloom_filters(j);
-        hash_table->addProbeSideBloomFilter(bloom_filters[probe_side_bloom_filter.probe_side_bloom_filter_id()].get());
-
-        // Add the attribute ids corresponding to this probe bloom filter.
-        std::vector<attribute_id> probe_attribute_ids;
-        for (int k = 0; k < probe_side_bloom_filter.probe_side_attr_ids_size(); ++k) {
-          const attribute_id probe_attribute_id = probe_side_bloom_filter.probe_side_attr_ids(k);
-          probe_attribute_ids.push_back(probe_attribute_id);
-        }
-        hash_table->addProbeSideAttributeIds(std::move(probe_attribute_ids));
-      }
-    }
-
     return hash_table;
   }