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/08/11 20:27:41 UTC

[01/16] incubator-quickstep git commit: Minor updates to Shiftboss. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/LIP-for-tpch abc9c0238 -> 39f574999 (forced update)


Minor updates to Shiftboss.


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

Branch: refs/heads/LIP-for-tpch
Commit: e443b2b8409a128cc5ba2bdf1a6d01ebf79e7e74
Parents: bd01748
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 10:33:10 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 10:33:10 2016 -0700

----------------------------------------------------------------------
 query_execution/QueryExecutionMessages.proto |  1 +
 query_execution/Shiftboss.cpp                | 89 ++++++++++++++++-------
 query_execution/Shiftboss.hpp                |  3 +
 3 files changed, 65 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e443b2b8/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index f6a8b73..f680d35 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -87,6 +87,7 @@ message ShiftbossRegistrationMessage {
 }
 
 message ShiftbossRegistrationResponseMessage {
+  required uint64 shiftboss_index = 1;
 }
 
 message QueryInitiateMessage {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e443b2b8/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index 120e8fb..24c91fe 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -56,6 +56,7 @@ using std::string;
 using std::unique_ptr;
 using std::vector;
 
+using tmb::MessageBus;
 using tmb::TaggedMessage;
 
 namespace quickstep {
@@ -78,6 +79,13 @@ void Shiftboss::run() {
     switch (annotated_message.tagged_message.message_type()) {
       case kShiftbossRegistrationResponseMessage: {
         foreman_client_id_ = annotated_message.sender;
+
+        const TaggedMessage &tagged_message = annotated_message.tagged_message;
+
+        serialization::ShiftbossRegistrationResponseMessage proto;
+        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+        shiftboss_index_ = proto.shiftboss_index();
         break;
       }
       case kQueryInitiateMessage: {
@@ -117,10 +125,14 @@ void Shiftboss::run() {
                   << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
                   << "') from Foreman to worker " << worker_index;
 
-        QueryExecutionUtil::SendTMBMessage(bus_,
-                                           shiftboss_client_id_,
-                                           workers_->getClientID(worker_index),
-                                           move(worker_tagged_message));
+        const MessageBus::SendStatus send_status =
+            QueryExecutionUtil::SendTMBMessage(bus_,
+                                               shiftboss_client_id_,
+                                               workers_->getClientID(worker_index),
+                                               move(worker_tagged_message));
+        CHECK(send_status == MessageBus::SendStatus::kOK)
+            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+            << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
         break;
       }
       case kInitiateRebuildMessage: {
@@ -147,10 +159,14 @@ void Shiftboss::run() {
                   << "' message from worker (client " << annotated_message.sender << ") to Foreman";
 
         DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
-        QueryExecutionUtil::SendTMBMessage(bus_,
-                                           shiftboss_client_id_,
-                                           foreman_client_id_,
-                                           move(annotated_message.tagged_message));
+        const MessageBus::SendStatus send_status =
+            QueryExecutionUtil::SendTMBMessage(bus_,
+                                               shiftboss_client_id_,
+                                               foreman_client_id_,
+                                               move(annotated_message.tagged_message));
+        CHECK(send_status == MessageBus::SendStatus::kOK)
+            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+            << " to Foreman with TMB client ID " << foreman_client_id_;
         break;
       }
       case kSaveQueryResultMessage: {
@@ -178,10 +194,14 @@ void Shiftboss::run() {
         LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
                   << "') sent SaveQueryResultResponseMessage (typed '" << kSaveQueryResultResponseMessage
                   << "') to Foreman";
-        QueryExecutionUtil::SendTMBMessage(bus_,
-                                           shiftboss_client_id_,
-                                           foreman_client_id_,
-                                           move(message_response));
+        const MessageBus::SendStatus send_status =
+            QueryExecutionUtil::SendTMBMessage(bus_,
+                                               shiftboss_client_id_,
+                                               foreman_client_id_,
+                                               move(message_response));
+        CHECK(send_status == MessageBus::SendStatus::kOK)
+            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+            << " to Foreman with TMB client ID " << foreman_client_id_;
         break;
       }
       case kPoisonMessage: {
@@ -192,12 +212,14 @@ void Shiftboss::run() {
         tmb::MessageStyle broadcast_style;
         broadcast_style.Broadcast(true);
 
-        tmb::MessageBus::SendStatus send_status =
+        const MessageBus::SendStatus send_status =
             bus_->Send(shiftboss_client_id_,
                        worker_addresses_,
                        broadcast_style,
                        move(annotated_message.tagged_message));
-        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
+        CHECK(send_status == MessageBus::SendStatus::kOK)
+            << "Message could not be broadcast from Shiftboss with TMB client ID " << shiftboss_client_id_
+            << " to All workers";
         return;
       }
       default: {
@@ -280,10 +302,14 @@ void Shiftboss::processQueryInitiateMessage(
                                  kQueryInitiateResponseMessage);
   free(proto_bytes);
 
-  QueryExecutionUtil::SendTMBMessage(bus_,
-                                     shiftboss_client_id_,
-                                     foreman_client_id_,
-                                     move(message_response));
+  const MessageBus::SendStatus send_status =
+      QueryExecutionUtil::SendTMBMessage(bus_,
+                                         shiftboss_client_id_,
+                                         foreman_client_id_,
+                                         move(message_response));
+  CHECK(send_status == MessageBus::SendStatus::kOK)
+      << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+      << " to Foreman with TMB client ID " << foreman_client_id_;
 }
 
 void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
@@ -307,8 +333,7 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
   proto.set_query_id(query_id);
   proto.set_operator_index(op_index);
   proto.set_num_rebuild_work_orders(partially_filled_block_refs.size());
-  // TODO(zuyu): Multiple Shiftboss support.
-  proto.set_shiftboss_index(0);
+  proto.set_shiftboss_index(shiftboss_index_);
 
   const size_t proto_length = proto.ByteSize();
   char *proto_bytes = static_cast<char*>(malloc(proto_length));
@@ -319,10 +344,14 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
                                  kInitiateRebuildResponseMessage);
   free(proto_bytes);
 
-  QueryExecutionUtil::SendTMBMessage(bus_,
-                                     shiftboss_client_id_,
-                                     foreman_client_id_,
-                                     move(message_response));
+  const MessageBus::SendStatus send_status =
+      QueryExecutionUtil::SendTMBMessage(bus_,
+                                         shiftboss_client_id_,
+                                         foreman_client_id_,
+                                         move(message_response));
+  CHECK(send_status == MessageBus::SendStatus::kOK)
+      << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+      << " to Foreman with TMB client ID " << foreman_client_id_;
 
   for (size_t i = 0; i < partially_filled_block_refs.size(); ++i) {
     // NOTE(zuyu): Worker releases the memory after the execution of
@@ -347,10 +376,14 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
               << "') sent RebuildWorkOrderMessage (typed '" << kRebuildWorkOrderMessage
               << "') to worker " << worker_index;
 
-    QueryExecutionUtil::SendTMBMessage(bus_,
-                                       shiftboss_client_id_,
-                                       workers_->getClientID(worker_index),
-                                       move(worker_tagged_message));
+    const MessageBus::SendStatus send_status =
+        QueryExecutionUtil::SendTMBMessage(bus_,
+                                           shiftboss_client_id_,
+                                           workers_->getClientID(worker_index),
+                                           move(worker_tagged_message));
+    CHECK(send_status == MessageBus::SendStatus::kOK)
+        << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
+        << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e443b2b8/query_execution/Shiftboss.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.hpp b/query_execution/Shiftboss.hpp
index 32d2408..9464a4d 100644
--- a/query_execution/Shiftboss.hpp
+++ b/query_execution/Shiftboss.hpp
@@ -223,6 +223,9 @@ class Shiftboss : public Thread {
 
   tmb::client_id shiftboss_client_id_, foreman_client_id_;
 
+  // Unique per Shiftboss instance.
+  std::uint64_t shiftboss_index_;
+
   // TMB recipients for all workers managed by this Shiftboss.
   tmb::Address worker_addresses_;
 


[08/16] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/PackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreValueAccessor.hpp b/storage/PackedRowStoreValueAccessor.hpp
index 80edecd..1520b3a 100644
--- a/storage/PackedRowStoreValueAccessor.hpp
+++ b/storage/PackedRowStoreValueAccessor.hpp
@@ -20,6 +20,8 @@
 #ifndef QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_
 #define QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_
 
+#include <utility>
+
 #include "catalog/CatalogRelationSchema.hpp"
 #include "catalog/CatalogTypedefs.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -42,7 +44,8 @@ class PackedRowStoreValueAccessorHelper {
       : relation_(relation),
         num_tuples_(num_tuples),
         tuple_storage_(tuple_storage),
-        null_bitmap_(null_bitmap) {
+        null_bitmap_(null_bitmap),
+        attr_max_lengths_(relation.getMaximumAttributeByteLengths()) {
   }
 
   inline tuple_id numPackedTuples() const {
@@ -67,6 +70,25 @@ class PackedRowStoreValueAccessorHelper {
            + relation_.getFixedLengthAttributeOffset(attr);  // Attribute offset within tuple.
   }
 
+  template <bool check_null>
+  inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+                                                                        const attribute_id attr) const {
+    DEBUG_ASSERT(tuple < num_tuples_);
+    DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
+    if (check_null) {
+      const int nullable_idx = relation_.getNullableAttributeIndex(attr);
+      if ((nullable_idx != -1)
+          && null_bitmap_->getBit(tuple * relation_.numNullableAttributes() + nullable_idx)) {
+        return std::make_pair(nullptr, 0);
+      }
+    }
+
+    return std::make_pair(static_cast<const char*>(tuple_storage_)
+                              + (tuple * relation_.getFixedByteLength())
+                              + relation_.getFixedLengthAttributeOffset(attr),
+                          attr_max_lengths_[attr]);
+  }
+
   inline TypedValue getAttributeValueTyped(const tuple_id tuple,
                                            const attribute_id attr) const {
     const Type &attr_type = relation_.getAttributeById(attr)->getType();
@@ -81,6 +103,7 @@ class PackedRowStoreValueAccessorHelper {
   const tuple_id num_tuples_;
   const void *tuple_storage_;
   const BitVector<false> *null_bitmap_;
+  const std::vector<std::size_t> &attr_max_lengths_;
 
   DISALLOW_COPY_AND_ASSIGN(PackedRowStoreValueAccessorHelper);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/SplitRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreValueAccessor.hpp b/storage/SplitRowStoreValueAccessor.hpp
index 61bb7bf..e2d2b47 100644
--- a/storage/SplitRowStoreValueAccessor.hpp
+++ b/storage/SplitRowStoreValueAccessor.hpp
@@ -102,6 +102,11 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
     return getUntypedValueAtAbsolutePosition<check_null>(attr_id, current_position_);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+    return getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, current_position_);
+  }
+
   inline TypedValue getTypedValue(const attribute_id attr_id) const {
     return getTypedValueAtAbsolutePosition(attr_id, current_position_);
   }
@@ -142,6 +147,44 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
     }
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+                                                                                            const tuple_id tid) const {
+    DEBUG_ASSERT(occupancy_bitmap_.getBit(tid));
+    DEBUG_ASSERT(relation_.hasAttributeWithId(attr_id));
+    const char *tuple_slot = static_cast<const char*>(tuple_storage_)
+                             + tuple_slot_bytes_ * tid;
+    if (check_null) {
+      const int nullable_idx = relation_.getNullableAttributeIndex(attr_id);
+      if (nullable_idx != -1) {
+        // const_cast is safe here. We will only be using read-only methods of
+        // BitVector.
+        BitVector<true> tuple_null_bitmap(const_cast<void*>(static_cast<const void*>(tuple_slot)),
+                                          relation_.numNullableAttributes());
+        if (tuple_null_bitmap.getBit(nullable_idx)) {
+          return std::make_pair(nullptr, 0);
+        }
+      }
+    }
+
+    const int variable_length_idx = relation_.getVariableLengthAttributeIndex(attr_id);
+    if (variable_length_idx == -1) {
+      // Fixed-length, stored in-line in slot.
+      return std::make_pair(tuple_slot + per_tuple_null_bitmap_bytes_
+                                       + relation_.getFixedLengthAttributeOffset(attr_id),
+                            attr_max_lengths_[attr_id]);
+
+    } else {
+      // Variable-length, stored at back of block.
+      const std::uint32_t *pos_ptr = reinterpret_cast<const std::uint32_t*>(
+          tuple_slot + per_tuple_null_bitmap_bytes_
+                     + relation_.getFixedByteLength()
+                     + variable_length_idx * 2 * sizeof(std::uint32_t));
+      return std::make_pair(static_cast<const char*>(tuple_storage_) + pos_ptr[0],
+                            pos_ptr[1]);
+    }
+  }
+
   inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
                                                     const tuple_id tid) const {
     DEBUG_ASSERT(occupancy_bitmap_.getBit(tid));
@@ -319,6 +362,7 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
         tuple_storage_(tuple_storage),
         tuple_slot_bytes_(tuple_slot_bytes),
         per_tuple_null_bitmap_bytes_(per_tuple_null_bitmap_bytes),
+        attr_max_lengths_(relation.getMaximumAttributeByteLengths()),
         current_position_(std::numeric_limits<std::size_t>::max()) {
   }
 
@@ -329,6 +373,7 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
   const void *tuple_storage_;
   const std::size_t tuple_slot_bytes_;
   const std::size_t per_tuple_null_bitmap_bytes_;
+  const std::vector<std::size_t> &attr_max_lengths_;
 
   std::size_t current_position_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 21aa12c..8370418 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -389,15 +389,7 @@ AggregationState* StorageBlock::aggregate(
     const AggregationHandle &handle,
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
     const std::vector<attribute_id> *arguments_as_attributes,
-    const Predicate *predicate,
     std::unique_ptr<TupleIdSequence> *reuse_matches) const {
-  // If there is a filter predicate that hasn't already been evaluated,
-  // evaluate it now and save the results for other aggregates on this same
-  // block.
-  if (predicate && !*reuse_matches) {
-    reuse_matches->reset(getMatchesForPredicate(predicate));
-  }
-
 #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.
@@ -418,7 +410,6 @@ void StorageBlock::aggregateGroupBy(
     const AggregationHandle &handle,
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
     const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const Predicate *predicate,
     AggregationStateHashTableBase *hash_table,
     std::unique_ptr<TupleIdSequence> *reuse_matches,
     std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
@@ -440,14 +431,7 @@ void StorageBlock::aggregateGroupBy(
   ColumnVectorsValueAccessor temp_result;
   {
     std::unique_ptr<ValueAccessor> accessor;
-    if (predicate) {
-      if (!*reuse_matches) {
-        // If there is a filter predicate that hasn't already been evaluated,
-        // evaluate it now and save the results for other aggregates on this
-        // same block.
-        reuse_matches->reset(getMatchesForPredicate(predicate));
-      }
-
+    if (reuse_matches) {
       // Create a filtered ValueAccessor that only iterates over predicate
       // matches.
       accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));
@@ -499,7 +483,6 @@ void StorageBlock::aggregateDistinct(
     const std::vector<std::unique_ptr<const Scalar>> &arguments,
     const std::vector<attribute_id> *arguments_as_attributes,
     const std::vector<std::unique_ptr<const Scalar>> &group_by,
-    const Predicate *predicate,
     AggregationStateHashTableBase *distinctify_hash_table,
     std::unique_ptr<TupleIdSequence> *reuse_matches,
     std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const {
@@ -514,14 +497,7 @@ void StorageBlock::aggregateDistinct(
   ColumnVectorsValueAccessor temp_result;
   {
     std::unique_ptr<ValueAccessor> accessor;
-    if (predicate) {
-      if (!*reuse_matches) {
-        // If there is a filter predicate that hasn't already been evaluated,
-        // evaluate it now and save the results for other aggregates on this
-        // same block.
-        reuse_matches->reset(getMatchesForPredicate(predicate));
-      }
-
+    if (reuse_matches) {
       // Create a filtered ValueAccessor that only iterates over predicate
       // matches.
       accessor.reset(tuple_store_->createValueAccessor(reuse_matches->get()));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 97b4773..2a20cb5 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -412,7 +412,6 @@ class StorageBlock : public StorageBlockBase {
       const AggregationHandle &handle,
       const std::vector<std::unique_ptr<const Scalar>> &arguments,
       const std::vector<attribute_id> *arguments_as_attributes,
-      const Predicate *predicate,
       std::unique_ptr<TupleIdSequence> *reuse_matches) const;
 
   /**
@@ -462,7 +461,6 @@ class StorageBlock : public StorageBlockBase {
   void aggregateGroupBy(const AggregationHandle &handle,
                         const std::vector<std::unique_ptr<const Scalar>> &arguments,
                         const std::vector<std::unique_ptr<const Scalar>> &group_by,
-                        const Predicate *predicate,
                         AggregationStateHashTableBase *hash_table,
                         std::unique_ptr<TupleIdSequence> *reuse_matches,
                         std::vector<std::unique_ptr<ColumnVector>>
@@ -507,7 +505,6 @@ class StorageBlock : public StorageBlockBase {
                          const std::vector<std::unique_ptr<const Scalar>> &arguments,
                          const std::vector<attribute_id> *arguments_as_attributes,
                          const std::vector<std::unique_ptr<const Scalar>> &group_by,
-                         const Predicate *predicate,
                          AggregationStateHashTableBase *distinctify_hash_table,
                          std::unique_ptr<TupleIdSequence> *reuse_matches,
                          std::vector<std::unique_ptr<ColumnVector>> *reuse_group_by_vectors) const;
@@ -590,6 +587,8 @@ class StorageBlock : public StorageBlockBase {
    **/
   const std::size_t getNumTuples() const;
 
+  TupleIdSequence* getMatchesForPredicate(const Predicate *predicate) const;
+
  private:
   static TupleStorageSubBlock* CreateTupleStorageSubBlock(
       const CatalogRelationSchema &relation,
@@ -629,8 +628,6 @@ class StorageBlock : public StorageBlockBase {
   // StorageBlock's header.
   bool rebuildIndexes(bool short_circuit);
 
-  TupleIdSequence* getMatchesForPredicate(const Predicate *predicate) const;
-
   std::unordered_map<attribute_id, TypedValue>* generateUpdatedValues(
       const ValueAccessor &accessor,
       const tuple_id tuple,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/ValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessor.hpp b/storage/ValueAccessor.hpp
index 70d4405..b107390 100644
--- a/storage/ValueAccessor.hpp
+++ b/storage/ValueAccessor.hpp
@@ -377,6 +377,11 @@ class TupleIdSequenceAdapterValueAccessor : public ValueAccessor {
     return accessor_->template getUntypedValueAtAbsolutePosition<check_null>(attr_id, *current_position_);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+    return accessor_->template getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, *current_position_);
+  }
+
   inline TypedValue getTypedValue(const attribute_id attr_id) const {
     return accessor_->getTypedValueAtAbsolutePosition(attr_id, *current_position_);
   }
@@ -389,6 +394,13 @@ class TupleIdSequenceAdapterValueAccessor : public ValueAccessor {
   }
 
   // Pass-through.
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+                                                                                            const tuple_id tid) const {
+    return accessor_->template getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, tid);
+  }
+
+  // Pass-through.
   inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
                                                     const tuple_id tid) const {
     return accessor_->getTypedValueAtAbsolutePosition(attr_id, tid);
@@ -562,6 +574,12 @@ class OrderedTupleIdSequenceAdapterValueAccessor : public ValueAccessor {
                                                                              id_sequence_[current_position_]);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+    return accessor_->template getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(
+        attr_id, id_sequence_[current_position_]);
+  }
+
   inline TypedValue getTypedValue(const attribute_id attr_id) const {
     return accessor_->getTypedValueAtAbsolutePosition(attr_id, id_sequence_[current_position_]);
   }
@@ -573,6 +591,13 @@ class OrderedTupleIdSequenceAdapterValueAccessor : public ValueAccessor {
                 "OrderedTupleIdSequenceAdapterValueAccessor");
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+                                                                                            const tuple_id tid) const {
+    FATAL_ERROR("getUntypedValueAndByteLengthAtAbsolutePosition() not implemented in "
+                "OrderedTupleIdSequenceAdapterValueAccessor");
+  }
+
   inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
                                                     const tuple_id tid) const {
     FATAL_ERROR("getTypedValueAtAbsolutePosition() not implemented in "
@@ -739,6 +764,11 @@ class PackedTupleStorageSubBlockValueAccessor : public ValueAccessor {
     return getUntypedValueAtAbsolutePosition<check_null>(attr_id, current_tuple_);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+    return getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, current_tuple_);
+  }
+
   inline TypedValue getTypedValue(const attribute_id attr_id) const {
     return getTypedValueAtAbsolutePosition(attr_id, current_tuple_);
   }
@@ -749,6 +779,12 @@ class PackedTupleStorageSubBlockValueAccessor : public ValueAccessor {
     return helper_.template getAttributeValue<check_null>(tid, attr_id);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+                                                                                            const tuple_id tid) const {
+    return helper_.template getAttributeValueAndByteLength<check_null>(tid, attr_id);
+  }
+
   inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
                                                     const tuple_id tid) const {
     return helper_.getAttributeValueTyped(tid, attr_id);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index fc65656..0817054 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -195,6 +195,22 @@ class NativeColumnVector : public ColumnVector {
   }
 
   /**
+   * @brief Get the untyped pointer to a value as well as the value's byte length
+   *        in this NativeColumnVector as a pair.
+   *
+   * @param position The position of the value to get.
+   * @return A pair containing the untyped pointer to the value at position and
+   *         the value's byte length.
+   **/
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const std::size_t position) const {
+    DCHECK_LT(position, actual_length_);
+    return (check_null && null_bitmap_ && null_bitmap_->getBit(position))
+        ? std::make_pair(nullptr, 0)
+        : std::make_pair(static_cast<const char*>(values_) + (position * type_length_), type_length_);
+  }
+
+  /**
    * @brief Get a value in this NativeColumnVector as a TypedValue.
    *
    * @param position The position of the value to get.
@@ -455,6 +471,25 @@ class IndirectColumnVector : public ColumnVector {
   }
 
   /**
+   * @brief Get the untyped pointer to a value as well as the value's byte length
+   *        in this IndirectColumnVector as a pair.
+   *
+   * @param position The position of the value to get.
+   * @return A pair containing the untyped pointer to the value at position and
+   *         the value's byte length.
+   **/
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const std::size_t position) const {
+    DCHECK_LT(position, values_.size());
+    if (check_null && type_is_nullable_ && values_[position].isNull()) {
+      return std::make_pair(nullptr, 0);
+    } else {
+      const TypedValue &value = values_[position];
+      return std::make_pair(value.getDataPtr(), value.getDataSize());
+    }
+  }
+
+  /**
    * @brief Get a value in this IndirectColumnVector as a TypedValue.
    *
    * @param position The position of the value to get.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/types/containers/ColumnVectorsValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVectorsValueAccessor.hpp b/types/containers/ColumnVectorsValueAccessor.hpp
index 2300f3b..19e57a9 100644
--- a/types/containers/ColumnVectorsValueAccessor.hpp
+++ b/types/containers/ColumnVectorsValueAccessor.hpp
@@ -126,6 +126,11 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
     return getUntypedValueAtAbsolutePosition<check_null>(attr_id, current_position_);
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLength(const attribute_id attr_id) const {
+    return getUntypedValueAndByteLengthAtAbsolutePosition<check_null>(attr_id, current_position_);
+  }
+
   inline TypedValue getTypedValue(const attribute_id attr_id) const {
     return getTypedValueAtAbsolutePosition(attr_id, current_position_);
   }
@@ -142,6 +147,18 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
     }
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthAtAbsolutePosition(const attribute_id attr_id,
+                                                                                            const tuple_id tid) const {
+    DCHECK(attributeIdInRange(attr_id));
+    DCHECK(tupleIdInRange(tid));
+    if (column_native_[attr_id]) {
+      return static_cast<const NativeColumnVector&>(*columns_[attr_id]).getUntypedValueAndByteLength<check_null>(tid);
+    } else {
+      return static_cast<const IndirectColumnVector&>(*columns_[attr_id]).getUntypedValueAndByteLength<check_null>(tid);
+    }
+  }
+
   inline TypedValue getTypedValueAtAbsolutePosition(const attribute_id attr_id,
                                                     const tuple_id tid) const {
     DCHECK(attributeIdInRange(attr_id));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/BloomFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/BloomFilter.hpp b/utility/BloomFilter.hpp
index 8d62da9..749d33a 100644
--- a/utility/BloomFilter.hpp
+++ b/utility/BloomFilter.hpp
@@ -23,6 +23,7 @@
 #include <algorithm>
 #include <cstddef>
 #include <cstdint>
+#include <cstring>
 #include <memory>
 #include <utility>
 #include <vector>
@@ -41,11 +42,358 @@ namespace quickstep {
  *  @{
  */
 
+class BloomFilterOriginal;
+class BloomFilterBlocked;
+typedef BloomFilterBlocked BloomFilter;
+
+/**
+ * @brief A "blocked" version of Bloom Filter based on this paper:
+ *        Putze, Felix, Peter Sanders, and Johannes Singler.
+ *        "Cache-, hash-and space-efficient bloom filters."
+ *        International Workshop on Experimental and Efficient Algorithms.
+ *        Springer Berlin Heidelberg, 2007.
+ **/
+class BloomFilterBlocked {
+ public:
+  static const std::uint8_t kNumBitsPerByte = 8;
+  static const std::uint8_t kMaxNumHashFns = 4;
+
+  // This union allows us to read/write position in convenient fashion,
+  // through nested structs and their bitfield members
+  //
+  // A position can simply be a 32-bit hash
+  // Or it can be a cache line (block of 512 bits) and position within it
+  // Or it can be a byte (block of 8 bits) and position within it
+  union Position {
+    std::uint32_t hash;
+    struct CacheLinePosition {
+      unsigned index_in_line : 9;
+      unsigned line_num : 23;
+    } cache_line_pos;
+    struct BytePosition {
+      unsigned index_in_byte : 3;
+      unsigned byte_num : 29;
+    } byte_pos;
+  };
+
+  // This Bloom filter implementation requires the bit array to be a
+  // multiple of the cache-line size. So we either have to round up to a 
+  // multiple (default behavior) or round down to a multiple.
+  // Rounding up is usually preferable but rounding down is necessary when
+  // we are given a bit array that we don't control the size of, in the
+  // constructor.
+  static std::uint64_t getNearestAllowedSize(
+      const std::uint64_t approx_size,
+      bool round_down = false) {
+    if (round_down)
+      return (approx_size / kCacheLineBytes) * kCacheLineBytes;
+    return ((approx_size + kCacheLineBytes - 1)/ kCacheLineBytes) * kCacheLineBytes;
+  }
+
+
+  /**
+   * @brief Constructor.
+   * @note When no bit_array is being passed to the constructor,
+   *       then the bit_array is owned and managed by this class.
+   *
+   * @param hash_fn_count The number of hash functions used by this bloom filter.
+   * @param bit_array_size_in_bytes Size of the bit array.
+   **/
+  BloomFilterBlocked(const std::uint8_t hash_fn_count,
+              const std::uint64_t bit_array_size_in_bytes)
+      : hash_fn_count_(hash_fn_count),
+        array_size_in_bytes_(getNearestAllowedSize(bit_array_size_in_bytes)),
+        is_bit_array_owner_(true),
+        bit_array_(new std::uint8_t[array_size_in_bytes_]) {
+    reset();
+  }
+
+  /**
+   * @brief Constructor.
+   * @note When a bit_array is passed as an argument to the constructor,
+   *       then the ownership of the bit array lies with the caller.
+   *
+   * @param hash_fn_count The number of hash functions used by this bloom filter.
+   * @param bit_array_size_in_bytes Size of the bit array.
+   * @param bit_array A pointer to the memory region that is used to store bit array.
+   * @param is_initialized A boolean that indicates whether to zero-out the region
+   *                       before use or not.
+   **/
+  BloomFilterBlocked(const std::uint8_t hash_fn_count,
+              const std::uint64_t bit_array_size_in_bytes,
+              std::uint8_t *bit_array,
+              const bool is_initialized)
+      : hash_fn_count_(hash_fn_count),
+        array_size_in_bytes_(getNearestAllowedSize(bit_array_size_in_bytes, true)),
+        is_bit_array_owner_(false),
+        bit_array_(bit_array) {  // Owned by the calling method.
+    if (!is_initialized) {
+      reset();
+    }
+  }
+
+  /**
+   * @brief Constructor.
+   * @note When a bloom filter proto is passed as an initializer,
+   *       then the bit_array is owned and managed by this class.
+   *
+   * @param bloom_filter_proto The protobuf representation of a
+   *        bloom filter configuration.
+   **/
+  explicit BloomFilterBlocked(const serialization::BloomFilter &bloom_filter_proto)
+      : hash_fn_count_(bloom_filter_proto.number_of_hashes()),
+        array_size_in_bytes_(bloom_filter_proto.bloom_filter_size()),
+        is_bit_array_owner_(true),
+        bit_array_(new std::uint8_t[array_size_in_bytes_]) {
+    reset();
+  }
+
+  /**
+   * @brief Destructor.
+   **/
+  ~BloomFilterBlocked() {
+    if (is_bit_array_owner_) {
+      bit_array_.reset();
+    } else {
+      bit_array_.release();
+    }
+  }
+
+  static bool ProtoIsValid(const serialization::BloomFilter &bloom_filter_proto) {
+    return bloom_filter_proto.IsInitialized();
+  }
+
+  /**
+   * @brief Zeros out the contents of the bit array.
+   **/
+  inline void reset() {
+    // Initialize the bit_array with all zeros.
+    std::fill_n(bit_array_.get(), array_size_in_bytes_, 0x00);
+    inserted_element_count_ = 0;
+  }
+
+  /**
+   * @brief Get the number of hash functions used in this bloom filter.
+   *
+   * @return Returns the number of hash functions.
+   **/
+  inline std::uint8_t getNumberOfHashes() const {
+    return hash_fn_count_;
+  }
+
+  /**
+   * @brief Get the size of the bit array in bytes for this bloom filter.
+   *
+   * @return Returns the bit array size (in bytes).
+   **/
+  inline std::uint64_t getBitArraySize() const {
+    return array_size_in_bytes_;
+  }
+
+  /**
+   * @brief Get the constant pointer to the bit array for this bloom filter
+   *
+   * @return Returns constant pointer to the bit array.
+   **/
+  inline const std::uint8_t* getBitArray() const {
+    return bit_array_.get();
+  }
+
+  template <typename T>
+  void insert(const T &value) {
+    insert(reinterpret_cast<const std::uint8_t *>(&value), sizeof(T));
+  }
+
+  /**
+   * @brief Inserts a given value into the bloom filter in a thread-safe manner.
+   *
+   * @param key_begin A pointer to the value being inserted.
+   * @param length Size of the value being inserted in bytes.
+   */
+  inline void insert(const std::uint8_t *key_begin, const std::size_t length) {
+      SpinSharedMutexExclusiveLock<false> exclusive_writer_lock(bloom_filter_insert_mutex_);
+      insertUnSafe(key_begin, length);
+  }
+
+  template <typename T>
+  void insertUnSafe(const T &value) {
+    insertUnSafe(reinterpret_cast<const std::uint8_t *>(&value), sizeof(T));
+  }
+
+  /**
+   * @brief Inserts a given value into the bloom filter.
+   * @Warning This is a faster thread-unsafe version of the insert() function.
+   *          The caller needs to ensure the thread safety.
+   *
+   * @param key_begin A pointer to the value being inserted.
+   * @param length Size of the value being inserted in bytes.
+   */
+  inline void insertUnSafe(const std::uint8_t *key_begin, const std::size_t length) {
+    Position first_pos = getFirstPosition(key_begin, length);
+    setBitAtPosition(first_pos);
+    Position other_pos;
+    for (std::uint8_t i = 1; i <hash_fn_count_; ++i) {
+      other_pos = getOtherPosition(key_begin, length, first_pos, i);
+      setBitAtPosition(other_pos);
+    }
+    ++inserted_element_count_;
+  }
+
+  template <typename T>
+  bool contains(const T &value) {
+    return contains(reinterpret_cast<const std::uint8_t *>(&value), sizeof(T));
+  }
+
+  /**
+   * @brief Test membership of a given value in the bloom filter.
+   *        If true is returned, then a value may or may not be present in the bloom filter.
+   *        If false is returned, a value is certainly not present in the bloom filter.
+   *
+   * @note The membersip test does not require any locks, because the assumption is that
+   *       the bloom filter will only be used after it has been built.
+   *
+   * @param key_begin A pointer to the value being tested for membership.
+   * @param length Size of the value being inserted in bytes.
+   */
+  inline bool contains(
+      const std::uint8_t *__restrict__ key_begin,
+      const std::size_t length) const {
+    Position first_pos = getFirstPosition(key_begin, length);
+    if (!getBitAtPosition(first_pos)) {
+      return false;
+    }
+    Position other_pos;
+    for (std::uint8_t i = 1; i < hash_fn_count_; ++i) {
+      other_pos = getOtherPosition(key_begin, length, first_pos, i);
+      if (!getBitAtPosition(other_pos)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * @brief Perform a bitwise-OR of the given Bloom filter with this bloom filter.
+   *        Essentially, it does a union of this bloom filter with the passed bloom filter.
+   *
+   * @param bloom_filter A const pointer to the bloom filter object to do bitwise-OR with.
+   */
+  inline void bitwiseOr(const BloomFilterBlocked *bloom_filter) {
+    SpinSharedMutexExclusiveLock<false> exclusive_writer_lock(bloom_filter_insert_mutex_);
+    for (std::size_t byte_index = 0; byte_index < bloom_filter->getBitArraySize(); ++byte_index) {
+      (bit_array_.get())[byte_index] |= bloom_filter->getBitArray()[byte_index];
+    }
+  }
+
+  /**
+   * @brief Return the number of elements currently inserted into bloom filter.
+   *
+   * @return The number of elements inserted into bloom filter.
+   **/
+  inline std::uint32_t element_count() const {
+    return inserted_element_count_;
+  }
+
+ protected:
+  Position getFirstPosition(const std::uint8_t *begin, std::size_t length) const {
+    Position pos;
+    pos.hash = hash_identity(begin, length);
+    return pos;
+  }
+
+  Position getOtherPosition(
+      const std::uint8_t *begin,
+      std::size_t length,
+      const Position first_pos,
+      const std::uint8_t index) const {
+    Position pos;
+    pos.hash = hash_multiplicative(begin, length, hash_fn_[index-1]);
+    pos.cache_line_pos.line_num = first_pos.cache_line_pos.line_num;
+    return pos;
+  }
+
+  void fillPosition(
+      const std::uint8_t *begin,
+      std::size_t length,
+      const std::uint8_t index,
+      Position positions[]) const {
+    if (index == 0)
+      positions[0].hash = hash_identity(begin, length);
+    else {
+      positions[index].hash = hash_multiplicative(begin, length, hash_fn_[index-1]);
+      positions[index].cache_line_pos.line_num = positions[0].cache_line_pos.line_num;
+    }
+  }
+
+  void setBitAtPosition(const Position &pos) {
+    (bit_array_.get())[pos.byte_pos.byte_num] |= (1 << pos.byte_pos.index_in_byte);
+  }
+
+  bool getBitAtPosition(const Position &pos) const {
+    return (bit_array_.get())[pos.byte_pos.byte_num] & (1 << pos.byte_pos.index_in_byte);
+  }
+
+  inline std::uint32_t hash_identity(
+      const std::uint8_t *__restrict__ begin,
+      std::size_t length) const {
+    std::uint32_t hash;
+    if (length >= 4)
+      hash = *reinterpret_cast<const std::uint32_t*> (begin);
+    else
+      std::memcpy(&hash, begin, length);
+    return hash % (array_size_in_bytes_ * kNumBitsPerByte);
+  }
+
+  inline std::uint32_t hash_multiplicative(
+      const std::uint8_t *__restrict__ begin,
+      std::size_t length,
+      const std::uint64_t multiplier) const {
+    std::uint32_t hash = 0;
+    std::size_t bytes_hashed = 0;
+    if (length >= 4) {
+      while (bytes_hashed < length) {
+        auto val = *reinterpret_cast<const std::uint32_t *>(begin + bytes_hashed);
+        hash += (multiplier * val) >> 24;
+        bytes_hashed += 4;
+      }
+    }
+    while (bytes_hashed < length) {
+      std::uint8_t val = *(begin + bytes_hashed);
+      hash += (multiplier * val) >> 24;
+      bytes_hashed++;
+    }
+    return hash;//  % (array_size_in_bytes_ * kNumBitsPerByte);
+  }
+
+ private:
+  const std::uint32_t hash_fn_count_;
+  const std::uint64_t array_size_in_bytes_;
+  std::uint32_t inserted_element_count_;
+  const bool is_bit_array_owner_;
+
+  static constexpr std::uint64_t kKnuthGoldenRatioNumber = 2654435761;
+  const std::uint64_t hash_fn_[kMaxNumHashFns] = { // hash_fn_[i] is 2**(i+1) - 1
+    0x00000001 * kKnuthGoldenRatioNumber, // 0x00000003, 0x00000007, 0x0000000f,
+    // 0x0000001f * kKnuthGoldenRatioNumber, // 0x0000003f, 0x0000007f, 0x000000ff,
+    0x000001ff * kKnuthGoldenRatioNumber, // 0x000003ff, 0x000007ff, 0x00000fff,
+    // 0x00001fff * kKnuthGoldenRatioNumber, // 0x00003fff, 0x00007fff, 0x0000ffff,
+    0x0001ffff * kKnuthGoldenRatioNumber, // 0x0003ffff, 0x0007ffff, 0x000fffff,
+    // 0x001fffff * kKnuthGoldenRatioNumber, // 0x003fffff, 0x007fffff, 0x00ffffff,
+    0x01ffffff * kKnuthGoldenRatioNumber, // 0x03ffffff, 0x07ffffff, 0x0fffffff,
+    // 0x1fffffff * kKnuthGoldenRatioNumber  // 0x3fffffff, 0x7fffffff, 0xffffffff
+    };
+
+  alignas(kCacheLineBytes) std::unique_ptr<std::uint8_t> bit_array_;
+  alignas(kCacheLineBytes) mutable SpinSharedMutex<false> bloom_filter_insert_mutex_;
+
+  DISALLOW_COPY_AND_ASSIGN(BloomFilterBlocked);
+};
+
 /**
  * @brief A simple Bloom Filter implementation with basic primitives
  *        based on Partow's Bloom Filter implementation.
  **/
-class BloomFilter {
+class BloomFilterOriginal {
  public:
   static const uint32_t kNumBitsPerByte = 8;
 
@@ -54,21 +402,17 @@ class BloomFilter {
    * @note When no bit_array is being passed to the constructor,
    *       then the bit_array is owned and managed by this class.
    *
-   * @param random_seed A random_seed that generates unique hash functions.
    * @param hash_fn_count The number of hash functions used by this bloom filter.
    * @param bit_array_size_in_bytes Size of the bit array.
    **/
-  BloomFilter(const std::uint64_t random_seed,
-              const std::size_t hash_fn_count,
+  BloomFilterOriginal(const std::size_t hash_fn_count,
               const std::uint64_t bit_array_size_in_bytes)
-      : random_seed_(random_seed),
-        hash_fn_count_(hash_fn_count),
+      : hash_fn_count_(hash_fn_count),
         array_size_in_bytes_(bit_array_size_in_bytes),
         array_size_(array_size_in_bytes_ * kNumBitsPerByte),
         bit_array_(new std::uint8_t[array_size_in_bytes_]),
         is_bit_array_owner_(true) {
     reset();
-    generate_unique_hash_fn();
   }
 
   /**
@@ -76,20 +420,17 @@ class BloomFilter {
    * @note When a bit_array is passed as an argument to the constructor,
    *       then the ownership of the bit array lies with the caller.
    *
-   * @param random_seed A random_seed that generates unique hash functions.
    * @param hash_fn_count The number of hash functions used by this bloom filter.
    * @param bit_array_size_in_bytes Size of the bit array.
    * @param bit_array A pointer to the memory region that is used to store bit array.
    * @param is_initialized A boolean that indicates whether to zero-out the region
    *                       before use or not.
    **/
-  BloomFilter(const std::uint64_t random_seed,
-              const std::size_t hash_fn_count,
+  BloomFilterOriginal(const std::size_t hash_fn_count,
               const std::uint64_t bit_array_size_in_bytes,
               std::uint8_t *bit_array,
               const bool is_initialized)
-      : random_seed_(random_seed),
-        hash_fn_count_(hash_fn_count),
+      : hash_fn_count_(hash_fn_count),
         array_size_in_bytes_(bit_array_size_in_bytes),
         array_size_(bit_array_size_in_bytes * kNumBitsPerByte),
         bit_array_(bit_array),  // Owned by the calling method.
@@ -97,7 +438,6 @@ class BloomFilter {
     if (!is_initialized) {
       reset();
     }
-    generate_unique_hash_fn();
   }
 
   /**
@@ -108,21 +448,19 @@ class BloomFilter {
    * @param bloom_filter_proto The protobuf representation of a
    *        bloom filter configuration.
    **/
-  explicit BloomFilter(const serialization::BloomFilter &bloom_filter_proto)
-      : random_seed_(bloom_filter_proto.bloom_filter_seed()),
-        hash_fn_count_(bloom_filter_proto.number_of_hashes()),
+  explicit BloomFilterOriginal(const serialization::BloomFilter &bloom_filter_proto)
+      : hash_fn_count_(bloom_filter_proto.number_of_hashes()),
         array_size_in_bytes_(bloom_filter_proto.bloom_filter_size()),
         array_size_(array_size_in_bytes_ * kNumBitsPerByte),
         bit_array_(new std::uint8_t[array_size_in_bytes_]),
         is_bit_array_owner_(true) {
     reset();
-    generate_unique_hash_fn();
   }
 
   /**
    * @brief Destructor.
    **/
-  ~BloomFilter() {
+  ~BloomFilterOriginal() {
     if (is_bit_array_owner_) {
       bit_array_.reset();
     } else {
@@ -144,15 +482,6 @@ class BloomFilter {
   }
 
   /**
-   * @brief Get the random seed that was used to initialize this bloom filter.
-   *
-   * @return Returns the random seed.
-   **/
-  inline std::uint64_t getRandomSeed() const {
-    return random_seed_;
-  }
-
-  /**
    * @brief Get the number of hash functions used in this bloom filter.
    *
    * @return Returns the number of hash functions.
@@ -195,7 +524,7 @@ class BloomFilter {
 
     // Determine all the bit positions that are required to be set.
     for (std::size_t i = 0; i < hash_fn_count_; ++i) {
-      compute_indices(hash_ap(key_begin, length, hash_fn_[i]), &bit_index, &bit);
+      compute_indices(hash_multiplicative(key_begin, length, hash_fn_[i]), &bit_index, &bit);
       modified_bit_positions.push_back(std::make_pair(bit_index, bit));
     }
 
@@ -240,7 +569,7 @@ class BloomFilter {
     std::size_t bit = 0;
 
     for (std::size_t i = 0; i < hash_fn_count_; ++i) {
-      compute_indices(hash_ap(key_begin, length, hash_fn_[i]), &bit_index, &bit);
+      compute_indices(hash_multiplicative(key_begin, length, hash_fn_[i]), &bit_index, &bit);
       (bit_array_.get())[bit_index / kNumBitsPerByte] |= (1 << bit);
     }
 
@@ -262,7 +591,7 @@ class BloomFilter {
     std::size_t bit_index = 0;
     std::size_t bit = 0;
     for (std::size_t i = 0; i < hash_fn_count_; ++i) {
-      compute_indices(hash_ap(key_begin, length, hash_fn_[i]), &bit_index, &bit);
+      compute_indices(hash_multiplicative(key_begin, length, hash_fn_[i]), &bit_index, &bit);
       if (((bit_array_.get())[bit_index / kNumBitsPerByte] & (1 << bit)) != (1 << bit)) {
         return false;
       }
@@ -276,7 +605,7 @@ class BloomFilter {
    *
    * @param bloom_filter A const pointer to the bloom filter object to do bitwise-OR with.
    */
-  inline void bitwiseOr(const BloomFilter *bloom_filter) {
+  inline void bitwiseOr(const BloomFilterOriginal *bloom_filter) {
     SpinSharedMutexExclusiveLock<false> exclusive_writer_lock(bloom_filter_insert_mutex_);
     for (std::size_t byte_index = 0; byte_index < bloom_filter->getBitArraySize(); ++byte_index) {
       (bit_array_.get())[byte_index] |= bloom_filter->getBitArray()[byte_index];
@@ -298,95 +627,28 @@ class BloomFilter {
     *bit = *bit_index % kNumBitsPerByte;
   }
 
-  void generate_unique_hash_fn() {
-    hash_fn_.reserve(hash_fn_count_);
-    const std::uint32_t predef_hash_fn_count = 128;
-    static const std::uint32_t predef_hash_fn[predef_hash_fn_count] = {
-       0xAAAAAAAA, 0x55555555, 0x33333333, 0xCCCCCCCC,
-       0x66666666, 0x99999999, 0xB5B5B5B5, 0x4B4B4B4B,
-       0xAA55AA55, 0x55335533, 0x33CC33CC, 0xCC66CC66,
-       0x66996699, 0x99B599B5, 0xB54BB54B, 0x4BAA4BAA,
-       0xAA33AA33, 0x55CC55CC, 0x33663366, 0xCC99CC99,
-       0x66B566B5, 0x994B994B, 0xB5AAB5AA, 0xAAAAAA33,
-       0x555555CC, 0x33333366, 0xCCCCCC99, 0x666666B5,
-       0x9999994B, 0xB5B5B5AA, 0xFFFFFFFF, 0xFFFF0000,
-       0xB823D5EB, 0xC1191CDF, 0xF623AEB3, 0xDB58499F,
-       0xC8D42E70, 0xB173F616, 0xA91A5967, 0xDA427D63,
-       0xB1E8A2EA, 0xF6C0D155, 0x4909FEA3, 0xA68CC6A7,
-       0xC395E782, 0xA26057EB, 0x0CD5DA28, 0x467C5492,
-       0xF15E6982, 0x61C6FAD3, 0x9615E352, 0x6E9E355A,
-       0x689B563E, 0x0C9831A8, 0x6753C18B, 0xA622689B,
-       0x8CA63C47, 0x42CC2884, 0x8E89919B, 0x6EDBD7D3,
-       0x15B6796C, 0x1D6FDFE4, 0x63FF9092, 0xE7401432,
-       0xEFFE9412, 0xAEAEDF79, 0x9F245A31, 0x83C136FC,
-       0xC3DA4A8C, 0xA5112C8C, 0x5271F491, 0x9A948DAB,
-       0xCEE59A8D, 0xB5F525AB, 0x59D13217, 0x24E7C331,
-       0x697C2103, 0x84B0A460, 0x86156DA9, 0xAEF2AC68,
-       0x23243DA5, 0x3F649643, 0x5FA495A8, 0x67710DF8,
-       0x9A6C499E, 0xDCFB0227, 0x46A43433, 0x1832B07A,
-       0xC46AFF3C, 0xB9C8FFF0, 0xC9500467, 0x34431BDF,
-       0xB652432B, 0xE367F12B, 0x427F4C1B, 0x224C006E,
-       0x2E7E5A89, 0x96F99AA5, 0x0BEB452A, 0x2FD87C39,
-       0x74B2E1FB, 0x222EFD24, 0xF357F60C, 0x440FCB1E,
-       0x8BBE030F, 0x6704DC29, 0x1144D12F, 0x948B1355,
-       0x6D8FD7E9, 0x1C11A014, 0xADD1592F, 0xFB3C712E,
-       0xFC77642F, 0xF9C4CE8C, 0x31312FB9, 0x08B0DD79,
-       0x318FA6E7, 0xC040D23D, 0xC0589AA7, 0x0CA5C075,
-       0xF874B172, 0x0CF914D5, 0x784D3280, 0x4E8CFEBC,
-       0xC569F575, 0xCDB2A091, 0x2CC016B4, 0x5C5F4421
-    };
-    if (hash_fn_count_ <= predef_hash_fn_count) {
-      std::copy(predef_hash_fn, predef_hash_fn + hash_fn_count_, hash_fn_.begin());
-      for (std::uint32_t i = 0; i < hash_fn_.size(); ++i) {
-        hash_fn_[i] = hash_fn_[i] * hash_fn_[(i + 3) % hash_fn_count_] + static_cast<std::uint32_t>(random_seed_);
+  inline std::uint32_t hash_multiplicative(
+      const std::uint8_t *begin,
+      std::size_t remaining_length,
+      const std::uint64_t multiplier) const {
+    std::uint32_t hash = 0;
+    std::size_t bytes_hashed = 0;
+    if (remaining_length >= 4) {
+      while (bytes_hashed < remaining_length) {
+        auto val = *reinterpret_cast<const std::uint32_t *>(begin + bytes_hashed);
+        hash += (multiplier * val) >> 32;
+        bytes_hashed += 4;
       }
-    } else {
-      LOG(FATAL) << "Requested number of hash functions is too large.";
     }
-  }
-
-  inline std::uint32_t hash_ap(const std::uint8_t *begin, std::size_t remaining_length, std::uint32_t hash) const {
-    const std::uint8_t *itr = begin;
-    std::uint32_t loop = 0;
-    while (remaining_length >= 8) {
-      const std::uint32_t &i1 = *(reinterpret_cast<const std::uint32_t*>(itr)); itr += sizeof(std::uint32_t);
-      const std::uint32_t &i2 = *(reinterpret_cast<const std::uint32_t*>(itr)); itr += sizeof(std::uint32_t);
-      hash ^= (hash <<  7) ^  i1 * (hash >> 3) ^ (~((hash << 11) + (i2 ^ (hash >> 5))));
-      remaining_length -= 8;
-    }
-    if (remaining_length) {
-      if (remaining_length >= 4) {
-        const std::uint32_t &i = *(reinterpret_cast<const std::uint32_t*>(itr));
-        if (loop & 0x01) {
-          hash ^= (hash <<  7) ^  i * (hash >> 3);
-        } else {
-          hash ^= (~((hash << 11) + (i ^ (hash >> 5))));
-        }
-        ++loop;
-        remaining_length -= 4;
-        itr += sizeof(std::uint32_t);
-      }
-      if (remaining_length >= 2) {
-        const std::uint16_t &i = *(reinterpret_cast<const std::uint16_t*>(itr));
-        if (loop & 0x01) {
-          hash ^= (hash <<  7) ^  i * (hash >> 3);
-        } else {
-          hash ^= (~((hash << 11) + (i ^ (hash >> 5))));
-        }
-        ++loop;
-        remaining_length -= 2;
-        itr += sizeof(std::uint16_t);
-      }
-      if (remaining_length) {
-        hash += ((*itr) ^ (hash * 0xA5A5A5A5)) + loop;
-      }
+    while (bytes_hashed < remaining_length) {
+      std::uint8_t val = *(begin + bytes_hashed);
+      hash += (multiplier * val) >> 32;
+      bytes_hashed++;
     }
     return hash;
   }
 
  private:
-  const std::uint64_t random_seed_;
-  std::vector<std::uint32_t> hash_fn_;
   const std::uint32_t hash_fn_count_;
   std::uint64_t array_size_in_bytes_;
   std::uint64_t array_size_;
@@ -394,9 +656,21 @@ class BloomFilter {
   std::uint32_t inserted_element_count_;
   const bool is_bit_array_owner_;
 
+  static constexpr std::uint64_t kKnuthGoldenRatioNumber = 2654435761;
+  static constexpr std::size_t kMaxNumHashFns = 8;
+  const std::uint64_t hash_fn_[kMaxNumHashFns] = { // hash_fn_[i] is 2**(i+1) - 1
+    0x00000001 * kKnuthGoldenRatioNumber, // 0x00000003, 0x00000007, 0x0000000f,
+    0x0000001f * kKnuthGoldenRatioNumber, // 0x0000003f, 0x0000007f, 0x000000ff,
+    0x000001ff * kKnuthGoldenRatioNumber, // 0x000003ff, 0x000007ff, 0x00000fff,
+    0x00001fff * kKnuthGoldenRatioNumber, // 0x00003fff, 0x00007fff, 0x0000ffff,
+    0x0001ffff * kKnuthGoldenRatioNumber, // 0x0003ffff, 0x0007ffff, 0x000fffff,
+    0x001fffff * kKnuthGoldenRatioNumber, // 0x003fffff, 0x007fffff, 0x00ffffff,
+    0x01ffffff * kKnuthGoldenRatioNumber, // 0x03ffffff, 0x07ffffff, 0x0fffffff,
+    0x1fffffff * kKnuthGoldenRatioNumber  // 0x3fffffff, 0x7fffffff, 0xffffffff
+    };
   alignas(kCacheLineBytes) mutable SpinSharedMutex<false> bloom_filter_insert_mutex_;
 
-  DISALLOW_COPY_AND_ASSIGN(BloomFilter);
+  DISALLOW_COPY_AND_ASSIGN(BloomFilterOriginal);
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/BloomFilter.proto
----------------------------------------------------------------------
diff --git a/utility/BloomFilter.proto b/utility/BloomFilter.proto
index 0f67878..1a8dbf2 100644
--- a/utility/BloomFilter.proto
+++ b/utility/BloomFilter.proto
@@ -23,10 +23,8 @@ message BloomFilter {
   // The default values were determined from empirical experiments.
   // These values control the amount of false positivity that
   // is expected from Bloom Filter.
-  // - Default seed for initializing family of hashes = 0xA5A5A5A55A5A5A5A.
   // - Default bloom filter size = 10 KB.
   // - Default number of hash functions used in bloom filter = 5.
-  optional fixed64 bloom_filter_seed = 1 [default = 0xA5A5A5A55A5A5A5A];
-  optional uint32 bloom_filter_size = 2 [default = 10000];
-  optional uint32 number_of_hashes = 3 [default = 5];
+  optional uint32 bloom_filter_size = 1 [default = 10000];
+  optional uint32 number_of_hashes = 2 [default = 5];
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/BloomFilterAdapter.hpp
----------------------------------------------------------------------
diff --git a/utility/BloomFilterAdapter.hpp b/utility/BloomFilterAdapter.hpp
new file mode 100644
index 0000000..f094307
--- /dev/null
+++ b/utility/BloomFilterAdapter.hpp
@@ -0,0 +1,142 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_BLOOM_FILTER_ADAPTER_HPP
+#define QUICKSTEP_UTILITY_BLOOM_FILTER_ADAPTER_HPP
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "utility/BloomFilter.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+class BloomFilterAdapter {
+ public:
+  BloomFilterAdapter(const std::vector<const BloomFilter*> &bloom_filters,
+                     const std::vector<attribute_id> &attribute_ids,
+                     const std::vector<std::size_t> &attr_sizes) {
+    DCHECK_EQ(bloom_filters.size(), attribute_ids.size());
+    DCHECK_EQ(bloom_filters.size(), attr_sizes.size());
+
+    bloom_filter_entries_.reserve(bloom_filters.size());
+    for (std::size_t i = 0; i < bloom_filters.size(); ++i) {
+      bloom_filter_entries_.emplace_back(
+          new BloomFilterEntry(
+              bloom_filters[i], attribute_ids[i], attr_sizes[i]));
+    }
+  }
+
+  ~BloomFilterAdapter() {
+    for (auto &entry : bloom_filter_entries_) {
+      delete entry;
+    }
+  }
+
+  template <bool adapt_filters, typename ValueAccessorT>
+  inline std::size_t bulkProbe(const ValueAccessorT *accessor,
+                               std::vector<tuple_id> &batch,
+                               const std::size_t batch_size) {
+    std::size_t out_size = batch_size;
+    for (auto &entry : bloom_filter_entries_) {
+      out_size = bulkProbeBloomFilterEntry<adapt_filters>(*entry, accessor, batch, out_size);
+    }
+    adaptEntryOrder();
+    return out_size;
+  }
+
+ private:
+  struct BloomFilterEntry {
+    BloomFilterEntry(const BloomFilter *in_bloom_filter,
+                     const attribute_id &in_attribute_id,
+                     const std::size_t &in_attribute_size)
+        : bloom_filter(in_bloom_filter),
+          attribute_id(in_attribute_id),
+          attribute_size(in_attribute_size),
+          miss(0),
+          cnt(0) {
+    }
+
+    static bool isBetterThan(const BloomFilterEntry *a,
+                             const BloomFilterEntry *b) {
+      return a->miss_rate > b->miss_rate;
+    }
+
+    const BloomFilter *bloom_filter;
+    const attribute_id attribute_id;
+    const std::size_t attribute_size;
+    std::uint32_t miss;
+    std::uint32_t cnt;
+    float miss_rate;
+  };
+
+  template <bool adapt_filters, typename ValueAccessorT>
+  inline std::size_t bulkProbeBloomFilterEntry(
+      BloomFilterEntry &entry,
+      const ValueAccessorT *accessor,
+      std::vector<tuple_id> &batch,
+      const std::size_t in_size) {
+    std::size_t out_size = 0;
+    const BloomFilter *bloom_filter = entry.bloom_filter;
+
+    for (std::size_t t = 0; t < in_size; ++t) {
+      const tuple_id tid = batch[t];
+      const auto value = static_cast<const std::uint8_t*>(
+          accessor->getUntypedValueAtAbsolutePosition(entry.attribute_id, tid));
+      if (bloom_filter->contains(value, entry.attribute_size)) {
+        batch[out_size] = tid;
+        ++out_size;
+      }
+    }
+    if (adapt_filters) {
+      entry.cnt += in_size;
+      entry.miss += (in_size - out_size);
+    }
+    return out_size;
+  }
+
+  inline void adaptEntryOrder() {
+    for (auto &entry : bloom_filter_entries_) {
+      entry->miss_rate = static_cast<float>(entry->miss) / entry->cnt;
+    }
+    std::sort(bloom_filter_entries_.begin(),
+              bloom_filter_entries_.end(),
+              BloomFilterEntry::isBetterThan);
+  }
+
+  std::vector<BloomFilterEntry *> bloom_filter_entries_;
+
+  DISALLOW_COPY_AND_ASSIGN(BloomFilterAdapter);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_BLOOM_FILTER_ADAPTER_HPP

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index ae1179d..46389f0 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -161,6 +161,7 @@ add_library(quickstep_utility_Alignment ../empty_src.cpp Alignment.hpp)
 add_library(quickstep_utility_BitManipulation ../empty_src.cpp BitManipulation.hpp)
 add_library(quickstep_utility_BitVector ../empty_src.cpp BitVector.hpp)
 add_library(quickstep_utility_BloomFilter ../empty_src.cpp BloomFilter.hpp)
+add_library(quickstep_utility_BloomFilterAdapter ../empty_src.cpp BloomFilterAdapter.hpp)
 add_library(quickstep_utility_BloomFilter_proto
             ${quickstep_utility_BloomFilter_proto_srcs}
             ${quickstep_utility_BloomFilter_proto_hdrs})
@@ -168,6 +169,8 @@ 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_DAG ../empty_src.cpp DAG.hpp)
+add_library(quickstep_utility_DisjointTreeForest ../empty_src.cpp DisjointTreeForest.hpp)
+add_library(quickstep_utility_EventProfiler EventProfiler.cpp EventProfiler.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)
 add_library(quickstep_utility_ExecutionDAGVisualizer
             ExecutionDAGVisualizer.cpp
@@ -221,6 +224,10 @@ target_link_libraries(quickstep_utility_BloomFilter
                       quickstep_threading_SpinSharedMutex
                       quickstep_utility_BloomFilter_proto
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_BloomFilterAdapter
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_utility_BloomFilter
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_BloomFilter_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_utility_CalculateInstalledMemory
@@ -230,6 +237,9 @@ target_link_libraries(quickstep_utility_CheckSnprintf
 target_link_libraries(quickstep_utility_DAG
                       glog
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_DisjointTreeForest)
+target_link_libraries(quickstep_utility_EventProfiler
+                      quickstep_threading_Mutex)
 target_link_libraries(quickstep_utility_ExecutionDAGVisualizer
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_queryexecution_QueryExecutionTypedefs
@@ -312,11 +322,14 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_BitManipulation
                       quickstep_utility_BitVector
                       quickstep_utility_BloomFilter
+                      quickstep_utility_BloomFilterAdapter
                       quickstep_utility_BloomFilter_proto
                       quickstep_utility_CalculateInstalledMemory
                       quickstep_utility_Cast
                       quickstep_utility_CheckSnprintf
                       quickstep_utility_DAG
+                      quickstep_utility_DisjointTreeForest
+                      quickstep_utility_EventProfiler
                       quickstep_utility_EqualsAnyConstant
                       quickstep_utility_ExecutionDAGVisualizer
                       quickstep_utility_Glob

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/DisjointTreeForest.hpp
----------------------------------------------------------------------
diff --git a/utility/DisjointTreeForest.hpp b/utility/DisjointTreeForest.hpp
new file mode 100644
index 0000000..f5722ba
--- /dev/null
+++ b/utility/DisjointTreeForest.hpp
@@ -0,0 +1,116 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_DISJOINT_TREE_FOREST_HPP_
+#define QUICKSTEP_UTILITY_DISJOINT_TREE_FOREST_HPP_
+
+#include <cstddef>
+#include <limits>
+#include <utility>
+#include <unordered_map>
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief A.k.a. union-find set.
+ */
+template <typename ElementT,
+          class MapperT = std::unordered_map<ElementT, std::size_t>>
+class DisjointTreeForest {
+ public:
+  inline bool hasElement(const ElementT &element) const {
+    return elements_map_.find(element) != elements_map_.end();
+  }
+
+  inline void makeSet(const ElementT &element) {
+    if (!hasElement(element)) {
+      std::size_t loc = nodes_.size();
+      nodes_.emplace_back(0, loc);
+      elements_map_.emplace(element, loc);
+    }
+  }
+
+  inline std::size_t find(const ElementT &element) {
+    const std::size_t node_id = elements_map_.at(element);
+    std::size_t root_id = node_id;
+    std::size_t parent_id;
+    while ((parent_id = nodes_[root_id].parent) != root_id) {
+      root_id = parent_id;
+    }
+    compress_path(node_id, root_id);
+    return root_id;
+  }
+
+  inline void merge(const ElementT &element1, const ElementT &element2) {
+    std::size_t root_id1 = find(element1);
+    std::size_t root_id2 = find(element2);
+    if (root_id1 != root_id2) {
+      Node &n1 = nodes_[root_id1];
+      Node &n2 = nodes_[root_id2];
+      if (n1.rank > n2.rank) {
+        n2.parent = root_id1;
+      } else if (n1.rank < n2.rank) {
+        n1.parent = root_id2;
+      } else {
+        n1.parent = root_id2;
+        n2.rank += 1;
+      }
+    }
+  }
+
+  inline bool isConnected(const ElementT &element1, const ElementT &element2) {
+    return find(element1) == find(element2);
+  }
+
+ private:
+  struct Node {
+    Node(const std::size_t rank_in, const std::size_t parent_in)
+        : rank(rank_in), parent(parent_in) {
+    }
+    std::size_t rank;
+    std::size_t parent;
+  };
+
+  inline void compress_path(const std::size_t leaf_node_id, const std::size_t root_node_id) {
+    std::size_t node_id = leaf_node_id;
+    std::size_t max_rank = 0;
+    while (node_id != root_node_id) {
+      const Node &node = nodes_[node_id];
+      max_rank = std::max(max_rank, node.rank);
+
+      const std::size_t parent_id = node.parent;
+      nodes_[node_id].parent = root_node_id;
+      node_id = parent_id;
+    }
+    nodes_[root_node_id].rank = max_rank + 1;
+  }
+
+  std::vector<Node> nodes_;
+  MapperT elements_map_;
+
+  static constexpr std::size_t kInvalid = std::numeric_limits<std::size_t>::max();
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_DISJOINT_TREE_FOREST_HPP_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/EventProfiler.cpp
----------------------------------------------------------------------
diff --git a/utility/EventProfiler.cpp b/utility/EventProfiler.cpp
new file mode 100644
index 0000000..728ebff
--- /dev/null
+++ b/utility/EventProfiler.cpp
@@ -0,0 +1,29 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "utility/EventProfiler.hpp"
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+namespace quickstep {
+
+EventProfiler<int, std::size_t> simple_profiler;
+EventProfiler<std::size_t> relop_profiler;
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/EventProfiler.hpp
----------------------------------------------------------------------
diff --git a/utility/EventProfiler.hpp b/utility/EventProfiler.hpp
new file mode 100644
index 0000000..70024e6
--- /dev/null
+++ b/utility/EventProfiler.hpp
@@ -0,0 +1,188 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_EVENT_PROFILER_HPP_
+#define QUICKSTEP_UTILITY_EVENT_PROFILER_HPP_
+
+#include <chrono>
+#include <cstddef>
+#include <cstring>
+#include <ctime>
+#include <iomanip>
+#include <map>
+#include <ostream>
+#include <thread>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include "threading/Mutex.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+using clock = std::chrono::steady_clock;
+
+template <typename TagT, typename ...PayloadT>
+class EventProfiler {
+
+ public:
+  EventProfiler()
+      : zero_time_(clock::now()) {
+  }
+
+  struct EventInfo {
+    clock::time_point start_time;
+    clock::time_point end_time;
+    bool is_finished;
+    std::tuple<PayloadT...> payload;
+
+    explicit EventInfo(const clock::time_point &start_time_in)
+        : start_time(start_time_in),
+          is_finished(false) {
+    }
+
+    EventInfo()
+        : start_time(clock::now()),
+          is_finished(false) {
+    }
+
+    inline void setPayload(PayloadT &&...in_payload) {
+      payload = std::make_tuple(in_payload...);
+    }
+
+    inline void endEvent() {
+      end_time = clock::now();
+      is_finished = true;
+    }
+  };
+
+  struct EventContainer {
+    EventContainer()
+        : context(0) {}
+
+    inline void startEvent(const TagT &tag) {
+      events[tag].emplace_back(clock::now());
+    }
+
+    inline void endEvent(const TagT &tag) {
+      auto &event_info = events.at(tag).back();
+      event_info.is_finished = true;
+      event_info.end_time = clock::now();
+    }
+
+    inline std::vector<EventInfo> *getEventLine(const TagT &tag) {
+      return &events[tag];
+    }
+
+    inline void setContext(int context_in) {
+      context = context_in;
+    }
+
+    inline int getContext() const {
+      return context;
+    }
+
+    std::map<TagT, std::vector<EventInfo>> events;
+    int context;
+  };
+
+  EventContainer *getContainer() {
+    MutexLock lock(mutex_);
+    return &thread_map_[std::this_thread::get_id()];
+  }
+
+  void writeToStream(std::ostream &os) const {
+    time_t rawtime;
+    time(&rawtime);
+    char event_id[32];
+    strftime(event_id, sizeof event_id, "%Y-%m-%d %H:%M:%S", localtime(&rawtime));
+
+    int thread_id = 0;
+    for (const auto &thread_ctx : thread_map_) {
+      for (const auto &event_group : thread_ctx.second.events) {
+        for (const auto &event_info : event_group.second) {
+          CHECK(event_info.is_finished) << "Unfinished profiling event";
+
+          os << std::setprecision(12)
+             << event_id << ","
+             << thread_id << "," << event_group.first << ",";
+
+          PrintTuple(os, event_info.payload, ",");
+
+          os << std::chrono::duration<double>(event_info.start_time - zero_time_).count()
+             << ","
+             << std::chrono::duration<double>(event_info.end_time - zero_time_).count()
+             << "\n";
+        }
+      }
+      ++thread_id;
+    }
+  }
+
+  void clear() {
+    zero_time_ = clock::now();
+    thread_map_.clear();
+  }
+
+  const std::map<std::thread::id, EventContainer> &containers() {
+    return thread_map_;
+  }
+
+  const clock::time_point &zero_time() {
+    return zero_time_;
+  }
+
+ private:
+  template<class Tuple, std::size_t N>
+  struct TuplePrinter {
+    static void Print(std::ostream &os, const Tuple &t, const std::string &sep) {
+      TuplePrinter<Tuple, N-1>::Print(os, t, sep);
+      os << std::get<N-1>(t) << sep;
+    }
+  };
+
+  template<class Tuple>
+  struct TuplePrinter<Tuple, 1> {
+    static void Print(std::ostream &os, const Tuple &t, const std::string &sep) {
+      os << std::get<0>(t) << sep;
+    }
+  };
+
+  template<class... Args>
+  static void PrintTuple(std::ostream &os, const std::tuple<Args...>& t, const std::string &sep) {
+    TuplePrinter<decltype(t), sizeof...(Args)>::Print(os, t, sep);
+  }
+
+  clock::time_point zero_time_;
+  std::map<std::thread::id, EventContainer> thread_map_;
+  Mutex mutex_;
+};
+
+extern EventProfiler<int, std::size_t> simple_profiler;
+extern EventProfiler<std::size_t> relop_profiler;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_EVENT_PROFILER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index 50cf7f0..b90a8dc 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -21,6 +21,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <set>
 #include <sstream>
 #include <string>
 #include <unordered_map>
@@ -30,6 +31,7 @@
 
 #include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
@@ -103,6 +105,10 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
   int node_id = ++id_counter_;
   node_id_map_.emplace(input, node_id);
 
+  std::set<E::ExprId> referenced_ids;
+  for (const auto &attr : input->getReferencedAttributes()) {
+    referenced_ids.emplace(attr->id());
+  }
   for (const auto &child : input->children()) {
     visit(child);
 
@@ -113,10 +119,8 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
     edge_info.src_node_id = child_id;
     edge_info.dst_node_id = node_id;
 
-    // Print output attributes except for TableReference -- there are just too many
-    // attributes out of TableReference.
-    if (child->getPhysicalType() != P::PhysicalType::kTableReference) {
-      for (const auto &attr : child->getOutputAttributes()) {
+    for (const auto &attr : child->getOutputAttributes()) {
+      if (referenced_ids.find(attr->id()) != referenced_ids.end()) {
         edge_info.labels.emplace_back(attr->attribute_alias());
       }
     }
@@ -147,6 +151,36 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
         node_info.labels.emplace_back(
             left_attributes[i]->attribute_alias() + " = " + right_attributes[i]->attribute_alias());
       }
+      if (hash_join->left()->impliesUniqueAttributes(left_attributes)) {
+        node_info.labels.emplace_back("LEFT join attrs unique");
+      }
+      if (hash_join->right()->impliesUniqueAttributes(right_attributes)) {
+        node_info.labels.emplace_back("RIGHT join attrs unique");
+      }
+
+      const auto &bf_config = hash_join->bloom_filter_config();
+      for (const auto &bf : bf_config.build_side_bloom_filters) {
+        node_info.labels.emplace_back(
+            std::string("[BF build] ") + bf.attribute->attribute_alias());
+      }
+      for (const auto &bf : bf_config.probe_side_bloom_filters) {
+        node_info.labels.emplace_back(
+            std::string("[BF probe] ") + bf.attribute->attribute_alias());
+      }
+
+      break;
+    }
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr aggregate =
+        std::static_pointer_cast<const P::Aggregate>(input);
+      node_info.labels.emplace_back(input->getName());
+
+      const auto &bf_config = aggregate->bloom_filter_config();
+      for (const auto &bf : bf_config.probe_side_bloom_filters) {
+        node_info.labels.emplace_back(
+            std::string("[BF probe] ") + bf.attribute->attribute_alias());
+      }
+
       break;
     }
     default: {


[15/16] incubator-quickstep git commit: Attach bloom filters to select

Posted by ji...@apache.org.
Attach bloom filters to select


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

Branch: refs/heads/LIP-for-tpch
Commit: ef4cd1ed9b0a48fdde7a753dbb309b213a7b2701
Parents: c588775
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Aug 4 18:05:13 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500

----------------------------------------------------------------------
 expressions/Expressions.proto                   |   9 +-
 expressions/predicate/BloomFiltersPredicate.cpp |  68 +++++++++++
 expressions/predicate/BloomFiltersPredicate.hpp |  94 +++++++++++++++
 query_optimizer/ExecutionGenerator.cpp          |  15 +++
 query_optimizer/ExecutionHeuristics.cpp         |  24 ++++
 query_optimizer/ExecutionHeuristics.hpp         |  28 +++++
 query_optimizer/physical/Selection.hpp          |  16 ++-
 query_optimizer/rules/AttachBloomFilters.cpp    | 117 +++++++++++++++---
 query_optimizer/rules/AttachBloomFilters.hpp    |   6 +-
 .../StarSchemaHashJoinOrderOptimization.cpp     |  34 +++---
 .../StarSchemaHashJoinOrderOptimization.hpp     |  14 ++-
 relational_operators/SelectOperator.cpp         |  57 ++++++++-
 relational_operators/SelectOperator.hpp         |  25 ++++
 storage/AggregationOperationState.cpp           |   2 +
 storage/StorageBlock.cpp                        | 119 ++++++++++++++++++-
 storage/StorageBlock.hpp                        |   8 +-
 utility/PlanVisualizer.cpp                      |  22 +++-
 utility/PlanVisualizer.hpp                      |   1 +
 18 files changed, 604 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/expressions/Expressions.proto
----------------------------------------------------------------------
diff --git a/expressions/Expressions.proto b/expressions/Expressions.proto
index 8d923c5..34c49e0 100644
--- a/expressions/Expressions.proto
+++ b/expressions/Expressions.proto
@@ -30,10 +30,11 @@ message Predicate {
   enum PredicateType {
     TRUE = 0;
     FALSE = 1;
-    COMPARISON = 2;
-    NEGATION = 3;
-    CONJUNCTION = 4;
-    DISJUNCTION = 5;
+    BLOOM_FILTERS = 2;
+    COMPARISON = 3;
+    NEGATION = 4;
+    CONJUNCTION = 5;
+    DISJUNCTION = 6;
   }
 
   required PredicateType predicate_type = 1;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/expressions/predicate/BloomFiltersPredicate.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/BloomFiltersPredicate.cpp b/expressions/predicate/BloomFiltersPredicate.cpp
new file mode 100644
index 0000000..17ff796
--- /dev/null
+++ b/expressions/predicate/BloomFiltersPredicate.cpp
@@ -0,0 +1,68 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015 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.
+ **/
+
+#include "expressions/predicate/BloomFiltersPredicate.hpp"
+
+#include "expressions/Expressions.pb.h"
+#include "expressions/predicate/Predicate.hpp"
+#include "storage/TupleIdSequence.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class ValueAccessor;
+
+serialization::Predicate BloomFiltersPredicate::getProto() const {
+  serialization::Predicate proto;
+  proto.set_predicate_type(serialization::Predicate::BLOOM_FILTERS);
+  return proto;
+}
+
+Predicate* BloomFiltersPredicate::clone() const {
+  LOG(FATAL) << "Not implemented\n";
+  return nullptr;
+}
+
+bool BloomFiltersPredicate::matchesForSingleTuple(const ValueAccessor &accessor,
+                                                  const tuple_id tuple) const {
+  LOG(FATAL) << "Not implemented\n";
+  return false;
+}
+
+bool BloomFiltersPredicate::matchesForJoinedTuples(
+    const ValueAccessor &left_accessor,
+    const relation_id left_relation_id,
+    const tuple_id left_tuple_id,
+    const ValueAccessor &right_accessor,
+    const relation_id right_relation_id,
+    const tuple_id right_tuple_id) const {
+  LOG(FATAL) << "Not implemented\n";
+  return false;
+}
+
+TupleIdSequence* BloomFiltersPredicate::getAllMatches(
+    ValueAccessor *accessor,
+    const SubBlocksReference *sub_blocks_ref,
+    const TupleIdSequence *filter,
+    const TupleIdSequence *existence_map) const {
+  LOG(FATAL) << "Not implemented\n";
+  return nullptr;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/expressions/predicate/BloomFiltersPredicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/BloomFiltersPredicate.hpp b/expressions/predicate/BloomFiltersPredicate.hpp
new file mode 100644
index 0000000..3c3acf4
--- /dev/null
+++ b/expressions/predicate/BloomFiltersPredicate.hpp
@@ -0,0 +1,94 @@
+/**
+ *   Copyright 2011-2015 Quickstep Technologies LLC.
+ *   Copyright 2015 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_EXPRESSIONS_PREDICATE_BLOOM_FILTERS_PREDICATE_HPP_
+#define QUICKSTEP_EXPRESSIONS_PREDICATE_BLOOM_FILTERS_PREDICATE_HPP_
+
+#include <memory>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/Expressions.pb.h"
+#include "expressions/predicate/Predicate.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "utility/BloomFilter.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class TupleIdSequence;
+class ValueAccessor;
+
+struct SubBlocksReference;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+class BloomFiltersPredicate : public Predicate {
+ public:
+  BloomFiltersPredicate() {
+  }
+
+  ~BloomFiltersPredicate() override {
+  }
+
+  serialization::Predicate getProto() const override;
+
+  Predicate* clone() const override;
+
+  PredicateType getPredicateType() const override {
+    return kBloomFilters;
+  }
+
+  bool matchesForSingleTuple(const ValueAccessor &accessor,
+                             const tuple_id tuple) const override;
+
+  bool matchesForJoinedTuples(
+      const ValueAccessor &left_accessor,
+      const relation_id left_relation_id,
+      const tuple_id left_tuple_id,
+      const ValueAccessor &right_accessor,
+      const relation_id right_relation_id,
+      const tuple_id right_tuple_id) const override;
+
+  TupleIdSequence* getAllMatches(ValueAccessor *accessor,
+                                 const SubBlocksReference *sub_blocks_ref,
+                                 const TupleIdSequence *filter,
+                                 const TupleIdSequence *existence_map) const override;
+
+  void addBloomFilter(const BloomFilter *bloom_filter) {
+    bloom_filters_.emplace_back(bloom_filter);
+  }
+
+  void addAttributeId(const attribute_id probe_attribute_id) {
+    bloom_filter_attribute_ids_.push_back(probe_attribute_id);
+  }
+
+ private:
+  std::vector<const BloomFilter *> bloom_filters_;
+  std::vector<attribute_id> bloom_filter_attribute_ids_;
+
+  friend class PredicateTest;
+
+  DISALLOW_COPY_AND_ASSIGN(BloomFiltersPredicate);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_PREDICATE_BLOOM_FILTERS_PREDICATE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index a194d46..21c6e30 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -507,6 +507,16 @@ void ExecutionGenerator::convertSelection(
     }
   }
 
+  const P::BloomFilterConfig &bloom_filter_config =
+      physical_selection->bloom_filter_config();
+  std::vector<attribute_id> bloom_filter_attribute_ids;
+
+  for (const auto &bf : bloom_filter_config.probe_side_bloom_filters) {
+    const CatalogAttribute *bf_catalog_attribute
+        = attribute_substitution_map_[bf.attribute->id()];
+    bloom_filter_attribute_ids.emplace_back(bf_catalog_attribute->getID());
+  }
+
   // Convert the project expressions proto.
   const QueryContext::scalar_group_id project_expressions_group_index =
       query_context_proto_->scalar_groups_size();
@@ -572,6 +582,11 @@ void ExecutionGenerator::convertSelection(
       std::forward_as_tuple(select_index,
                             output_relation));
   temporary_relation_info_vec_.emplace_back(select_index, output_relation);
+
+  execution_heuristics_->addSelectInfo(select_index,
+                                       bloom_filter_config,
+                                       std::move(bloom_filter_attribute_ids),
+                                       op);
 }
 
 void ExecutionGenerator::convertSharedSubplanReference(const physical::SharedSubplanReferencePtr &physical_plan) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/ExecutionHeuristics.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.cpp b/query_optimizer/ExecutionHeuristics.cpp
index 0bef716..d5d7640 100644
--- a/query_optimizer/ExecutionHeuristics.cpp
+++ b/query_optimizer/ExecutionHeuristics.cpp
@@ -127,6 +127,30 @@ void ExecutionHeuristics::optimizeExecutionPlan(QueryPlan *query_plan,
                                       true /* is_pipeline_breaker */);
     }
   }
+
+  for (const auto &info : selects_) {
+    const auto &bloom_filter_config = info.bloom_filter_config_;
+
+    for (std::size_t i = 0; i < info.bloom_filter_ids_.size(); ++i) {
+      const auto &bf =
+          bloom_filter_config.probe_side_bloom_filters[i];
+      std::cerr << "Select probe " << bf.attribute->toString()
+                << " @" << bf.builder << "\n";
+
+      const auto &build_side_info =
+           bloom_filter_map.at(
+               std::make_pair(bf.source_attribute->id(),
+                              bf.builder));
+      info.select_operator_->addBloomFilter(
+          build_side_info.first, info.bloom_filter_ids_[i]);
+//      std::cerr << "Select probe attr_id = "
+//                << info.bloom_filter_ids_[i] << "\n";
+
+      query_plan->addDirectDependency(info.select_operator_index_,
+                                      build_side_info.second,
+                                      true /* is_pipeline_breaker */);
+    }
+  }
 }
 
 void ExecutionHeuristics::setBloomFilterProperties(serialization::BloomFilter *bloom_filter_proto,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/ExecutionHeuristics.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.hpp b/query_optimizer/ExecutionHeuristics.hpp
index 9e5efc5..9b021a3 100644
--- a/query_optimizer/ExecutionHeuristics.hpp
+++ b/query_optimizer/ExecutionHeuristics.hpp
@@ -28,6 +28,7 @@
 #include "query_execution/QueryContext.pb.h"
 #include "query_optimizer/QueryPlan.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
+#include "relational_operators/SelectOperator.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -112,6 +113,22 @@ class ExecutionHeuristics {
     const QueryContext::aggregation_state_id aggregate_state_id_;
   };
 
+  struct SelectInfo {
+    SelectInfo(const QueryPlan::DAGNodeIndex select_operator_index,
+               const physical::BloomFilterConfig &bloom_filter_config,
+               std::vector<attribute_id> &&bloom_filter_ids,
+               SelectOperator *select_operator)
+        : select_operator_index_(select_operator_index),
+          bloom_filter_config_(bloom_filter_config),
+          bloom_filter_ids_(bloom_filter_ids),
+          select_operator_(select_operator) {
+    }
+
+    const QueryPlan::DAGNodeIndex select_operator_index_;
+    const physical::BloomFilterConfig &bloom_filter_config_;
+    const std::vector<attribute_id> bloom_filter_ids_;
+    SelectOperator *select_operator_;
+  };
 
   /**
    * @brief Constructor.
@@ -161,6 +178,16 @@ class ExecutionHeuristics {
                              aggregate_state_id);
   }
 
+  inline void addSelectInfo(const QueryPlan::DAGNodeIndex select_operator_index,
+                            const physical::BloomFilterConfig &bloom_filter_config,
+                            std::vector<attribute_id> &&bloom_filter_ids,
+                            SelectOperator *select_operator) {
+    selects_.emplace_back(select_operator_index,
+                          bloom_filter_config,
+                          std::move(bloom_filter_ids),
+                          select_operator);
+  }
+
   /**
    * @brief Optimize the execution plan based on heuristics generated
    *        during physical plan to execution plan conversion.
@@ -184,6 +211,7 @@ class ExecutionHeuristics {
  private:
   std::vector<HashJoinInfo> hash_joins_;
   std::vector<AggregateInfo> aggregates_;
+  std::vector<SelectInfo> selects_;
 
   DISALLOW_COPY_AND_ASSIGN(ExecutionHeuristics);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/physical/Selection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.hpp b/query_optimizer/physical/Selection.hpp
index f42fc71..bb50314 100644
--- a/query_optimizer/physical/Selection.hpp
+++ b/query_optimizer/physical/Selection.hpp
@@ -89,6 +89,10 @@ class Selection : public Physical {
   bool impliesUniqueAttributes(
       const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
 
+  const BloomFilterConfig &bloom_filter_config() const {
+    return bloom_filter_config_;
+  }
+
   /**
    * @brief Creates a Selection.
    *
@@ -100,9 +104,10 @@ class Selection : public Physical {
   static SelectionPtr Create(
       const PhysicalPtr &input,
       const std::vector<expressions::NamedExpressionPtr> &project_expressions,
-      const expressions::PredicatePtr &filter_predicate) {
+      const expressions::PredicatePtr &filter_predicate,
+      const BloomFilterConfig bloom_filter_config = BloomFilterConfig()) {
     return SelectionPtr(
-        new Selection(input, project_expressions, filter_predicate));
+        new Selection(input, project_expressions, filter_predicate, bloom_filter_config));
   }
 
   /**
@@ -143,15 +148,18 @@ class Selection : public Physical {
   Selection(
       const PhysicalPtr &input,
       const std::vector<expressions::NamedExpressionPtr> &project_expressions,
-      const expressions::PredicatePtr &filter_predicate)
+      const expressions::PredicatePtr &filter_predicate,
+      const BloomFilterConfig &bloom_filter_config)
       : project_expressions_(project_expressions),
-        filter_predicate_(filter_predicate) {
+        filter_predicate_(filter_predicate),
+        bloom_filter_config_(bloom_filter_config) {
     addChild(input);
   }
 
   std::vector<expressions::NamedExpressionPtr> project_expressions_;
   // Can be NULL. If NULL, the filter predicate is treated as the literal true.
   expressions::PredicatePtr filter_predicate_;
+  BloomFilterConfig bloom_filter_config_;
 
   DISALLOW_COPY_AND_ASSIGN(Selection);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/rules/AttachBloomFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.cpp b/query_optimizer/rules/AttachBloomFilters.cpp
index 03a42a0..10ed512 100644
--- a/query_optimizer/rules/AttachBloomFilters.cpp
+++ b/query_optimizer/rules/AttachBloomFilters.cpp
@@ -17,6 +17,8 @@
 
 #include "query_optimizer/rules/AttachBloomFilters.hpp"
 
+#include <algorithm>
+#include <iterator>
 #include <memory>
 #include <set>
 #include <unordered_set>
@@ -34,8 +36,6 @@
 #include "query_optimizer/physical/PhysicalType.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
 
-#include "glog/logging.h"
-
 namespace quickstep {
 namespace optimizer {
 
@@ -70,7 +70,10 @@ P::PhysicalPtr AttachBloomFilters::apply(const P::PhysicalPtr &input) {
 //    std::cerr << "********\n";
 //  }
 
-  return visitAndAttach(input);
+  std::set<E::ExprId> used_bloom_filters;
+  decideAttach(input, &used_bloom_filters);
+
+  return performAttach(input);
 }
 
 void AttachBloomFilters::visitProducer(const P::PhysicalPtr &node, const int depth) {
@@ -140,6 +143,52 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
   // Bloom filters from parent
   const auto &parent_bloom_filters = consumers_[node];
   if (!parent_bloom_filters.empty()) {
+//    if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+//      const P::HashJoinPtr hash_join =
+//          std::static_pointer_cast<const P::HashJoin>(node);
+//      const std::vector<const std::vector<E::AttributeReferencePtr>*> join_attributes =
+//          { &hash_join->left_join_attributes(), &hash_join->right_join_attributes() };
+//
+//      for (std::size_t i = 0; i < 2; ++i) {
+//        const auto child = hash_join->children()[i];
+//        std::unordered_set<E::ExprId> child_output_attribute_ids;
+//        for (const auto &attr : child->getOutputAttributes()) {
+//          child_output_attribute_ids.emplace(attr->id());
+//        }
+//
+//        std::unordered_map<E::ExprId, E::AttributeReferencePtr> join_attribute_map;
+//        for (std::size_t k = 0; k < hash_join->left_join_attributes().size(); ++k) {
+//          join_attribute_map.emplace(
+//              join_attributes[1-i]->at(k)->id(),
+//              join_attributes[i]->at(k));
+//        }
+//
+//        std::vector<BloomFilterInfo> bloom_filters;
+//        for (const auto &info : parent_bloom_filters) {
+//          if (child_output_attribute_ids.find(info.attribute->id())
+//                  != child_output_attribute_ids.end()) {
+//            bloom_filters.emplace_back(info.source,
+//                                       info.attribute,
+//                                       info.depth,
+//                                       info.selectivity,
+//                                       false,
+//                                       info.source_attribute);
+//          } else {
+//            auto attr_it = join_attribute_map.find(info.attribute->id());
+//            if (attr_it != join_attribute_map.end()) {
+//              bloom_filters.emplace_back(info.source,
+//                                         attr_it->second,
+//                                         info.depth,
+//                                         info.selectivity,
+//                                         false,
+//                                         info.source_attribute);
+//
+//            }
+//          }
+//        }
+//        consumers_.emplace(child, std::move(bloom_filters));
+//      }
+//    }
     for (const auto &child : node->children()) {
       std::unordered_set<E::ExprId> child_output_attribute_ids;
       for (const auto &attr : child->getOutputAttributes()) {
@@ -195,6 +244,21 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
     }
   }
 
+  for (const auto &child : node->children()) {
+    visitConsumer(child);
+  }
+}
+
+void AttachBloomFilters::decideAttach(
+    const P::PhysicalPtr &node,
+    std::set<E::ExprId> *used_bloom_filters) {
+  for (const auto &child : node->children()) {
+    std::set<E::ExprId> child_bloom_filters;
+    decideAttach(child, &child_bloom_filters);
+    used_bloom_filters->insert(child_bloom_filters.begin(),
+                               child_bloom_filters.end());
+  }
+
   P::PhysicalPtr consumer_child = nullptr;
   if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
     consumer_child = std::static_pointer_cast<const P::HashJoin>(node)->left();
@@ -202,6 +266,9 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
   if (node->getPhysicalType() == P::PhysicalType::kAggregate) {
     consumer_child = std::static_pointer_cast<const P::Aggregate>(node)->input();
   }
+  if (node->getPhysicalType() == P::PhysicalType::kSelection) {
+    consumer_child = std::static_pointer_cast<const P::Selection>(node)->input();
+  }
 
   if (consumer_child != nullptr) {
     // Decide attaches
@@ -222,27 +289,27 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
 
       auto &probe_attaches = getBloomFilterConfig(node);
       for (const auto &pair : filters) {
-        auto &build_attaches = getBloomFilterConfig(pair.second->source);
-        build_attaches.addBuildSideBloomFilter(
-            pair.second->source_attribute);
-        probe_attaches.addProbeSideBloomFilter(
-            pair.first,
-            pair.second->source_attribute,
-            pair.second->source);
+        const E::ExprId source_attr_id = pair.second->source_attribute->id();
+        if (used_bloom_filters->find(source_attr_id) == used_bloom_filters->end()) {
+          auto &build_attaches = getBloomFilterConfig(pair.second->source);
+          build_attaches.addBuildSideBloomFilter(
+              pair.second->source_attribute);
+          probe_attaches.addProbeSideBloomFilter(
+              pair.first,
+              pair.second->source_attribute,
+              pair.second->source);
+          used_bloom_filters->emplace(source_attr_id);
+        }
       }
     }
   }
-
-  for (const auto &child : node->children()) {
-    visitConsumer(child);
-  }
 }
 
-P::PhysicalPtr AttachBloomFilters::visitAndAttach(const physical::PhysicalPtr &node) {
+P::PhysicalPtr AttachBloomFilters::performAttach(const physical::PhysicalPtr &node) {
   std::vector<P::PhysicalPtr> new_children;
   bool has_changed = false;
   for (const auto &child : node->children()) {
-    P::PhysicalPtr new_child = visitAndAttach(child);
+    P::PhysicalPtr new_child = performAttach(child);
     if (new_child != child) {
       has_changed = true;
     }
@@ -290,6 +357,24 @@ P::PhysicalPtr AttachBloomFilters::visitAndAttach(const physical::PhysicalPtr &n
     }
   }
 
+  if (node->getPhysicalType() == P::PhysicalType::kSelection) {
+    const auto attach_it = attaches_.find(node);
+    if (attach_it != attaches_.end()) {
+//      for (const auto& item : attach_it->second.probe_side_bloom_filters) {
+//        std::cout << "Attach probe from " << item.builder
+//                  << " to " << node << "\n";
+//      }
+
+      const P::SelectionPtr selection =
+          std::static_pointer_cast<const P::Selection>(node);
+      return P::Selection::Create(
+          selection->input(),
+          selection->project_expressions(),
+          selection->filter_predicate(),
+          attach_it->second);
+    }
+  }
+
   if (has_changed) {
     return node->copyWithNewChildren(new_children);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/rules/AttachBloomFilters.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.hpp b/query_optimizer/rules/AttachBloomFilters.hpp
index e4437f7..5131bdd 100644
--- a/query_optimizer/rules/AttachBloomFilters.hpp
+++ b/query_optimizer/rules/AttachBloomFilters.hpp
@@ -21,6 +21,7 @@
 #include <algorithm>
 #include <cstddef>
 #include <memory>
+#include <set>
 #include <string>
 #include <unordered_map>
 #include <unordered_set>
@@ -97,7 +98,10 @@ class AttachBloomFilters : public Rule<physical::Physical> {
 
   void visitConsumer(const physical::PhysicalPtr &node);
 
-  physical::PhysicalPtr visitAndAttach(const physical::PhysicalPtr &node);
+  void decideAttach(const physical::PhysicalPtr &node,
+                    std::set<expressions::ExprId> *used_bloom_filters);
+
+  physical::PhysicalPtr performAttach(const physical::PhysicalPtr &node);
 
   physical::BloomFilterConfig &getBloomFilterConfig(const physical::PhysicalPtr &node);
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index 15a7154..9e8d794 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -221,11 +221,14 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
                   attribute_id_to_reference_map.at(build_it->second));
             }
           }
-          if (!build_attrs.empty()
-              && build_table_info->table->impliesUniqueAttributes(build_attrs)) {
-            std::unique_ptr<JoinPair> new_join(
-                new JoinPair(probe_table_info, build_table_info));
-            if (best_join == nullptr || new_join->isBetterThan(*best_join)) {
+
+          const bool build_side_unique =
+              !build_attrs.empty() && build_table_info->table->impliesUniqueAttributes(build_attrs);
+          std::unique_ptr<JoinPair> new_join(
+              new JoinPair(probe_table_info,
+                           build_table_info,
+                           build_side_unique));
+          if (best_join == nullptr || new_join->isBetterThan(*best_join)) {
 //              if (best_join != nullptr) {
 //                std::cerr << "(" << best_join->probe->estimated_selectivity
 //                          << ", " << best_join->probe->estimated_cardinality << ")"
@@ -241,25 +244,24 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
 //                        << "(" << new_join->build->estimated_selectivity
 //                        << ", " << new_join->build->estimated_cardinality << ")"
 //                        << "\n****\n";
-              best_join.reset(new_join.release());
-            }
+            best_join.reset(new_join.release());
           }
         }
       }
     }
 
-    TableInfo *selected_probe_table_info = nullptr;
-    TableInfo *selected_build_table_info = nullptr;
+    CHECK(best_join != nullptr);
 
-    if (best_join != nullptr) {
-      selected_probe_table_info = best_join->probe;
-      selected_build_table_info = best_join->build;
+    TableInfo *selected_probe_table_info = best_join->probe;
+    TableInfo *selected_build_table_info = best_join->build;
+    std::cerr << "card: " << selected_probe_table_info->estimated_cardinality << "\n";
+    std::cerr << "card: " << selected_build_table_info->estimated_cardinality << "\n";
+    std::cerr << "--------\n";
+    if (!best_join->build_side_unique &&
+        selected_probe_table_info->estimated_cardinality < selected_build_table_info->estimated_cardinality) {
+      std::swap(selected_probe_table_info, selected_build_table_info);
     }
 
-    // TODO(jianqiao): Handle the case when there is no primary key-foreign key information available.
-    CHECK(selected_probe_table_info != nullptr);
-    CHECK(selected_build_table_info != nullptr);
-
 //    std::cerr << selected_probe_table_info->estimated_selectivity
 //              << " -- "
 //              << selected_build_table_info->estimated_selectivity

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
index eb21d03..0466765 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
@@ -96,12 +96,21 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
   };
 
   struct JoinPair {
-    JoinPair(TableInfo *probe_in, TableInfo *build_in)
-        : probe(probe_in), build(build_in) {
+    JoinPair(TableInfo *probe_in,
+             TableInfo *build_in,
+             const bool build_side_unique_in)
+        : probe(probe_in),
+          build(build_in),
+          build_side_unique(build_side_unique_in) {
     }
 
     inline bool isBetterThan(const JoinPair &rhs) const {
       const auto &lhs = *this;
+
+      if (lhs.build_side_unique != rhs.build_side_unique) {
+        return lhs.build_side_unique;
+      }
+
       const bool lhs_has_large_output =
           lhs.build->estimated_num_output_attributes
               + lhs.probe->estimated_num_output_attributes > 5;
@@ -151,6 +160,7 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
 
     TableInfo *probe;
     TableInfo *build;
+    const bool build_side_unique;
   };
 
   physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index d56326e..5a67e2d 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -42,6 +42,8 @@ class Predicate;
 void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
                                    StorageManager *storage_manager,
                                    const Predicate *predicate,
+                                   const std::vector<const BloomFilter *> &bloom_filters,
+                                   const std::vector<attribute_id> &bloom_filter_attribute_ids,
                                    const std::vector<std::unique_ptr<const Scalar>> *selection,
                                    InsertDestination *output_destination) {
   if (input_relation_is_stored_) {
@@ -50,6 +52,8 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
                                                         input_relation_,
                                                         input_block_id,
                                                         predicate,
+                                                        bloom_filters,
+                                                        bloom_filter_attribute_ids,
                                                         simple_projection_,
                                                         simple_selection_,
                                                         selection,
@@ -65,6 +69,8 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
               input_relation_,
               input_relation_block_ids_[num_workorders_generated_],
               predicate,
+              bloom_filters,
+              bloom_filter_attribute_ids,
               simple_projection_,
               simple_selection_,
               selection,
@@ -80,6 +86,8 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
 void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
                                                  StorageManager *storage_manager,
                                                  const Predicate *predicate,
+                                                 const std::vector<const BloomFilter *> &bloom_filters,
+                                                 const std::vector<attribute_id> &bloom_filter_attribute_ids,
                                                  const std::vector<std::unique_ptr<const Scalar>> *selection,
                                                  InsertDestination *output_destination) {
   DCHECK(placement_scheme_ != nullptr);
@@ -94,6 +102,8 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
                 input_relation_,
                 input_block_id,
                 predicate,
+                bloom_filters,
+                bloom_filter_attribute_ids,
                 simple_projection_,
                 simple_selection_,
                 selection,
@@ -115,6 +125,8 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
                 input_relation_,
                 block_in_partition,
                 predicate,
+                bloom_filters,
+                bloom_filter_attribute_ids,
                 simple_projection_,
                 simple_selection_,
                 selection,
@@ -137,6 +149,15 @@ bool SelectOperator::getAllWorkOrders(
     tmb::MessageBus *bus) {
   DCHECK(query_context != nullptr);
 
+  if (bloom_filters_ == nullptr) {
+    bloom_filters_.reset(new std::vector<const BloomFilter*>());
+    for (const auto bloom_filter_id : bloom_filter_ids_) {
+      // Add the pointer to the probe bloom filter within the list of probe bloom filters to use.
+      bloom_filters_->emplace_back(
+          query_context->getBloomFilter(bloom_filter_id));
+    }
+  }
+
   const Predicate *predicate =
       query_context->getPredicate(predicate_index_);
   const std::vector<std::unique_ptr<const Scalar>> *selection =
@@ -151,11 +172,23 @@ bool SelectOperator::getAllWorkOrders(
       if (input_relation_.hasPartitionScheme()) {
 #ifdef QUICKSTEP_HAVE_LIBNUMA
         if (input_relation_.hasNUMAPlacementScheme()) {
-          addPartitionAwareWorkOrders(container, storage_manager, predicate, selection, output_destination);
+          addPartitionAwareWorkOrders(container,
+                                      storage_manager,
+                                      predicate,
+                                      *bloom_filters_,
+                                      bloom_filter_attribute_ids_,
+                                      selection,
+                                      output_destination);
         }
 #endif
       } else {
-        addWorkOrders(container, storage_manager, predicate, selection, output_destination);
+        addWorkOrders(container,
+                      storage_manager,
+                      predicate,
+                      *bloom_filters_,
+                      bloom_filter_attribute_ids_,
+                      selection,
+                      output_destination);
       }
       started_ = true;
     }
@@ -164,11 +197,23 @@ bool SelectOperator::getAllWorkOrders(
     if (input_relation_.hasPartitionScheme()) {
 #ifdef QUICKSTEP_HAVE_LIBNUMA
         if (input_relation_.hasNUMAPlacementScheme()) {
-          addPartitionAwareWorkOrders(container, storage_manager, predicate, selection, output_destination);
+          addPartitionAwareWorkOrders(container,
+                                      storage_manager,
+                                      predicate,
+                                      *bloom_filters_,
+                                      bloom_filter_attribute_ids_,
+                                      selection,
+                                      output_destination);
         }
 #endif
     } else {
-        addWorkOrders(container, storage_manager, predicate, selection, output_destination);
+        addWorkOrders(container,
+                      storage_manager,
+                      predicate,
+                      *bloom_filters_,
+                      bloom_filter_attribute_ids_,
+                      selection,
+                      output_destination);
     }
     return done_feeding_input_relation_;
   }
@@ -222,10 +267,14 @@ void SelectWorkOrder::execute() {
   if (simple_projection_) {
     block->selectSimple(simple_selection_,
                         predicate_,
+                        bloom_filters_,
+                        bloom_filter_attribute_ids_,
                         output_destination_);
   } else {
     block->select(*DCHECK_NOTNULL(selection_),
                   predicate_,
+                  bloom_filters_,
+                  bloom_filter_attribute_ids_,
                   output_destination_);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 0f5c712..bc587a1 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -37,6 +37,7 @@
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.hpp"
 #include "storage/StorageBlockInfo.hpp"
+#include "utility/BloomFilterAdapter.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -47,6 +48,7 @@ namespace tmb { class MessageBus; }
 
 namespace quickstep {
 
+class BloomFitler;
 class CatalogRelationSchema;
 class InsertDestination;
 class Predicate;
@@ -246,15 +248,25 @@ class SelectOperator : public RelationalOperator {
     return output_relation_.getID();
   }
 
+  void addBloomFilter(const QueryContext::bloom_filter_id bloom_filter_id,
+                      const attribute_id bloom_filter_attribute_id) {
+    bloom_filter_ids_.emplace_back(bloom_filter_id);
+    bloom_filter_attribute_ids_.emplace_back(bloom_filter_attribute_id);
+  }
+
   void addWorkOrders(WorkOrdersContainer *container,
                      StorageManager *storage_manager,
                      const Predicate *predicate,
+                     const std::vector<const BloomFilter *> &bloom_filters,
+                     const std::vector<attribute_id> &bloom_filter_attribute_ids,
                      const std::vector<std::unique_ptr<const Scalar>> *selection,
                      InsertDestination *output_destination);
 
   void addPartitionAwareWorkOrders(WorkOrdersContainer *container,
                                    StorageManager *storage_manager,
                                    const Predicate *predicate,
+                                   const std::vector<const BloomFilter *> &bloom_filters,
+                                   const std::vector<attribute_id> &bloom_filter_attribute_ids,
                                    const std::vector<std::unique_ptr<const Scalar>> *selection,
                                    InsertDestination *output_destination);
 
@@ -270,6 +282,9 @@ class SelectOperator : public RelationalOperator {
   const CatalogRelation &output_relation_;
   const QueryContext::insert_destination_id output_destination_index_;
   const QueryContext::predicate_id predicate_index_;
+  std::vector<QueryContext::bloom_filter_id> bloom_filter_ids_;
+  std::vector<attribute_id> bloom_filter_attribute_ids_;
+  std::unique_ptr<std::vector<const BloomFilter*>> bloom_filters_;
 
   const QueryContext::scalar_group_id selection_index_;
   const std::vector<attribute_id> simple_selection_;
@@ -323,6 +338,8 @@ class SelectWorkOrder : public WorkOrder {
                   const CatalogRelationSchema &input_relation,
                   const block_id input_block_id,
                   const Predicate *predicate,
+                  const std::vector<const BloomFilter *> &bloom_filters,
+                  const std::vector<attribute_id> &bloom_filter_attribute_ids,
                   const bool simple_projection,
                   const std::vector<attribute_id> &simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
@@ -333,6 +350,8 @@ class SelectWorkOrder : public WorkOrder {
         input_relation_(input_relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
+        bloom_filters_(bloom_filters),
+        bloom_filter_attribute_ids_(bloom_filter_attribute_ids),
         simple_projection_(simple_projection),
         simple_selection_(simple_selection),
         selection_(selection),
@@ -365,6 +384,8 @@ class SelectWorkOrder : public WorkOrder {
                   const CatalogRelationSchema &input_relation,
                   const block_id input_block_id,
                   const Predicate *predicate,
+                  const std::vector<const BloomFilter *> &bloom_filters,
+                  const std::vector<attribute_id> &bloom_filter_attribute_ids,
                   const bool simple_projection,
                   std::vector<attribute_id> &&simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
@@ -375,6 +396,8 @@ class SelectWorkOrder : public WorkOrder {
         input_relation_(input_relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
+        bloom_filters_(bloom_filters),
+        bloom_filter_attribute_ids_(bloom_filter_attribute_ids),
         simple_projection_(simple_projection),
         simple_selection_(std::move(simple_selection)),
         selection_(selection),
@@ -399,6 +422,8 @@ class SelectWorkOrder : public WorkOrder {
   const CatalogRelationSchema &input_relation_;
   const block_id input_block_id_;
   const Predicate *predicate_;
+  const std::vector<const BloomFilter *> &bloom_filters_;
+  const std::vector<attribute_id> &bloom_filter_attribute_ids_;
 
   const bool simple_projection_;
   const std::vector<attribute_id> simple_selection_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index d225258..d85b5c4 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -54,6 +54,8 @@
 #include "types/containers/Tuple.hpp"
 #include "utility/BloomFilterAdapter.hpp"
 
+#include "gflags/gflags.h"
+
 #include "glog/logging.h"
 
 using std::unique_ptr;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 8370418..74ff5b6 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -59,8 +59,12 @@
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
 #include "types/containers/Tuple.hpp"
 #include "types/operations/comparisons/ComparisonUtil.hpp"
+#include "utility/BloomFilter.hpp"
+#include "utility/BloomFilterAdapter.hpp"
 #include "utility/Macros.hpp"
 
+#include "gflags/gflags.h"
+
 #include "glog/logging.h"
 
 #ifdef QUICKSTEP_HAVE_BITWEAVING
@@ -78,6 +82,8 @@ using std::vector;
 
 namespace quickstep {
 
+DECLARE_int64(bloom_adapter_batch_size);
+
 class Type;
 
 StorageBlock::StorageBlock(const CatalogRelationSchema &relation,
@@ -341,6 +347,8 @@ void StorageBlock::sample(const bool is_block_sample,
 
 void StorageBlock::select(const vector<unique_ptr<const Scalar>> &selection,
                           const Predicate *predicate,
+                          const std::vector<const BloomFilter *> &bloom_filters,
+                          const std::vector<attribute_id> &bloom_filter_attribute_ids,
                           InsertDestinationInterface *destination) const {
   ColumnVectorsValueAccessor temp_result;
   {
@@ -350,10 +358,58 @@ void StorageBlock::select(const vector<unique_ptr<const Scalar>> &selection,
 
     std::unique_ptr<TupleIdSequence> matches;
     std::unique_ptr<ValueAccessor> accessor;
-    if (predicate == nullptr) {
+
+    if (bloom_filters.size() > 0) {
+      const std::size_t num_tuples = tuple_store_->numTuples();
+      matches.reset(new TupleIdSequence(num_tuples));
+//      std::cerr << "Before: " << num_tuples << "\n";
       accessor.reset(tuple_store_->createValueAccessor());
+      InvokeOnAnyValueAccessor(
+          accessor.get(),
+          [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+        std::vector<std::size_t> attr_size_vector;
+        attr_size_vector.reserve(bloom_filter_attribute_ids.size());
+        for (const auto &attr : bloom_filter_attribute_ids) {
+          auto val_and_size =
+              accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, attr);
+          attr_size_vector.emplace_back(val_and_size.second);
+        }
+
+        std::unique_ptr<BloomFilterAdapter> bloom_filter_adapter;
+        bloom_filter_adapter.reset(new BloomFilterAdapter(
+            bloom_filters, bloom_filter_attribute_ids, attr_size_vector));
+
+        std::uint32_t batch_size_try = FLAGS_bloom_adapter_batch_size;
+        std::uint32_t num_tuples_left = accessor->getNumTuples();
+        std::vector<tuple_id> batch(num_tuples_left);
+
+        do {
+          std::uint32_t batch_size =
+              batch_size_try < num_tuples_left ? batch_size_try : num_tuples_left;
+          for (std::size_t i = 0; i < batch_size; ++i) {
+            accessor->next();
+            batch.push_back(accessor->getCurrentPosition());
+          }
+
+          std::size_t num_hits =
+              bloom_filter_adapter->bulkProbe<true>(accessor, batch, batch_size);
+          for (std::size_t t = 0; t < num_hits; ++t){
+            matches->set(batch[t], true);
+          }
+
+          batch.clear();
+          num_tuples_left -= batch_size;
+          batch_size_try = batch_size * 2;
+        } while (num_tuples_left > 0);
+      });
+//      std::cerr << "After: " << matches->numTuples() << "\n";
+    }
+
+    if (predicate == nullptr) {
+      accessor.reset(tuple_store_->createValueAccessor(matches.get()));
     } else {
-      matches.reset(getMatchesForPredicate(predicate));
+      auto *new_matches = getMatchesForPredicate(predicate, matches.get());
+      matches.reset(new_matches);
       accessor.reset(tuple_store_->createValueAccessor(matches.get()));
     }
 
@@ -371,13 +427,63 @@ void StorageBlock::select(const vector<unique_ptr<const Scalar>> &selection,
 
 void StorageBlock::selectSimple(const std::vector<attribute_id> &selection,
                                 const Predicate *predicate,
+                                const std::vector<const BloomFilter *> &bloom_filters,
+                                const std::vector<attribute_id> &bloom_filter_attribute_ids,
                                 InsertDestinationInterface *destination) const {
   std::unique_ptr<ValueAccessor> accessor;
   std::unique_ptr<TupleIdSequence> matches;
-  if (predicate == nullptr) {
+
+  if (bloom_filters.size() > 0) {
+    const std::size_t num_tuples = tuple_store_->numTuples();
+    matches.reset(new TupleIdSequence(num_tuples));
+//    std::cerr << "Before: " << num_tuples << "\n";
     accessor.reset(tuple_store_->createValueAccessor());
+    InvokeOnAnyValueAccessor(
+        accessor.get(),
+        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+      std::vector<std::size_t> attr_size_vector;
+      attr_size_vector.reserve(bloom_filter_attribute_ids.size());
+      for (const auto &attr : bloom_filter_attribute_ids) {
+        auto val_and_size =
+            accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, attr);
+        attr_size_vector.emplace_back(val_and_size.second);
+      }
+
+      std::unique_ptr<BloomFilterAdapter> bloom_filter_adapter;
+      bloom_filter_adapter.reset(new BloomFilterAdapter(
+          bloom_filters, bloom_filter_attribute_ids, attr_size_vector));
+
+      std::uint32_t batch_size_try = FLAGS_bloom_adapter_batch_size;
+      std::uint32_t num_tuples_left = accessor->getNumTuples();
+      std::vector<tuple_id> batch(num_tuples_left);
+
+      do {
+        std::uint32_t batch_size =
+            batch_size_try < num_tuples_left ? batch_size_try : num_tuples_left;
+        for (std::size_t i = 0; i < batch_size; ++i) {
+          accessor->next();
+          batch.push_back(accessor->getCurrentPosition());
+        }
+
+        std::size_t num_hits =
+            bloom_filter_adapter->bulkProbe<true>(accessor, batch, batch_size);
+        for (std::size_t t = 0; t < num_hits; ++t){
+          matches->set(batch[t], true);
+        }
+
+        batch.clear();
+        num_tuples_left -= batch_size;
+        batch_size_try = batch_size * 2;
+      } while (num_tuples_left > 0);
+    });
+//    std::cerr << "After: " << matches->numTuples() << "\n";
+  }
+
+  if (predicate == nullptr) {
+    accessor.reset(tuple_store_->createValueAccessor(matches.get()));
   } else {
-    matches.reset(getMatchesForPredicate(predicate));
+    auto *new_matches = getMatchesForPredicate(predicate, matches.get());
+    matches.reset(new_matches);
     accessor.reset(tuple_store_->createValueAccessor(matches.get()));
   }
 
@@ -1219,12 +1325,13 @@ bool StorageBlock::rebuildIndexes(bool short_circuit) {
   return all_indices_consistent_;
 }
 
-TupleIdSequence* StorageBlock::getMatchesForPredicate(const Predicate *predicate) const {
+TupleIdSequence* StorageBlock::getMatchesForPredicate(const Predicate *predicate,
+                                                      const TupleIdSequence *sequence) const {
   if (predicate == nullptr) {
     return tuple_store_->getExistenceMap();
   }
 
-  std::unique_ptr<ValueAccessor> value_accessor(tuple_store_->createValueAccessor());
+  std::unique_ptr<ValueAccessor> value_accessor(tuple_store_->createValueAccessor(sequence));
   std::unique_ptr<TupleIdSequence> existence_map;
   if (!tuple_store_->isPacked()) {
     existence_map.reset(tuple_store_->getExistenceMap());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 2a20cb5..5cca51c 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -34,6 +34,7 @@
 #include "storage/StorageBlockLayout.pb.h"
 #include "storage/TupleIdSequence.hpp"
 #include "storage/TupleStorageSubBlock.hpp"
+#include "utility/BloomFilter.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
 
@@ -349,6 +350,8 @@ class StorageBlock : public StorageBlockBase {
    **/
   void select(const std::vector<std::unique_ptr<const Scalar>> &selection,
               const Predicate *predicate,
+              const std::vector<const BloomFilter *> &bloom_filters,
+              const std::vector<attribute_id> &bloom_filter_attribute_ids,
               InsertDestinationInterface *destination) const;
 
   /**
@@ -372,6 +375,8 @@ class StorageBlock : public StorageBlockBase {
    **/
   void selectSimple(const std::vector<attribute_id> &selection,
                     const Predicate *predicate,
+                    const std::vector<const BloomFilter *> &bloom_filters,
+                    const std::vector<attribute_id> &bloom_filter_attribute_ids,
                     InsertDestinationInterface *destination) const;
 
   /**
@@ -587,7 +592,8 @@ class StorageBlock : public StorageBlockBase {
    **/
   const std::size_t getNumTuples() const;
 
-  TupleIdSequence* getMatchesForPredicate(const Predicate *predicate) const;
+  TupleIdSequence* getMatchesForPredicate(const Predicate *predicate,
+                                          const TupleIdSequence *sequence = nullptr) const;
 
  private:
   static TupleStorageSubBlock* CreateTupleStorageSubBlock(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index b90a8dc..9af00b4 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -35,6 +35,7 @@
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/Selection.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
 #include "utility/StringUtil.hpp"
@@ -88,6 +89,9 @@ std::string PlanVisualizer::visualize(const P::PhysicalPtr &input) {
   for (const EdgeInfo &edge_info : edges_) {
     graph_oss << "  " << edge_info.src_node_id << " -> "
               << edge_info.dst_node_id << " [";
+    if (edge_info.dashed) {
+      graph_oss << "style=dashed ";
+    }
     if (!edge_info.labels.empty()) {
       graph_oss << "label=\""
                 << EscapeSpecialChars(JoinToString(edge_info.labels, "&#10;"))
@@ -118,6 +122,12 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
     EdgeInfo &edge_info = edges_.back();
     edge_info.src_node_id = child_id;
     edge_info.dst_node_id = node_id;
+    edge_info.dashed = false;
+
+    if (input->getPhysicalType() == P::PhysicalType::kHashJoin &&
+        child == input->children()[1]) {
+      edge_info.dashed = true;
+    }
 
     for (const auto &attr : child->getOutputAttributes()) {
       if (referenced_ids.find(attr->id()) != referenced_ids.end()) {
@@ -167,7 +177,6 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
         node_info.labels.emplace_back(
             std::string("[BF probe] ") + bf.attribute->attribute_alias());
       }
-
       break;
     }
     case P::PhysicalType::kAggregate: {
@@ -180,7 +189,18 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
         node_info.labels.emplace_back(
             std::string("[BF probe] ") + bf.attribute->attribute_alias());
       }
+      break;
+    }
+    case P::PhysicalType::kSelection: {
+      const P::SelectionPtr selection =
+        std::static_pointer_cast<const P::Selection>(input);
+      node_info.labels.emplace_back(input->getName());
 
+      const auto &bf_config = selection->bloom_filter_config();
+      for (const auto &bf : bf_config.probe_side_bloom_filters) {
+        node_info.labels.emplace_back(
+            std::string("[BF probe] ") + bf.attribute->attribute_alias());
+      }
       break;
     }
     default: {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ef4cd1ed/utility/PlanVisualizer.hpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.hpp b/utility/PlanVisualizer.hpp
index 1c0df77..e4e3957 100644
--- a/utility/PlanVisualizer.hpp
+++ b/utility/PlanVisualizer.hpp
@@ -73,6 +73,7 @@ class PlanVisualizer {
     int src_node_id;
     int dst_node_id;
     std::vector<std::string> labels;
+    bool dashed;
   };
 
   void visit(const optimizer::physical::PhysicalPtr &input);


[10/16] incubator-quickstep git commit: Initial commit

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


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

Branch: refs/heads/LIP-for-tpch
Commit: b7150fb13b117841919ac8dfcc7aca9043a30e82
Parents: d9135a8
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sat Jun 11 23:14:00 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500

----------------------------------------------------------------------
 CMakeLists.txt                                  |   1 +
 catalog/CMakeLists.txt                          |   9 +
 catalog/Catalog.proto                           |   5 +
 catalog/CatalogRelationConstraints.cpp          |  55 ++
 catalog/CatalogRelationConstraints.hpp          | 110 ++++
 catalog/CatalogRelationSchema.cpp               |  15 +
 catalog/CatalogRelationSchema.hpp               |  16 +-
 cli/CommandExecutor.cpp                         |  25 +-
 cli/QuickstepCli.cpp                            |  65 +++
 compression/CompressionDictionaryLite.hpp       |  42 ++
 query_execution/CMakeLists.txt                  |   1 +
 query_execution/QueryContext.cpp                |  11 +-
 query_execution/Worker.cpp                      |   5 +
 query_optimizer/CMakeLists.txt                  |   2 +
 query_optimizer/ExecutionGenerator.cpp          |  74 +--
 query_optimizer/ExecutionGenerator.hpp          |   2 +-
 query_optimizer/ExecutionHeuristics.cpp         | 171 ++++---
 query_optimizer/ExecutionHeuristics.hpp         |  79 ++-
 query_optimizer/PhysicalGenerator.cpp           |   7 +-
 query_optimizer/cost_model/SimpleCostModel.cpp  |   4 +-
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  42 +-
 query_optimizer/expressions/ExpressionUtil.hpp  |   8 +-
 query_optimizer/physical/Aggregate.cpp          |   5 +
 query_optimizer/physical/Aggregate.hpp          |  23 +-
 query_optimizer/physical/HashJoin.cpp           |  27 +
 query_optimizer/physical/HashJoin.hpp           |  23 +-
 query_optimizer/physical/Physical.hpp           |  55 ++
 query_optimizer/physical/Selection.cpp          |   6 +
 query_optimizer/physical/Selection.hpp          |   3 +
 query_optimizer/physical/TableReference.cpp     |  18 +
 query_optimizer/physical/TableReference.hpp     |   3 +
 query_optimizer/rules/AttachBloomFilters.cpp    | 308 ++++++++++++
 query_optimizer/rules/AttachBloomFilters.hpp    | 118 +++++
 query_optimizer/rules/CMakeLists.txt            |  17 +
 .../StarSchemaHashJoinOrderOptimization.cpp     | 277 ++++++----
 .../StarSchemaHashJoinOrderOptimization.hpp     | 100 ++--
 .../tests/ExecutionHeuristics_unittest.cpp      |   3 +-
 relational_operators/HashJoinOperator.cpp       |  10 +
 relational_operators/HashJoinOperator.hpp       |  25 +-
 relational_operators/WorkOrder.hpp              |  11 +-
 storage/AggregationOperationState.cpp           |  98 +++-
 storage/AggregationOperationState.hpp           |  10 +-
 storage/AggregationOperationState.proto         |   6 +
 storage/BasicColumnStoreValueAccessor.hpp       |  26 +-
 storage/BloomFilterIndexSubBlock.cpp            |   4 +-
 storage/BloomFilterIndexSubBlock.hpp            |   6 -
 storage/CMakeLists.txt                          |   2 +
 storage/CompressedColumnStoreValueAccessor.hpp  |  22 +
 .../CompressedPackedRowStoreValueAccessor.hpp   |  22 +
 storage/HashTable.hpp                           | 185 ++++---
 storage/HashTable.proto                         |  10 +-
 storage/HashTableFactory.hpp                    |  23 +-
 storage/PackedRowStoreValueAccessor.hpp         |  25 +-
 storage/SplitRowStoreValueAccessor.hpp          |  45 ++
 storage/StorageBlock.cpp                        |  28 +-
 storage/StorageBlock.hpp                        |   7 +-
 storage/ValueAccessor.hpp                       |  36 ++
 types/containers/ColumnVector.hpp               |  35 ++
 types/containers/ColumnVectorsValueAccessor.hpp |  17 +
 utility/BloomFilter.hpp                         | 502 ++++++++++++++-----
 utility/BloomFilter.proto                       |   6 +-
 utility/BloomFilterAdapter.hpp                  | 142 ++++++
 utility/CMakeLists.txt                          |  13 +
 utility/DisjointTreeForest.hpp                  | 116 +++++
 utility/EventProfiler.cpp                       |  29 ++
 utility/EventProfiler.hpp                       | 188 +++++++
 utility/PlanVisualizer.cpp                      |  42 +-
 67 files changed, 2867 insertions(+), 559 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 487aaf9..27db15e 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -770,6 +770,7 @@ target_link_libraries(quickstep_cli_shell
                       quickstep_queryoptimizer_QueryProcessor
                       quickstep_storage_PreloaderThread
                       quickstep_threading_ThreadIDBasedMap
+                      quickstep_utility_EventProfiler
                       quickstep_utility_ExecutionDAGVisualizer
                       quickstep_utility_Macros
                       quickstep_utility_PtrVector

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index dd4ef99..fc01458 100644
--- a/catalog/CMakeLists.txt
+++ b/catalog/CMakeLists.txt
@@ -35,6 +35,9 @@ add_library(quickstep_catalog_CatalogDatabaseCache CatalogDatabaseCache.cpp Cata
 add_library(quickstep_catalog_CatalogDatabaseLite ../empty_src.cpp CatalogDatabaseLite.hpp)
 add_library(quickstep_catalog_CatalogErrors ../empty_src.cpp CatalogErrors.hpp)
 add_library(quickstep_catalog_CatalogRelation CatalogRelation.cpp CatalogRelation.hpp)
+add_library(quickstep_catalog_CatalogRelationConstraints
+            CatalogRelationConstraints.cpp
+            CatalogRelationConstraints.hpp)
 add_library(quickstep_catalog_CatalogRelationSchema
             CatalogRelationSchema.cpp
             CatalogRelationSchema.hpp)
@@ -117,6 +120,10 @@ target_link_libraries(quickstep_catalog_CatalogRelation
                       quickstep_threading_SpinSharedMutex
                       quickstep_utility_Macros
                       quickstep_utility_PtrVector)
+target_link_libraries(quickstep_catalog_CatalogRelationConstraints
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_catalog_Catalog_proto
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_catalog_CatalogRelationStatistics
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_Catalog_proto
@@ -136,6 +143,7 @@ target_link_libraries(quickstep_catalog_CatalogRelationSchema
                       glog
                       quickstep_catalog_CatalogAttribute
                       quickstep_catalog_CatalogErrors
+                      quickstep_catalog_CatalogRelationConstraints
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_Catalog_proto
                       quickstep_types_Type
@@ -182,6 +190,7 @@ target_link_libraries(quickstep_catalog
                       quickstep_catalog_CatalogDatabaseLite
                       quickstep_catalog_CatalogErrors
                       quickstep_catalog_CatalogRelation
+                      quickstep_catalog_CatalogRelationConstraints
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogRelationStatistics
                       quickstep_catalog_CatalogTypedefs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index 90ce37e..a4cc3ce 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -80,6 +80,10 @@ message IndexScheme {
   repeated IndexEntry index_entries = 1;
 }
 
+message CatalogRelationConstraints {
+  repeated int32 primary_key = 1;
+}
+
 message CatalogRelationStatistics {
   optional fixed64 num_tuples = 1;
 
@@ -96,6 +100,7 @@ message CatalogRelationSchema {
   required bool temporary = 3;
 
   repeated CatalogAttribute attributes = 4;
+  optional CatalogRelationConstraints constraints = 5;
 
   extensions 16 to max;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/CatalogRelationConstraints.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationConstraints.cpp b/catalog/CatalogRelationConstraints.cpp
new file mode 100644
index 0000000..4525a98
--- /dev/null
+++ b/catalog/CatalogRelationConstraints.cpp
@@ -0,0 +1,55 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "catalog/CatalogRelationConstraints.hpp"
+
+#include "catalog/Catalog.pb.h"
+
+namespace quickstep {
+
+CatalogRelationConstraints::CatalogRelationConstraints(
+    const serialization::CatalogRelationConstraints &proto) {
+  if (proto.primary_key_size() > 0) {
+    primary_key_.reset(new std::set<attribute_id>());
+    for (std::size_t i = 0; i < proto.primary_key_size(); ++i) {
+      primary_key_->emplace(proto.primary_key(i));
+    }
+  }
+}
+
+serialization::CatalogRelationConstraints CatalogRelationConstraints::getProto() const {
+  serialization::CatalogRelationConstraints proto;
+  if (primary_key_ != nullptr) {
+    for (const auto attr_id : *primary_key_) {
+      proto.add_primary_key(attr_id);
+    }
+  }
+  return proto;
+}
+
+bool CatalogRelationConstraints::ProtoIsValid(
+    const serialization::CatalogRelationConstraints &proto,
+    const std::size_t num_attributes) {
+  for (std::size_t i = 0; i < proto.primary_key_size(); ++i) {
+    if (proto.primary_key(i) >= num_attributes) {
+      return false;
+    }
+  }
+  return true;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/CatalogRelationConstraints.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationConstraints.hpp b/catalog/CatalogRelationConstraints.hpp
new file mode 100644
index 0000000..896c072
--- /dev/null
+++ b/catalog/CatalogRelationConstraints.hpp
@@ -0,0 +1,110 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_
+#define QUICKSTEP_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <memory>
+#include <set>
+#include <utility>
+
+#include "catalog/Catalog.pb.h"
+#include "catalog/CatalogTypedefs.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Catalog
+ *  @{
+ */
+
+/**
+ * @brief Constraints on a catalog relation.
+ **/
+class CatalogRelationConstraints {
+ public:
+  /**
+   * @brief Constructor.
+   **/
+  CatalogRelationConstraints() {}
+
+  /**
+   * @brief Reconstruct a CatalogRelationConstraints object from its serialized
+   *        Protocol Buffer form.
+   *
+   * @param proto The Protocol Buffer serialization of a CatalogRelationConstraints
+   *        object, previously produced by getProto().
+   **/
+  explicit CatalogRelationConstraints(const serialization::CatalogRelationConstraints &proto);
+
+  /**
+   * @brief Serialize the CatalogRelationConstraints object as Protocol Buffer.
+   *
+   * @return The Protocol Buffer representation of the CatalogRelationConstraints
+   *         object.
+   **/
+  serialization::CatalogRelationConstraints getProto() const;
+
+  static bool ProtoIsValid(const serialization::CatalogRelationConstraints &proto,
+                           const std::size_t num_attributes);
+
+  bool hasPrimaryKey() const {
+    return (primary_key_ != nullptr);
+  }
+
+  const std::set<attribute_id>* getPrimaryKey() const {
+    return primary_key_.get();
+  }
+
+  template <typename IterableT>
+  void setPrimaryKey(IterableT &&primary_key) {
+    CHECK(!primary_key.empty());
+    primary_key_.reset(
+        new std::set<attribute_id>(primary_key.begin(), primary_key.end()));
+  }
+
+  void removePrimaryKey() {
+    primary_key_.reset();
+  }
+
+  bool impliesUniqueAttributes(const std::set<attribute_id> &attributes) const {
+    if (primary_key_ == nullptr) {
+      return false;
+    }
+
+    std::vector<attribute_id> attr_intersection;
+    std::set_intersection(primary_key_->begin(), primary_key_->end(),
+                          attributes.begin(), attributes.end(),
+                          std::back_inserter(attr_intersection));
+    return (attr_intersection.size() == primary_key_->size());
+  }
+
+ private:
+  std::unique_ptr<std::set<attribute_id>> primary_key_;
+
+  DISALLOW_COPY_AND_ASSIGN(CatalogRelationConstraints);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/CatalogRelationSchema.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationSchema.cpp b/catalog/CatalogRelationSchema.cpp
index 5568cef..d474ef6 100644
--- a/catalog/CatalogRelationSchema.cpp
+++ b/catalog/CatalogRelationSchema.cpp
@@ -27,6 +27,7 @@
 #include "catalog/Catalog.pb.h"
 #include "catalog/CatalogAttribute.hpp"
 #include "catalog/CatalogErrors.hpp"
+#include "catalog/CatalogRelationConstraints.hpp"
 #include "catalog/CatalogTypedefs.hpp"
 #include "types/Type.hpp"
 #include "utility/PtrVector.hpp"
@@ -70,6 +71,12 @@ CatalogRelationSchema::CatalogRelationSchema(const serialization::CatalogRelatio
       attr_vec_.push_back(nullptr);
     }
   }
+
+  if (proto.has_constraints()) {
+    constraints_.reset(new CatalogRelationConstraints(proto.constraints()));
+  } else {
+    constraints_.reset(new CatalogRelationConstraints());
+  }
 }
 
 bool CatalogRelationSchema::ProtoIsValid(const serialization::CatalogRelationSchema &proto) {
@@ -84,6 +91,12 @@ bool CatalogRelationSchema::ProtoIsValid(const serialization::CatalogRelationSch
     }
   }
 
+  if (proto.has_constraints()
+      && !CatalogRelationConstraints::ProtoIsValid(proto.constraints(),
+                                                   proto.attributes_size())) {
+    return false;
+  }
+
   return true;
 }
 
@@ -104,6 +117,8 @@ serialization::CatalogRelationSchema CatalogRelationSchema::getProto() const {
     }
   }
 
+  proto.mutable_constraints()->CopyFrom(constraints_->getProto());
+
   return proto;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/catalog/CatalogRelationSchema.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationSchema.hpp b/catalog/CatalogRelationSchema.hpp
index d8b6128..0c7e97b 100644
--- a/catalog/CatalogRelationSchema.hpp
+++ b/catalog/CatalogRelationSchema.hpp
@@ -21,12 +21,14 @@
 #define QUICKSTEP_CATALOG_CATALOG_RELATION_SCHEMA_HPP_
 
 #include <cstddef>
+#include <memory>
 #include <string>
 #include <unordered_map>
 #include <vector>
 
 #include "catalog/Catalog.pb.h"
 #include "catalog/CatalogAttribute.hpp"
+#include "catalog/CatalogRelationConstraints.hpp"
 #include "catalog/CatalogTypedefs.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
@@ -427,6 +429,14 @@ class CatalogRelationSchema {
     return max_byte_lengths_;
   }
 
+  const CatalogRelationConstraints& getConstraints() const {
+    return *constraints_;
+  }
+
+  CatalogRelationConstraints* getConstraintsMutable() {
+    return constraints_.get();
+  }
+
  protected:
   /**
    * @brief Create a new relation.
@@ -456,7 +466,8 @@ class CatalogRelationSchema {
         min_variable_byte_length_excluding_nullable_(0),
         estimated_variable_byte_length_(0),
         current_nullable_attribute_index_(-1),
-        current_variable_length_attribute_index_(-1) {
+        current_variable_length_attribute_index_(-1),
+        constraints_(new CatalogRelationConstraints()) {
   }
 
   /**
@@ -532,6 +543,9 @@ class CatalogRelationSchema {
   std::vector<int> variable_length_attribute_indices_;
   int current_variable_length_attribute_index_;
 
+  // Primary key, foreign keys, etc.
+  std::unique_ptr<CatalogRelationConstraints> constraints_;
+
  private:
   friend class CatalogDatabase;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 78fbe6f..f27f1ef 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -258,7 +258,8 @@ inline TypedValue executeQueryForSingleResult(
   return value;
 }
 
-void executeAnalyze(const tmb::client_id main_thread_client_id,
+void executeAnalyze(const PtrVector<ParseString> *arguments,
+                    const tmb::client_id main_thread_client_id,
                     const tmb::client_id foreman_client_id,
                     MessageBus *bus,
                     QueryProcessor *query_processor,
@@ -267,8 +268,19 @@ void executeAnalyze(const tmb::client_id main_thread_client_id,
   StorageManager *storage_manager = query_processor->getStorageManager();
 
   std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
-  std::vector<std::reference_wrapper<const CatalogRelation>> relations(
-      database.begin(), database.end());
+  std::vector<std::reference_wrapper<const CatalogRelation>> relations;
+  if (arguments->size() == 0) {
+    relations.insert(relations.begin(), database.begin(), database.end());
+  } else {
+    for (const auto &rel_name : *arguments) {
+      const CatalogRelation *rel = database.getRelationByName(rel_name.value());
+      if (rel == nullptr) {
+        THROW_SQL_ERROR_AT(&rel_name) << "Table does not exist";
+      } else {
+        relations.emplace_back(*rel);
+      }
+    }
+  }
 
   // Analyze each relation in the database.
   for (const CatalogRelation &relation : relations) {
@@ -348,8 +360,11 @@ void executeCommand(const ParseStatement &statement,
       executeDescribeTable(arguments, catalog_database, out);
     }
   } else if (command_str == C::kAnalyzeCommand) {
-    executeAnalyze(
-        main_thread_client_id, foreman_client_id, bus, query_processor, out);
+    executeAnalyze(arguments,
+                   main_thread_client_id,
+                   foreman_client_id,
+                   bus,
+                   query_processor, out);
   } else {
     THROW_SQL_ERROR_AT(command.command()) << "Invalid Command";
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 8269197..a3dfae4 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -52,6 +52,9 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include <gperftools/profiler.h>
 #endif
 
+#include "catalog/CatalogDatabase.hpp"
+#include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogRelationConstraints.hpp"
 #include "cli/DefaultsConfigurator.hpp"
 #include "cli/InputParserUtil.hpp"
 #include "cli/PrintToScreen.hpp"
@@ -75,6 +78,7 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 
 #include "storage/PreloaderThread.hpp"
 #include "threading/ThreadIDBasedMap.hpp"
+#include "utility/EventProfiler.hpp"
 #include "utility/ExecutionDAGVisualizer.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
@@ -90,6 +94,8 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include "tmb/message_bus.h"
 #include "tmb/message_style.h"
 
+#include "google/protobuf/text_format.h"
+
 namespace quickstep {
 class CatalogRelation;
 }
@@ -190,9 +196,48 @@ DEFINE_bool(visualize_execution_dag, false,
             "If true, visualize the execution plan DAG into a graph in DOT "
             "format (DOT is a plain text graph description language) which is "
             "then printed via stderr.");
+DEFINE_string(profile_output, "",
+              "Output file name for writing the profiled events.");
 
 }  // namespace quickstep
 
+void addPrimaryKeyInfoForTPCHTables(quickstep::CatalogDatabase *database) {
+  const std::vector<std::pair<std::string, std::vector<std::string>>> rel_pkeys = {
+      { "region", { "r_regionkey" } },
+      { "nation", { "n_nationkey" } },
+      { "supplier", { "s_suppkey" } },
+      { "customer", { "c_custkey" } },
+      { "part", { "p_partkey" } },
+      { "partsupp", { "ps_partkey", "ps_suppkey" } },
+      { "orders", { "o_orderkey" } }
+  };
+  for (const auto &rel_pair : rel_pkeys) {
+    CatalogRelation *rel = database->getRelationByNameMutable(rel_pair.first);
+    std::vector<quickstep::attribute_id> attrs;
+    for (const auto &pkey : rel_pair.second) {
+      attrs.emplace_back(rel->getAttributeByName(pkey)->getID());
+    }
+    rel->getConstraintsMutable()->setPrimaryKey(attrs);
+  }
+}
+
+void addPrimaryKeyInfoForSSBTables(quickstep::CatalogDatabase *database) {
+  const std::vector<std::pair<std::string, std::vector<std::string>>> rel_pkeys = {
+      { "supplier", { "s_suppkey" } },
+      { "customer", { "c_custkey" } },
+      { "part", { "p_partkey" } },
+      { "ddate", { "d_datekey" } }
+  };
+  for (const auto &rel_pair : rel_pkeys) {
+    CatalogRelation *rel = database->getRelationByNameMutable(rel_pair.first);
+    std::vector<quickstep::attribute_id> attrs;
+    for (const auto &pkey : rel_pair.second) {
+      attrs.emplace_back(rel->getAttributeByName(pkey)->getID());
+    }
+    rel->getConstraintsMutable()->setPrimaryKey(attrs);
+  }
+}
+
 int main(int argc, char* argv[]) {
   google::InitGoogleLogging(argv[0]);
   gflags::ParseCommandLineFlags(&argc, &argv, true);
@@ -300,6 +345,15 @@ int main(int argc, char* argv[]) {
     LOG(FATAL) << "NON-STANDARD EXCEPTION DURING STARTUP";
   }
 
+//  addPrimaryKeyInfoForTPCHTables(query_processor->getDefaultDatabase());
+//  addPrimaryKeyInfoForSSBTables(query_processor->getDefaultDatabase());
+//  std::string proto_str;
+//  google::protobuf::TextFormat::PrintToString(
+//      query_processor->getDefaultDatabase()->getProto(), &proto_str);
+//  std::cerr << proto_str << "\n";
+//  query_processor->markCatalogAltered();
+//  query_processor->saveCatalog();
+
   // Parse the CPU affinities for workers and the preloader thread, if enabled
   // to warm up the buffer pool.
   const vector<int> worker_cpu_affinities =
@@ -449,6 +503,7 @@ int main(int argc, char* argv[]) {
               new quickstep::ExecutionDAGVisualizer(*query_handle->getQueryPlanMutable()));
         }
 
+        quickstep::simple_profiler.clear();
         start = std::chrono::steady_clock::now();
         QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
             main_thread_client_id,
@@ -461,6 +516,11 @@ int main(int argc, char* argv[]) {
               main_thread_client_id, &bus);
           end = std::chrono::steady_clock::now();
 
+          if (quickstep::FLAGS_visualize_dag) {
+            quickstep::DAGVisualizer visualizer(*query_handle->getQueryPlanMutable());
+            std::cerr << "\n" << visualizer.toDOT() << "\n";
+          }
+
           const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
           if (query_result_relation) {
             PrintToScreen::PrintRelation(*query_result_relation,
@@ -492,6 +552,11 @@ int main(int argc, char* argv[]) {
             dag_visualizer->bindProfilingStats(profiling_stats);
             std::cerr << "\n" << dag_visualizer->toDOT() << "\n";
           }
+          if (!quickstep::FLAGS_profile_output.empty()) {
+            std::ofstream ofs(quickstep::FLAGS_profile_output, std::ios::out);
+            quickstep::simple_profiler.writeToStream(ofs);
+            ofs.close();
+          }
         } catch (const std::exception &e) {
           fprintf(stderr, "QUERY EXECUTION ERROR: %s\n", e.what());
           break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/compression/CompressionDictionaryLite.hpp
----------------------------------------------------------------------
diff --git a/compression/CompressionDictionaryLite.hpp b/compression/CompressionDictionaryLite.hpp
index c4c338e..7eb0e34 100644
--- a/compression/CompressionDictionaryLite.hpp
+++ b/compression/CompressionDictionaryLite.hpp
@@ -176,6 +176,15 @@ class CompressionDictionaryLite {
     }
   }
 
+  template <bool check_null = true>
+  inline std::pair<const void*, std::size_t> getUntypedValueAndByteLengthForCode(const std::uint32_t code) const {
+    if (type_is_variable_length_) {
+      return variableLengthGetUntypedValueAndByteLengthHelper<std::uint32_t, check_null>(code);
+    } else {
+      return fixedLengthGetUntypedValueAndByteLengthHelper<std::uint32_t, check_null>(code);
+    }
+  }
+
   /**
    * @brief Get the value represented by the specified code as a TypedValue.
    * @note This version is for codes of 8 bits or less. Also see
@@ -257,6 +266,39 @@ class CompressionDictionaryLite {
     return retval;
   }
 
+  template <typename CodeType, bool check_null = true>
+  inline std::pair<const void*, std::size_t> fixedLengthGetUntypedValueAndByteLengthHelper(
+      const CodeType code) const {
+    if (check_null && (code == getNullCode())) {
+      return std::make_pair(nullptr, 0);
+    }
+    DCHECK_LT(code, numberOfCodes());
+    return std::make_pair(static_cast<const char*>(dictionary_memory_)
+                              + 2 * sizeof(std::uint32_t)        // Header.
+                              + code * type_fixed_byte_length_,  // Index into value array.
+                          type_fixed_byte_length_);
+  }
+
+  template <typename CodeType, bool check_null = true>
+  inline std::pair<const void*, std::size_t> variableLengthGetUntypedValueAndByteLengthHelper(
+      const CodeType code) const {
+    if (check_null && (code == getNullCode())) {
+      return std::make_pair(nullptr, 0);
+    }
+    DCHECK_LT(code, numberOfCodes());
+
+    const std::uint32_t value_offset = static_cast<const std::uint32_t*>(dictionary_memory_)[code + 2];
+    const void *data_ptr = variable_length_data_region_ + value_offset;
+    DCHECK_LT(data_ptr, static_cast<const char*>(dictionary_memory_) + dictionary_memory_size_);
+
+    std::size_t data_size = (code == *static_cast<const std::uint32_t*>(dictionary_memory_) - 1) ?
+        (static_cast<const char*>(dictionary_memory_)
+            + dictionary_memory_size_
+            - static_cast<const char*>(data_ptr))
+        : (static_cast<const std::uint32_t*>(dictionary_memory_)[code + 3] - value_offset);
+    return std::make_pair(data_ptr, data_size);
+  }
+
   template <typename CodeType>
   inline TypedValue fixedLengthGetTypedValueHelper(const CodeType code) const {
     if (code == getNullCode()) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 74fcafb..ff0fe08 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -282,6 +282,7 @@ target_link_libraries(quickstep_queryexecution_Worker
                       quickstep_threading_Thread
                       quickstep_threading_ThreadIDBasedMap
                       quickstep_threading_ThreadUtil
+                      quickstep_utility_EventProfiler
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_queryexecution_WorkerDirectory

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_execution/QueryContext.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.cpp b/query_execution/QueryContext.cpp
index 2572e18..65405a6 100644
--- a/query_execution/QueryContext.cpp
+++ b/query_execution/QueryContext.cpp
@@ -61,15 +61,16 @@ QueryContext::QueryContext(const serialization::QueryContext &proto,
       << "Attempted to create QueryContext from an invalid proto description:\n"
       << proto.DebugString();
 
+  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.aggregation_states_size(); ++i) {
     aggregation_states_.emplace_back(
         AggregationOperationState::ReconstructFromProto(proto.aggregation_states(i),
                                                         database,
-                                                        storage_manager));
-  }
-
-  for (int i = 0; i < proto.bloom_filters_size(); ++i) {
-    bloom_filters_.emplace_back(new BloomFilter(proto.bloom_filters(i)));
+                                                        storage_manager,
+                                                        bloom_filters_));
   }
 
   for (int i = 0; i < proto.generator_functions_size(); ++i) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index 0b1efba..a1d3685 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -32,6 +32,7 @@
 #include "relational_operators/WorkOrder.hpp"
 #include "threading/ThreadIDBasedMap.hpp"
 #include "threading/ThreadUtil.hpp"
+#include "utility/EventProfiler.hpp"
 
 #include "glog/logging.h"
 
@@ -119,8 +120,12 @@ void Worker::executeWorkOrderHelper(const TaggedMessage &tagged_message,
   const size_t query_id_for_workorder = worker_message.getWorkOrder()->getQueryID();
 
   // Start measuring the execution time.
+  auto *container = relop_profiler.getContainer();
+  auto *line = container->getEventLine(worker_message.getRelationalOpIndex());
   start = std::chrono::steady_clock::now();
+  line->emplace_back();
   worker_message.getWorkOrder()->execute();
+  line->back().endEvent();
   end = std::chrono::steady_clock::now();
   delete worker_message.getWorkOrder();
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index c24ee89..aaad17c 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -79,6 +79,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_QueryPlan
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_costmodel_SimpleCostModel
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
                       quickstep_queryoptimizer_expressions_AggregateFunction
                       quickstep_queryoptimizer_expressions_Alias
                       quickstep_queryoptimizer_expressions_AttributeReference
@@ -196,6 +197,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_LogicalToPhysicalMapper
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_rules_AttachBloomFilters
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index a4b538e..6dbba51 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -59,6 +59,7 @@
 #include "query_optimizer/QueryHandle.hpp"
 #include "query_optimizer/QueryPlan.hpp"
 #include "query_optimizer/cost_model/SimpleCostModel.hpp"
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
 #include "query_optimizer/expressions/AggregateFunction.hpp"
 #include "query_optimizer/expressions/Alias.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
@@ -167,6 +168,8 @@ void ExecutionGenerator::generatePlan(const P::PhysicalPtr &physical_plan) {
 
   cost_model_.reset(
       new cost::SimpleCostModel(top_level_physical_plan_->shared_subplans()));
+  star_schema_cost_model_.reset(
+      new cost::StarSchemaSimpleCostModel(top_level_physical_plan_->shared_subplans()));
 
   const CatalogRelation *result_relation = nullptr;
 
@@ -600,8 +603,10 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   std::vector<attribute_id> probe_attribute_ids;
   std::vector<attribute_id> build_attribute_ids;
 
-  std::vector<attribute_id> probe_original_attribute_ids;
-  std::vector<attribute_id> build_original_attribute_ids;
+  const P::BloomFilterConfig &bloom_filter_config =
+      physical_plan->bloom_filter_config();
+  std::vector<attribute_id> probe_side_bloom_filter_attribute_ids;
+  std::vector<attribute_id> build_side_bloom_filter_attribute_ids;
 
   const CatalogRelation *referenced_stored_probe_relation = nullptr;
   const CatalogRelation *referenced_stored_build_relation = nullptr;
@@ -616,18 +621,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   const std::vector<E::AttributeReferencePtr> &left_join_attributes =
       physical_plan->left_join_attributes();
   for (const E::AttributeReferencePtr &left_join_attribute : left_join_attributes) {
-    // Try to determine the original stored relation referenced in the Hash Join.
-    referenced_stored_probe_relation =
-        optimizer_context_->catalog_database()->getRelationByName(left_join_attribute->relation_name());
-    if (referenced_stored_probe_relation == nullptr) {
-      // Hash Join optimizations are not possible, if the referenced relation cannot be determined.
-      skip_hash_join_optimization = true;
-    } else {
-      const attribute_id probe_operator_attribute_id =
-          referenced_stored_probe_relation->getAttributeByName(left_join_attribute->attribute_name())->getID();
-      probe_original_attribute_ids.emplace_back(probe_operator_attribute_id);
-    }
-
     const CatalogAttribute *probe_catalog_attribute
         = attribute_substitution_map_[left_join_attribute->id()];
     probe_attribute_ids.emplace_back(probe_catalog_attribute->getID());
@@ -640,18 +633,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   const std::vector<E::AttributeReferencePtr> &right_join_attributes =
       physical_plan->right_join_attributes();
   for (const E::AttributeReferencePtr &right_join_attribute : right_join_attributes) {
-    // Try to determine the original stored relation referenced in the Hash Join.
-    referenced_stored_build_relation =
-        optimizer_context_->catalog_database()->getRelationByName(right_join_attribute->relation_name());
-    if (referenced_stored_build_relation == nullptr) {
-      // Hash Join optimizations are not possible, if the referenced relation cannot be determined.
-      skip_hash_join_optimization = true;
-    } else {
-      const attribute_id build_operator_attribute_id =
-          referenced_stored_build_relation->getAttributeByName(right_join_attribute->attribute_name())->getID();
-      build_original_attribute_ids.emplace_back(build_operator_attribute_id);
-    }
-
     const CatalogAttribute *build_catalog_attribute
         = attribute_substitution_map_[right_join_attribute->id()];
     build_attribute_ids.emplace_back(build_catalog_attribute->getID());
@@ -661,6 +642,20 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
     }
   }
 
+  for (const auto &bf : bloom_filter_config.probe_side_bloom_filters) {
+    const CatalogAttribute *probe_bf_catalog_attribute
+        = attribute_substitution_map_[bf.attribute->id()];
+    probe_side_bloom_filter_attribute_ids.emplace_back(
+        probe_bf_catalog_attribute->getID());
+  }
+
+  for (const auto &bf : bloom_filter_config.build_side_bloom_filters) {
+    const CatalogAttribute *build_bf_catalog_attribute
+        = attribute_substitution_map_[bf.attribute->id()];
+    build_side_bloom_filter_attribute_ids.emplace_back(
+        build_bf_catalog_attribute->getID());
+  }
+
   // Remember key types for call to SimplifyHashTableImplTypeProto() below.
   std::vector<const Type*> key_types;
   for (std::vector<E::AttributeReferencePtr>::size_type attr_idx = 0;
@@ -675,6 +670,8 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
     key_types.push_back(&left_attribute_type);
   }
 
+  std::size_t build_cardinality = cost_model_->estimateCardinality(build_physical);
+
   // Convert the residual predicate proto.
   QueryContext::predicate_id residual_predicate_index = QueryContext::kInvalidPredicateId;
   if (physical_plan->residual_predicate()) {
@@ -835,9 +832,11 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
                                            join_operator_index,
                                            referenced_stored_build_relation,
                                            referenced_stored_probe_relation,
-                                           std::move(build_original_attribute_ids),
-                                           std::move(probe_original_attribute_ids),
-                                           join_hash_table_index);
+                                           bloom_filter_config,
+                                           std::move(build_side_bloom_filter_attribute_ids),
+                                           std::move(probe_side_bloom_filter_attribute_ids),
+                                           join_hash_table_index,
+                                           star_schema_cost_model_->estimateCardinality(build_physical));
   }
 }
 
@@ -1351,6 +1350,16 @@ void ExecutionGenerator::convertAggregate(
       findRelationInfoOutputByPhysical(physical_plan->input());
   aggr_state_proto->set_relation_id(input_relation_info->relation->getID());
 
+  const P::BloomFilterConfig &bloom_filter_config =
+      physical_plan->bloom_filter_config();
+  std::vector<attribute_id> bloom_filter_attribute_ids;
+
+  for (const auto &bf : bloom_filter_config.probe_side_bloom_filters) {
+    const CatalogAttribute *bf_catalog_attribute
+        = attribute_substitution_map_[bf.attribute->id()];
+    bloom_filter_attribute_ids.emplace_back(bf_catalog_attribute->getID());
+  }
+
   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;
@@ -1465,6 +1474,13 @@ void ExecutionGenerator::convertAggregate(
       std::forward_as_tuple(finalize_aggregation_operator_index, output_relation));
   temporary_relation_info_vec_.emplace_back(finalize_aggregation_operator_index,
                                             output_relation);
+
+  if (FLAGS_optimize_joins) {
+    execution_heuristics_->addAggregateInfo(aggregation_operator_index,
+                                            bloom_filter_config,
+                                            std::move(bloom_filter_attribute_ids),
+                                            aggr_state_index);
+  }
 }
 
 void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index d5c7b06..8358233 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -37,7 +37,6 @@
 #include "query_optimizer/QueryHandle.hpp"
 #include "query_optimizer/QueryPlan.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
-#include "query_optimizer/cost_model/SimpleCostModel.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/NamedExpression.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
@@ -423,6 +422,7 @@ class ExecutionGenerator {
    * @brief The cost model to use for creating the execution plan.
    */
   std::unique_ptr<cost::CostModel> cost_model_;
+  std::unique_ptr<cost::CostModel> star_schema_cost_model_;
 
   physical::TopLevelPlanPtr top_level_physical_plan_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/ExecutionHeuristics.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.cpp b/query_optimizer/ExecutionHeuristics.cpp
index 4fd7320..0bef716 100644
--- a/query_optimizer/ExecutionHeuristics.cpp
+++ b/query_optimizer/ExecutionHeuristics.cpp
@@ -27,6 +27,8 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/QueryContext.pb.h"
 #include "query_optimizer/QueryPlan.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -34,95 +36,106 @@
 namespace quickstep {
 namespace optimizer {
 
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+static const std::size_t kNumBitsPerByte = 8;
+DEFINE_double(bloom_num_bits_per_tuple, kNumBitsPerByte,
+    "Number of bits per tuple used to size the Bloom filter.");
+
+DEFINE_int32(bloom_num_hash_fns, 1,
+    "Number of hash functions used in the Bloom filter.");
+
 void ExecutionHeuristics::optimizeExecutionPlan(QueryPlan *query_plan,
                                                 serialization::QueryContext *query_context_proto) {
-  // Currently this only optimizes left deep joins using bloom filters.
-  // It uses a simple algorithm to discover the left deep joins.
-  // It starts with the first hash join in the plan and keeps on iterating
-  // over the next hash joins, till a probe on a different relation id is found.
-  // The set of hash joins found in this way forms a chain and can be recognized
-  // as a left deep join. It becomes a candidate for optimization.
-
-  // The optimization is done by modifying each of the build operators in the chain
-  // to generate a bloom filter on the build key during their hash table creation.
-  // The leaf-level probe operator is then modified to query all the bloom
-  // filters generated from all the build operators in the chain. These
-  // bloom filters are queried to test the membership of the probe key
-  // just prior to probing the hash table.
-
-  QueryPlan::DAGNodeIndex origin_node = 0;
-  while (origin_node < hash_joins_.size() - 1) {
-    std::vector<std::size_t> chained_nodes;
-    chained_nodes.push_back(origin_node);
-    for (std::size_t i = origin_node + 1; i < hash_joins_.size(); ++i) {
-      const relation_id checked_relation_id = hash_joins_[origin_node].referenced_stored_probe_relation_->getID();
-      const relation_id expected_relation_id = hash_joins_[i].referenced_stored_probe_relation_->getID();
-      if (checked_relation_id == expected_relation_id) {
-        chained_nodes.push_back(i);
-      } else {
-        break;
-      }
+  std::map<std::pair<E::ExprId, P::PhysicalPtr>,
+           std::pair<QueryContext::bloom_filter_id, QueryPlan::DAGNodeIndex>> bloom_filter_map;
+  for (const auto &info : hash_joins_) {
+    auto *hash_table_proto =
+        query_context_proto->mutable_join_hash_tables(info.join_hash_table_id_);
+    const auto &bloom_filter_config = info.bloom_filter_config_;
+
+    for (std::size_t i = 0; i < info.build_side_bloom_filter_ids_.size(); ++i) {
+      const QueryContext::bloom_filter_id bloom_filter_id = query_context_proto->bloom_filters_size();
+      serialization::BloomFilter *bloom_filter_proto = query_context_proto->add_bloom_filters();
+      setBloomFilterProperties(bloom_filter_proto, info.estimated_build_relation_cardinality_);
+
+      const auto &build_side_bf =
+          bloom_filter_config.build_side_bloom_filters[i];
+      bloom_filter_map.emplace(
+          std::make_pair(build_side_bf.attribute->id(),
+                         bloom_filter_config.builder),
+          std::make_pair(bloom_filter_id, info.build_operator_index_));
+
+      auto *build_side_bloom_filter = hash_table_proto->add_build_side_bloom_filters();
+      build_side_bloom_filter->set_bloom_filter_id(bloom_filter_id);
+      build_side_bloom_filter->set_attr_id(info.build_side_bloom_filter_ids_[i]);
+      std::cerr << "Build " << build_side_bf.attribute->toString()
+                << " @" << bloom_filter_config.builder << "\n";
     }
+  }
 
-    // Only chains of length greater than one are suitable candidates for semi-join optimization.
-    if (chained_nodes.size() > 1) {
-      std::unordered_map<QueryContext::bloom_filter_id, std::vector<attribute_id>> probe_bloom_filter_info;
-      for (const std::size_t node : chained_nodes) {
-        // Provision for a new bloom filter to be used by the build operator.
-        const QueryContext::bloom_filter_id bloom_filter_id =  query_context_proto->bloom_filters_size();
-        serialization::BloomFilter *bloom_filter_proto = query_context_proto->add_bloom_filters();
-
-        // Modify the bloom filter properties based on the statistics of the relation.
-        setBloomFilterProperties(bloom_filter_proto, hash_joins_[node].referenced_stored_build_relation_);
-
-        // Add build-side bloom filter information to the corresponding hash table proto.
-        query_context_proto->mutable_join_hash_tables(hash_joins_[node].join_hash_table_id_)
-            ->add_build_side_bloom_filter_id(bloom_filter_id);
-
-        probe_bloom_filter_info.insert(std::make_pair(bloom_filter_id, hash_joins_[node].probe_attributes_));
-      }
-
-      // Add probe-side bloom filter information to the corresponding hash table proto for each build-side bloom filter.
-      for (const std::pair<QueryContext::bloom_filter_id, std::vector<attribute_id>>
-               &bloom_filter_info : probe_bloom_filter_info) {
-        auto *probe_side_bloom_filter =
-            query_context_proto->mutable_join_hash_tables(hash_joins_[origin_node].join_hash_table_id_)
-                                  ->add_probe_side_bloom_filters();
-        probe_side_bloom_filter->set_probe_side_bloom_filter_id(bloom_filter_info.first);
-        for (const attribute_id &probe_attribute_id : bloom_filter_info.second) {
-          probe_side_bloom_filter->add_probe_side_attr_ids(probe_attribute_id);
-        }
-      }
-
-      // Add node dependencies from chained build nodes to origin node probe.
-      for (std::size_t i = 1; i < chained_nodes.size(); ++i) {  // Note: It starts from index 1.
-        query_plan->addDirectDependency(hash_joins_[origin_node].join_operator_index_,
-                                        hash_joins_[origin_node + i].build_operator_index_,
-                                        true /* is_pipeline_breaker */);
-      }
+  for (const auto &info : hash_joins_) {
+    auto *hash_table_proto =
+        query_context_proto->mutable_join_hash_tables(info.join_hash_table_id_);
+    const auto &bloom_filter_config = info.bloom_filter_config_;
+
+    for (std::size_t i = 0; i < info.probe_side_bloom_filter_ids_.size(); ++i) {
+      auto *probe_side_bloom_filter = hash_table_proto->add_probe_side_bloom_filters();
+      const auto &probe_side_bf =
+          bloom_filter_config.probe_side_bloom_filters[i];
+      std::cerr << "HashJoin probe " << probe_side_bf.attribute->toString()
+                << " @" << probe_side_bf.builder << "\n";
+
+      const auto &build_side_info =
+           bloom_filter_map.at(
+               std::make_pair(probe_side_bf.source_attribute->id(),
+                              probe_side_bf.builder));
+      probe_side_bloom_filter->set_bloom_filter_id(build_side_info.first);
+      probe_side_bloom_filter->set_attr_id(info.probe_side_bloom_filter_ids_[i]);
+//      std::cerr << "HashJoin probe attr_id = " << info.probe_side_bloom_filter_ids_[i] << "\n";
+
+      query_plan->addDirectDependency(info.join_operator_index_,
+                                      build_side_info.second,
+                                      true /* is_pipeline_breaker */);
     }
+  }
 
-    // Update the origin node.
-    origin_node = chained_nodes.back() + 1;
+  for (const auto &info : aggregates_) {
+    auto *aggregate_proto =
+        query_context_proto->mutable_aggregation_states(info.aggregate_state_id_);
+    const auto &bloom_filter_config = info.bloom_filter_config_;
+
+    for (std::size_t i = 0; i < info.bloom_filter_ids_.size(); ++i) {
+      auto *bloom_filter = aggregate_proto->add_bloom_filters();
+      const auto &bf =
+          bloom_filter_config.probe_side_bloom_filters[i];
+      std::cerr << "Aggregate probe " << bf.attribute->toString()
+                << " @" << bf.builder << "\n";
+
+      const auto &build_side_info =
+           bloom_filter_map.at(
+               std::make_pair(bf.source_attribute->id(),
+                              bf.builder));
+      bloom_filter->set_bloom_filter_id(build_side_info.first);
+      bloom_filter->set_attr_id(info.bloom_filter_ids_[i]);
+//      std::cerr << "Aggregate probe attr_id = "
+//                << info.bloom_filter_ids_[i] << "\n";
+
+      query_plan->addDirectDependency(info.aggregate_operator_index_,
+                                      build_side_info.second,
+                                      true /* is_pipeline_breaker */);
+    }
   }
 }
 
 void ExecutionHeuristics::setBloomFilterProperties(serialization::BloomFilter *bloom_filter_proto,
-                                                   const CatalogRelation *relation) {
-  const std::size_t cardinality = relation->estimateTupleCardinality();
-  if (cardinality < kOneThousand) {
-    bloom_filter_proto->set_bloom_filter_size(kOneThousand / kCompressionFactor);
-    bloom_filter_proto->set_number_of_hashes(kVeryLowSparsityHash);
-  } else if (cardinality < kTenThousand) {
-    bloom_filter_proto->set_bloom_filter_size(kTenThousand / kCompressionFactor);
-    bloom_filter_proto->set_number_of_hashes(kLowSparsityHash);
-  } else if (cardinality < kHundredThousand) {
-    bloom_filter_proto->set_bloom_filter_size(kHundredThousand / kCompressionFactor);
-    bloom_filter_proto->set_number_of_hashes(kMediumSparsityHash);
-  } else {
-    bloom_filter_proto->set_bloom_filter_size(kMillion / kCompressionFactor);
-    bloom_filter_proto->set_number_of_hashes(kHighSparsityHash);
-  }
+                                                   const std::size_t cardinality) {
+  bloom_filter_proto->set_bloom_filter_size(
+      BloomFilter::getNearestAllowedSize(
+          (FLAGS_bloom_num_bits_per_tuple * cardinality) / kNumBitsPerByte));
+//  std::cerr << "bf size = " << bloom_filter_proto->bloom_filter_size() << "\n";
+  bloom_filter_proto->set_number_of_hashes(FLAGS_bloom_num_hash_fns);
 }
 
 }  // namespace optimizer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/ExecutionHeuristics.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.hpp b/query_optimizer/ExecutionHeuristics.hpp
index 8ad3b7a..9e5efc5 100644
--- a/query_optimizer/ExecutionHeuristics.hpp
+++ b/query_optimizer/ExecutionHeuristics.hpp
@@ -27,6 +27,7 @@
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryContext.pb.h"
 #include "query_optimizer/QueryPlan.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -51,7 +52,7 @@ class ExecutionHeuristics {
   static const std::size_t kHundredThousand = 100000;
   static const std::size_t kMillion = 1000000;
 
-  static const std::size_t kCompressionFactor = 10;
+  static const std::size_t kCompressionFactor = 1;
 
   static const std::size_t kVeryLowSparsityHash = 1;
   static const std::size_t kLowSparsityHash = 2;
@@ -67,25 +68,48 @@ class ExecutionHeuristics {
                  const QueryPlan::DAGNodeIndex join_operator_index,
                  const CatalogRelation *referenced_stored_build_relation,
                  const CatalogRelation *referenced_stored_probe_relation,
-                 std::vector<attribute_id> &&build_attributes,
-                 std::vector<attribute_id> &&probe_attributes,
-                 const QueryContext::join_hash_table_id join_hash_table_id)
+                 const physical::BloomFilterConfig &bloom_filter_config,
+                 std::vector<attribute_id> &&build_side_bloom_filter_ids,
+                 std::vector<attribute_id> &&probe_side_bloom_filter_ids,
+                 const QueryContext::join_hash_table_id join_hash_table_id,
+                 const std::size_t estimated_build_relation_cardinality)
         : build_operator_index_(build_operator_index),
           join_operator_index_(join_operator_index),
           referenced_stored_build_relation_(referenced_stored_build_relation),
           referenced_stored_probe_relation_(referenced_stored_probe_relation),
-          build_attributes_(std::move(build_attributes)),
-          probe_attributes_(std::move(probe_attributes)),
-          join_hash_table_id_(join_hash_table_id) {
+          bloom_filter_config_(bloom_filter_config),
+          build_side_bloom_filter_ids_(std::move(build_side_bloom_filter_ids)),
+          probe_side_bloom_filter_ids_(std::move(probe_side_bloom_filter_ids)),
+          join_hash_table_id_(join_hash_table_id),
+          estimated_build_relation_cardinality_(estimated_build_relation_cardinality) {
     }
 
     const QueryPlan::DAGNodeIndex build_operator_index_;
     const QueryPlan::DAGNodeIndex join_operator_index_;
     const CatalogRelation *referenced_stored_build_relation_;
     const CatalogRelation *referenced_stored_probe_relation_;
-    const std::vector<attribute_id> build_attributes_;
-    const std::vector<attribute_id> probe_attributes_;
+    const physical::BloomFilterConfig &bloom_filter_config_;
+    const std::vector<attribute_id> build_side_bloom_filter_ids_;
+    const std::vector<attribute_id> probe_side_bloom_filter_ids_;
     const QueryContext::join_hash_table_id join_hash_table_id_;
+    const std::size_t estimated_build_relation_cardinality_;
+  };
+
+  struct AggregateInfo {
+    AggregateInfo(const QueryPlan::DAGNodeIndex aggregate_operator_index,
+                  const physical::BloomFilterConfig &bloom_filter_config,
+                  std::vector<attribute_id> &&bloom_filter_ids,
+                  const QueryContext::aggregation_state_id aggregate_state_id)
+        : aggregate_operator_index_(aggregate_operator_index),
+          bloom_filter_config_(bloom_filter_config),
+          bloom_filter_ids_(bloom_filter_ids),
+          aggregate_state_id_(aggregate_state_id) {
+    }
+
+    const QueryPlan::DAGNodeIndex aggregate_operator_index_;
+    const physical::BloomFilterConfig &bloom_filter_config_;
+    const std::vector<attribute_id> bloom_filter_ids_;
+    const QueryContext::aggregation_state_id aggregate_state_id_;
   };
 
 
@@ -111,16 +135,30 @@ class ExecutionHeuristics {
                               const QueryPlan::DAGNodeIndex join_operator_index,
                               const CatalogRelation *referenced_stored_build_relation,
                               const CatalogRelation *referenced_stored_probe_relation,
-                              std::vector<attribute_id> &&build_attributes,
-                              std::vector<attribute_id> &&probe_attributes,
-                              const QueryContext::join_hash_table_id join_hash_table_id) {
-    hash_joins_.push_back(HashJoinInfo(build_operator_index,
-                                       join_operator_index,
-                                       referenced_stored_build_relation,
-                                       referenced_stored_probe_relation,
-                                       std::move(build_attributes),
-                                       std::move(probe_attributes),
-                                       join_hash_table_id));
+                              const physical::BloomFilterConfig &bloom_filter_config,
+                              std::vector<attribute_id> &&build_side_bloom_filter_ids,
+                              std::vector<attribute_id> &&probe_side_bloom_filter_ids,
+                              const QueryContext::join_hash_table_id join_hash_table_id,
+                              const std::size_t estimated_build_relation_cardinality) {
+    hash_joins_.emplace_back(build_operator_index,
+                             join_operator_index,
+                             referenced_stored_build_relation,
+                             referenced_stored_probe_relation,
+                             bloom_filter_config,
+                             std::move(build_side_bloom_filter_ids),
+                             std::move(probe_side_bloom_filter_ids),
+                             join_hash_table_id,
+                             estimated_build_relation_cardinality);
+  }
+
+  inline void addAggregateInfo(const QueryPlan::DAGNodeIndex aggregate_operator_index,
+                               const physical::BloomFilterConfig &bloom_filter_config,
+                               std::vector<attribute_id> &&bloom_filter_ids,
+                               const QueryContext::aggregation_state_id aggregate_state_id) {
+    aggregates_.emplace_back(aggregate_operator_index,
+                             bloom_filter_config,
+                             std::move(bloom_filter_ids),
+                             aggregate_state_id);
   }
 
   /**
@@ -141,10 +179,11 @@ class ExecutionHeuristics {
    * @param relation The catalog relation on which bloom filter is being built.
    **/
   void setBloomFilterProperties(serialization::BloomFilter *bloom_filter_proto,
-                                const CatalogRelation *relation);
+                                const std::size_t cardinality);
 
  private:
   std::vector<HashJoinInfo> hash_joins_;
+  std::vector<AggregateInfo> aggregates_;
 
   DISALLOW_COPY_AND_ASSIGN(ExecutionHeuristics);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 8f19702..9ee685d 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -26,6 +26,7 @@
 #include "query_optimizer/Validator.hpp"
 #include "query_optimizer/logical/Logical.hpp"
 #include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/AttachBloomFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
@@ -96,10 +97,12 @@ P::PhysicalPtr PhysicalGenerator::generateInitialPlan(
 P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   std::vector<std::unique_ptr<Rule<P::Physical>>> rules;
   if (FLAGS_reorder_hash_joins) {
+    rules.emplace_back(new PruneColumns());
     rules.emplace_back(new StarSchemaHashJoinOrderOptimization());
   }
   rules.emplace_back(new PruneColumns());
-  rules.emplace_back(new SwapProbeBuild());
+  // rules.emplace_back(new SwapProbeBuild());
+  rules.emplace_back(new AttachBloomFilters());
 
   for (std::unique_ptr<Rule<P::Physical>> &rule : rules) {
     physical_plan_ = rule->apply(physical_plan_);
@@ -110,7 +113,7 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   DVLOG(4) << "Optimized physical plan:\n" << physical_plan_->toString();
 
   if (FLAGS_visualize_plan) {
-  quickstep::PlanVisualizer plan_visualizer;
+    quickstep::PlanVisualizer plan_visualizer;
     std::cerr << "\n" << plan_visualizer.visualize(physical_plan_) << "\n";
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index f313c90..45e2f00 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -88,7 +88,7 @@ std::size_t SimpleCostModel::estimateCardinalityForTopLevelPlan(
 
 std::size_t SimpleCostModel::estimateCardinalityForTableReference(
     const P::TableReferencePtr &physical_plan) {
-  return physical_plan->relation()->estimateTupleCardinality();
+  return physical_plan->relation()->getStatistics().getNumTuples();
 }
 
 std::size_t SimpleCostModel::estimateCardinalityForSelection(
@@ -119,7 +119,7 @@ std::size_t SimpleCostModel::estimateCardinalityForAggregate(
     return 1;
   }
   return std::max(static_cast<std::size_t>(1),
-                  estimateCardinality(physical_plan->input()) / 10);
+                  estimateCardinality(physical_plan->input()));
 }
 
 std::size_t SimpleCostModel::estimateCardinalityForWindowAggregate(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 911a765..9eea27c 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -123,12 +123,26 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForTableGenerator(
 
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForHashJoin(
     const P::HashJoinPtr &physical_plan) {
-  std::size_t left_cardinality = estimateCardinality(physical_plan->left());
-  std::size_t right_cardinality = estimateCardinality(physical_plan->right());
-  double left_selectivity = estimateSelectivity(physical_plan->left());
-  double right_selectivity = estimateSelectivity(physical_plan->right());
-  return std::max(static_cast<std::size_t>(left_cardinality * right_selectivity) + 1,
-                  static_cast<std::size_t>(right_cardinality * left_selectivity) + 1);
+  const P::PhysicalPtr &left_child = physical_plan->left();
+  const P::PhysicalPtr &right_child = physical_plan->right();
+
+  std::size_t left_cardinality = estimateCardinality(left_child);
+  std::size_t right_cardinality = estimateCardinality(right_child);
+
+  std::size_t estimated_cardinality = std::max(left_cardinality, right_cardinality);
+  if (left_child->impliesUniqueAttributes(physical_plan->left_join_attributes())) {
+    double left_selectivity = estimateSelectivity(left_child);
+    estimated_cardinality =
+        std::min(estimated_cardinality,
+                 static_cast<std::size_t>(right_cardinality * left_selectivity));
+  }
+  if (right_child->impliesUniqueAttributes(physical_plan->right_join_attributes())) {
+    double right_selectivity = estimateSelectivity(right_child);
+    estimated_cardinality =
+        std::min(estimated_cardinality,
+                 static_cast<std::size_t>(left_cardinality * right_selectivity));
+  }
+  return estimated_cardinality;
 }
 
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForNestedLoopsJoin(
@@ -143,7 +157,7 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForAggregate(
     return 1;
   }
   return std::max(static_cast<std::size_t>(1),
-                  estimateCardinality(physical_plan->input()) / 10);
+                  estimateCardinality(physical_plan->input()) / 100);
 }
 
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForWindowAggregate(
@@ -161,8 +175,14 @@ double StarSchemaSimpleCostModel::estimateSelectivity(
     case P::PhysicalType::kHashJoin: {
       const P::HashJoinPtr &hash_join =
           std::static_pointer_cast<const P::HashJoin>(physical_plan);
-      return std::min(estimateSelectivity(hash_join->left()),
-                      estimateSelectivity(hash_join->right()));
+      double left_selectivity = estimateSelectivity(hash_join->left());
+      double right_selectivity = estimateSelectivity(hash_join->right());
+      double min_sel = std::min(left_selectivity, right_selectivity);
+      double max_sel = std::max(left_selectivity, right_selectivity);
+      if (max_sel < 1) {
+        min_sel *= std::max(max_sel, 0.9);
+      }
+      return min_sel;
     }
     case P::PhysicalType::kNestedLoopsJoin: {
       const P::NestedLoopsJoinPtr &nested_loop_join =
@@ -215,7 +235,7 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
     case E::ExpressionType::kComparisonExpression: {
       // Case 1 - Number of distinct values statistics available
       //   Case 1.1 - Equality comparison: 1.0 / num_distinct_values
-      //   Case 1.2 - Otherwise: 5.0 / num_distinct_values
+      //   Case 1.2 - Otherwise: 0.5
       // Case 2 - Number of distinct values statistics not available
       //   Case 2.1 - Equality comparison: 0.1
       //   Case 2.2 - Otherwise: 0.5
@@ -231,7 +251,7 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
           double unit_selectivity = 1.0 / it->second;
           return comparison_expression->isEqualityComparisonPredicate()
                      ? unit_selectivity
-                     : std::min(0.5, unit_selectivity * 5.0);
+                     : 0.5;
         }
       }
 

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/Aggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Aggregate.cpp b/query_optimizer/physical/Aggregate.cpp
index 35476af..6d07a52 100644
--- a/query_optimizer/physical/Aggregate.cpp
+++ b/query_optimizer/physical/Aggregate.cpp
@@ -87,6 +87,11 @@ std::vector<E::AttributeReferencePtr> Aggregate::getReferencedAttributes()
   return referenced_attributes;
 }
 
+bool Aggregate::impliesUniqueAttributes(
+    const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+  return E::SubsetOfExpressions(grouping_expressions_, attributes);
+}
+
 void Aggregate::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/Aggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Aggregate.hpp b/query_optimizer/physical/Aggregate.hpp
index de36cc3..855d783 100644
--- a/query_optimizer/physical/Aggregate.hpp
+++ b/query_optimizer/physical/Aggregate.hpp
@@ -100,6 +100,13 @@ class Aggregate : public Physical {
     return false;
   }
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
+  const BloomFilterConfig &bloom_filter_config() const {
+    return bloom_filter_config_;
+  }
+
   /**
    * @brief Creates an Aggregate physical node.
    *
@@ -113,9 +120,14 @@ class Aggregate : public Physical {
       PhysicalPtr input,
       const std::vector<expressions::NamedExpressionPtr> &grouping_expressions,
       const std::vector<expressions::AliasPtr> &aggregate_expressions,
-      const expressions::PredicatePtr &filter_predicate) {
+      const expressions::PredicatePtr &filter_predicate,
+      const BloomFilterConfig bloom_filter_config = BloomFilterConfig()) {
     return AggregatePtr(
-        new Aggregate(input, grouping_expressions, aggregate_expressions, filter_predicate));
+        new Aggregate(input,
+                      grouping_expressions,
+                      aggregate_expressions,
+                      filter_predicate,
+                      bloom_filter_config));
   }
 
  protected:
@@ -132,11 +144,13 @@ class Aggregate : public Physical {
       PhysicalPtr input,
       const std::vector<expressions::NamedExpressionPtr> &grouping_expressions,
       const std::vector<expressions::AliasPtr> &aggregate_expressions,
-      const expressions::PredicatePtr &filter_predicate)
+      const expressions::PredicatePtr &filter_predicate,
+      const BloomFilterConfig &bloom_filter_config)
       : input_(input),
         grouping_expressions_(grouping_expressions),
         aggregate_expressions_(aggregate_expressions),
-        filter_predicate_(filter_predicate) {
+        filter_predicate_(filter_predicate),
+        bloom_filter_config_(bloom_filter_config) {
     addChild(input_);
   }
 
@@ -144,6 +158,7 @@ class Aggregate : public Physical {
   std::vector<expressions::NamedExpressionPtr> grouping_expressions_;
   std::vector<expressions::AliasPtr> aggregate_expressions_;
   expressions::PredicatePtr filter_predicate_;
+  BloomFilterConfig bloom_filter_config_;
 
   DISALLOW_COPY_AND_ASSIGN(Aggregate);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/HashJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.cpp b/query_optimizer/physical/HashJoin.cpp
index e186072..883c87a 100644
--- a/query_optimizer/physical/HashJoin.cpp
+++ b/query_optimizer/physical/HashJoin.cpp
@@ -85,6 +85,15 @@ bool HashJoin::maybeCopyWithPrunedExpressions(
   return false;
 }
 
+bool HashJoin::impliesUniqueAttributes(
+    const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+  return (left()->impliesUniqueAttributes(left_join_attributes_)
+              && right()->impliesUniqueAttributes(attributes))
+         || (right()->impliesUniqueAttributes(right_join_attributes_)
+                 && left()->impliesUniqueAttributes(attributes));
+
+}
+
 void HashJoin::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,
@@ -106,6 +115,24 @@ void HashJoin::getFieldStringItems(
   container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(left_join_attributes_));
   container_child_field_names->push_back("right_join_attributes");
   container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(right_join_attributes_));
+
+  if (!bloom_filter_config_.build_side_bloom_filters.empty()) {
+    container_child_field_names->push_back("build_side_bloom_filters");
+    container_child_fields->emplace_back();
+    auto &container = container_child_fields->back();
+    for (const auto& bf : bloom_filter_config_.build_side_bloom_filters) {
+      container.emplace_back(bf.attribute);
+    }
+  }
+
+  if (!bloom_filter_config_.probe_side_bloom_filters.empty()) {
+    container_child_field_names->push_back("probe_side_bloom_filters");
+    container_child_fields->emplace_back();
+    auto &container = container_child_fields->back();
+    for (const auto& bf : bloom_filter_config_.probe_side_bloom_filters) {
+      container.emplace_back(bf.attribute);
+    }
+  }
 }
 
 }  // namespace physical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/HashJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.hpp b/query_optimizer/physical/HashJoin.hpp
index c513f77..a830d0b 100644
--- a/query_optimizer/physical/HashJoin.hpp
+++ b/query_optimizer/physical/HashJoin.hpp
@@ -116,7 +116,8 @@ class HashJoin : public BinaryJoin {
                   right_join_attributes_,
                   residual_predicate_,
                   project_expressions(),
-                  join_type_);
+                  join_type_,
+                  bloom_filter_config_);
   }
 
   std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override;
@@ -125,6 +126,13 @@ class HashJoin : public BinaryJoin {
       const expressions::UnorderedNamedExpressionSet &referenced_expressions,
       PhysicalPtr *output) const override;
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
+  const BloomFilterConfig &bloom_filter_config() const {
+    return bloom_filter_config_;
+  }
+
   /**
    * @brief Creates a physical HashJoin. The left/right operand does not correspond to
    *        probe/build operand.
@@ -145,7 +153,8 @@ class HashJoin : public BinaryJoin {
       const std::vector<expressions::AttributeReferencePtr> &right_join_attributes,
       const expressions::PredicatePtr &residual_predicate,
       const std::vector<expressions::NamedExpressionPtr> &project_expressions,
-      const JoinType join_type) {
+      const JoinType join_type,
+      const BloomFilterConfig bloom_filter_config = BloomFilterConfig()) {
     return HashJoinPtr(
         new HashJoin(left,
                      right,
@@ -153,7 +162,8 @@ class HashJoin : public BinaryJoin {
                      right_join_attributes,
                      residual_predicate,
                      project_expressions,
-                     join_type));
+                     join_type,
+                     bloom_filter_config));
   }
 
  protected:
@@ -173,18 +183,21 @@ class HashJoin : public BinaryJoin {
       const std::vector<expressions::AttributeReferencePtr> &right_join_attributes,
       const expressions::PredicatePtr &residual_predicate,
       const std::vector<expressions::NamedExpressionPtr> &project_expressions,
-      const JoinType join_type)
+      const JoinType join_type,
+      const BloomFilterConfig &bloom_filter_config)
       : BinaryJoin(left, right, project_expressions),
         left_join_attributes_(left_join_attributes),
         right_join_attributes_(right_join_attributes),
         residual_predicate_(residual_predicate),
-        join_type_(join_type) {
+        join_type_(join_type),
+        bloom_filter_config_(bloom_filter_config) {
   }
 
   std::vector<expressions::AttributeReferencePtr> left_join_attributes_;
   std::vector<expressions::AttributeReferencePtr> right_join_attributes_;
   expressions::PredicatePtr residual_predicate_;
   JoinType join_type_;
+  BloomFilterConfig bloom_filter_config_;
 
   DISALLOW_COPY_AND_ASSIGN(HashJoin);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/Physical.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Physical.hpp b/query_optimizer/physical/Physical.hpp
index 4bed593..f70d142 100644
--- a/query_optimizer/physical/Physical.hpp
+++ b/query_optimizer/physical/Physical.hpp
@@ -41,6 +41,56 @@ namespace physical {
 class Physical;
 typedef std::shared_ptr<const Physical> PhysicalPtr;
 
+struct BloomFilterConfig {
+  struct BuildSide {
+    BuildSide(const expressions::AttributeReferencePtr &attribute_in)
+        : attribute(attribute_in) {
+    }
+    expressions::AttributeReferencePtr attribute;
+  };
+  struct ProbeSide {
+    ProbeSide(const expressions::AttributeReferencePtr &attribute_in,
+              const expressions::AttributeReferencePtr &source_attribute_in,
+              const physical::PhysicalPtr &builder_in)
+        : attribute(attribute_in),
+          source_attribute(source_attribute_in),
+          builder(builder_in) {
+    }
+    expressions::AttributeReferencePtr attribute;
+    expressions::AttributeReferencePtr source_attribute;
+    PhysicalPtr builder;
+  };
+  BloomFilterConfig() {}
+  BloomFilterConfig(const PhysicalPtr &builder_in)
+      : builder(builder_in) {
+  }
+  BloomFilterConfig(const PhysicalPtr &builder_in,
+                    const std::vector<BuildSide> &build_side_bloom_filters_in,
+                    const std::vector<ProbeSide> &probe_side_bloom_filters_in)
+      : builder(builder_in),
+        build_side_bloom_filters(build_side_bloom_filters_in),
+        probe_side_bloom_filters(probe_side_bloom_filters_in) {
+  }
+  void addBuildSideBloomFilter(const expressions::AttributeReferencePtr &attribute_in) {
+    for (const auto &build_bf : build_side_bloom_filters) {
+      if (attribute_in == build_bf.attribute) {
+        return;
+      }
+    }
+    build_side_bloom_filters.emplace_back(attribute_in);
+  }
+  void addProbeSideBloomFilter(const expressions::AttributeReferencePtr &attribute_in,
+                               const expressions::AttributeReferencePtr &source_attribute_in,
+                               const physical::PhysicalPtr &builder_in) {
+    probe_side_bloom_filters.emplace_back(attribute_in,
+                                          source_attribute_in,
+                                          builder_in);
+  }
+  PhysicalPtr builder;
+  std::vector<BuildSide> build_side_bloom_filters;
+  std::vector<ProbeSide> probe_side_bloom_filters;
+};
+
 /**
  * @brief Base class for physical plan nodes.
  */
@@ -86,6 +136,11 @@ class Physical : public OptimizerTree<Physical> {
       const expressions::UnorderedNamedExpressionSet &referenced_expressions,
       PhysicalPtr *output) const = 0;
 
+  virtual bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+    return false;
+  }
+
  protected:
   /**
    * @brief Constructor.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/Selection.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.cpp b/query_optimizer/physical/Selection.cpp
index 36ade04..73af500 100644
--- a/query_optimizer/physical/Selection.cpp
+++ b/query_optimizer/physical/Selection.cpp
@@ -82,6 +82,12 @@ bool Selection::maybeCopyWithPrunedExpressions(
   return false;
 }
 
+bool Selection::impliesUniqueAttributes(
+    const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+  return input()->impliesUniqueAttributes(attributes);
+}
+
+
 void Selection::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/Selection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.hpp b/query_optimizer/physical/Selection.hpp
index b6874a1..f42fc71 100644
--- a/query_optimizer/physical/Selection.hpp
+++ b/query_optimizer/physical/Selection.hpp
@@ -86,6 +86,9 @@ class Selection : public Physical {
       const expressions::UnorderedNamedExpressionSet &referenced_attributes,
       PhysicalPtr *output) const override;
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
   /**
    * @brief Creates a Selection.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/TableReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableReference.cpp b/query_optimizer/physical/TableReference.cpp
index bfd6464..399ee51 100644
--- a/query_optimizer/physical/TableReference.cpp
+++ b/query_optimizer/physical/TableReference.cpp
@@ -20,6 +20,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 
 #include <string>
+#include <set>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
@@ -32,6 +33,23 @@ namespace physical {
 
 namespace E = ::quickstep::optimizer::expressions;
 
+bool TableReference::impliesUniqueAttributes(
+    const std::vector<expressions::AttributeReferencePtr> &attributes) const {
+  std::set<E::ExprId> attr_ids;
+  for (const auto &attr : attributes) {
+    attr_ids.emplace(attr->id());
+  }
+
+  std::set<attribute_id> rel_attr_ids;
+  for (std::size_t i = 0; i < attribute_list_.size(); ++i) {
+    if (attr_ids.find(attribute_list_[i]->id()) != attr_ids.end()) {
+      rel_attr_ids.emplace(i);
+    }
+  }
+
+  return relation_->getConstraints().impliesUniqueAttributes(rel_attr_ids);
+}
+
 void TableReference::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/physical/TableReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableReference.hpp b/query_optimizer/physical/TableReference.hpp
index 638d73b..7643c07 100644
--- a/query_optimizer/physical/TableReference.hpp
+++ b/query_optimizer/physical/TableReference.hpp
@@ -90,6 +90,9 @@ class TableReference : public Physical {
     return false;
   }
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
   /**
    * @brief Creates a TableReference.
    *


[16/16] incubator-quickstep git commit: Initial work on better estimation

Posted by ji...@apache.org.
Initial work on better estimation


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

Branch: refs/heads/LIP-for-tpch
Commit: 39f574999a280e0fbcb1792db84f584deca20675
Parents: 9b90665
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Aug 11 15:20:53 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500

----------------------------------------------------------------------
 cli/QuickstepCli.cpp                            |  8 ++---
 .../cost_model/StarSchemaSimpleCostModel.cpp    | 36 ++++++++++++++++++++
 .../cost_model/StarSchemaSimpleCostModel.hpp    | 10 ++++++
 utility/PlanVisualizer.cpp                      | 25 ++++++++++++--
 4 files changed, 73 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/39f57499/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index bf0e2c7..e7ffc0c 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -347,10 +347,10 @@ int main(int argc, char* argv[]) {
 
 //  addPrimaryKeyInfoForTPCHTables(query_processor->getDefaultDatabase());
 //  addPrimaryKeyInfoForSSBTables(query_processor->getDefaultDatabase());
-  std::string proto_str;
-  google::protobuf::TextFormat::PrintToString(
-      query_processor->getDefaultDatabase()->getProto(), &proto_str);
-  std::cerr << proto_str << "\n";
+//  std::string proto_str;
+//  google::protobuf::TextFormat::PrintToString(
+//      query_processor->getDefaultDatabase()->getProto(), &proto_str);
+//  std::cerr << proto_str << "\n";
 //  query_processor->markCatalogAltered();
 //  query_processor->saveCatalog();
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/39f57499/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 9eea27c..ba7a3c6 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -25,6 +25,8 @@
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogRelationStatistics.hpp"
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
@@ -283,6 +285,40 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
   return 1.0;
 }
 
+
+void StarSchemaSimpleCostModel::getStatistics(
+    const physical::PhysicalPtr &physical_plan,
+    const expressions::AttributeReferencePtr &attribute,
+    const CatalogRelationStatistics** stat,
+    attribute_id* attr_id) {
+  switch (physical_plan->getPhysicalType()) {
+    case P::PhysicalType::kTableReference: {
+      const P::TableReferencePtr table_reference =
+          std::static_pointer_cast<const P::TableReference>(physical_plan);
+      const CatalogRelation *catalog_relation = table_reference->relation();
+      const std::vector<E::AttributeReferencePtr> &attributes =
+          table_reference->attribute_list();
+      for (std::size_t i = 0; i < attributes.size(); ++i) {
+        if (attributes[i]->id() == attribute->id()) {
+          *stat = &catalog_relation->getStatistics();
+          *attr_id = i;
+        }
+      }
+    }
+    default:
+      break;
+  }
+
+  for (const auto &child : physical_plan->children()) {
+    for (const auto &attr : child->getOutputAttributes()) {
+      if (attr->id() == attribute->id()) {
+        getStatistics(child, attribute, stat, attr_id);
+      }
+    }
+  }
+}
+
+
 }  // namespace cost
 }  // namespace optimizer
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/39f57499/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index 4314b92..fdd2b36 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -24,7 +24,9 @@
 #include <unordered_map>
 #include <vector>
 
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
@@ -39,6 +41,9 @@
 #include "utility/Macros.hpp"
 
 namespace quickstep {
+
+class CatalogRelationStatistics;
+
 namespace optimizer {
 namespace cost {
 
@@ -75,6 +80,11 @@ class StarSchemaSimpleCostModel : public CostModel {
    */
   double estimateSelectivity(const physical::PhysicalPtr &physical_plan);
 
+  void getStatistics(const physical::PhysicalPtr &physical_plan,
+                     const expressions::AttributeReferencePtr &attribute,
+                     const CatalogRelationStatistics** stat,
+                     attribute_id* attr_id);
+
  private:
   std::size_t estimateCardinalityForTopLevelPlan(
       const physical::TopLevelPlanPtr &physical_plan);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/39f57499/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index 9af00b4..2039a69 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -28,7 +28,8 @@
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
-
+#include "catalog/CatalogRelationStatistics.hpp"
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
@@ -131,7 +132,27 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
 
     for (const auto &attr : child->getOutputAttributes()) {
       if (referenced_ids.find(attr->id()) != referenced_ids.end()) {
-        edge_info.labels.emplace_back(attr->attribute_alias());
+        std::string attr_info = attr->attribute_alias();
+        const CatalogRelationStatistics *stat = nullptr;
+        attribute_id attr_id = 0;
+        cost_model_->getStatistics(child, attr, &stat, &attr_id);
+        if (stat != nullptr) {
+          if (stat->hasNumDistinctValues(attr_id)) {
+            attr_info.append(", # distinct = ");
+            attr_info.append(std::to_string(static_cast<std::size_t>(
+                stat->getNumDistinctValues(attr_id) * cost_model_->estimateSelectivity(child))));
+          }
+          const Type& attr_type = attr->getValueType();
+          if (stat->hasMinValue(attr_id)) {
+            attr_info.append(", min = ");
+            attr_info.append(attr_type.printValueToString(stat->getMinValue(attr_id)));
+          }
+          if (stat->hasMaxValue(attr_id)) {
+            attr_info.append(", max = ");
+            attr_info.append(attr_type.printValueToString(stat->getMaxValue(attr_id)));
+          }
+        }
+        edge_info.labels.emplace_back(attr_info);
       }
     }
   }


[04/16] incubator-quickstep git commit: Deserialized Window Aggr WorkOrder.

Posted by ji...@apache.org.
Deserialized Window Aggr WorkOrder.


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

Branch: refs/heads/LIP-for-tpch
Commit: 85e02de49205409accfef3737dadfe95aad1f5c0
Parents: 658cb61
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 23:14:08 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 23:14:08 2016 -0700

----------------------------------------------------------------------
 relational_operators/CMakeLists.txt       |  1 +
 relational_operators/WorkOrderFactory.cpp | 25 +++++++++++++++++++++++++
 2 files changed, 26 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/85e02de4/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 9696392..43a42f9 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -467,6 +467,7 @@ target_link_libraries(quickstep_relationaloperators_WorkOrderFactory
                       quickstep_relationaloperators_TableGeneratorOperator
                       quickstep_relationaloperators_TextScanOperator
                       quickstep_relationaloperators_UpdateOperator
+                      quickstep_relationaloperators_WindowAggregationOperator
                       quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_utility_Macros

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/85e02de4/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index f920cac..721d735 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -46,6 +46,7 @@
 #include "relational_operators/TableGeneratorOperator.hpp"
 #include "relational_operators/TextScanOperator.hpp"
 #include "relational_operators/UpdateOperator.hpp"
+#include "relational_operators/WindowAggregationOperator.hpp"
 #include "relational_operators/WorkOrder.pb.h"
 #include "storage/StorageBlockInfo.hpp"
 
@@ -419,6 +420,22 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           shiftboss_client_id,
           bus);
     }
+    case serialization::WINDOW_AGGREGATION: {
+      LOG(INFO) << "Creating WindowAggregationWorkOrder";
+      vector<block_id> blocks;
+      for (int i = 0; i < proto.ExtensionSize(serialization::WindowAggregationWorkOrder::block_ids); ++i) {
+        blocks.push_back(
+            proto.GetExtension(serialization::WindowAggregationWorkOrder::block_ids, i));
+      }
+
+      return new WindowAggregationWorkOrder(
+          proto.query_id(),
+          query_context->getWindowAggregationState(
+              proto.GetExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index)),
+          move(blocks),
+          query_context->getInsertDestination(
+              proto.GetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index)));
+    }
     default:
       LOG(FATAL) << "Unknown WorkOrder Type in WorkOrderFactory::ReconstructFromProto";
   }
@@ -697,6 +714,14 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
              proto.HasExtension(serialization::UpdateWorkOrder::operator_index) &&
              proto.HasExtension(serialization::UpdateWorkOrder::block_id);
     }
+    case serialization::WINDOW_AGGREGATION: {
+      return proto.HasExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index) &&
+             query_context.isValidWindowAggregationStateId(
+                 proto.GetExtension(serialization::WindowAggregationWorkOrder::window_aggr_state_index)) &&
+             proto.HasExtension(serialization::WindowAggregationWorkOrder::insert_destination_index) &&
+             query_context.isValidInsertDestinationId(
+                 proto.GetExtension(serialization::WindowAggregationWorkOrder::insert_destination_index));
+    }
     default:
       return false;
   }


[14/16] incubator-quickstep git commit: MinMaxStatistics

Posted by ji...@apache.org.
MinMaxStatistics


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

Branch: refs/heads/LIP-for-tpch
Commit: 9b90665796962baa9a80ef5240db99d6b38a9016
Parents: ebdc041
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Aug 11 03:21:30 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500

----------------------------------------------------------------------
 catalog/Catalog.proto                 | 10 +--
 catalog/CatalogRelationStatistics.cpp | 50 ++++++++++++---
 catalog/CatalogRelationStatistics.hpp | 93 +++++++++++++++++++++++-----
 cli/CommandExecutor.cpp               | 98 ++++++++++++++++++++++--------
 cli/QuickstepCli.cpp                  |  8 +--
 5 files changed, 201 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index a4cc3ce..6423a7c 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -85,13 +85,15 @@ message CatalogRelationConstraints {
 }
 
 message CatalogRelationStatistics {
-  optional fixed64 num_tuples = 1;
+  optional TypedValue num_tuples = 1;
 
-  message NumDistinctValuesEntry {
+  message ColumnStats {
     required int32 attr_id = 1;
-    required fixed64 num_distinct_values = 2;
+    optional TypedValue num_distinct_values = 2;
+    optional TypedValue min_value = 3;
+    optional TypedValue max_value = 4;
   }
-  repeated NumDistinctValuesEntry num_distinct_values_map = 2;
+  repeated ColumnStats column_stats = 2;
 }
 
 message CatalogRelationSchema {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/catalog/CatalogRelationStatistics.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.cpp b/catalog/CatalogRelationStatistics.cpp
index 6a51570..04939a5 100644
--- a/catalog/CatalogRelationStatistics.cpp
+++ b/catalog/CatalogRelationStatistics.cpp
@@ -20,30 +20,60 @@
 #include "catalog/CatalogRelationStatistics.hpp"
 
 #include "catalog/Catalog.pb.h"
+#include "types/NullType.hpp"
 
 namespace quickstep {
 
+const TypedValue CatalogRelationStatistics::kNullValue =
+    NullType::InstanceNullable().makeNullValue();
+
 CatalogRelationStatistics::CatalogRelationStatistics(
     const serialization::CatalogRelationStatistics &proto) {
   if (proto.has_num_tuples()) {
-    num_tuples_ = proto.num_tuples();
+    num_tuples_ = TypedValue::ReconstructFromProto(proto.num_tuples());
+  } else {
+    num_tuples_ = kNullValue;
   }
-  for (int i = 0; i < proto.num_distinct_values_map_size(); ++i) {
-    const auto &entry = proto.num_distinct_values_map(i);
-    num_distinct_values_map_.emplace(entry.attr_id(),
-                                     entry.num_distinct_values());
+
+  for (int i = 0; i < proto.column_stats_size(); ++i) {
+    const auto &stat_proto = proto.column_stats(i);
+    auto &stat = column_stats_[stat_proto.attr_id()];
+    if (stat_proto.has_num_distinct_values()) {
+      stat.num_distinct_values =
+          TypedValue::ReconstructFromProto(stat_proto.num_distinct_values());
+    }
+    if (stat_proto.has_min_value()) {
+      stat.min_value =
+          TypedValue::ReconstructFromProto(stat_proto.min_value());
+    }
+    if (stat_proto.has_max_value()) {
+      stat.max_value =
+          TypedValue::ReconstructFromProto(stat_proto.max_value());
+    }
   }
 }
 
 serialization::CatalogRelationStatistics CatalogRelationStatistics::getProto() const {
   serialization::CatalogRelationStatistics proto;
-  if (num_tuples_ != 0) {
-    proto.set_num_tuples(num_tuples_);
+  if (!num_tuples_.isNull()) {
+    proto.mutable_num_tuples()->CopyFrom(num_tuples_.getProto());
   }
-  for (const auto &pair : num_distinct_values_map_) {
-    auto entry = proto.add_num_distinct_values_map();
+  for (const auto &pair : column_stats_) {
+    auto entry = proto.add_column_stats();
     entry->set_attr_id(pair.first);
-    entry->set_num_distinct_values(pair.second);
+    const auto &stat = pair.second;
+    if (!stat.num_distinct_values.isNull()) {
+      entry->mutable_num_distinct_values()->CopyFrom(
+          stat.num_distinct_values.getProto());
+    }
+    if (!stat.min_value.isNull()) {
+      entry->mutable_min_value()->CopyFrom(
+          stat.min_value.getProto());
+    }
+    if (!stat.max_value.isNull()) {
+      entry->mutable_max_value()->CopyFrom(
+          stat.max_value.getProto());
+    }
   }
   return proto;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/catalog/CatalogRelationStatistics.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.hpp b/catalog/CatalogRelationStatistics.hpp
index f2056f3..532ea46 100644
--- a/catalog/CatalogRelationStatistics.hpp
+++ b/catalog/CatalogRelationStatistics.hpp
@@ -26,8 +26,13 @@
 
 #include "catalog/Catalog.pb.h"
 #include "catalog/CatalogTypedefs.hpp"
+#include "types/LongType.hpp"
+#include "types/NullType.hpp"
+#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 /** \addtogroup Catalog
@@ -44,7 +49,7 @@ class CatalogRelationStatistics {
    * @brief Constructor.
    **/
   CatalogRelationStatistics()
-      : num_tuples_(0) {}
+      : num_tuples_(kNullValue) {}
 
   /**
    * @brief Reconstruct a CatalogRelationStatistics object from its serialized
@@ -68,8 +73,12 @@ class CatalogRelationStatistics {
    *
    * @param num_tuples The number of tuples statistic.
    */
-  void setNumTuples(std::size_t num_tuples) {
-    num_tuples_ = num_tuples;
+  void setNumTuples(const std::size_t num_tuples) {
+    num_tuples_ = LongType::InstanceNonNullable().makeValue(&num_tuples);
+  }
+
+  bool hasNumTuples() const {
+    return !num_tuples_.isNull();
   }
 
   /**
@@ -78,7 +87,8 @@ class CatalogRelationStatistics {
    * @return The number of tuples. Returns 0 if the statistic is not set.
    */
   std::size_t getNumTuples() const {
-    return num_tuples_;
+    DCHECK(hasNumTuples());
+    return num_tuples_.getLiteral<std::int64_t>();
   }
 
   /**
@@ -87,8 +97,15 @@ class CatalogRelationStatistics {
    * @param attr_id The id of the column.
    * @param num_distinct_values The number of distinct values statistic.
    */
-  void setNumDistinctValues(attribute_id attr_id, std::size_t num_distinct_values) {
-    num_distinct_values_map_[attr_id] = num_distinct_values;
+  void setNumDistinctValues(const attribute_id attr_id,
+                            const std::size_t num_distinct_values) {
+    column_stats_[attr_id].num_distinct_values =
+        LongType::InstanceNonNullable().makeValue(&num_distinct_values);
+  }
+
+  bool hasNumDistinctValues(const attribute_id attr_id) const {
+    const auto stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->num_distinct_values.isNull());
   }
 
   /**
@@ -98,21 +115,65 @@ class CatalogRelationStatistics {
    * @return The number of distinct values statistic for the column. Returns 0
    *         if the statistic is not set.
    */
-  std::size_t getNumDistinctValues(attribute_id attr_id) const {
-    const auto it = num_distinct_values_map_.find(attr_id);
-    if (it == num_distinct_values_map_.end()) {
-      return static_cast<std::size_t>(0);
-    } else {
-      return it->second;
-    }
+  std::size_t getNumDistinctValues(const attribute_id attr_id) const {
+    DCHECK(hasNumDistinctValues(attr_id));
+    return column_stats_.at(attr_id).num_distinct_values.getLiteral<std::int64_t>();
+  }
+
+  void setMinValue(const attribute_id attr_id,
+                   const TypedValue &min_value) {
+    column_stats_[attr_id].min_value = min_value;
+  }
+
+  bool hasMinValue(const attribute_id attr_id) const {
+    const auto stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->min_value.isNull());
+  }
+
+  const TypedValue& getMinValue(const attribute_id attr_id) const {
+    DCHECK(hasMinValue(attr_id));
+    return column_stats_.at(attr_id).min_value;
+  }
+
+  void setMaxValue(const attribute_id attr_id,
+                   const TypedValue &max_value) {
+    column_stats_[attr_id].max_value = max_value;
+  }
+
+  bool hasMaxValue(const attribute_id attr_id) const {
+    const auto stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->max_value.isNull());
+  }
+
+  const TypedValue& getMaxValue(const attribute_id attr_id) const {
+    DCHECK(hasMaxValue(attr_id));
+    return column_stats_.at(attr_id).max_value;
   }
 
  private:
+  struct ColumnStats {
+    ColumnStats()
+        : num_distinct_values(kNullValue),
+          min_value(kNullValue),
+          max_value(kNullValue) {
+    }
+    TypedValue num_distinct_values;
+    TypedValue min_value;
+    TypedValue max_value;
+  };
+
+  inline const ColumnStats* getColumnStats(const attribute_id attr_id) const {
+    const auto it = column_stats_.find(attr_id);
+    return (it == column_stats_.end() ? nullptr : &it->second);
+  }
+
+  static const TypedValue kNullValue;
+
   // Total number of tuples in the relation.
-  std::size_t num_tuples_;
+  TypedValue num_tuples_;
 
-  // Number of distinct values for each column.
-  std::unordered_map<attribute_id, std::size_t> num_distinct_values_map_;
+  // Statistics for each column.
+  std::unordered_map<attribute_id, ColumnStats> column_stats_;
 
   DISALLOW_COPY_AND_ASSIGN(CatalogRelationStatistics);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index f27f1ef..feafec1 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -196,10 +196,7 @@ void executeDescribeTable(
   }
 }
 
-/**
- * @brief A helper function that executes a SQL query to obtain a scalar result.
- */
-inline TypedValue executeQueryForSingleResult(
+inline std::vector<TypedValue> executeQueryForSingleRow(
     const tmb::client_id main_thread_client_id,
     const tmb::client_id foreman_client_id,
     const std::string &query_string,
@@ -232,22 +229,29 @@ inline TypedValue executeQueryForSingleResult(
   const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
   DCHECK(query_result_relation != nullptr);
 
-  TypedValue value;
+  std::vector<TypedValue> values;
   {
     std::vector<block_id> blocks = query_result_relation->getBlocksSnapshot();
     DCHECK_EQ(1u, blocks.size());
+
     BlockReference block = storage_manager->getBlock(blocks[0], *query_result_relation);
     const TupleStorageSubBlock &tuple_store = block->getTupleStorageSubBlock();
     DCHECK_EQ(1, tuple_store.numTuples());
-    DCHECK_EQ(1u, tuple_store.getRelation().size());
 
+    const std::size_t num_columns = tuple_store.getRelation().size();
     if (tuple_store.isPacked()) {
-      value = tuple_store.getAttributeValueTyped(0, 0);
+      for (std::size_t i = 0; i < num_columns; ++i) {
+        values.emplace_back(tuple_store.getAttributeValueTyped(0, i));
+        values[i].ensureNotReference();
+      }
     } else {
       std::unique_ptr<TupleIdSequence> existence_map(tuple_store.getExistenceMap());
-      value = tuple_store.getAttributeValueTyped(*existence_map->begin(), 0);
+      for (std::size_t i = 0; i < num_columns; ++i) {
+        values.emplace_back(
+            tuple_store.getAttributeValueTyped(*existence_map->begin(), i));
+        values[i].ensureNotReference();
+      }
     }
-    value.ensureNotReference();
   }
 
   // Drop the result relation.
@@ -255,7 +259,31 @@ inline TypedValue executeQueryForSingleResult(
                      query_processor->getDefaultDatabase(),
                      query_processor->getStorageManager());
 
-  return value;
+  return values;
+}
+
+/**
+ * @brief A helper function that executes a SQL query to obtain a scalar result.
+ */
+inline TypedValue executeQueryForSingleResult(
+    const tmb::client_id main_thread_client_id,
+    const tmb::client_id foreman_client_id,
+    const std::string &query_string,
+    tmb::MessageBus *bus,
+    StorageManager *storage_manager,
+    QueryProcessor *query_processor,
+    SqlParserWrapper *parser_wrapper) {
+  std::vector<TypedValue> results =
+      executeQueryForSingleRow(
+          main_thread_client_id,
+          foreman_client_id,
+          query_string,
+          bus,
+          storage_manager,
+          query_processor,
+          parser_wrapper);
+  DCHECK_EQ(1u, results.size());
+  return results[0];
 }
 
 void executeAnalyze(const PtrVector<ParseString> *arguments,
@@ -292,25 +320,47 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
 
     // Get the number of distinct values for each column.
     for (const CatalogAttribute &attribute : relation) {
+      const Type &attr_type = attribute.getType();
+      bool is_min_max_applicable =
+          (attr_type.getSuperTypeID() == Type::SuperTypeID::kNumeric);
+
       std::string query_string = "SELECT COUNT(DISTINCT ";
       query_string.append(attribute.getName());
-      query_string.append(") FROM ");
+      query_string.append(")");
+      if (is_min_max_applicable) {
+        query_string.append(", MIN(");
+        query_string.append(attribute.getName());
+        query_string.append("), MAX(");
+        query_string.append(attribute.getName());
+        query_string.append(")");
+      }
+      query_string.append(" FROM ");
       query_string.append(relation.getName());
       query_string.append(";");
 
-      TypedValue num_distinct_values =
-          executeQueryForSingleResult(main_thread_client_id,
-                                      foreman_client_id,
-                                      query_string,
-                                      bus,
-                                      storage_manager,
-                                      query_processor,
-                                      parser_wrapper.get());
-
-      DCHECK(num_distinct_values.getTypeID() == TypeID::kLong);
-      mutable_relation->getStatisticsMutable()->setNumDistinctValues(
-          attribute.getID(),
-          num_distinct_values.getLiteral<std::int64_t>());
+      std::vector<TypedValue> results =
+          executeQueryForSingleRow(main_thread_client_id,
+                                   foreman_client_id,
+                                   query_string,
+                                   bus,
+                                   storage_manager,
+                                   query_processor,
+                                   parser_wrapper.get());
+
+      auto *stat = mutable_relation->getStatisticsMutable();
+      const attribute_id attr_id = attribute.getID();
+
+      DCHECK(results[0].getTypeID() == TypeID::kLong);
+      stat->setNumDistinctValues(attr_id,
+                                 results[0].getLiteral<std::int64_t>());
+
+      if (is_min_max_applicable) {
+        DCHECK_GE(results.size(), 3u);
+        DCHECK(results[1].getTypeID() == attr_type.getTypeID());
+        DCHECK(results[2].getTypeID() == attr_type.getTypeID());
+        stat->setMinValue(attr_id, results[1]);
+        stat->setMaxValue(attr_id, results[2]);
+      }
     }
 
     // Get the number of tuples for the relation.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index e7ffc0c..bf0e2c7 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -347,10 +347,10 @@ int main(int argc, char* argv[]) {
 
 //  addPrimaryKeyInfoForTPCHTables(query_processor->getDefaultDatabase());
 //  addPrimaryKeyInfoForSSBTables(query_processor->getDefaultDatabase());
-//  std::string proto_str;
-//  google::protobuf::TextFormat::PrintToString(
-//      query_processor->getDefaultDatabase()->getProto(), &proto_str);
-//  std::cerr << proto_str << "\n";
+  std::string proto_str;
+  google::protobuf::TextFormat::PrintToString(
+      query_processor->getDefaultDatabase()->getProto(), &proto_str);
+  std::cerr << proto_str << "\n";
 //  query_processor->markCatalogAltered();
 //  query_processor->saveCatalog();
 


[12/16] incubator-quickstep git commit: Updates

Posted by ji...@apache.org.
Updates


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

Branch: refs/heads/LIP-for-tpch
Commit: c588775bcb71eb23bba8cda313e4dbc83b23c78c
Parents: b7150fb
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Wed Aug 3 20:56:18 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500

----------------------------------------------------------------------
 cli/QuickstepCli.cpp                   | 5 -----
 query_optimizer/ExecutionGenerator.cpp | 2 --
 2 files changed, 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c588775b/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index a3dfae4..e7ffc0c 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -516,11 +516,6 @@ int main(int argc, char* argv[]) {
               main_thread_client_id, &bus);
           end = std::chrono::steady_clock::now();
 
-          if (quickstep::FLAGS_visualize_dag) {
-            quickstep::DAGVisualizer visualizer(*query_handle->getQueryPlanMutable());
-            std::cerr << "\n" << visualizer.toDOT() << "\n";
-          }
-
           const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
           if (query_result_relation) {
             PrintToScreen::PrintRelation(*query_result_relation,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c588775b/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 6dbba51..a194d46 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -670,8 +670,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
     key_types.push_back(&left_attribute_type);
   }
 
-  std::size_t build_cardinality = cost_model_->estimateCardinality(build_physical);
-
   // Convert the residual predicate proto.
   QueryContext::predicate_id residual_predicate_index = QueryContext::kInvalidPredicateId;
   if (physical_plan->residual_predicate()) {


[13/16] incubator-quickstep git commit: Updates to build bloom filters

Posted by ji...@apache.org.
Updates to build bloom filters


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

Branch: refs/heads/LIP-for-tpch
Commit: 6f3129f723f8b76c818368c0f7c30393db6565af
Parents: ef4cd1e
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Aug 8 12:08:32 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500

----------------------------------------------------------------------
 query_execution/QueryManagerBase.cpp            |  2 +-
 query_optimizer/ExecutionGenerator.cpp          | 26 ++++---
 query_optimizer/ExecutionHeuristics.cpp         |  4 +-
 query_optimizer/cost_model/SimpleCostModel.cpp  |  2 +-
 query_optimizer/rules/AttachBloomFilters.cpp    | 41 ++++++-----
 .../StarSchemaHashJoinOrderOptimization.cpp     |  6 +-
 relational_operators/BuildHashOperator.cpp      | 12 +++-
 relational_operators/BuildHashOperator.hpp      |  4 ++
 relational_operators/CreateIndexOperator.cpp    |  2 +-
 relational_operators/CreateIndexOperator.hpp    |  2 +-
 relational_operators/CreateTableOperator.cpp    |  2 +-
 relational_operators/CreateTableOperator.hpp    |  2 +-
 relational_operators/DropTableOperator.cpp      |  2 +-
 relational_operators/DropTableOperator.hpp      |  2 +-
 relational_operators/RelationalOperator.hpp     |  4 +-
 storage/HashTable.hpp                           | 76 ++++++++++++++------
 storage/StorageBlock.cpp                        | 12 ++--
 storage/StorageBlock.hpp                        |  2 +-
 utility/DAG.hpp                                 |  4 +-
 19 files changed, 129 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_execution/QueryManagerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.cpp b/query_execution/QueryManagerBase.cpp
index 8e37da8..d49ee91 100644
--- a/query_execution/QueryManagerBase.cpp
+++ b/query_execution/QueryManagerBase.cpp
@@ -209,7 +209,7 @@ void QueryManagerBase::markOperatorFinished(const dag_node_index index) {
   query_exec_state_->setExecutionFinished(index);
 
   RelationalOperator *op = query_dag_->getNodePayloadMutable(index);
-  op->updateCatalogOnCompletion();
+  op->actionOnCompletion();
 
   const relation_id output_rel = op->getOutputRelationID();
   for (const pair<dag_node_index, bool> &dependent_link : query_dag_->getDependents(index)) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 21c6e30..02deb3a 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -631,8 +631,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   bool any_probe_attributes_nullable = false;
   bool any_build_attributes_nullable = false;
 
-  bool skip_hash_join_optimization = false;
-
   const std::vector<E::AttributeReferencePtr> &left_join_attributes =
       physical_plan->left_join_attributes();
   for (const E::AttributeReferencePtr &left_join_attribute : left_join_attributes) {
@@ -840,17 +838,15 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   temporary_relation_info_vec_.emplace_back(join_operator_index, output_relation);
 
   // Add heuristics for the Hash Join, if enabled.
-  if (FLAGS_optimize_joins && !skip_hash_join_optimization) {
-    execution_heuristics_->addHashJoinInfo(build_operator_index,
-                                           join_operator_index,
-                                           referenced_stored_build_relation,
-                                           referenced_stored_probe_relation,
-                                           bloom_filter_config,
-                                           std::move(build_side_bloom_filter_attribute_ids),
-                                           std::move(probe_side_bloom_filter_attribute_ids),
-                                           join_hash_table_index,
-                                           star_schema_cost_model_->estimateCardinality(build_physical));
-  }
+  execution_heuristics_->addHashJoinInfo(build_operator_index,
+                                         join_operator_index,
+                                         referenced_stored_build_relation,
+                                         referenced_stored_probe_relation,
+                                         bloom_filter_config,
+                                         std::move(build_side_bloom_filter_attribute_ids),
+                                         std::move(probe_side_bloom_filter_attribute_ids),
+                                         join_hash_table_index,
+                                         star_schema_cost_model_->estimateCardinality(build_physical));
 }
 
 void ExecutionGenerator::convertNestedLoopsJoin(
@@ -1443,7 +1439,9 @@ void ExecutionGenerator::convertAggregate(
     aggr_state_proto->mutable_predicate()->CopyFrom(predicate->getProto());
   }
 
-  aggr_state_proto->set_estimated_num_entries(cost_model_->estimateCardinality(physical_plan));
+//  aggr_state_proto->set_estimated_num_entries(cost_model_->estimateCardinality(physical_plan));
+  aggr_state_proto->set_estimated_num_entries(
+      star_schema_cost_model_->estimateCardinality(physical_plan) * 10);
 
   const QueryPlan::DAGNodeIndex aggregation_operator_index =
       execution_plan_->addRelationalOperator(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_optimizer/ExecutionHeuristics.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.cpp b/query_optimizer/ExecutionHeuristics.cpp
index d5d7640..81e7362 100644
--- a/query_optimizer/ExecutionHeuristics.cpp
+++ b/query_optimizer/ExecutionHeuristics.cpp
@@ -70,8 +70,8 @@ void ExecutionHeuristics::optimizeExecutionPlan(QueryPlan *query_plan,
       auto *build_side_bloom_filter = hash_table_proto->add_build_side_bloom_filters();
       build_side_bloom_filter->set_bloom_filter_id(bloom_filter_id);
       build_side_bloom_filter->set_attr_id(info.build_side_bloom_filter_ids_[i]);
-      std::cerr << "Build " << build_side_bf.attribute->toString()
-                << " @" << bloom_filter_config.builder << "\n";
+//      std::cerr << "Build " << build_side_bf.attribute->toString()
+//                << " @" << bloom_filter_config.builder << "\n";
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index 45e2f00..f3d4fee 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -119,7 +119,7 @@ std::size_t SimpleCostModel::estimateCardinalityForAggregate(
     return 1;
   }
   return std::max(static_cast<std::size_t>(1),
-                  estimateCardinality(physical_plan->input()));
+                  estimateCardinality(physical_plan->input()) / 10);
 }
 
 std::size_t SimpleCostModel::estimateCardinalityForWindowAggregate(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_optimizer/rules/AttachBloomFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.cpp b/query_optimizer/rules/AttachBloomFilters.cpp
index 10ed512..898d831 100644
--- a/query_optimizer/rules/AttachBloomFilters.cpp
+++ b/query_optimizer/rules/AttachBloomFilters.cpp
@@ -143,12 +143,11 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
   // Bloom filters from parent
   const auto &parent_bloom_filters = consumers_[node];
   if (!parent_bloom_filters.empty()) {
-//    if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
-//      const P::HashJoinPtr hash_join =
-//          std::static_pointer_cast<const P::HashJoin>(node);
+//    P::HashJoinPtr hash_join;
+//    if (P::SomeHashJoin::MatchesWithConditionalCast(node, &hash_join) &&
+//        hash_join->join_type() == P::HashJoin::JoinType::kInnerJoin) {
 //      const std::vector<const std::vector<E::AttributeReferencePtr>*> join_attributes =
 //          { &hash_join->left_join_attributes(), &hash_join->right_join_attributes() };
-//
 //      for (std::size_t i = 0; i < 2; ++i) {
 //        const auto child = hash_join->children()[i];
 //        std::unordered_set<E::ExprId> child_output_attribute_ids;
@@ -188,7 +187,7 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
 //        }
 //        consumers_.emplace(child, std::move(bloom_filters));
 //      }
-//    }
+//    } else {
     for (const auto &child : node->children()) {
       std::unordered_set<E::ExprId> child_output_attribute_ids;
       for (const auto &attr : child->getOutputAttributes()) {
@@ -209,6 +208,7 @@ void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
       }
       consumers_.emplace(child, std::move(bloom_filters));
     }
+//    }
   }
 
   // Bloom filters from build side to probe side via HashJoin
@@ -260,14 +260,18 @@ void AttachBloomFilters::decideAttach(
   }
 
   P::PhysicalPtr consumer_child = nullptr;
-  if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
-    consumer_child = std::static_pointer_cast<const P::HashJoin>(node)->left();
-  }
-  if (node->getPhysicalType() == P::PhysicalType::kAggregate) {
-    consumer_child = std::static_pointer_cast<const P::Aggregate>(node)->input();
-  }
-  if (node->getPhysicalType() == P::PhysicalType::kSelection) {
-    consumer_child = std::static_pointer_cast<const P::Selection>(node)->input();
+  switch (node->getPhysicalType()) {
+    case P::PhysicalType::kHashJoin:
+      consumer_child = std::static_pointer_cast<const P::HashJoin>(node)->left();
+      break;
+    case P::PhysicalType::kAggregate:
+      consumer_child = std::static_pointer_cast<const P::Aggregate>(node)->input();
+      break;
+    case P::PhysicalType::kSelection:
+      consumer_child = std::static_pointer_cast<const P::Selection>(node)->input();
+      break;
+    default:
+      break;
   }
 
   if (consumer_child != nullptr) {
@@ -320,7 +324,7 @@ P::PhysicalPtr AttachBloomFilters::performAttach(const physical::PhysicalPtr &no
     const auto attach_it = attaches_.find(node);
     if (attach_it != attaches_.end()) {
 //      for (const auto& item : attach_it->second.probe_side_bloom_filters) {
-//        std::cout << "Attach probe from " << item.builder
+//        std::cerr << "Attach probe from " << item.builder
 //                  << " to " << node << "\n";
 //      }
 
@@ -342,14 +346,14 @@ P::PhysicalPtr AttachBloomFilters::performAttach(const physical::PhysicalPtr &no
     const auto attach_it = attaches_.find(node);
     if (attach_it != attaches_.end()) {
 //      for (const auto& item : attach_it->second.probe_side_bloom_filters) {
-//        std::cout << "Attach probe from " << item.builder
+//        std::cerr << "Attach probe from " << item.builder
 //                  << " to " << node << "\n";
 //      }
 
       const P::AggregatePtr aggregate =
           std::static_pointer_cast<const P::Aggregate>(node);
       return P::Aggregate::Create(
-          aggregate->input(),
+          new_children[0],
           aggregate->grouping_expressions(),
           aggregate->aggregate_expressions(),
           aggregate->filter_predicate(),
@@ -361,14 +365,14 @@ P::PhysicalPtr AttachBloomFilters::performAttach(const physical::PhysicalPtr &no
     const auto attach_it = attaches_.find(node);
     if (attach_it != attaches_.end()) {
 //      for (const auto& item : attach_it->second.probe_side_bloom_filters) {
-//        std::cout << "Attach probe from " << item.builder
+//        std::cerr << "Attach probe from " << item.builder
 //                  << " to " << node << "\n";
 //      }
 
       const P::SelectionPtr selection =
           std::static_pointer_cast<const P::Selection>(node);
       return P::Selection::Create(
-          selection->input(),
+          new_children[0],
           selection->project_expressions(),
           selection->filter_predicate(),
           attach_it->second);
@@ -378,7 +382,6 @@ P::PhysicalPtr AttachBloomFilters::performAttach(const physical::PhysicalPtr &no
   if (has_changed) {
     return node->copyWithNewChildren(new_children);
   }
-
   return node;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index 9e8d794..1e38f63 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -254,9 +254,9 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
 
     TableInfo *selected_probe_table_info = best_join->probe;
     TableInfo *selected_build_table_info = best_join->build;
-    std::cerr << "card: " << selected_probe_table_info->estimated_cardinality << "\n";
-    std::cerr << "card: " << selected_build_table_info->estimated_cardinality << "\n";
-    std::cerr << "--------\n";
+//    std::cerr << "card: " << selected_probe_table_info->estimated_cardinality << "\n";
+//    std::cerr << "card: " << selected_build_table_info->estimated_cardinality << "\n";
+//    std::cerr << "--------\n";
     if (!best_join->build_side_unique &&
         selected_probe_table_info->estimated_cardinality < selected_build_table_info->estimated_cardinality) {
       std::swap(selected_probe_table_info, selected_build_table_info);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index 465621c..b4e20e4 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -68,7 +68,10 @@ bool BuildHashOperator::getAllWorkOrders(
     tmb::MessageBus *bus) {
   DCHECK(query_context != nullptr);
 
-  JoinHashTable *hash_table = query_context->getJoinHashTable(hash_table_index_);
+  if (hash_table_ == nullptr) {
+    hash_table_ = query_context->getJoinHashTable(hash_table_index_);
+  }
+
   if (input_relation_is_stored_) {
     if (!started_) {
       for (const block_id input_block_id : input_relation_block_ids_) {
@@ -78,7 +81,7 @@ bool BuildHashOperator::getAllWorkOrders(
                                    join_key_attributes_,
                                    any_join_key_attributes_nullable_,
                                    input_block_id,
-                                   hash_table,
+                                   hash_table_,
                                    storage_manager),
             op_index_);
       }
@@ -94,7 +97,7 @@ bool BuildHashOperator::getAllWorkOrders(
               join_key_attributes_,
               any_join_key_attributes_nullable_,
               input_relation_block_ids_[num_workorders_generated_],
-              hash_table,
+              hash_table_,
               storage_manager),
           op_index_);
       ++num_workorders_generated_;
@@ -140,6 +143,9 @@ serialization::WorkOrder* BuildHashOperator::createWorkOrderProto(const block_id
   return proto;
 }
 
+void BuildHashOperator::actionOnCompletion() {
+  hash_table_->finalizeBuildSideThreadLocalBloomFilters();
+}
 
 void BuildHashWorkOrder::execute() {
   BlockReference block(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 4a80a8a..15b23f5 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -89,6 +89,7 @@ class BuildHashOperator : public RelationalOperator {
       join_key_attributes_(join_key_attributes),
       any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
       hash_table_index_(hash_table_index),
+      hash_table_(nullptr),
       input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
                                                          : std::vector<block_id>()),
       num_workorders_generated_(0),
@@ -124,6 +125,8 @@ class BuildHashOperator : public RelationalOperator {
                                      partially_filled_blocks->end());
   }
 
+  void actionOnCompletion() override;
+
  private:
   /**
    * @brief Create Work Order proto.
@@ -137,6 +140,7 @@ class BuildHashOperator : public RelationalOperator {
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
   const QueryContext::join_hash_table_id hash_table_index_;
+  JoinHashTable *hash_table_;
 
   std::vector<block_id> input_relation_block_ids_;
   std::vector<block_id>::size_type num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/CreateIndexOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.cpp b/relational_operators/CreateIndexOperator.cpp
index ab3624c..dd311bf 100644
--- a/relational_operators/CreateIndexOperator.cpp
+++ b/relational_operators/CreateIndexOperator.cpp
@@ -33,7 +33,7 @@ bool CreateIndexOperator::getAllWorkOrders(WorkOrdersContainer *container,
   return true;
 }
 
-void CreateIndexOperator::updateCatalogOnCompletion() {
+void CreateIndexOperator::actionOnCompletion() {
   relation_->addIndex(index_name_, std::move(index_description_));
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index fa992c9..1f6775a 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -90,7 +90,7 @@ class CreateIndexOperator : public RelationalOperator {
     return true;
   }
 
-  void updateCatalogOnCompletion() override;
+  void actionOnCompletion() override;
 
  private:
   CatalogRelation *relation_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/CreateTableOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.cpp b/relational_operators/CreateTableOperator.cpp
index 261bec1..4cc615b 100644
--- a/relational_operators/CreateTableOperator.cpp
+++ b/relational_operators/CreateTableOperator.cpp
@@ -36,7 +36,7 @@ bool CreateTableOperator::getAllWorkOrders(
   return true;
 }
 
-void CreateTableOperator::updateCatalogOnCompletion() {
+void CreateTableOperator::actionOnCompletion() {
   database_->addRelation(relation_.release());
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 3a2e29b..9b889ca 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -89,7 +89,7 @@ class CreateTableOperator : public RelationalOperator {
     return true;
   }
 
-  void updateCatalogOnCompletion() override;
+  void actionOnCompletion() override;
 
  private:
   std::unique_ptr<CatalogRelation> relation_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/DropTableOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.cpp b/relational_operators/DropTableOperator.cpp
index 5cd5ebc..a5e3d7c 100644
--- a/relational_operators/DropTableOperator.cpp
+++ b/relational_operators/DropTableOperator.cpp
@@ -80,7 +80,7 @@ bool DropTableOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *containe
   return work_generated_;
 }
 
-void DropTableOperator::updateCatalogOnCompletion() {
+void DropTableOperator::actionOnCompletion() {
   const relation_id rel_id = relation_.getID();
   if (only_drop_blocks_) {
     database_->getRelationByIdMutable(rel_id)->clearBlocks();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index e713bd6..534a266 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -89,7 +89,7 @@ class DropTableOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void updateCatalogOnCompletion() override;
+  void actionOnCompletion() override;
 
  private:
   const CatalogRelation &relation_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index f0303e5..ac5dd54 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -114,11 +114,11 @@ class RelationalOperator {
   virtual bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) = 0;
 
   /**
-   * @brief Update Catalog upon the completion of this RelationalOperator, if
+   * @brief Perform action upon the completion of this RelationalOperator, if
    *        necessary.
    *
    **/
-  virtual void updateCatalogOnCompletion() {
+  virtual void actionOnCompletion() {
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/storage/HashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTable.hpp b/storage/HashTable.hpp
index 3538181..2ef9359 100644
--- a/storage/HashTable.hpp
+++ b/storage/HashTable.hpp
@@ -23,7 +23,9 @@
 #include <atomic>
 #include <cstddef>
 #include <cstdlib>
+#include <map>
 #include <memory>
+#include <thread>
 #include <type_traits>
 #include <vector>
 
@@ -37,6 +39,7 @@
 #include "storage/ValueAccessor.hpp"
 #include "storage/ValueAccessorUtil.hpp"
 #include "threading/SpinSharedMutex.hpp"
+#include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/BloomFilter.hpp"
@@ -1028,6 +1031,17 @@ class HashTable : public HashTableBase<resizable,
     build_attribute_ids_.push_back(build_attribute_id);
   }
 
+  inline void finalizeBuildSideThreadLocalBloomFilters() {
+    if (has_build_side_bloom_filter_) {
+      for (const auto &thread_local_bf_pair : thread_local_bloom_filters_) {
+        for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
+          build_bloom_filters_[i]->bitwiseOr(
+              thread_local_bf_pair.second[i].get());
+        }
+      }
+    }
+  }
+
   /**
    * @brief This function adds a pointer to the list of bloom filters to be
    *        used during the probe phase of this hash table.
@@ -1338,6 +1352,8 @@ class HashTable : public HashTableBase<resizable,
   bool has_build_side_bloom_filter_ = false;
   bool has_probe_side_bloom_filter_ = false;
   std::vector<BloomFilter *> build_bloom_filters_;
+  std::map<std::thread::id, std::vector<std::unique_ptr<BloomFilter>>> thread_local_bloom_filters_;
+  SpinMutex bloom_filter_mutex_;
   std::vector<attribute_id> build_attribute_ids_;
   std::vector<const BloomFilter*> probe_bloom_filters_;
   std::vector<attribute_id> probe_attribute_ids_;
@@ -1487,22 +1503,19 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
       }
     }
 
+    BloomFilter *thread_local_bloom_filter = nullptr;
     if (has_build_side_bloom_filter_) {
-      for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
-        auto *build_bloom_filter = build_bloom_filters_[i];
-        std::unique_ptr<BloomFilter> thread_local_bloom_filter(
-            new BloomFilter(build_bloom_filter->getNumberOfHashes(),
-                            build_bloom_filter->getBitArraySize()));
-        const auto &build_attr = build_attribute_ids_[i];
-        const std::size_t attr_size =
-            accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, build_attr).second;
-        while (accessor->next()) {
-          thread_local_bloom_filter->insertUnSafe(
-              static_cast<const std::uint8_t *>(accessor->getUntypedValue(build_attr)),
-              attr_size);
-        }
-        build_bloom_filter->bitwiseOr(thread_local_bloom_filter.get());
-        accessor->beginIteration();
+      const auto tid = std::this_thread::get_id();
+      SpinMutexLock lock(bloom_filter_mutex_);
+      auto bf_it = thread_local_bloom_filters_.find(tid);
+      if (bf_it == thread_local_bloom_filters_.end()) {
+        auto &bf_vector = thread_local_bloom_filters_[tid];
+        bf_vector.emplace_back(
+            std::make_unique<BloomFilter>(build_bloom_filters_[0]->getNumberOfHashes(),
+                                          build_bloom_filters_[0]->getBitArraySize()));
+        thread_local_bloom_filter = bf_vector[0].get();
+      } else {
+        thread_local_bloom_filter = bf_it->second[0].get();
       }
     }
 
@@ -1521,6 +1534,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::kDuplicateKey) {
               DEBUG_ASSERT(!using_prealloc);
               return result;
@@ -1546,6 +1564,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;
         }
@@ -1618,12 +1641,26 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
     }
 
     if (has_build_side_bloom_filter_) {
+      const auto tid = std::this_thread::get_id();
+      std::vector<std::unique_ptr<BloomFilter>> *thread_local_bf_vector;
+      {
+        SpinMutexLock lock(bloom_filter_mutex_);
+        auto bf_it = thread_local_bloom_filters_.find(tid);
+        if (bf_it == thread_local_bloom_filters_.end()) {
+          thread_local_bf_vector = &thread_local_bloom_filters_[tid];
+          for (const auto &build_side_bf : build_bloom_filters_) {
+            thread_local_bf_vector->emplace_back(
+                std::make_unique<BloomFilter>(build_side_bf->getNumberOfHashes(),
+                                              build_side_bf->getBitArraySize()));
+          }
+        } else {
+          thread_local_bf_vector = &bf_it->second;
+        }
+      }
+
       for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
-        auto *build_bloom_filter = build_bloom_filters_[i];
-        std::unique_ptr<BloomFilter> thread_local_bloom_filter(
-            new BloomFilter(build_bloom_filter->getNumberOfHashes(),
-                            build_bloom_filter->getBitArraySize()));
         const auto &build_attr = build_attribute_ids_[i];
+        BloomFilter *thread_local_bloom_filter = (*thread_local_bf_vector)[i].get();
         const std::size_t attr_size =
             accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, build_attr).second;
         while (accessor->next()) {
@@ -1631,7 +1668,6 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
               static_cast<const std::uint8_t *>(accessor->getUntypedValue(build_attr)),
               attr_size);
         }
-        build_bloom_filter->bitwiseOr(thread_local_bloom_filter.get());
         accessor->beginIteration();
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 74ff5b6..7bbba9a 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -1326,12 +1326,16 @@ bool StorageBlock::rebuildIndexes(bool short_circuit) {
 }
 
 TupleIdSequence* StorageBlock::getMatchesForPredicate(const Predicate *predicate,
-                                                      const TupleIdSequence *sequence) const {
+                                                      const TupleIdSequence *filter) const {
   if (predicate == nullptr) {
-    return tuple_store_->getExistenceMap();
+    TupleIdSequence *sequence = tuple_store_->getExistenceMap();
+    if (filter != nullptr) {
+      sequence->intersectWith(*filter);
+    }
+    return sequence;
   }
 
-  std::unique_ptr<ValueAccessor> value_accessor(tuple_store_->createValueAccessor(sequence));
+  std::unique_ptr<ValueAccessor> value_accessor(tuple_store_->createValueAccessor());
   std::unique_ptr<TupleIdSequence> existence_map;
   if (!tuple_store_->isPacked()) {
     existence_map.reset(tuple_store_->getExistenceMap());
@@ -1341,7 +1345,7 @@ TupleIdSequence* StorageBlock::getMatchesForPredicate(const Predicate *predicate
                                     indices_consistent_);
   return predicate->getAllMatches(value_accessor.get(),
                                   &sub_blocks_ref,
-                                  nullptr,
+                                  filter,
                                   existence_map.get());
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/storage/StorageBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.hpp b/storage/StorageBlock.hpp
index 5cca51c..4284ea1 100644
--- a/storage/StorageBlock.hpp
+++ b/storage/StorageBlock.hpp
@@ -593,7 +593,7 @@ class StorageBlock : public StorageBlockBase {
   const std::size_t getNumTuples() const;
 
   TupleIdSequence* getMatchesForPredicate(const Predicate *predicate,
-                                          const TupleIdSequence *sequence = nullptr) const;
+                                          const TupleIdSequence *filter = nullptr) const;
 
  private:
   static TupleStorageSubBlock* CreateTupleStorageSubBlock(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6f3129f7/utility/DAG.hpp
----------------------------------------------------------------------
diff --git a/utility/DAG.hpp b/utility/DAG.hpp
index a1f2619..1d1caa1 100644
--- a/utility/DAG.hpp
+++ b/utility/DAG.hpp
@@ -293,8 +293,8 @@ class DAG {
      *                      node at node_index.
      **/
      inline void addDependent(const size_type_nodes node_index, const LinkMetadataT &link_metadata) {
-       DCHECK(dependents_with_metadata_.find(node_index) == dependents_with_metadata_.end());
-       dependents_with_metadata_.emplace(node_index, link_metadata);
+       // DCHECK(dependents_with_metadata_.find(node_index) == dependents_with_metadata_.end());
+       dependents_with_metadata_[node_index] = link_metadata;
      }
 
     /**


[11/16] incubator-quickstep git commit: Minor updates to mutex

Posted by ji...@apache.org.
Minor updates to mutex


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

Branch: refs/heads/LIP-for-tpch
Commit: ebdc041f5be5fab9c8dd702a50811326d4178bac
Parents: 6f3129f
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Aug 8 12:57:04 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500

----------------------------------------------------------------------
 storage/HashTable.hpp | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ebdc041f/storage/HashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTable.hpp b/storage/HashTable.hpp
index 2ef9359..1caa183 100644
--- a/storage/HashTable.hpp
+++ b/storage/HashTable.hpp
@@ -38,8 +38,8 @@
 #include "storage/TupleReference.hpp"
 #include "storage/ValueAccessor.hpp"
 #include "storage/ValueAccessorUtil.hpp"
+#include "threading/Mutex.hpp"
 #include "threading/SpinSharedMutex.hpp"
-#include "threading/SpinMutex.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/BloomFilter.hpp"
@@ -1353,7 +1353,7 @@ class HashTable : public HashTableBase<resizable,
   bool has_probe_side_bloom_filter_ = false;
   std::vector<BloomFilter *> build_bloom_filters_;
   std::map<std::thread::id, std::vector<std::unique_ptr<BloomFilter>>> thread_local_bloom_filters_;
-  SpinMutex bloom_filter_mutex_;
+  Mutex bloom_filter_mutex_;
   std::vector<attribute_id> build_attribute_ids_;
   std::vector<const BloomFilter*> probe_bloom_filters_;
   std::vector<attribute_id> probe_attribute_ids_;
@@ -1506,7 +1506,7 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
     BloomFilter *thread_local_bloom_filter = nullptr;
     if (has_build_side_bloom_filter_) {
       const auto tid = std::this_thread::get_id();
-      SpinMutexLock lock(bloom_filter_mutex_);
+      MutexLock lock(bloom_filter_mutex_);
       auto bf_it = thread_local_bloom_filters_.find(tid);
       if (bf_it == thread_local_bloom_filters_.end()) {
         auto &bf_vector = thread_local_bloom_filters_[tid];
@@ -1644,7 +1644,7 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
       const auto tid = std::this_thread::get_id();
       std::vector<std::unique_ptr<BloomFilter>> *thread_local_bf_vector;
       {
-        SpinMutexLock lock(bloom_filter_mutex_);
+        MutexLock lock(bloom_filter_mutex_);
         auto bf_it = thread_local_bloom_filters_.find(tid);
         if (bf_it == thread_local_bloom_filters_.end()) {
           thread_local_bf_vector = &thread_local_bloom_filters_[tid];


[06/16] incubator-quickstep git commit: Removed an unused message type.

Posted by ji...@apache.org.
Removed an unused message type.


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

Branch: refs/heads/LIP-for-tpch
Commit: 6168996216af8278d5c789c67aa4ec8325fab483
Parents: 2c0ce6a
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 15:32:34 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Tue Aug 9 10:44:50 2016 -0700

----------------------------------------------------------------------
 query_execution/ForemanSingleNode.cpp        |  4 +---
 query_execution/PolicyEnforcerBase.cpp       | 13 -------------
 query_execution/QueryExecutionMessages.proto |  5 -----
 query_execution/QueryExecutionTypedefs.hpp   |  1 -
 query_execution/Shiftboss.cpp                |  1 -
 query_execution/Shiftboss.hpp                |  1 -
 query_execution/Worker.hpp                   |  1 -
 7 files changed, 1 insertion(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/ForemanSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.cpp b/query_execution/ForemanSingleNode.cpp
index 23db379..d064a6f 100644
--- a/query_execution/ForemanSingleNode.cpp
+++ b/query_execution/ForemanSingleNode.cpp
@@ -87,7 +87,6 @@ ForemanSingleNode::ForemanSingleNode(
       kPoisonMessage,
       kRebuildWorkOrderCompleteMessage,
       kWorkOrderFeedbackMessage,
-      kWorkOrdersAvailableMessage,
       kWorkOrderCompleteMessage};
 
   for (const auto message_type : receiver_message_types) {
@@ -122,8 +121,7 @@ void ForemanSingleNode::run() {
       case kDataPipelineMessage:
       case kRebuildWorkOrderCompleteMessage:
       case kWorkOrderCompleteMessage:
-      case kWorkOrderFeedbackMessage:
-      case kWorkOrdersAvailableMessage: {
+      case kWorkOrderFeedbackMessage: {
         policy_enforcer_->processMessage(tagged_message);
         break;
       }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index 78f7b44..4174bd6 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -107,19 +107,6 @@ void PolicyEnforcerBase::processMessage(const TaggedMessage &tagged_message) {
           op_index, proto.block_id(), proto.relation_id());
       break;
     }
-    case kWorkOrdersAvailableMessage: {
-      serialization::WorkOrdersAvailableMessage proto;
-      CHECK(proto.ParseFromArray(tagged_message.message(),
-                                 tagged_message.message_bytes()));
-      query_id = proto.query_id();
-      DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
-
-      op_index = proto.operator_index();
-
-      // Check if new work orders are available.
-      admitted_queries_[query_id]->fetchNormalWorkOrders(op_index);
-      break;
-    }
     case kWorkOrderFeedbackMessage: {
       WorkOrder::FeedbackMessage msg(
           const_cast<void *>(tagged_message.message()),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 20b684e..060efa1 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -74,11 +74,6 @@ message DataPipelineMessage {
   required uint64 query_id = 4;
 }
 
-message WorkOrdersAvailableMessage {
-  required uint64 operator_index = 1;
-  required uint64 query_id = 2;
-}
-
 // Distributed version related messages.
 message ShiftbossRegistrationMessage {
   // The total Work Order processing capacity in Shiftboss, which equals to the

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index d154d84..33a93b0 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -69,7 +69,6 @@ enum QueryExecutionMessageType : message_type_id {
   kWorkOrderCompleteMessage,  // From Worker to Foreman.
   kCatalogRelationNewBlockMessage,  // From InsertDestination to Foreman.
   kDataPipelineMessage,  // From InsertDestination or some WorkOrders to Foreman.
-  kWorkOrdersAvailableMessage,  // From some WorkOrders to Foreman.
   kWorkOrderFeedbackMessage,  // From some WorkOrders to Foreman on behalf of
                               // their corresponding RelationalOperators.
   kRebuildWorkOrderMessage,  // From Foreman to Worker.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index 24c91fe..bd83dd4 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -152,7 +152,6 @@ void Shiftboss::run() {
       case kWorkOrderCompleteMessage:  // Fall through.
       case kRebuildWorkOrderCompleteMessage:
       case kDataPipelineMessage:
-      case kWorkOrdersAvailableMessage:
       case kWorkOrderFeedbackMessage: {
         LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
                   << "') forwarded typed '" << annotated_message.tagged_message.message_type()

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/Shiftboss.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.hpp b/query_execution/Shiftboss.hpp
index 9464a4d..30a8d1a 100644
--- a/query_execution/Shiftboss.hpp
+++ b/query_execution/Shiftboss.hpp
@@ -103,7 +103,6 @@ class Shiftboss : public Thread {
     // Message sent to Foreman.
     bus_->RegisterClientAsSender(shiftboss_client_id_, kCatalogRelationNewBlockMessage);
     bus_->RegisterClientAsSender(shiftboss_client_id_, kDataPipelineMessage);
-    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrdersAvailableMessage);
     bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderFeedbackMessage);
 
     // Forward the following message types from Foreman to Workers.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/61689962/query_execution/Worker.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.hpp b/query_execution/Worker.hpp
index 44a7447..aa39bb3 100644
--- a/query_execution/Worker.hpp
+++ b/query_execution/Worker.hpp
@@ -68,7 +68,6 @@ class Worker : public Thread {
                                  kRebuildWorkOrderCompleteMessage);
     bus_->RegisterClientAsSender(worker_client_id_, kCatalogRelationNewBlockMessage);
     bus_->RegisterClientAsSender(worker_client_id_, kDataPipelineMessage);
-    bus_->RegisterClientAsSender(worker_client_id_, kWorkOrdersAvailableMessage);
     bus_->RegisterClientAsSender(worker_client_id_, kWorkOrderFeedbackMessage);
 
     bus_->RegisterClientAsReceiver(worker_client_id_, kWorkOrderMessage);


[05/16] incubator-quickstep git commit: Fixed bugs in creating WorkOrderProtos.

Posted by ji...@apache.org.
Fixed bugs in creating WorkOrderProtos.


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

Branch: refs/heads/LIP-for-tpch
Commit: 2c0ce6a3bcf2ec40b0c32d077552cda3e225f787
Parents: 85e02de
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 18:42:32 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Tue Aug 9 10:42:25 2016 -0700

----------------------------------------------------------------------
 relational_operators/HashJoinOperator.cpp | 1 +
 relational_operators/WorkOrderFactory.cpp | 4 +---
 2 files changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2c0ce6a3/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 7851f41..779c0fe 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -393,6 +393,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *
 serialization::WorkOrder* HashJoinOperator::createOuterJoinWorkOrderProto(const block_id block) {
   serialization::WorkOrder *proto = new serialization::WorkOrder;
   proto->set_work_order_type(serialization::HASH_JOIN);
+  proto->set_query_id(query_id_);
 
   proto->SetExtension(serialization::HashJoinWorkOrder::hash_join_work_order_type,
                       serialization::HashJoinWorkOrder::HASH_OUTER_JOIN);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/2c0ce6a3/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 721d735..7d7af59 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -533,13 +533,11 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
         return false;
       }
 
-      const CatalogRelationSchema &build_relation = catalog_database.getRelationSchemaById(build_relation_id);
       const CatalogRelationSchema &probe_relation = catalog_database.getRelationSchemaById(probe_relation_id);
       for (int i = 0; i < proto.ExtensionSize(serialization::HashJoinWorkOrder::join_key_attributes); ++i) {
         const attribute_id attr_id =
             proto.GetExtension(serialization::HashJoinWorkOrder::join_key_attributes, i);
-        if (!build_relation.hasAttributeWithId(attr_id) ||
-            !probe_relation.hasAttributeWithId(attr_id)) {
+        if (!probe_relation.hasAttributeWithId(attr_id)) {
           return false;
         }
       }


[09/16] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/rules/AttachBloomFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.cpp b/query_optimizer/rules/AttachBloomFilters.cpp
new file mode 100644
index 0000000..03a42a0
--- /dev/null
+++ b/query_optimizer/rules/AttachBloomFilters.cpp
@@ -0,0 +1,308 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "query_optimizer/rules/AttachBloomFilters.hpp"
+
+#include <memory>
+#include <set>
+#include <unordered_set>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.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/TopLevelPlan.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr AttachBloomFilters::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()));
+
+  visitProducer(input, 0);
+  visitConsumer(input);
+
+//  for (const auto &info_vec_pair : consumers_) {
+//    std::cerr << "--------\n"
+//              << "Node " << info_vec_pair.first->getName()
+//              << " " << info_vec_pair.first << "\n";
+//
+//    for (const auto &info : info_vec_pair.second) {
+//      std::cerr << info.attribute->attribute_alias();
+//      if (info.attribute->id() != info.source_attribute->id()) {
+//        std::cerr << "{FROM " << info.source_attribute->attribute_alias() << "}";
+//      }
+//      if (info.from_sibling) {
+//        std::cerr << " sibling";
+//      }
+//      std::cerr << " @" << info.source << "[" << info.depth << "]"
+//                << ": " << info.selectivity << "\n";
+//    }
+//    std::cerr << "********\n";
+//  }
+
+  return visitAndAttach(input);
+}
+
+void AttachBloomFilters::visitProducer(const P::PhysicalPtr &node, const int depth) {
+  for (const P::PhysicalPtr &child : node->children()) {
+    visitProducer(child, depth+1);
+  }
+
+  std::vector<BloomFilterInfo> bloom_filters;
+
+  if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+    const P::HashJoinPtr &hash_join =
+        std::static_pointer_cast<const P::HashJoin>(node);
+    const P::PhysicalPtr &build_node = hash_join->right();
+    double selectivity = cost_model_->estimateSelectivity(build_node);
+    if (selectivity < 1.0) {
+      auto &build_node_info = producers_[build_node];
+      for (const auto &attr : hash_join->right_join_attributes()) {
+        build_node_info.emplace_back(node, attr, depth, selectivity, false);
+      }
+    }
+  }
+
+  const std::vector<E::AttributeReferencePtr> output_attributes(
+      node->getOutputAttributes());
+  std::unordered_set<E::ExprId> output_attribute_ids;
+  for (const auto &attr : output_attributes) {
+    output_attribute_ids.emplace(attr->id());
+  }
+
+  // First check inherited bloom filters
+  std::vector<const BloomFilterInfo*> candidates;
+  switch (node->getPhysicalType()) {
+    case P::PhysicalType::kAggregate:
+    case P::PhysicalType::kSelection:
+    case P::PhysicalType::kHashJoin: {
+      for (const P::PhysicalPtr &child : node->children()) {
+        for (const BloomFilterInfo &info : producers_[child]) {
+          candidates.emplace_back(&info);
+        }
+      }
+    }
+    default:
+      break;
+  }
+
+  for (const BloomFilterInfo *info : candidates) {
+    if (output_attribute_ids.find(info->attribute->id()) != output_attribute_ids.end()) {
+      bloom_filters.emplace_back(
+          info->source, info->attribute, info->depth, info->selectivity, false);
+    }
+  }
+
+  // Self-produced bloom filters
+//  double selectivity = cost_model_->estimateSelectivity(node);
+//  if (selectivity < 1.0) {
+//    for (const auto &attr : output_attributes) {
+//      bloom_filters.emplace_back(node, attr, depth, selectivity, false);
+//    }
+//  }
+
+  producers_.emplace(node, std::move(bloom_filters));
+}
+
+void AttachBloomFilters::visitConsumer(const P::PhysicalPtr &node) {
+  std::vector<BloomFilterInfo> bloom_filters;
+
+  // Bloom filters from parent
+  const auto &parent_bloom_filters = consumers_[node];
+  if (!parent_bloom_filters.empty()) {
+    for (const auto &child : node->children()) {
+      std::unordered_set<E::ExprId> child_output_attribute_ids;
+      for (const auto &attr : child->getOutputAttributes()) {
+        child_output_attribute_ids.emplace(attr->id());
+      }
+
+      std::vector<BloomFilterInfo> bloom_filters;
+      for (const auto &info : parent_bloom_filters) {
+        if (child_output_attribute_ids.find(info.attribute->id())
+                != child_output_attribute_ids.end()) {
+          bloom_filters.emplace_back(info.source,
+                                     info.attribute,
+                                     info.depth,
+                                     info.selectivity,
+                                     false,
+                                     info.source_attribute);
+        }
+      }
+      consumers_.emplace(child, std::move(bloom_filters));
+    }
+  }
+
+  // Bloom filters from build side to probe side via HashJoin
+  if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+    const P::HashJoinPtr hash_join =
+        std::static_pointer_cast<const P::HashJoin>(node);
+    if (hash_join->join_type() == P::HashJoin::JoinType::kInnerJoin ||
+        hash_join->join_type() == P::HashJoin::JoinType::kLeftSemiJoin) {
+      const P::PhysicalPtr &producer_child = hash_join->right();
+      const P::PhysicalPtr &consumer_child = hash_join->left();
+      std::unordered_map<E::ExprId, E::AttributeReferencePtr> join_attribute_pairs;
+      for (std::size_t i = 0; i < hash_join->left_join_attributes().size(); ++i) {
+        const E::AttributeReferencePtr probe_join_attribute =
+            hash_join->left_join_attributes()[i];
+        const E::AttributeReferencePtr build_join_attribute =
+            hash_join->right_join_attributes()[i];
+        join_attribute_pairs.emplace(build_join_attribute->id(),
+                                     probe_join_attribute);
+      }
+
+      auto &consumer_bloom_filters = consumers_[consumer_child];
+      for (const auto &info : producers_[producer_child]) {
+        const auto pair_it = join_attribute_pairs.find(info.attribute->id());
+        if (pair_it != join_attribute_pairs.end()) {
+          consumer_bloom_filters.emplace_back(info.source,
+                                              pair_it->second,
+                                              info.depth,
+                                              info.selectivity,
+                                              true,
+                                              info.attribute);
+        }
+      }
+    }
+  }
+
+  P::PhysicalPtr consumer_child = nullptr;
+  if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+    consumer_child = std::static_pointer_cast<const P::HashJoin>(node)->left();
+  }
+  if (node->getPhysicalType() == P::PhysicalType::kAggregate) {
+    consumer_child = std::static_pointer_cast<const P::Aggregate>(node)->input();
+  }
+
+  if (consumer_child != nullptr) {
+    // Decide attaches
+    auto &consumer_bloom_filters = consumers_[consumer_child];
+    if (cost_model_->estimateCardinality(consumer_child) > 10000000 &&
+        !consumer_bloom_filters.empty()) {
+      std::map<E::AttributeReferencePtr, const BloomFilterInfo*> filters;
+      for (const auto &info : consumer_bloom_filters) {
+        auto it = filters.find(info.attribute);
+        if (it == filters.end()) {
+          filters.emplace(info.attribute, &info);
+        } else {
+          if (BloomFilterInfo::isBetterThan(&info, it->second)) {
+            it->second = &info;
+          }
+        }
+      }
+
+      auto &probe_attaches = getBloomFilterConfig(node);
+      for (const auto &pair : filters) {
+        auto &build_attaches = getBloomFilterConfig(pair.second->source);
+        build_attaches.addBuildSideBloomFilter(
+            pair.second->source_attribute);
+        probe_attaches.addProbeSideBloomFilter(
+            pair.first,
+            pair.second->source_attribute,
+            pair.second->source);
+      }
+    }
+  }
+
+  for (const auto &child : node->children()) {
+    visitConsumer(child);
+  }
+}
+
+P::PhysicalPtr AttachBloomFilters::visitAndAttach(const physical::PhysicalPtr &node) {
+  std::vector<P::PhysicalPtr> new_children;
+  bool has_changed = false;
+  for (const auto &child : node->children()) {
+    P::PhysicalPtr new_child = visitAndAttach(child);
+    if (new_child != child) {
+      has_changed = true;
+    }
+    new_children.emplace_back(new_child);
+  }
+
+  if (node->getPhysicalType() == P::PhysicalType::kHashJoin) {
+    const auto attach_it = attaches_.find(node);
+    if (attach_it != attaches_.end()) {
+//      for (const auto& item : attach_it->second.probe_side_bloom_filters) {
+//        std::cout << "Attach probe from " << item.builder
+//                  << " to " << node << "\n";
+//      }
+
+      const P::HashJoinPtr hash_join =
+          std::static_pointer_cast<const P::HashJoin>(node);
+      return P::HashJoin::Create(
+          new_children[0],
+          new_children[1],
+          hash_join->left_join_attributes(),
+          hash_join->right_join_attributes(),
+          hash_join->residual_predicate(),
+          hash_join->project_expressions(),
+          hash_join->join_type(),
+          attach_it->second);
+    }
+  }
+
+  if (node->getPhysicalType() == P::PhysicalType::kAggregate) {
+    const auto attach_it = attaches_.find(node);
+    if (attach_it != attaches_.end()) {
+//      for (const auto& item : attach_it->second.probe_side_bloom_filters) {
+//        std::cout << "Attach probe from " << item.builder
+//                  << " to " << node << "\n";
+//      }
+
+      const P::AggregatePtr aggregate =
+          std::static_pointer_cast<const P::Aggregate>(node);
+      return P::Aggregate::Create(
+          aggregate->input(),
+          aggregate->grouping_expressions(),
+          aggregate->aggregate_expressions(),
+          aggregate->filter_predicate(),
+          attach_it->second);
+    }
+  }
+
+  if (has_changed) {
+    return node->copyWithNewChildren(new_children);
+  }
+
+  return node;
+}
+
+P::BloomFilterConfig& AttachBloomFilters::getBloomFilterConfig(const physical::PhysicalPtr &node) {
+  if (attaches_.find(node) == attaches_.end()) {
+    attaches_.emplace(node, node);
+  }
+  return attaches_[node];
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/rules/AttachBloomFilters.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachBloomFilters.hpp b/query_optimizer/rules/AttachBloomFilters.hpp
new file mode 100644
index 0000000..e4437f7
--- /dev/null
+++ b/query_optimizer/rules/AttachBloomFilters.hpp
@@ -0,0 +1,118 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_RULES_ATTACH_BLOOM_FILTERS_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_ATTACH_BLOOM_FILTERS_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#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"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief TODO
+ */
+class AttachBloomFilters : public Rule<physical::Physical> {
+ public:
+  AttachBloomFilters() {}
+
+  ~AttachBloomFilters() override {}
+
+  std::string getName() const override {
+    return "AttachBloomFilters";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  struct BloomFilterInfo {
+    BloomFilterInfo(const physical::PhysicalPtr &source_in,
+                    const expressions::AttributeReferencePtr &attribute_in,
+                    const int depth_in,
+                    const double selectivity_in,
+                    const bool from_sibling_in,
+                    const expressions::AttributeReferencePtr &source_attribute_in = nullptr)
+        : source(source_in),
+          attribute(attribute_in),
+          depth(depth_in),
+          selectivity(selectivity_in),
+          from_sibling(from_sibling_in),
+          source_attribute(
+              source_attribute_in == nullptr
+                  ? attribute_in
+                  : source_attribute_in) {
+
+    }
+    static bool isBetterThan(const BloomFilterInfo *a,
+                             const BloomFilterInfo *b) {
+      if (a->selectivity == b->selectivity) {
+        return a->depth > b->depth;
+      } else {
+        return a->selectivity < b->selectivity;
+      }
+    }
+    physical::PhysicalPtr source;
+    expressions::AttributeReferencePtr attribute;
+    int depth;
+    double selectivity;
+    bool from_sibling;
+    expressions::AttributeReferencePtr source_attribute;
+  };
+
+  void visitProducer(const physical::PhysicalPtr &node, const int depth);
+
+  void visitConsumer(const physical::PhysicalPtr &node);
+
+  physical::PhysicalPtr visitAndAttach(const physical::PhysicalPtr &node);
+
+  physical::BloomFilterConfig &getBloomFilterConfig(const physical::PhysicalPtr &node);
+
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+
+  std::map<physical::PhysicalPtr, std::vector<BloomFilterInfo>> producers_;
+  std::map<physical::PhysicalPtr, std::vector<BloomFilterInfo>> consumers_;
+  std::map<physical::PhysicalPtr, physical::BloomFilterConfig> attaches_;
+
+  DISALLOW_COPY_AND_ASSIGN(AttachBloomFilters);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_QUERY_OPTIMIZER_RULES_ATTACH_BLOOM_FILTERS_HPP_ */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index d9709ce..54b1e59 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -18,6 +18,7 @@
 add_subdirectory(tests)
 
 # Declare micro-libs:
+add_library(quickstep_queryoptimizer_rules_AttachBloomFilters AttachBloomFilters.cpp AttachBloomFilters.hpp)
 add_library(quickstep_queryoptimizer_rules_BottomUpRule ../../empty_src.cpp BottomUpRule.hpp)
 add_library(quickstep_queryoptimizer_rules_CollapseProject CollapseProject.cpp CollapseProject.hpp)
 add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp GenerateJoins.hpp)
@@ -36,6 +37,20 @@ add_library(quickstep_queryoptimizer_rules_UnnestSubqueries UnnestSubqueries.cpp
 
 
 # Link dependencies:
+target_link_libraries(quickstep_queryoptimizer_rules_AttachBloomFilters
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_BottomUpRule
                       glog
                       quickstep_queryoptimizer_rules_Rule
@@ -127,6 +142,7 @@ target_link_libraries(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOpti
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_DisjointTreeForest
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_costmodel_SimpleCostModel
@@ -187,6 +203,7 @@ target_link_libraries(quickstep_queryoptimizer_rules_UpdateExpression
 # Module all-in-one library:
 add_library(quickstep_queryoptimizer_rules ../../empty_src.cpp OptimizerRulesModule.hpp)
 target_link_libraries(quickstep_queryoptimizer_rules
+                      quickstep_queryoptimizer_rules_AttachBloomFilters
                       quickstep_queryoptimizer_rules_BottomUpRule
                       quickstep_queryoptimizer_rules_CollapseProject
                       quickstep_queryoptimizer_rules_GenerateJoins

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index 946d316..15a7154 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -33,6 +33,7 @@
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "utility/DisjointTreeForest.hpp"
 
 #include "glog/logging.h"
 
@@ -74,6 +75,9 @@ P::PhysicalPtr StarSchemaHashJoinOrderOptimization::applyInternal(const P::Physi
     JoinGroupInfo *join_group = nullptr;
     if (parent_join_group == nullptr || !is_valid_cascading_hash_join) {
       new_join_group.reset(new JoinGroupInfo());
+      for (const auto &attr : input->getReferencedAttributes()) {
+        new_join_group->referenced_attributes.emplace(attr->id());
+      }
       join_group = new_join_group.get();
     } else {
       join_group = parent_join_group;
@@ -146,7 +150,10 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
         i,
         tables[i],
         cost_model_->estimateCardinality(tables[i]),
-        cost_model_->estimateSelectivity(tables[i]));
+        cost_model_->estimateSelectivity(tables[i]),
+        CountSharedAttributes(join_group.referenced_attributes,
+                              tables[i]->getOutputAttributes()),
+        tables[i]->getPhysicalType() == physical::PhysicalType::kAggregate);
   }
 
   // Auxiliary mapping info.
@@ -163,9 +170,19 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
     }
   }
 
-  // Create a join graph where tables are vertices, and add an edge between vertices
-  // t1 and t2 for each join predicate t1.x = t2.y
-  std::vector<std::unordered_set<std::size_t>> join_graph(table_info_storage.size());
+  std::set<TableInfo*> remaining_tables;
+  for (auto &table_info : table_info_storage) {
+    remaining_tables.emplace(&table_info);
+  }
+
+  DisjointTreeForest<E::ExprId> join_attribute_forest;
+  for (const auto &attr_id_pair : join_group.join_attribute_pairs) {
+    join_attribute_forest.makeSet(attr_id_pair.first);
+    join_attribute_forest.makeSet(attr_id_pair.second);
+    join_attribute_forest.merge(attr_id_pair.first, attr_id_pair.second);
+  }
+
+  std::map<std::size_t, std::map<std::size_t, E::ExprId>> join_attribute_groups;
   for (const auto &attr_id_pair : join_group.join_attribute_pairs) {
     DCHECK(attribute_id_to_table_info_index_map.find(attr_id_pair.first)
                != attribute_id_to_table_info_index_map.end());
@@ -178,128 +195,169 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
         attribute_id_to_table_info_index_map[attr_id_pair.second];
     DCHECK_NE(first_table_idx, second_table_idx);
 
-    table_info_storage[first_table_idx].join_attribute_pairs.emplace(
-        attr_id_pair.first, attr_id_pair.second);
-    table_info_storage[second_table_idx].join_attribute_pairs.emplace(
-        attr_id_pair.second, attr_id_pair.first);
-
-    join_graph[first_table_idx].emplace(second_table_idx);
-    join_graph[second_table_idx].emplace(first_table_idx);
-  }
-
-  std::set<TableInfo*, TableInfoPtrLessComparator> table_info_ordered_by_priority;
-  for (std::size_t i = 0; i < table_info_storage.size(); ++i) {
-    table_info_ordered_by_priority.emplace(&table_info_storage[i]);
+    DCHECK_EQ(join_attribute_forest.find(attr_id_pair.first),
+              join_attribute_forest.find(attr_id_pair.second));
+    const std::size_t attr_group_id = join_attribute_forest.find(attr_id_pair.first);
+    auto &attr_group = join_attribute_groups[attr_group_id];
+    attr_group.emplace(first_table_idx, attr_id_pair.first);
+    attr_group.emplace(second_table_idx, attr_id_pair.second);
   }
 
-  // Contruct hash join tree.
   while (true) {
-    TableInfo *first_table_info = *table_info_ordered_by_priority.begin();
-    table_info_ordered_by_priority.erase(
-        table_info_ordered_by_priority.begin());
-    const std::size_t first_table_info_id = first_table_info->table_info_id;
-
-    TableInfo *second_table_info = nullptr;
-    std::set<TableInfo*, TableInfoPtrLessComparator>::iterator second_table_info_it;
-    for (auto candidate_table_info_it = table_info_ordered_by_priority.begin();
-         candidate_table_info_it != table_info_ordered_by_priority.end();
-         ++candidate_table_info_it) {
-      TableInfo *candidate_table_info = *candidate_table_info_it;
-      const std::size_t candidate_table_info_id = candidate_table_info->table_info_id;
-
-      if (join_graph[first_table_info_id].find(candidate_table_info_id)
-              == join_graph[first_table_info_id].end() &&
-          join_graph[candidate_table_info_id].find(first_table_info_id)
-              == join_graph[candidate_table_info_id].end()) {
-        continue;
-      } else if (second_table_info == nullptr) {
-        second_table_info = candidate_table_info;
-        second_table_info_it = candidate_table_info_it;
-      }
-
-      bool is_likely_many_to_many_join = false;
-      for (const auto join_attr_pair : first_table_info->join_attribute_pairs) {
-        if (candidate_table_info->joined_attribute_set.find(join_attr_pair.second)
-                != candidate_table_info->joined_attribute_set.end()) {
-          is_likely_many_to_many_join = true;
-          break;
-        }
-      }
-      for (const auto join_attr_pair : candidate_table_info->join_attribute_pairs) {
-        if (first_table_info->joined_attribute_set.find(join_attr_pair.second)
-                != first_table_info->joined_attribute_set.end()) {
-          is_likely_many_to_many_join = true;
-          break;
+    // TODO(jianqiao): design better data structure to improve efficiency here.
+    std::unique_ptr<JoinPair> best_join = nullptr;
+    for (TableInfo *probe_table_info : remaining_tables) {
+      for (TableInfo *build_table_info : remaining_tables) {
+        if (probe_table_info != build_table_info) {
+          std::vector<E::AttributeReferencePtr> build_attrs;
+          const std::size_t probe_table_id = probe_table_info->table_info_id;
+          const std::size_t build_table_id = build_table_info->table_info_id;
+          for (const auto &attr_group_pair : join_attribute_groups) {
+            const auto &attr_group = attr_group_pair.second;
+            auto probe_it = attr_group.find(probe_table_id);
+            auto build_it = attr_group.find(build_table_id);
+            if (probe_it != attr_group.end() && build_it != attr_group.end()) {
+              build_attrs.emplace_back(
+                  attribute_id_to_reference_map.at(build_it->second));
+            }
+          }
+          if (!build_attrs.empty()
+              && build_table_info->table->impliesUniqueAttributes(build_attrs)) {
+            std::unique_ptr<JoinPair> new_join(
+                new JoinPair(probe_table_info, build_table_info));
+            if (best_join == nullptr || new_join->isBetterThan(*best_join)) {
+//              if (best_join != nullptr) {
+//                std::cerr << "(" << best_join->probe->estimated_selectivity
+//                          << ", " << best_join->probe->estimated_cardinality << ")"
+//                          << " -- "
+//                          << "(" << best_join->build->estimated_selectivity
+//                          << ", " << best_join->build->estimated_cardinality << ")"
+//                          << "\n";
+//                std::cerr << "REPLACED WITH\n";
+//              }
+//              std::cerr << "(" << new_join->probe->estimated_selectivity
+//                        << ", " << new_join->probe->estimated_cardinality << ")"
+//                        << " -- "
+//                        << "(" << new_join->build->estimated_selectivity
+//                        << ", " << new_join->build->estimated_cardinality << ")"
+//                        << "\n****\n";
+              best_join.reset(new_join.release());
+            }
+          }
         }
       }
-      if (!is_likely_many_to_many_join) {
-        second_table_info = candidate_table_info;
-        second_table_info_it = candidate_table_info_it;
-        break;
-      }
     }
-    DCHECK(second_table_info != nullptr);
-    table_info_ordered_by_priority.erase(second_table_info_it);
 
-    const P::PhysicalPtr &left_child = first_table_info->table;
-    const P::PhysicalPtr &right_child = second_table_info->table;
+    TableInfo *selected_probe_table_info = nullptr;
+    TableInfo *selected_build_table_info = nullptr;
+
+    if (best_join != nullptr) {
+      selected_probe_table_info = best_join->probe;
+      selected_build_table_info = best_join->build;
+    }
+
+    // TODO(jianqiao): Handle the case when there is no primary key-foreign key information available.
+    CHECK(selected_probe_table_info != nullptr);
+    CHECK(selected_build_table_info != nullptr);
+
+//    std::cerr << selected_probe_table_info->estimated_selectivity
+//              << " -- "
+//              << selected_build_table_info->estimated_selectivity
+//              << "\n";
+
+//    std::cerr << selected_probe_table_info->estimated_num_output_attributes
+//              << " -- "
+//              << selected_build_table_info->estimated_num_output_attributes
+//              << "\n";
+
+    remaining_tables.erase(selected_probe_table_info);
+    remaining_tables.erase(selected_build_table_info);
+
+    const P::PhysicalPtr &probe_child = selected_probe_table_info->table;
+    const P::PhysicalPtr &build_child = selected_build_table_info->table;
     std::vector<E::NamedExpressionPtr> output_attributes;
-    for (const E::AttributeReferencePtr &left_attr : left_child->getOutputAttributes()) {
-      output_attributes.emplace_back(left_attr);
+    for (const E::AttributeReferencePtr &probe_attr : probe_child->getOutputAttributes()) {
+      output_attributes.emplace_back(probe_attr);
     }
-    for (const E::AttributeReferencePtr &right_attr : right_child->getOutputAttributes()) {
-      output_attributes.emplace_back(right_attr);
+    for (const E::AttributeReferencePtr &build_attr : build_child->getOutputAttributes()) {
+      output_attributes.emplace_back(build_attr);
     }
 
-    std::vector<E::AttributeReferencePtr> left_join_attributes;
-    std::vector<E::AttributeReferencePtr> right_join_attributes;
-    std::unordered_set<expressions::ExprId> new_joined_attribute_set;
-    for (const auto &join_attr_pair : first_table_info->join_attribute_pairs) {
-      if (second_table_info->join_attribute_pairs.find(join_attr_pair.second)
-              != second_table_info->join_attribute_pairs.end()) {
-        left_join_attributes.emplace_back(
-            attribute_id_to_reference_map[join_attr_pair.first]);
-        right_join_attributes.emplace_back(
-            attribute_id_to_reference_map[join_attr_pair.second]);
-
-        new_joined_attribute_set.emplace(join_attr_pair.first);
-        new_joined_attribute_set.emplace(join_attr_pair.second);
+    std::vector<E::AttributeReferencePtr> probe_attributes;
+    std::vector<E::AttributeReferencePtr> build_attributes;
+    const std::size_t probe_table_id = selected_probe_table_info->table_info_id;
+    const std::size_t build_table_id = selected_build_table_info->table_info_id;
+    for (const auto &attr_group_pair : join_attribute_groups) {
+      const auto &attr_group = attr_group_pair.second;
+      auto probe_it = attr_group.find(probe_table_id);
+      auto build_it = attr_group.find(build_table_id);
+      if (probe_it != attr_group.end() && build_it != attr_group.end()) {
+        probe_attributes.emplace_back(
+            attribute_id_to_reference_map.at(probe_it->second));
+        build_attributes.emplace_back(
+            attribute_id_to_reference_map.at(build_it->second));
       }
     }
-    DCHECK_GE(left_join_attributes.size(), static_cast<std::size_t>(1));
 
-    if (table_info_ordered_by_priority.size() > 0) {
+    if (remaining_tables.size() > 0) {
       P::PhysicalPtr output =
-          P::HashJoin::Create(left_child,
-                              right_child,
-                              left_join_attributes,
-                              right_join_attributes,
+          P::HashJoin::Create(probe_child,
+                              build_child,
+                              probe_attributes,
+                              build_attributes,
                               nullptr,
                               output_attributes,
                               P::HashJoin::JoinType::kInnerJoin);
 
-      second_table_info->table = output;
+//      P::PhysicalPtr output;
+//      if (selected_build_table_info->estimated_num_output_attributes >= 4 &&
+//          selected_probe_table_info->estimated_num_output_attributes < 4) {
+//        output = P::HashJoin::Create(build_child,
+//                                     probe_child,
+//                                     build_attributes,
+//                                     probe_attributes,
+//                                     nullptr,
+//                                     output_attributes,
+//                                     P::HashJoin::JoinType::kInnerJoin);
+//      } else {
+//        output = P::HashJoin::Create(probe_child,
+//                                     build_child,
+//                                     probe_attributes,
+//                                     build_attributes,
+//                                     nullptr,
+//                                     output_attributes,
+//                                     P::HashJoin::JoinType::kInnerJoin);
+//      }
+
+      selected_probe_table_info->table = output;
 
       // TODO(jianqiao): Cache the estimated cardinality for each plan in cost
       // model to avoid duplicated estimation.
-      second_table_info->estimated_cardinality = cost_model_->estimateCardinality(output);
-
-      second_table_info->join_attribute_pairs.insert(first_table_info->join_attribute_pairs.begin(),
-                                                     first_table_info->join_attribute_pairs.end());
-      second_table_info->joined_attribute_set.insert(first_table_info->joined_attribute_set.begin(),
-                                                     first_table_info->joined_attribute_set.end());
-      second_table_info->joined_attribute_set.insert(new_joined_attribute_set.begin(),
-                                                     new_joined_attribute_set.end());
-      table_info_ordered_by_priority.emplace(second_table_info);
-
-      join_graph[second_table_info->table_info_id].insert(join_graph[first_table_info_id].begin(),
-                                                          join_graph[first_table_info_id].end());
-
+      selected_probe_table_info->estimated_cardinality = cost_model_->estimateCardinality(output);
+      selected_probe_table_info->estimated_selectivity = cost_model_->estimateSelectivity(output);
+
+      selected_probe_table_info->estimated_num_output_attributes =
+          CountSharedAttributes(join_group.referenced_attributes,
+                                output->getOutputAttributes());
+      selected_probe_table_info->is_aggregation = false;
+
+      remaining_tables.emplace(selected_probe_table_info);
+
+      // Update join attribute groups.
+      for (auto &attr_group_pair : join_attribute_groups) {
+        auto &attr_group = attr_group_pair.second;
+        auto build_it = attr_group.find(build_table_id);
+        if (build_it != attr_group.end()) {
+          const E::ExprId attr_id = build_it->second;
+          attr_group.erase(build_it);
+          attr_group.emplace(probe_table_id, attr_id);
+        }
+      }
     } else {
-      return P::HashJoin::Create(left_child,
-                                 right_child,
-                                 left_join_attributes,
-                                 right_join_attributes,
+      return P::HashJoin::Create(probe_child,
+                                 build_child,
+                                 probe_attributes,
+                                 build_attributes,
                                  residual_predicate,
                                  project_expressions,
                                  P::HashJoin::JoinType::kInnerJoin);
@@ -307,5 +365,18 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
   }
 }
 
+std::size_t StarSchemaHashJoinOrderOptimization::CountSharedAttributes(
+    const std::unordered_set<expressions::ExprId> &attr_set1,
+    const std::vector<expressions::AttributeReferencePtr> &attr_set2) {
+  std::size_t cnt = 0;
+  for (const auto &attr : attr_set2) {
+    if (attr_set1.find(attr->id()) != attr_set1.end()) {
+      ++cnt;
+    }
+  }
+  return cnt;
+}
+
+
 }  // namespace optimizer
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
index 4d6765c..eb21d03 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
@@ -64,6 +64,7 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
    * @brief A group of tables to form a hash join tree.
    */
   struct JoinGroupInfo {
+    std::unordered_set<expressions::ExprId> referenced_attributes;
     std::vector<physical::PhysicalPtr> tables;
     std::vector<std::pair<expressions::ExprId, expressions::ExprId>> join_attribute_pairs;
   };
@@ -72,49 +73,84 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
    * @brief Auxiliary information of a table for the optimizer.
    */
   struct TableInfo {
-    TableInfo(const std::size_t in_table_info_id,
-              const physical::PhysicalPtr &in_table,
-              const std::size_t in_estimated_cardinality,
-              const double in_estimated_selectivity)
-        : table_info_id(in_table_info_id),
-          table(in_table),
-          estimated_cardinality(in_estimated_cardinality),
-          estimated_selectivity(in_estimated_selectivity) {
+    TableInfo(const std::size_t table_info_id_in,
+              const physical::PhysicalPtr &table_in,
+              const std::size_t estimated_cardinality_in,
+              const double estimated_selectivity_in,
+              const std::size_t estimated_num_output_attributes_in,
+              const bool is_aggregation_in)
+        : table_info_id(table_info_id_in),
+          table(table_in),
+          estimated_cardinality(estimated_cardinality_in),
+          estimated_selectivity(estimated_selectivity_in),
+          estimated_num_output_attributes(estimated_num_output_attributes_in),
+          is_aggregation(is_aggregation_in) {
     }
 
     const std::size_t table_info_id;
     physical::PhysicalPtr table;
     std::size_t estimated_cardinality;
     double estimated_selectivity;
-    std::unordered_multimap<expressions::ExprId, expressions::ExprId> join_attribute_pairs;
-    std::unordered_set<expressions::ExprId> joined_attribute_set;
+    std::size_t estimated_num_output_attributes;
+    bool is_aggregation;
   };
 
-  /**
-   * @brief Comparator that compares the join priorities between two tables.
-   */
-  struct TableInfoPtrLessComparator {
-    inline bool operator() (const TableInfo *lhs, const TableInfo *rhs) {
-      bool swapped = false;
-      if (lhs->estimated_cardinality > rhs->estimated_cardinality) {
-        std::swap(lhs, rhs);
-        swapped = true;
+  struct JoinPair {
+    JoinPair(TableInfo *probe_in, TableInfo *build_in)
+        : probe(probe_in), build(build_in) {
+    }
+
+    inline bool isBetterThan(const JoinPair &rhs) const {
+      const auto &lhs = *this;
+      const bool lhs_has_large_output =
+          lhs.build->estimated_num_output_attributes
+              + lhs.probe->estimated_num_output_attributes > 5;
+      const bool rhs_has_large_output =
+          rhs.build->estimated_num_output_attributes
+              + rhs.probe->estimated_num_output_attributes > 5;
+      if (lhs_has_large_output || rhs_has_large_output) {
+        if (lhs_has_large_output != rhs_has_large_output) {
+          return rhs_has_large_output;
+        }
+        double lhs_selectivity =
+            lhs.build->estimated_selectivity * lhs.probe->estimated_selectivity;
+        double rhs_selectivity =
+            rhs.build->estimated_selectivity * rhs.probe->estimated_selectivity;
+        if (lhs_selectivity != rhs_selectivity) {
+          return lhs_selectivity < rhs_selectivity;
+        }
       }
 
-      if (lhs->estimated_selectivity < rhs->estimated_selectivity) {
-        return !swapped;
-      } else if (lhs->estimated_cardinality < 1000u &&
-                 rhs->estimated_cardinality > 10000u &&
-                 lhs->estimated_selectivity < rhs->estimated_selectivity * 1.5) {
-        return !swapped;
-      } else if (lhs->estimated_selectivity > rhs->estimated_selectivity) {
-        return swapped;
-      } else if (lhs->estimated_cardinality != rhs->estimated_cardinality) {
-        return !swapped;
+      const bool lhs_has_small_build =
+          !lhs_has_large_output && lhs.build->estimated_cardinality < 0x100;
+      const bool rhs_has_small_build =
+          !rhs_has_large_output && rhs.build->estimated_cardinality < 0x100;
+      if (lhs_has_small_build != rhs_has_small_build) {
+        return lhs_has_small_build;
+      }
+
+      if (lhs.probe->is_aggregation != rhs.probe->is_aggregation) {
+        return lhs.probe->is_aggregation;
+      }
+
+      if (lhs.probe->estimated_cardinality != rhs.probe->estimated_cardinality) {
+        return lhs.probe->estimated_cardinality < rhs.probe->estimated_cardinality;
+      }
+      if (lhs.build->estimated_selectivity != rhs.build->estimated_selectivity) {
+        return lhs.build->estimated_selectivity < rhs.build->estimated_selectivity;
+      }
+      if (lhs.build->estimated_cardinality != rhs.build->estimated_cardinality) {
+        return lhs.build->estimated_cardinality < rhs.build->estimated_cardinality;
+      }
+      if (lhs.probe->table != rhs.probe->table) {
+        return lhs.probe->table < rhs.probe->table;
       } else {
-        return swapped ^ (lhs->table < rhs->table);
+        return lhs.build->table < rhs.build->table;
       }
     }
+
+    TableInfo *probe;
+    TableInfo *build;
   };
 
   physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input,
@@ -125,6 +161,10 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
       const expressions::PredicatePtr &residual_predicate,
       const std::vector<expressions::NamedExpressionPtr> &project_expressions);
 
+  static std::size_t CountSharedAttributes(
+      const std::unordered_set<expressions::ExprId> &attr_set1,
+      const std::vector<expressions::AttributeReferencePtr> &attr_set2);
+
   std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
 
   DISALLOW_COPY_AND_ASSIGN(StarSchemaHashJoinOrderOptimization);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
index 73b3e84..efea423 100644
--- a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
+++ b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
@@ -72,7 +72,8 @@ class ExecutionHeuristicsTest : public ::testing::Test {
                                           probe_relation,
                                           std::move(build_attribute_ids),
                                           std::move(probe_attribute_ids),
-                                          join_hash_table_id);
+                                          join_hash_table_id,
+                                          build_relation->estimateTupleCardinality());
   }
 
   QueryPlan::DAGNodeIndex createDummyBuildHashOperator(QueryPlan *query_plan,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 779c0fe..9b573ac 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -59,6 +59,11 @@ using std::vector;
 
 namespace quickstep {
 
+DEFINE_int64(bloom_adapter_batch_size, 64,
+             "Number of tuples to probe in bulk in Bloom filter adapter.");
+DEFINE_bool(adapt_bloom_filters, true,
+            "Whether to adaptively adjust the ordering of bloom filters.");
+
 namespace {
 
 // Functor passed to HashTable::getAllFromValueAccessor() to collect matching
@@ -75,6 +80,11 @@ class MapBasedJoinedTupleCollector {
     joined_tuples_[tref.block].emplace_back(tref.tuple, accessor.getCurrentPosition());
   }
 
+  inline void operator()(const tuple_id probe_tid,
+                         const TupleReference &build_tref) {
+    joined_tuples_[build_tref.block].emplace_back(build_tref.tuple, probe_tid);
+  }
+
   // Get a mutable pointer to the collected map of joined tuple ID pairs. The
   // key is inner block_id, values are vectors of joined tuple ID pairs with
   // tuple ID from the inner block on the left and the outer block on the

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index fa393b6..30571a1 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -307,8 +307,9 @@ class HashInnerJoinWorkOrder : public WorkOrder {
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
-      StorageManager *storage_manager)
-      : WorkOrder(query_id),
+      StorageManager *storage_manager,
+      const int op_index = -1)
+      : WorkOrder(query_id, op_index),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
@@ -354,8 +355,9 @@ class HashInnerJoinWorkOrder : public WorkOrder {
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
-      StorageManager *storage_manager)
-      : WorkOrder(query_id),
+      StorageManager *storage_manager,
+      const int op_index = -1)
+      : WorkOrder(query_id, op_index),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
@@ -435,8 +437,9 @@ class HashSemiJoinWorkOrder : public WorkOrder {
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
-      StorageManager *storage_manager)
-      : WorkOrder(query_id),
+      StorageManager *storage_manager,
+      const int op_index = -1)
+      : WorkOrder(query_id, op_index),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
@@ -482,8 +485,9 @@ class HashSemiJoinWorkOrder : public WorkOrder {
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
-      StorageManager *storage_manager)
-      : WorkOrder(query_id),
+      StorageManager *storage_manager,
+      const int op_index = -1)
+      : WorkOrder(query_id, op_index),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
@@ -559,8 +563,9 @@ class HashAntiJoinWorkOrder : public WorkOrder {
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
-      StorageManager *storage_manager)
-      : WorkOrder(query_id),
+      StorageManager *storage_manager,
+      const int op_index = -1)
+      : WorkOrder(query_id, op_index),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index c1b9b68..2eeca06 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -300,16 +300,23 @@ class WorkOrder {
     return query_id_;
   }
 
+  inline const int getOperatorIndex() const {
+    return op_index_;
+  }
+
  protected:
   /**
    * @brief Constructor.
    *
    * @param query_id The ID of the query to which this WorkOrder belongs.
    **/
-  explicit WorkOrder(const std::size_t query_id)
-      : query_id_(query_id) {}
+  explicit WorkOrder(const std::size_t query_id,
+                     const int op_index = -1)
+      : query_id_(query_id),
+        op_index_(op_index) {}
 
   const std::size_t query_id_;
+  const int op_index_;
   // A vector of preferred NUMA node IDs where this workorder should be executed.
   // These node IDs typically indicate the NUMA node IDs of the input(s) of the
   // workorder. Derived classes should ensure that there are no duplicate entries

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 3f6e23a..d225258 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -46,10 +46,13 @@
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
+#include "storage/ValueAccessor.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/BloomFilterAdapter.hpp"
 
 #include "glog/logging.h"
 
@@ -57,6 +60,8 @@ using std::unique_ptr;
 
 namespace quickstep {
 
+DECLARE_int64(bloom_adapter_batch_size);
+
 AggregationOperationState::AggregationOperationState(
     const CatalogRelationSchema &input_relation,
     const std::vector<const AggregateFunction*> &aggregate_functions,
@@ -64,12 +69,16 @@ AggregationOperationState::AggregationOperationState(
     std::vector<bool> &&is_distinct,
     std::vector<std::unique_ptr<const Scalar>> &&group_by,
     const Predicate *predicate,
+    std::vector<const BloomFilter *> &&bloom_filters,
+    std::vector<attribute_id> &&bloom_filter_attribute_ids,
     const std::size_t estimated_num_entries,
     const HashTableImplType hash_table_impl_type,
     const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
     StorageManager *storage_manager)
     : input_relation_(input_relation),
       predicate_(predicate),
+      bloom_filters_(std::move(bloom_filters)),
+      bloom_filter_attribute_ids_(std::move(bloom_filter_attribute_ids)),
       group_by_list_(std::move(group_by)),
       arguments_(std::move(arguments)),
       is_distinct_(std::move(is_distinct)),
@@ -183,7 +192,8 @@ AggregationOperationState::AggregationOperationState(
 AggregationOperationState* AggregationOperationState::ReconstructFromProto(
     const serialization::AggregationOperationState &proto,
     const CatalogDatabaseLite &database,
-    StorageManager *storage_manager) {
+    StorageManager *storage_manager,
+    const std::vector<std::unique_ptr<BloomFilter>> &bloom_filters) {
   DCHECK(ProtoIsValid(proto, database));
 
   // Rebuild contructor arguments from their representation in 'proto'.
@@ -232,12 +242,24 @@ AggregationOperationState* AggregationOperationState::ReconstructFromProto(
                                                database));
   }
 
+  std::vector<const BloomFilter*> bloom_filter_vector;
+  std::vector<attribute_id> bloom_filter_attribute_ids;
+  for (int i = 0; i < proto.bloom_filters_size(); ++i) {
+    // Add the pointer to the probe bloom filter within the list of probe bloom filters to use.
+    const auto bloom_filter_proto = proto.bloom_filters(i);
+    bloom_filter_vector.emplace_back(
+        bloom_filters[bloom_filter_proto.bloom_filter_id()].get());
+    bloom_filter_attribute_ids.emplace_back(bloom_filter_proto.attr_id());
+  }
+
   return new AggregationOperationState(database.getRelationSchemaById(proto.relation_id()),
                                        aggregate_functions,
                                        std::move(arguments),
                                        std::move(is_distinct),
                                        std::move(group_by_expressions),
                                        predicate.release(),
+                                       std::move(bloom_filter_vector),
+                                       std::move(bloom_filter_attribute_ids),
                                        proto.estimated_num_entries(),
                                        HashTableImplTypeFromProto(proto.hash_table_impl_type()),
                                        distinctify_hash_table_impl_types,
@@ -340,6 +362,10 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
   // tuples so that it can be reused across multiple aggregates (i.e. we only
   // pay the cost of evaluating the predicate once).
   std::unique_ptr<TupleIdSequence> reuse_matches;
+  if (predicate_) {
+    reuse_matches.reset(block->getMatchesForPredicate(predicate_.get()));
+  }
+
   for (std::size_t agg_idx = 0;
        agg_idx < handles_.size();
        ++agg_idx) {
@@ -358,7 +384,6 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
                                arguments_[agg_idx],
                                local_arguments_as_attributes,
                                {}, /* group_by */
-                               predicate_.get(),
                                distinctify_hashtables_[agg_idx].get(),
                                &reuse_matches,
                                nullptr /* reuse_group_by_vectors */);
@@ -369,7 +394,6 @@ void AggregationOperationState::aggregateBlockSingleState(const block_id input_b
           block->aggregate(*handles_[agg_idx],
                            arguments_[agg_idx],
                            local_arguments_as_attributes,
-                           predicate_.get(),
                            &reuse_matches));
     }
   }
@@ -391,6 +415,72 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
   // GROUP BY expressions once).
   std::vector<std::unique_ptr<ColumnVector>> reuse_group_by_vectors;
 
+  if (predicate_) {
+    reuse_matches.reset(block->getMatchesForPredicate(predicate_.get()));
+  }
+
+  if (bloom_filters_.size() > 0) {
+    const std::size_t num_tuples = block->getNumTuples();
+//    std::cerr << "Before: " << num_tuples << " -- "
+//              << (reuse_matches ? reuse_matches->numTuples() : num_tuples)
+//              << "\n";
+    std::unique_ptr<ValueAccessor> accessor;
+    if (reuse_matches) {
+      accessor.reset(
+          block->getTupleStorageSubBlock().createValueAccessor(reuse_matches.get()));
+    } else {
+      accessor.reset(
+          block->getTupleStorageSubBlock().createValueAccessor());
+    }
+    InvokeOnAnyValueAccessor(
+        accessor.get(),
+        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+      std::unique_ptr<TupleIdSequence> filtered(new TupleIdSequence(num_tuples));
+
+      std::vector<std::size_t> attr_size_vector;
+      attr_size_vector.reserve(bloom_filter_attribute_ids_.size());
+      for (const auto &attr : bloom_filter_attribute_ids_) {
+        auto val_and_size =
+            accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, attr);
+        attr_size_vector.emplace_back(val_and_size.second);
+      }
+
+      std::unique_ptr<BloomFilterAdapter> bloom_filter_adapter;
+      bloom_filter_adapter.reset(new BloomFilterAdapter(
+          bloom_filters_, bloom_filter_attribute_ids_, attr_size_vector));
+
+      std::uint32_t batch_size_try = FLAGS_bloom_adapter_batch_size;
+      std::uint32_t num_tuples_left = accessor->getNumTuples();
+      std::vector<tuple_id> batch(num_tuples_left);
+
+      do {
+        std::uint32_t batch_size =
+            batch_size_try < num_tuples_left ? batch_size_try : num_tuples_left;
+        for (std::size_t i = 0; i < batch_size; ++i) {
+          accessor->next();
+          batch.push_back(accessor->getCurrentPosition());
+        }
+
+        std::size_t num_hits =
+            bloom_filter_adapter->bulkProbe<true>(accessor, batch, batch_size);
+        for (std::size_t t = 0; t < num_hits; ++t){
+          filtered->set(batch[t], true);
+        }
+
+        batch.clear();
+        num_tuples_left -= batch_size;
+        batch_size_try = batch_size * 2;
+      } while (num_tuples_left > 0);
+
+      if (reuse_matches) {
+        reuse_matches->intersectWith(*filtered);
+      } else {
+        reuse_matches.reset(filtered.release());
+      }
+    });
+//    std::cerr << "After: " << reuse_matches->numTuples() << "\n";
+  }
+
   for (std::size_t agg_idx = 0;
        agg_idx < handles_.size();
        ++agg_idx) {
@@ -402,7 +492,6 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
                                arguments_[agg_idx],
                                nullptr, /* arguments_as_attributes */
                                group_by_list_,
-                               predicate_.get(),
                                distinctify_hashtables_[agg_idx].get(),
                                &reuse_matches,
                                &reuse_group_by_vectors);
@@ -416,7 +505,6 @@ void AggregationOperationState::aggregateBlockHashTable(const block_id input_blo
       block->aggregateGroupBy(*handles_[agg_idx],
                               arguments_[agg_idx],
                               group_by_list_,
-                              predicate_.get(),
                               agg_hash_table,
                               &reuse_matches,
                               &reuse_group_by_vectors);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index ecd116b..30b8f76 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -33,6 +33,7 @@
 #include "storage/HashTableBase.hpp"
 #include "storage/HashTablePool.hpp"
 #include "storage/StorageBlockInfo.hpp"
+#include "utility/BloomFilter.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -108,6 +109,8 @@ class AggregationOperationState {
                             std::vector<bool> &&is_distinct,
                             std::vector<std::unique_ptr<const Scalar>> &&group_by,
                             const Predicate *predicate,
+                            std::vector<const BloomFilter *> &&bloom_filters,
+                            std::vector<attribute_id> &&bloom_filter_attribute_ids,
                             const std::size_t estimated_num_entries,
                             const HashTableImplType hash_table_impl_type,
                             const std::vector<HashTableImplType> &distinctify_hash_table_impl_types,
@@ -131,7 +134,8 @@ class AggregationOperationState {
   static AggregationOperationState* ReconstructFromProto(
       const serialization::AggregationOperationState &proto,
       const CatalogDatabaseLite &database,
-      StorageManager *storage_manager);
+      StorageManager *storage_manager,
+      const std::vector<std::unique_ptr<BloomFilter>> &bloom_filters);
 
   /**
    * @brief Check whether a serialization::AggregationOperationState is
@@ -181,6 +185,10 @@ class AggregationOperationState {
   // filter predicate (if any), and the list of GROUP BY expressions (if any).
   const CatalogRelationSchema &input_relation_;
   std::unique_ptr<const Predicate> predicate_;
+
+  std::vector<const BloomFilter*> bloom_filters_;
+  std::vector<attribute_id> bloom_filter_attribute_ids_;
+
   std::vector<std::unique_ptr<const Scalar>> group_by_list_;
 
   // Each individual aggregate in this operation has an AggregationHandle and

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/AggregationOperationState.proto
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.proto b/storage/AggregationOperationState.proto
index 7521d73..04b661c 100644
--- a/storage/AggregationOperationState.proto
+++ b/storage/AggregationOperationState.proto
@@ -42,4 +42,10 @@ message AggregationOperationState {
 
   // Each DISTINCT aggregation has its distinctify hash table impl type.
   repeated HashTableImplType distinctify_hash_table_impl_types = 7;
+
+  message BloomFilter {
+    required uint32 bloom_filter_id = 1;
+    required uint32 attr_id = 2;
+  }
+  repeated BloomFilter bloom_filters = 8;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/BasicColumnStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/BasicColumnStoreValueAccessor.hpp b/storage/BasicColumnStoreValueAccessor.hpp
index 0560d99..f4c6cd9 100644
--- a/storage/BasicColumnStoreValueAccessor.hpp
+++ b/storage/BasicColumnStoreValueAccessor.hpp
@@ -20,6 +20,8 @@
 #ifndef QUICKSTEP_STORAGE_BASIC_COLUMN_STORE_VALUE_ACCESSOR_HPP_
 #define QUICKSTEP_STORAGE_BASIC_COLUMN_STORE_VALUE_ACCESSOR_HPP_
 
+#include <cstddef>
+#include <utility>
 #include <vector>
 
 #include "catalog/CatalogRelationSchema.hpp"
@@ -45,7 +47,8 @@ class BasicColumnStoreValueAccessorHelper {
       : relation_(relation),
         num_tuples_(num_tuples),
         column_stripes_(column_stripes),
-        column_null_bitmaps_(column_null_bitmaps) {
+        column_null_bitmaps_(column_null_bitmaps),
+        attr_max_lengths_(relation.getMaximumAttributeByteLengths()) {
   }
 
   inline tuple_id numPackedTuples() const {
@@ -63,9 +66,23 @@ class BasicColumnStoreValueAccessorHelper {
       return nullptr;
     }
 
-    // TODO(chasseur): Consider cacheing the byte lengths of attributes.
-    return static_cast<const char*>(column_stripes_[attr])
-           + (tuple * relation_.getAttributeById(attr)->getType().maximumByteLength());
+    return static_cast<const char*>(column_stripes_[attr]) + (tuple * attr_max_lengths_[attr]);
+  }
+
+  template <bool check_null>
+  inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+                                                                            const attribute_id attr) const {
+    DEBUG_ASSERT(tuple < num_tuples_);
+    DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
+    if (check_null
+        && (!column_null_bitmaps_.elementIsNull(attr))
+        && column_null_bitmaps_[attr].getBit(tuple)) {
+      return std::make_pair(nullptr, 0);
+    }
+
+    const std::size_t attr_length = attr_max_lengths_[attr];
+    return std::make_pair(static_cast<const char*>(column_stripes_[attr]) + (tuple * attr_length),
+                          attr_length);
   }
 
   inline TypedValue getAttributeValueTyped(const tuple_id tuple,
@@ -82,6 +99,7 @@ class BasicColumnStoreValueAccessorHelper {
   const tuple_id num_tuples_;
   const std::vector<void*> &column_stripes_;
   const PtrVector<BitVector<false>, true> &column_null_bitmaps_;
+  const std::vector<std::size_t> &attr_max_lengths_;
 
   DISALLOW_COPY_AND_ASSIGN(BasicColumnStoreValueAccessorHelper);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/BloomFilterIndexSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/BloomFilterIndexSubBlock.cpp b/storage/BloomFilterIndexSubBlock.cpp
index 4351c05..6ba466e 100644
--- a/storage/BloomFilterIndexSubBlock.cpp
+++ b/storage/BloomFilterIndexSubBlock.cpp
@@ -57,7 +57,6 @@ BloomFilterIndexSubBlock::BloomFilterIndexSubBlock(const TupleStorageSubBlock &t
                     sub_block_memory_size),
       is_initialized_(false),
       is_consistent_(false),
-      random_seed_(kBloomFilterSeed),
       bit_array_size_in_bytes_(description.GetExtension(
                                    BloomFilterIndexSubBlockDescription::bloom_filter_size)) {
   CHECK(DescriptionIsValid(relation_, description_))
@@ -76,8 +75,7 @@ BloomFilterIndexSubBlock::BloomFilterIndexSubBlock(const TupleStorageSubBlock &t
   const std::uint32_t salt_count = description.GetExtension(BloomFilterIndexSubBlockDescription::number_of_hashes);
 
   // Initialize the bloom_filter_ data structure to operate on bit_array.
-  bloom_filter_.reset(new BloomFilter(random_seed_,
-                                      salt_count,
+  bloom_filter_.reset(new BloomFilter(salt_count,
                                       bit_array_size_in_bytes_,
                                       bit_array_.get(),
                                       is_bloom_filter_initialized));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/BloomFilterIndexSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/BloomFilterIndexSubBlock.hpp b/storage/BloomFilterIndexSubBlock.hpp
index 09d6225..4ffbc2c 100644
--- a/storage/BloomFilterIndexSubBlock.hpp
+++ b/storage/BloomFilterIndexSubBlock.hpp
@@ -67,11 +67,6 @@ class BloomFilterIndexSubBlock : public IndexSubBlock {
     kSelectivityNone
   };
 
-  /**
-   * @brief A random seed to initialize the bloom filter hash functions.
-   **/
-  static const std::uint64_t kBloomFilterSeed = 0xA5A5A5A55A5A5A5AULL;
-
   BloomFilterIndexSubBlock(const TupleStorageSubBlock &tuple_store,
                            const IndexSubBlockDescription &description,
                            const bool new_block,
@@ -181,7 +176,6 @@ class BloomFilterIndexSubBlock : public IndexSubBlock {
  private:
   bool is_initialized_;
   bool is_consistent_;
-  const std::uint64_t random_seed_;
   const std::uint64_t bit_array_size_in_bytes_;
   std::vector<attribute_id> indexed_attribute_ids_;
   std::unique_ptr<unsigned char> bit_array_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 65a7975..8fbe7de 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -679,6 +679,8 @@ target_link_libraries(quickstep_storage_HashTable
                       quickstep_types_Type
                       quickstep_types_TypedValue
                       quickstep_utility_BloomFilter
+                      quickstep_utility_BloomFilterAdapter
+                      quickstep_utility_EventProfiler
                       quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_HashTableBase

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/CompressedColumnStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedColumnStoreValueAccessor.hpp b/storage/CompressedColumnStoreValueAccessor.hpp
index 25e5eed..31ddef5 100644
--- a/storage/CompressedColumnStoreValueAccessor.hpp
+++ b/storage/CompressedColumnStoreValueAccessor.hpp
@@ -54,6 +54,7 @@ class CompressedColumnStoreValueAccessorHelper {
       const PtrVector<BitVector<false>, true> &uncompressed_column_null_bitmaps)
       : relation_(relation),
         num_tuples_(num_tuples),
+        attr_max_lengths_(relation.getMaximumAttributeByteLengths()),
         compression_info_(compression_info),
         dictionary_coded_attributes_(dictionary_coded_attributes),
         truncated_attributes_(truncated_attributes),
@@ -86,6 +87,26 @@ class CompressedColumnStoreValueAccessorHelper {
     }
   }
 
+  template <bool check_null>
+  inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+                                                                            const attribute_id attr) const {
+    if (dictionary_coded_attributes_[attr]) {
+      return dictionaries_.atUnchecked(attr).getUntypedValueAndByteLengthForCode<check_null>(
+          getCode(tuple, attr));
+    } else if (truncated_attributes_[attr]) {
+      if (truncated_attribute_is_int_[attr]) {
+        int_buffer_ = getCode(tuple, attr);
+        return std::make_pair(&int_buffer_, sizeof(int_buffer_));
+      } else {
+        long_buffer_ = getCode(tuple, attr);
+        return std::make_pair(&long_buffer_, sizeof(long_buffer_));
+      }
+    } else {
+      return std::make_pair(getAttributePtr<check_null>(tuple, attr),
+                            attr_max_lengths_[attr]);
+    }
+  }
+
   inline TypedValue getAttributeValueTyped(const tuple_id tuple,
                                            const attribute_id attr) const {
     if (dictionary_coded_attributes_[attr]) {
@@ -140,6 +161,7 @@ class CompressedColumnStoreValueAccessorHelper {
   const CatalogRelationSchema &relation_;
 
   const tuple_id num_tuples_;
+  const std::vector<std::size_t> &attr_max_lengths_;
 
   const CompressedBlockInfo &compression_info_;
   const std::vector<bool> &dictionary_coded_attributes_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/CompressedPackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedPackedRowStoreValueAccessor.hpp b/storage/CompressedPackedRowStoreValueAccessor.hpp
index 8858175..66b2f5f 100644
--- a/storage/CompressedPackedRowStoreValueAccessor.hpp
+++ b/storage/CompressedPackedRowStoreValueAccessor.hpp
@@ -60,6 +60,7 @@ class CompressedPackedRowStoreValueAccessorHelper {
         num_tuples_(num_tuples),
         tuple_length_bytes_(tuple_length_bytes),
         attribute_offsets_(attribute_offsets),
+        attr_max_lengths_(relation.getMaximumAttributeByteLengths()),
         compression_info_(compression_info),
         dictionary_coded_attributes_(dictionary_coded_attributes),
         truncated_attributes_(truncated_attributes),
@@ -94,6 +95,26 @@ class CompressedPackedRowStoreValueAccessorHelper {
     }
   }
 
+  template <bool check_null>
+  inline std::pair<const void*, std::size_t> getAttributeValueAndByteLength(const tuple_id tuple,
+                                                                            const attribute_id attr) const {
+    if (dictionary_coded_attributes_[attr]) {
+      return dictionaries_.atUnchecked(attr).getUntypedValueAndByteLengthForCode<check_null>(
+          getCode(tuple, attr));
+    } else if (truncated_attributes_[attr]) {
+      if (truncated_attribute_is_int_[attr]) {
+        int_buffer_ = getCode(tuple, attr);
+        return std::make_pair(&int_buffer_, sizeof(int_buffer_));
+      } else {
+        long_buffer_ = getCode(tuple, attr);
+        return std::make_pair(&long_buffer_, sizeof(long_buffer_));
+      }
+    } else {
+      return std::make_pair(getAttributePtr<check_null>(tuple, attr),
+                            attr_max_lengths_[attr]);
+    }
+  }
+
   inline TypedValue getAttributeValueTyped(const tuple_id tuple,
                                            const attribute_id attr) const {
     if (dictionary_coded_attributes_[attr]) {
@@ -152,6 +173,7 @@ class CompressedPackedRowStoreValueAccessorHelper {
   const tuple_id num_tuples_;
   const std::size_t tuple_length_bytes_;
   const std::vector<std::size_t> &attribute_offsets_;
+  const std::vector<std::size_t> &attr_max_lengths_;
 
   const CompressedBlockInfo &compression_info_;
   const std::vector<bool> &dictionary_coded_attributes_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/HashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTable.hpp b/storage/HashTable.hpp
index 9fa41a2..3538181 100644
--- a/storage/HashTable.hpp
+++ b/storage/HashTable.hpp
@@ -23,6 +23,7 @@
 #include <atomic>
 #include <cstddef>
 #include <cstdlib>
+#include <memory>
 #include <type_traits>
 #include <vector>
 
@@ -39,11 +40,14 @@
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/BloomFilter.hpp"
+#include "utility/BloomFilterAdapter.hpp"
 #include "utility/HashPair.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
+DECLARE_int64(bloom_adapter_batch_size);
+
 /** \addtogroup Storage
  *  @{
  */
@@ -1016,8 +1020,12 @@ class HashTable : public HashTableBase<resizable,
    *
    * @param bloom_filter The pointer to the bloom filter.
    **/
-  inline void setBuildSideBloomFilter(BloomFilter *bloom_filter) {
-    build_bloom_filter_ = bloom_filter;
+  inline void addBuildSideBloomFilter(BloomFilter *bloom_filter) {
+    build_bloom_filters_.emplace_back(bloom_filter);
+  }
+
+  inline void addBuildSideAttributeId(const attribute_id build_attribute_id) {
+    build_attribute_ids_.push_back(build_attribute_id);
   }
 
   /**
@@ -1042,8 +1050,8 @@ class HashTable : public HashTableBase<resizable,
    * @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);
+  inline void addProbeSideAttributeId(const attribute_id probe_attribute_id) {
+    probe_attribute_ids_.push_back(probe_attribute_id);
   }
 
  protected:
@@ -1329,9 +1337,10 @@ class HashTable : public HashTableBase<resizable,
   // 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<BloomFilter *> build_bloom_filters_;
+  std::vector<attribute_id> build_attribute_ids_;
   std::vector<const BloomFilter*> probe_bloom_filters_;
-  std::vector<std::vector<attribute_id>> probe_attribute_ids_;
+  std::vector<attribute_id> probe_attribute_ids_;
 
   DISALLOW_COPY_AND_ASSIGN(HashTable);
 };
@@ -1477,12 +1486,26 @@ 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()));
+      for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
+        auto *build_bloom_filter = build_bloom_filters_[i];
+        std::unique_ptr<BloomFilter> thread_local_bloom_filter(
+            new BloomFilter(build_bloom_filter->getNumberOfHashes(),
+                            build_bloom_filter->getBitArraySize()));
+        const auto &build_attr = build_attribute_ids_[i];
+        const std::size_t attr_size =
+            accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, build_attr).second;
+        while (accessor->next()) {
+          thread_local_bloom_filter->insertUnSafe(
+              static_cast<const std::uint8_t *>(accessor->getUntypedValue(build_attr)),
+              attr_size);
+        }
+        build_bloom_filter->bitwiseOr(thread_local_bloom_filter.get());
+        accessor->beginIteration();
+      }
     }
+
     if (resizable) {
       while (result == HashTablePutResult::kOutOfSpace) {
         {
@@ -1498,11 +1521,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;
@@ -1528,20 +1546,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;
   });
@@ -1607,6 +1616,26 @@ HashTablePutResult HashTable<ValueT, resizable, serializable, force_key_copy, al
                                                         &prealloc_state);
       }
     }
+
+    if (has_build_side_bloom_filter_) {
+      for (std::size_t i = 0; i < build_bloom_filters_.size(); ++i) {
+        auto *build_bloom_filter = build_bloom_filters_[i];
+        std::unique_ptr<BloomFilter> thread_local_bloom_filter(
+            new BloomFilter(build_bloom_filter->getNumberOfHashes(),
+                            build_bloom_filter->getBitArraySize()));
+        const auto &build_attr = build_attribute_ids_[i];
+        const std::size_t attr_size =
+            accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, build_attr).second;
+        while (accessor->next()) {
+          thread_local_bloom_filter->insertUnSafe(
+              static_cast<const std::uint8_t *>(accessor->getUntypedValue(build_attr)),
+              attr_size);
+        }
+        build_bloom_filter->bitwiseOr(thread_local_bloom_filter.get());
+        accessor->beginIteration();
+      }
+    }
+
     if (resizable) {
       while (result == HashTablePutResult::kOutOfSpace) {
         {
@@ -2229,6 +2258,7 @@ inline std::size_t HashTable<ValueT, resizable, serializable, force_key_copy, al
   }
 }
 
+
 template <typename ValueT,
           bool resizable,
           bool serializable,
@@ -2246,42 +2276,85 @@ void HashTable<ValueT, resizable, serializable, force_key_copy, allow_duplicate_
   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;
+    std::unique_ptr<BloomFilterAdapter> bloom_filter_adapter;
+    if (has_probe_side_bloom_filter_) {
+      // Find (and cache) the size of each attribute in the probe lists.
+      // NOTE(nav): This code uses the accessor to get the size,
+      // and hence only works if there's at least one tuple.
+      std::vector<std::size_t> attr_size_vector;
+      attr_size_vector.reserve(probe_attribute_ids_.size());
+      for (const auto &probe_attr : probe_attribute_ids_) {
+        auto val_and_size =
+            accessor->template getUntypedValueAndByteLengthAtAbsolutePosition<false>(0, probe_attr);
+//        std::cerr << "BF attr size = " << val_and_size.second << "\n";
+        attr_size_vector.emplace_back(val_and_size.second);
+      }
+
+      bloom_filter_adapter.reset(new BloomFilterAdapter(
+              probe_bloom_filters_, probe_attribute_ids_, attr_size_vector));
+
+      // We want to have large batch sizes for cache efficiency while probeing,
+      // but small batch sizes to ensure that the adaptation logic kicks in
+      // (and does early). We use exponentially increasing batch sizes to
+      // achieve a balance between the two.
+      //
+      // We also keep track of num_tuples_left in the block, to ensure that
+      // we don't reserve an unnecessarily large vector.
+      std::uint32_t batch_size_try = FLAGS_bloom_adapter_batch_size;
+      std::uint32_t num_tuples_left = accessor->getNumTuples();
+      std::vector<tuple_id> batch(num_tuples_left);
+
+      do {
+        std::uint32_t batch_size =
+            batch_size_try < num_tuples_left ? batch_size_try : num_tuples_left;
+        for (std::size_t i = 0; i < batch_size; ++i) {
+          accessor->next();
+          batch.push_back(accessor->getCurrentPosition());
+        }
+
+        std::size_t num_hits =
+            bloom_filter_adapter->bulkProbe<true>(accessor, batch, batch_size);
+
+        for (std::size_t t = 0; t < num_hits; ++t){
+          tuple_id probe_tid = batch[t];
+          TypedValue key = accessor->getTypedValueAtAbsolutePosition(key_attr_id, probe_tid);
+          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 ValueT *value;
+          while (this->getNextEntryForKey(key, adjusted_hash, &value, &entry_num)) {
+            (*functor)(probe_tid, *value);
+            if (!allow_duplicate_keys)
               break;
-            }
           }
         }
-        if (bloom_miss) {
-          continue;  // On a bloom filter miss, probing the hash table can be skipped.
-        }
-      }
+        batch.clear();
+        num_tuples_left -= batch_size;
+        batch_size_try = batch_size * 2;
+      } while (!accessor->iterationFinished());
+    }
 
-      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 ValueT *value;
-      while (this->getNextEntryForKey(key, adjusted_hash, &value, &entry_num)) {
-        (*functor)(*accessor, *value);
-        if (!allow_duplicate_keys) {
-          break;
+    else { // no Bloom filters to probe
+      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 ValueT *value;
+        while (this->getNextEntryForKey(key, adjusted_hash, &value, &entry_num)) {
+          (*functor)(*accessor, *value);
+          if (!allow_duplicate_keys)
+            break;
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/HashTable.proto
----------------------------------------------------------------------
diff --git a/storage/HashTable.proto b/storage/HashTable.proto
index ade30d8..209144d 100644
--- a/storage/HashTable.proto
+++ b/storage/HashTable.proto
@@ -34,10 +34,10 @@ 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;
+  message BloomFilter {
+    required uint32 bloom_filter_id = 1;
+    required uint32 attr_id = 2;
   }
-  repeated ProbeSideBloomFilter probe_side_bloom_filters = 6;
+  repeated BloomFilter probe_side_bloom_filters = 4;
+  repeated BloomFilter build_side_bloom_filters = 5;  
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b7150fb1/storage/HashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.hpp b/storage/HashTableFactory.hpp
index 40b39de..1b60c53 100644
--- a/storage/HashTableFactory.hpp
+++ b/storage/HashTableFactory.hpp
@@ -321,9 +321,15 @@ class HashTableFactory {
     //                 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) {
+    if (proto.build_side_bloom_filters_size() > 0) {
       hash_table->enableBuildSideBloomFilter();
-      hash_table->setBuildSideBloomFilter(bloom_filters[proto.build_side_bloom_filter_id(0)].get());
+      for (int j = 0; j < proto.build_side_bloom_filters_size(); ++j) {
+        const auto build_side_bloom_filter = proto.build_side_bloom_filters(j);
+        hash_table->addBuildSideBloomFilter(
+            bloom_filters[build_side_bloom_filter.bloom_filter_id()].get());
+
+        hash_table->addBuildSideAttributeId(build_side_bloom_filter.attr_id());
+      }
     }
 
     // Check if there are any probe side bloom filters defined on the hash table.
@@ -333,15 +339,10 @@ class HashTableFactory {
       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));
+        hash_table->addProbeSideBloomFilter(
+            bloom_filters[probe_side_bloom_filter.bloom_filter_id()].get());
+
+        hash_table->addProbeSideAttributeId(probe_side_bloom_filter.attr_id());
       }
     }
 



[07/16] incubator-quickstep git commit: Logged all sent messages using glog.

Posted by ji...@apache.org.
Logged all sent messages using glog.


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

Branch: refs/heads/LIP-for-tpch
Commit: d9135a8a2d11a1eabf6705c88391c498f4be38bb
Parents: 6168996
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Mon Aug 8 22:49:59 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Tue Aug 9 19:48:01 2016 -0700

----------------------------------------------------------------------
 query_execution/ForemanSingleNode.cpp         | 14 ++--
 query_execution/PolicyEnforcerDistributed.cpp | 38 ++++------
 query_execution/QueryExecutionUtil.hpp        |  6 +-
 query_execution/Shiftboss.cpp                 | 86 +++++++++-------------
 query_execution/Worker.cpp                    |  6 +-
 relational_operators/DeleteOperator.cpp       | 10 +--
 relational_operators/RebuildWorkOrder.hpp     |  7 +-
 relational_operators/UpdateOperator.cpp       | 10 +--
 relational_operators/WorkOrder.hpp            |  7 +-
 storage/InsertDestination.cpp                 | 18 +++--
 storage/InsertDestination.hpp                 | 10 +--
 11 files changed, 92 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/ForemanSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanSingleNode.cpp b/query_execution/ForemanSingleNode.cpp
index d064a6f..7596b00 100644
--- a/query_execution/ForemanSingleNode.cpp
+++ b/query_execution/ForemanSingleNode.cpp
@@ -168,16 +168,15 @@ void ForemanSingleNode::run() {
       // Signal the main thread that there are no queries to be executed.
       // Currently the message doesn't have any real content.
       TaggedMessage completion_tagged_message(kWorkloadCompletionMessage);
+      DLOG(INFO) << "ForemanSingleNode sent WorkloadCompletionMessage (typed '" << kWorkloadCompletionMessage
+                 << "') to CLI with TMB client ID " << main_thread_client_id_;
       const tmb::MessageBus::SendStatus send_status =
           QueryExecutionUtil::SendTMBMessage(
               bus_,
               foreman_client_id_,
               main_thread_client_id_,
               move(completion_tagged_message));
-      CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-          << "Message could not be sent from Foreman with TMB client ID "
-          << foreman_client_id_ << " to main thread with TMB client ID"
-          << main_thread_client_id_;
+      CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
     }
   }
 }
@@ -225,15 +224,14 @@ void ForemanSingleNode::sendWorkerMessage(const size_t worker_thread_index,
   }
   TaggedMessage worker_tagged_message(&message, sizeof(message), type);
 
+  DLOG(INFO) << "ForemanSingleNode sent WorkOrderMessage (typed '" << type
+             << "') to Worker with TMB client ID " << worker_directory_->getClientID(worker_thread_index);
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          foreman_client_id_,
                                          worker_directory_->getClientID(worker_thread_index),
                                          move(worker_tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
-      "Message could not be sent from Foreman with TMB client ID "
-      << foreman_client_id_ << " to Foreman with TMB client ID "
-      << worker_directory_->getClientID(worker_thread_index);
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 }
 
 const std::vector<WorkOrderTimeEntry>& ForemanSingleNode

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
index 6d0de47..c76a9e1 100644
--- a/query_execution/PolicyEnforcerDistributed.cpp
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -170,25 +170,22 @@ void PolicyEnforcerDistributed::initiateQueryInShiftboss(QueryHandle *query_hand
                         kQueryInitiateMessage);
   free(proto_bytes);
 
-  LOG(INFO) << "PolicyEnforcerDistributed sent QueryInitiateMessage (typed '" << kQueryInitiateMessage
-            << "') to Shiftboss 0";
-
   // TODO(zuyu): Multiple Shiftbosses support.
+  DLOG(INFO) << "PolicyEnforcerDistributed sent QueryInitiateMessage (typed '" << kQueryInitiateMessage
+             << "') to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          foreman_client_id_,
                                          shiftboss_directory_->getClientId(0),
                                          move(message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
-      << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 
   // Wait Shiftboss for QueryInitiateResponseMessage.
   const tmb::AnnotatedMessage annotated_message = bus_->Receive(foreman_client_id_, 0, true);
   const TaggedMessage &tagged_message = annotated_message.tagged_message;
   DCHECK_EQ(kQueryInitiateResponseMessage, tagged_message.message_type());
-  LOG(INFO) << "PolicyEnforcerDistributed received typed '" << tagged_message.message_type()
-            << "' message from client " << annotated_message.sender;
+  DLOG(INFO) << "PolicyEnforcerDistributed received typed '" << tagged_message.message_type()
+             << "' message from client " << annotated_message.sender;
 
   S::QueryInitiateResponseMessage proto_response;
   CHECK(proto_response.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
@@ -215,30 +212,27 @@ void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manage
                           kQueryTeardownMessage);
 
     // TODO(zuyu): Support multiple shiftbosses.
-    LOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
-              << "') to Shiftboss 0";
+    DLOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
+               << "') to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
     tmb::MessageBus::SendStatus send_status =
         QueryExecutionUtil::SendTMBMessage(bus_,
                                            foreman_client_id_,
                                            shiftboss_directory_->getClientId(0),
                                            move(message));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-        << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
-        << " to Shiftboss";
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 
     TaggedMessage cli_message(kQueryExecutionSuccessMessage);
 
     // Notify the CLI query execution successfully.
-    LOG(INFO) << "PolicyEnforcerDistributed sent QueryExecutionSuccessMessage (typed '" << kQueryExecutionSuccessMessage
-              << "') to CLI with TMB client id " << cli_id;
+    DLOG(INFO) << "PolicyEnforcerDistributed sent QueryExecutionSuccessMessage (typed '"
+               << kQueryExecutionSuccessMessage
+               << "') to CLI with TMB client id " << cli_id;
     send_status =
         QueryExecutionUtil::SendTMBMessage(bus_,
                                            foreman_client_id_,
                                            cli_id,
                                            move(cli_message));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-        << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
-        << " to CLI with TMB client ID " << cli_id;
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
     return;
   }
 
@@ -263,17 +257,15 @@ void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manage
                         kSaveQueryResultMessage);
   free(proto_bytes);
 
-  LOG(INFO) << "PolicyEnforcerDistributed sent SaveQueryResultMessage (typed '" << kSaveQueryResultMessage
-            << "') to Shiftboss 0";
   // TODO(zuyu): Support multiple shiftbosses.
+  DLOG(INFO) << "PolicyEnforcerDistributed sent SaveQueryResultMessage (typed '" << kSaveQueryResultMessage
+             << "') to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          foreman_client_id_,
                                          shiftboss_directory_->getClientId(0),
                                          move(message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
-      << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/QueryExecutionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionUtil.hpp b/query_execution/QueryExecutionUtil.hpp
index 906fb6b..feb4cc0 100644
--- a/query_execution/QueryExecutionUtil.hpp
+++ b/query_execution/QueryExecutionUtil.hpp
@@ -128,11 +128,11 @@ class QueryExecutionUtil {
     address.All(true);
     TaggedMessage poison_tagged_message(kPoisonMessage);
 
+    DLOG(INFO) << "TMB client ID " << sender_id
+               << " broadcast PoisonMessage (typed '" << kPoisonMessage << "') to all";
     const tmb::MessageBus::SendStatus send_status = bus->Send(
         sender_id, address, style, std::move(poison_tagged_message));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
-       "Broadcast poison message from sender with TMB client ID " << sender_id
-       << " failed";
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index bd83dd4..ddfd47f 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -73,9 +73,9 @@ void Shiftboss::run() {
     // Receive() is a blocking call, causing this thread to sleep until next
     // message is received.
     AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
-    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-              << "') received the typed '" << annotated_message.tagged_message.message_type()
-              << "' message from client " << annotated_message.sender;
+    DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+               << "') received the typed '" << annotated_message.tagged_message.message_type()
+               << "' message from client " << annotated_message.sender;
     switch (annotated_message.tagged_message.message_type()) {
       case kShiftbossRegistrationResponseMessage: {
         foreman_client_id_ = annotated_message.sender;
@@ -121,18 +121,16 @@ void Shiftboss::run() {
                                             kWorkOrderMessage);
 
         const size_t worker_index = getSchedulableWorker();
-        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
-                  << "') from Foreman to worker " << worker_index;
+        DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                   << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
+                   << "') from Foreman to worker " << worker_index;
 
         const MessageBus::SendStatus send_status =
             QueryExecutionUtil::SendTMBMessage(bus_,
                                                shiftboss_client_id_,
                                                workers_->getClientID(worker_index),
                                                move(worker_tagged_message));
-        CHECK(send_status == MessageBus::SendStatus::kOK)
-            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-            << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
+        CHECK(send_status == MessageBus::SendStatus::kOK);
         break;
       }
       case kInitiateRebuildMessage: {
@@ -153,9 +151,10 @@ void Shiftboss::run() {
       case kRebuildWorkOrderCompleteMessage:
       case kDataPipelineMessage:
       case kWorkOrderFeedbackMessage: {
-        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
-                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
+        DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                   << "') forwarded typed '" << annotated_message.tagged_message.message_type()
+                   << "' message from Worker with TMB client ID '" << annotated_message.sender
+                   << "' to Foreman with TMB client ID " << foreman_client_id_;
 
         DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
         const MessageBus::SendStatus send_status =
@@ -163,9 +162,7 @@ void Shiftboss::run() {
                                                shiftboss_client_id_,
                                                foreman_client_id_,
                                                move(annotated_message.tagged_message));
-        CHECK(send_status == MessageBus::SendStatus::kOK)
-            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-            << " to Foreman with TMB client ID " << foreman_client_id_;
+        CHECK(send_status == MessageBus::SendStatus::kOK);
         break;
       }
       case kSaveQueryResultMessage: {
@@ -190,23 +187,21 @@ void Shiftboss::run() {
                                        kSaveQueryResultResponseMessage);
         free(proto_response_bytes);
 
-        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-                  << "') sent SaveQueryResultResponseMessage (typed '" << kSaveQueryResultResponseMessage
-                  << "') to Foreman";
+        DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                   << "') sent SaveQueryResultResponseMessage (typed '" << kSaveQueryResultResponseMessage
+                   << "') to Foreman with TMB client ID " << foreman_client_id_;
         const MessageBus::SendStatus send_status =
             QueryExecutionUtil::SendTMBMessage(bus_,
                                                shiftboss_client_id_,
                                                foreman_client_id_,
                                                move(message_response));
-        CHECK(send_status == MessageBus::SendStatus::kOK)
-            << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-            << " to Foreman with TMB client ID " << foreman_client_id_;
+        CHECK(send_status == MessageBus::SendStatus::kOK);
         break;
       }
       case kPoisonMessage: {
-        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
-                  << "') from Foreman to all workers";
+        DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+                   << "') forwarded PoisonMessage (typed '" << kPoisonMessage
+                   << "') from Foreman to all workers";
 
         tmb::MessageStyle broadcast_style;
         broadcast_style.Broadcast(true);
@@ -216,9 +211,7 @@ void Shiftboss::run() {
                        worker_addresses_,
                        broadcast_style,
                        move(annotated_message.tagged_message));
-        CHECK(send_status == MessageBus::SendStatus::kOK)
-            << "Message could not be broadcast from Shiftboss with TMB client ID " << shiftboss_client_id_
-            << " to All workers";
+        CHECK(send_status == MessageBus::SendStatus::kOK);
         return;
       }
       default: {
@@ -245,10 +238,6 @@ size_t Shiftboss::getSchedulableWorker() {
 }
 
 void Shiftboss::registerWithForeman() {
-  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-            << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
-            << "') to all";
-
   tmb::Address all_addresses;
   all_addresses.All(true);
 
@@ -266,6 +255,9 @@ void Shiftboss::registerWithForeman() {
                         kShiftbossRegistrationMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+             << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
+             << "') to all";
   tmb::MessageBus::SendStatus send_status =
       bus_->Send(shiftboss_client_id_, all_addresses, style, move(message));
   DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
@@ -285,10 +277,6 @@ void Shiftboss::processQueryInitiateMessage(
                        bus_));
   query_contexts_.emplace(query_id, move(query_context));
 
-  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-            << "') sent QueryInitiateResponseMessage (typed '" << kQueryInitiateResponseMessage
-            << "') to Foreman";
-
   serialization::QueryInitiateResponseMessage proto;
   proto.set_query_id(query_id);
 
@@ -301,14 +289,15 @@ void Shiftboss::processQueryInitiateMessage(
                                  kQueryInitiateResponseMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+             << "') sent QueryInitiateResponseMessage (typed '" << kQueryInitiateResponseMessage
+             << "') to Foreman with TMB client ID " << foreman_client_id_;
   const MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          shiftboss_client_id_,
                                          foreman_client_id_,
                                          move(message_response));
-  CHECK(send_status == MessageBus::SendStatus::kOK)
-      << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-      << " to Foreman with TMB client ID " << foreman_client_id_;
+  CHECK(send_status == MessageBus::SendStatus::kOK);
 }
 
 void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
@@ -324,10 +313,6 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
   vector<MutableBlockReference> partially_filled_block_refs;
   insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs);
 
-  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-            << "') sent InitiateRebuildResponseMessage (typed '" << kInitiateRebuildResponseMessage
-            << "') to Foreman";
-
   serialization::InitiateRebuildResponseMessage proto;
   proto.set_query_id(query_id);
   proto.set_operator_index(op_index);
@@ -343,14 +328,15 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
                                  kInitiateRebuildResponseMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+             << "') sent InitiateRebuildResponseMessage (typed '" << kInitiateRebuildResponseMessage
+             << "') to Foreman with TMB client ID " << foreman_client_id_;
   const MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          shiftboss_client_id_,
                                          foreman_client_id_,
                                          move(message_response));
-  CHECK(send_status == MessageBus::SendStatus::kOK)
-      << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-      << " to Foreman with TMB client ID " << foreman_client_id_;
+  CHECK(send_status == MessageBus::SendStatus::kOK);
 
   for (size_t i = 0; i < partially_filled_block_refs.size(); ++i) {
     // NOTE(zuyu): Worker releases the memory after the execution of
@@ -371,18 +357,16 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
                                         kRebuildWorkOrderMessage);
 
     const size_t worker_index = getSchedulableWorker();
-    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
-              << "') sent RebuildWorkOrderMessage (typed '" << kRebuildWorkOrderMessage
-              << "') to worker " << worker_index;
+    DLOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
+               << "') sent RebuildWorkOrderMessage (typed '" << kRebuildWorkOrderMessage
+               << "') to worker " << worker_index;
 
     const MessageBus::SendStatus send_status =
         QueryExecutionUtil::SendTMBMessage(bus_,
                                            shiftboss_client_id_,
                                            workers_->getClientID(worker_index),
                                            move(worker_tagged_message));
-    CHECK(send_status == MessageBus::SendStatus::kOK)
-        << "Message could not be sent from Shiftboss with TMB client ID " << shiftboss_client_id_
-        << " to Worker with TMB client ID " << workers_->getClientID(worker_index);
+    CHECK(send_status == MessageBus::SendStatus::kOK);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index d497be6..0b1efba 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -101,12 +101,12 @@ void Worker::sendWorkOrderCompleteMessage(const tmb::client_id receiver,
       static_cast<const void *>(proto_bytes), proto_length, message_type);
   std::free(proto_bytes);
 
+  DLOG(INFO) << "Worker sent WorkOrderCompleteMessage (typed '" << message_type
+             << "') to Scheduler with TMB client ID " << receiver;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(
           bus_, worker_client_id_, receiver, std::move(tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "Message could not be sent from worker with TMB client ID "
-      << worker_client_id_ << " to Foreman with TMB client ID " << receiver;
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 }
 
 template <typename CompletionMessageProtoT>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 8197aef..24da9bf 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -146,17 +146,15 @@ void DeleteWorkOrder::execute() {
                                     kDataPipelineMessage);
   std::free(proto_bytes);
 
-  const tmb::client_id worker_thread_client_id = ClientIDMap::Instance()->getValue();
+  DLOG(INFO) << "DeleteWorkOrder sent DataPipelineMessage (typed '" << kDataPipelineMessage
+             << "') to Scheduler with TMB client ID " << scheduler_client_id_;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(
           bus_,
-          worker_thread_client_id,
+          ClientIDMap::Instance()->getValue(),
           scheduler_client_id_,
           std::move(tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could not"
-      " be sent from thread with TMB client ID " <<
-      worker_thread_client_id << " to Foreman with TMB client ID "
-      << scheduler_client_id_;
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index fe4be68..2cef1f1 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -101,15 +101,14 @@ class RebuildWorkOrder : public WorkOrder {
 
     // Refer to InsertDestination::sendBlockFilledMessage for the rationale
     // behind using the ClientIDMap map.
+    DLOG(INFO) << "RebuildWorkOrder sent DataPipelineMessage (typed '" << kDataPipelineMessage
+               << "') to Scheduler with TMB client ID " << scheduler_client_id_;
     const tmb::MessageBus::SendStatus send_status =
         QueryExecutionUtil::SendTMBMessage(bus_,
                                            ClientIDMap::Instance()->getValue(),
                                            scheduler_client_id_,
                                            std::move(tagged_message));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could "
-        " not be sent from thread with TMB client ID " <<
-        ClientIDMap::Instance()->getValue() << " to Foreman with TMB client ID "
-        << scheduler_client_id_;
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index bc29365..143c741 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -128,17 +128,15 @@ void UpdateWorkOrder::execute() {
                                     kDataPipelineMessage);
   std::free(proto_bytes);
 
-  const tmb::client_id worker_thread_client_id = ClientIDMap::Instance()->getValue();
+  DLOG(INFO) << "UpdateWorkOrder sent DataPipelineMessage (typed '" << kDataPipelineMessage
+             << "') to Scheduler with TMB client ID " << scheduler_client_id_;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(
           bus_,
-          worker_thread_client_id,
+          ClientIDMap::Instance()->getValue(),
           scheduler_client_id_,
           std::move(tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could not"
-      " be sent from thread with TMB client ID " <<
-      worker_thread_client_id << " to Foreman with TMB client ID "
-      << scheduler_client_id_;
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index 3cbab94..c1b9b68 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -44,7 +44,6 @@ namespace quickstep {
  *  @{
  */
 
-
 /**
  * @brief A single unit of work in a query plan, produced by a
  *        RelationalOperator. Where possible, WorkOrders should be of
@@ -284,14 +283,14 @@ class WorkOrder {
     tmb::MessageStyle single_receiver_style;
 
     DCHECK(bus != nullptr);
+    DLOG(INFO) << "WorkOrder sent WorkOrderFeedbackMessage (typed '" << kWorkOrderFeedbackMessage
+               << "') to Scheduler with TMB client ID " << receiver_id;
     const tmb::MessageBus::SendStatus send_status =
         bus->Send(sender_id,
                   receiver_address,
                   single_receiver_style,
                   std::move(msg));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) << "Message could"
-        " not be sent from thread with TMB client ID " << sender_id << " to"
-        " receiver thread with TMB client ID " << receiver_id;
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 9897aed..5e83453 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -282,13 +282,15 @@ MutableBlockReference AlwaysCreateBlockInsertDestination::createNewBlock() {
                            kCatalogRelationNewBlockMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "AlwaysCreateBlockInsertDestination sent CatalogRelationNewBlockMessage (typed '"
+             << kCatalogRelationNewBlockMessage
+             << "') to Scheduler with TMB client ID " << scheduler_client_id_;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          thread_id_map_.getValue(),
                                          scheduler_client_id_,
                                          move(tagged_msg));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "CatalogRelationNewBlockMessage could not be sent from InsertDestination to Foreman.";
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 
   return storage_manager_->getBlockMutable(new_id, relation_);
 }
@@ -330,13 +332,15 @@ MutableBlockReference BlockPoolInsertDestination::createNewBlock() {
                            kCatalogRelationNewBlockMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "BlockPoolInsertDestination sent CatalogRelationNewBlockMessage (typed '"
+             << kCatalogRelationNewBlockMessage
+             << "') to Scheduler with TMB client ID " << scheduler_client_id_;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          thread_id_map_.getValue(),
                                          scheduler_client_id_,
                                          move(tagged_msg));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "CatalogRelationNewBlockMessage could not be sent from InsertDestination to Foreman.";
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 
   return storage_manager_->getBlockMutable(new_id, relation_);
 }
@@ -445,13 +449,15 @@ MutableBlockReference PartitionAwareInsertDestination::createNewBlockInPartition
                            kCatalogRelationNewBlockMessage);
   free(proto_bytes);
 
+  DLOG(INFO) << "PartitionAwareInsertDestination sent CatalogRelationNewBlockMessage (typed '"
+             << kCatalogRelationNewBlockMessage
+             << "') to Scheduler with TMB client ID " << scheduler_client_id_;
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          thread_id_map_.getValue(),
                                          scheduler_client_id_,
                                          move(tagged_msg));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
-      << "CatalogRelationNewBlockMessage could not be sent from InsertDestination to Foreman.";
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
 
   return storage_manager_->getBlockMutable(new_id, relation_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d9135a8a/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index 3dae9a0..408e76b 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -249,16 +249,14 @@ class InsertDestination : public InsertDestinationInterface {
     // option 3.
     DCHECK(bus_ != nullptr);
 
-    const tmb::client_id worker_thread_client_id = thread_id_map_.getValue();
+    DLOG(INFO) << "InsertDestination sent DataPipelineMessage (typed '" << kDataPipelineMessage
+               << "') to Scheduler with TMB client ID " << scheduler_client_id_;
     const tmb::MessageBus::SendStatus send_status =
         QueryExecutionUtil::SendTMBMessage(bus_,
-                                           worker_thread_client_id,
+                                           thread_id_map_.getValue(),
                                            scheduler_client_id_,
                                            std::move(tagged_message));
-    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
-        "Message could not be sent from thread with TMB client ID "
-        << worker_thread_client_id << " to Scheduler with TMB client"
-        " ID " << scheduler_client_id_;
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK);
   }
 
   inline const std::size_t getQueryID() const {


[02/16] incubator-quickstep git commit: Added PolicyEnforcer implementation for the distributed version.

Posted by ji...@apache.org.
Added PolicyEnforcer implementation for 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/3c2749ea
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/3c2749ea
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/3c2749ea

Branch: refs/heads/LIP-for-tpch
Commit: 3c2749eafcff25283915bdef21822056d36f5281
Parents: e443b2b
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Thu Aug 4 11:45:51 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 10:35:45 2016 -0700

----------------------------------------------------------------------
 query_execution/CMakeLists.txt                |  24 ++
 query_execution/PolicyEnforcerBase.cpp        |   2 +
 query_execution/PolicyEnforcerBase.hpp        |   7 +
 query_execution/PolicyEnforcerDistributed.cpp | 279 +++++++++++++++++++++
 query_execution/PolicyEnforcerDistributed.hpp | 113 +++++++++
 query_execution/QueryExecutionMessages.proto  |  16 +-
 query_execution/QueryExecutionTypedefs.hpp    |   5 +
 query_execution/QueryManagerBase.cpp          |   3 +-
 query_execution/QueryManagerBase.hpp          |  11 +-
 9 files changed, 456 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 4b180e3..74fcafb 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -35,6 +35,9 @@ endif()
 add_library(quickstep_queryexecution_ForemanBase ../empty_src.cpp ForemanBase.hpp)
 add_library(quickstep_queryexecution_ForemanSingleNode ForemanSingleNode.cpp ForemanSingleNode.hpp)
 add_library(quickstep_queryexecution_PolicyEnforcerBase PolicyEnforcerBase.cpp PolicyEnforcerBase.hpp)
+if (ENABLE_DISTRIBUTED)
+  add_library(quickstep_queryexecution_PolicyEnforcerDistributed PolicyEnforcerDistributed.cpp PolicyEnforcerDistributed.hpp)
+endif(ENABLE_DISTRIBUTED)
 add_library(quickstep_queryexecution_PolicyEnforcerSingleNode PolicyEnforcerSingleNode.cpp PolicyEnforcerSingleNode.hpp)
 add_library(quickstep_queryexecution_QueryContext QueryContext.cpp QueryContext.hpp)
 add_library(quickstep_queryexecution_QueryContext_proto
@@ -110,6 +113,26 @@ target_link_libraries(quickstep_queryexecution_PolicyEnforcerBase
                       quickstep_storage_StorageBlockInfo
                       quickstep_utility_Macros
                       tmb)
+if (ENABLE_DISTRIBUTED)
+  target_link_libraries(quickstep_queryexecution_PolicyEnforcerDistributed
+                        glog
+                        quickstep_catalog_CatalogRelation
+                        quickstep_catalog_Catalog_proto
+                        quickstep_queryexecution_PolicyEnforcerBase
+                        quickstep_queryexecution_QueryContext_proto
+                        quickstep_queryexecution_QueryExecutionMessages_proto
+                        quickstep_queryexecution_QueryExecutionState
+                        quickstep_queryexecution_QueryExecutionTypedefs
+                        quickstep_queryexecution_QueryExecutionUtil
+                        quickstep_queryexecution_QueryManagerBase
+                        quickstep_queryexecution_QueryManagerDistributed
+                        quickstep_queryexecution_ShiftbossDirectory
+                        quickstep_queryoptimizer_QueryHandle
+                        quickstep_storage_StorageBlockInfo
+                        quickstep_utility_Macros
+                        tmb
+                        ${GFLAGS_LIB_NAME})
+endif(ENABLE_DISTRIBUTED)
 target_link_libraries(quickstep_queryexecution_PolicyEnforcerSingleNode
                       glog
                       quickstep_catalog_CatalogTypedefs
@@ -293,6 +316,7 @@ target_link_libraries(quickstep_queryexecution
 if (ENABLE_DISTRIBUTED)
   target_link_libraries(quickstep_queryexecution
                         quickstep_queryexecution_BlockLocator
+                        quickstep_queryexecution_PolicyEnforcerDistributed
                         quickstep_queryexecution_QueryManagerDistributed
                         quickstep_queryexecution_Shiftboss
                         quickstep_queryexecution_ShiftbossDirectory)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index bf6edf9..78f7b44 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -136,6 +136,8 @@ void PolicyEnforcerBase::processMessage(const TaggedMessage &tagged_message) {
   }
   if (admitted_queries_[query_id]->queryStatus(op_index) ==
           QueryManagerBase::QueryStatusCode::kQueryExecuted) {
+    onQueryCompletion(admitted_queries_[query_id].get());
+
     removeQuery(query_id);
     if (!waiting_queries_.empty()) {
       // Admit the earliest waiting query.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.hpp b/query_execution/PolicyEnforcerBase.hpp
index c75a531..e95799e 100644
--- a/query_execution/PolicyEnforcerBase.hpp
+++ b/query_execution/PolicyEnforcerBase.hpp
@@ -140,6 +140,13 @@ class PolicyEnforcerBase {
   static constexpr std::size_t kMaxConcurrentQueries = 1;
 
   /**
+   * @brief Add custom actions upon the completion of a query.
+   *
+   * @param query_manager The query manager.
+   **/
+  virtual void onQueryCompletion(QueryManagerBase *query_manager) {}
+
+  /**
    * @brief Record the execution time for a finished WorkOrder.
    *
    * TODO(harshad) - Extend the functionality to rebuild work orders.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
new file mode 100644
index 0000000..6d0de47
--- /dev/null
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -0,0 +1,279 @@
+/**
+ * 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.
+ **/
+
+#include "query_execution/PolicyEnforcerDistributed.hpp"
+
+#include <cstddef>
+#include <cstdlib>
+#include <memory>
+#include <queue>
+#include <utility>
+#include <unordered_map>
+#include <vector>
+
+#include "catalog/Catalog.pb.h"
+#include "catalog/CatalogRelation.hpp"
+#include "query_execution/QueryContext.pb.h"
+#include "query_execution/QueryExecutionMessages.pb.h"
+#include "query_execution/QueryExecutionState.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
+#include "query_execution/QueryManagerBase.hpp"
+#include "query_execution/QueryManagerDistributed.hpp"
+#include "query_optimizer/QueryHandle.hpp"
+#include "storage/StorageBlockInfo.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
+#include "tmb/tagged_message.h"
+
+using std::free;
+using std::malloc;
+using std::move;
+using std::size_t;
+using std::unique_ptr;
+using std::vector;
+
+using tmb::TaggedMessage;
+
+namespace quickstep {
+
+namespace S = serialization;
+
+DEFINE_uint64(max_msgs_per_dispatch_round, 20, "Maximum number of messages that"
+              " can be allocated in a single round of dispatch of messages to"
+              " the workers.");
+
+void PolicyEnforcerDistributed::getWorkOrderMessages(
+    vector<unique_ptr<S::WorkOrderMessage>> *work_order_messages) {
+  // Iterate over admitted queries until either there are no more
+  // messages available, or the maximum number of messages have
+  // been collected.
+  DCHECK(work_order_messages->empty());
+  // TODO(harshad) - Make this function generic enough so that it
+  // works well when multiple queries are getting executed.
+  if (admitted_queries_.empty()) {
+    LOG(WARNING) << "Requesting WorkerMessages when no query is running";
+    return;
+  }
+
+  const std::size_t per_query_share =
+      FLAGS_max_msgs_per_dispatch_round / admitted_queries_.size();
+  DCHECK_GT(per_query_share, 0u);
+
+  vector<std::size_t> finished_queries_ids;
+
+  for (const auto &admitted_query_info : admitted_queries_) {
+    QueryManagerBase *curr_query_manager = admitted_query_info.second.get();
+    DCHECK(curr_query_manager != nullptr);
+    std::size_t messages_collected_curr_query = 0;
+    while (messages_collected_curr_query < per_query_share) {
+      S::WorkOrderMessage *next_work_order_message =
+          static_cast<QueryManagerDistributed*>(curr_query_manager)->getNextWorkOrderMessage(0);
+      if (next_work_order_message != nullptr) {
+        ++messages_collected_curr_query;
+        work_order_messages->push_back(unique_ptr<S::WorkOrderMessage>(next_work_order_message));
+      } else {
+        // No more work ordes from the current query at this time.
+        // Check if the query's execution is over.
+        if (curr_query_manager->getQueryExecutionState().hasQueryExecutionFinished()) {
+          // If the query has been executed, remove it.
+          finished_queries_ids.push_back(admitted_query_info.first);
+        }
+        break;
+      }
+    }
+  }
+  for (const std::size_t finished_qid : finished_queries_ids) {
+    onQueryCompletion(admitted_queries_[finished_qid].get());
+    removeQuery(finished_qid);
+  }
+}
+
+bool PolicyEnforcerDistributed::admitQuery(QueryHandle *query_handle) {
+  if (admitted_queries_.size() < PolicyEnforcerBase::kMaxConcurrentQueries) {
+    // Ok to admit the query.
+    const std::size_t query_id = query_handle->query_id();
+    if (admitted_queries_.find(query_id) == admitted_queries_.end()) {
+      // NOTE(zuyu): Should call before constructing a 'QueryManager'.
+      // Otherwise, an InitiateRebuildMessage may be sent before 'QueryContext'
+      // initializes.
+      initiateQueryInShiftboss(query_handle);
+
+      // Query with the same ID not present, ok to admit.
+      admitted_queries_[query_id].reset(
+          new QueryManagerDistributed(query_handle, shiftboss_directory_, foreman_client_id_, bus_));
+      return true;
+    } else {
+      LOG(ERROR) << "Query with the same ID " << query_id << " exists";
+      return false;
+    }
+  } else {
+    // This query will have to wait.
+    waiting_queries_.push(query_handle);
+    return false;
+  }
+}
+
+void PolicyEnforcerDistributed::processInitiateRebuildResponseMessage(const tmb::TaggedMessage &tagged_message) {
+  S::InitiateRebuildResponseMessage proto;
+  CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+
+  const std::size_t query_id = proto.query_id();
+  DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
+
+  QueryManagerDistributed *query_manager = static_cast<QueryManagerDistributed*>(admitted_queries_[query_id].get());
+
+  const std::size_t num_rebuild_work_orders = proto.num_rebuild_work_orders();
+  query_manager->processInitiateRebuildResponseMessage(proto.operator_index(), num_rebuild_work_orders);
+  shiftboss_directory_->addNumQueuedWorkOrders(proto.shiftboss_index(), num_rebuild_work_orders);
+
+  if (query_manager->getQueryExecutionState().hasQueryExecutionFinished()) {
+    onQueryCompletion(query_manager);
+
+    removeQuery(query_id);
+    if (!waiting_queries_.empty()) {
+      // Admit the earliest waiting query.
+      QueryHandle *new_query = waiting_queries_.front();
+      waiting_queries_.pop();
+      admitQuery(new_query);
+    }
+  }
+}
+
+void PolicyEnforcerDistributed::initiateQueryInShiftboss(QueryHandle *query_handle) {
+  S::QueryInitiateMessage proto;
+  proto.set_query_id(query_handle->query_id());
+  proto.mutable_catalog_database_cache()->MergeFrom(query_handle->getCatalogDatabaseCacheProto());
+  proto.mutable_query_context()->MergeFrom(query_handle->getQueryContextProto());
+
+  const size_t proto_length = proto.ByteSize();
+  char *proto_bytes = static_cast<char*>(malloc(proto_length));
+  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+  TaggedMessage message(static_cast<const void*>(proto_bytes),
+                        proto_length,
+                        kQueryInitiateMessage);
+  free(proto_bytes);
+
+  LOG(INFO) << "PolicyEnforcerDistributed sent QueryInitiateMessage (typed '" << kQueryInitiateMessage
+            << "') to Shiftboss 0";
+
+  // TODO(zuyu): Multiple Shiftbosses support.
+  const tmb::MessageBus::SendStatus send_status =
+      QueryExecutionUtil::SendTMBMessage(bus_,
+                                         foreman_client_id_,
+                                         shiftboss_directory_->getClientId(0),
+                                         move(message));
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+      << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+      << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+
+  // Wait Shiftboss for QueryInitiateResponseMessage.
+  const tmb::AnnotatedMessage annotated_message = bus_->Receive(foreman_client_id_, 0, true);
+  const TaggedMessage &tagged_message = annotated_message.tagged_message;
+  DCHECK_EQ(kQueryInitiateResponseMessage, tagged_message.message_type());
+  LOG(INFO) << "PolicyEnforcerDistributed received typed '" << tagged_message.message_type()
+            << "' message from client " << annotated_message.sender;
+
+  S::QueryInitiateResponseMessage proto_response;
+  CHECK(proto_response.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
+}
+
+void PolicyEnforcerDistributed::onQueryCompletion(QueryManagerBase *query_manager) {
+  const QueryHandle *query_handle = query_manager->query_handle();
+
+  const CatalogRelation *query_result = query_handle->getQueryResultRelation();
+  const tmb::client_id cli_id = query_handle->getClientId();
+  const std::size_t query_id = query_handle->query_id();
+
+  if (query_result == nullptr) {
+    // Clean up query execution states, i.e., QueryContext, in Shiftboss.
+    serialization::QueryTeardownMessage proto;
+    proto.set_query_id(query_id);
+
+    const size_t proto_length = proto.ByteSize();
+    char *proto_bytes = static_cast<char*>(malloc(proto_length));
+    CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+    TaggedMessage message(static_cast<const void*>(proto_bytes),
+                          proto_length,
+                          kQueryTeardownMessage);
+
+    // TODO(zuyu): Support multiple shiftbosses.
+    LOG(INFO) << "PolicyEnforcerDistributed sent QueryTeardownMessage (typed '" << kQueryTeardownMessage
+              << "') to Shiftboss 0";
+    tmb::MessageBus::SendStatus send_status =
+        QueryExecutionUtil::SendTMBMessage(bus_,
+                                           foreman_client_id_,
+                                           shiftboss_directory_->getClientId(0),
+                                           move(message));
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+        << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+        << " to Shiftboss";
+
+    TaggedMessage cli_message(kQueryExecutionSuccessMessage);
+
+    // Notify the CLI query execution successfully.
+    LOG(INFO) << "PolicyEnforcerDistributed sent QueryExecutionSuccessMessage (typed '" << kQueryExecutionSuccessMessage
+              << "') to CLI with TMB client id " << cli_id;
+    send_status =
+        QueryExecutionUtil::SendTMBMessage(bus_,
+                                           foreman_client_id_,
+                                           cli_id,
+                                           move(cli_message));
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+        << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+        << " to CLI with TMB client ID " << cli_id;
+    return;
+  }
+
+  // NOTE(zuyu): SaveQueryResultMessage implicitly triggers QueryTeardown in Shiftboss.
+  S::SaveQueryResultMessage proto;
+  proto.set_query_id(query_id);
+  proto.set_relation_id(query_result->getID());
+
+  const vector<block_id> blocks(query_result->getBlocksSnapshot());
+  for (const block_id block : blocks) {
+    proto.add_blocks(block);
+  }
+
+  proto.set_cli_id(cli_id);
+
+  const size_t proto_length = proto.ByteSize();
+  char *proto_bytes = static_cast<char*>(malloc(proto_length));
+  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
+
+  TaggedMessage message(static_cast<const void*>(proto_bytes),
+                        proto_length,
+                        kSaveQueryResultMessage);
+  free(proto_bytes);
+
+  LOG(INFO) << "PolicyEnforcerDistributed sent SaveQueryResultMessage (typed '" << kSaveQueryResultMessage
+            << "') to Shiftboss 0";
+  // TODO(zuyu): Support multiple shiftbosses.
+  const tmb::MessageBus::SendStatus send_status =
+      QueryExecutionUtil::SendTMBMessage(bus_,
+                                         foreman_client_id_,
+                                         shiftboss_directory_->getClientId(0),
+                                         move(message));
+  CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+      << "Message could not be sent from Foreman with TMB client ID " << foreman_client_id_
+      << " to Shiftboss with TMB client ID " << shiftboss_directory_->getClientId(0);
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/PolicyEnforcerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.hpp b/query_execution/PolicyEnforcerDistributed.hpp
new file mode 100644
index 0000000..16ebe36
--- /dev/null
+++ b/query_execution/PolicyEnforcerDistributed.hpp
@@ -0,0 +1,113 @@
+/**
+ * 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_QUERY_EXECUTION_POLICY_ENFORCER_DISTRIBUTED_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_DISTRIBUTED_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <vector>
+
+#include "query_execution/PolicyEnforcerBase.hpp"
+#include "query_execution/ShiftbossDirectory.hpp"
+#include "utility/Macros.hpp"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb {
+class MessageBus;
+class TaggedMessage;
+}
+
+namespace quickstep {
+
+class CatalogDatabaseLite;
+class QueryHandle;
+class QueryManagerBase;
+
+namespace serialization { class WorkOrderMessage; }
+
+/** \addtogroup QueryExecution
+ *  @{
+ */
+
+/**
+ * @brief A class that ensures that a high level policy is maintained
+ *        in sharing resources among concurrent queries.
+ **/
+class PolicyEnforcerDistributed final : public PolicyEnforcerBase {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param foreman_client_id The TMB client ID of the Foreman.
+   * @param catalog_database The CatalogDatabase used.
+   * @param bus The TMB.
+   **/
+  PolicyEnforcerDistributed(const tmb::client_id foreman_client_id,
+                            CatalogDatabaseLite *catalog_database,
+                            ShiftbossDirectory *shiftboss_directory,
+                            tmb::MessageBus *bus,
+                            const bool profile_individual_workorders = false)
+      : PolicyEnforcerBase(catalog_database, profile_individual_workorders),
+        foreman_client_id_(foreman_client_id),
+        shiftboss_directory_(shiftboss_directory),
+        bus_(bus) {}
+
+  /**
+   * @brief Destructor.
+   **/
+  ~PolicyEnforcerDistributed() override {}
+
+  bool admitQuery(QueryHandle *query_handle) override;
+
+  /**
+   * @brief Get work order messages to be dispatched. These messages come from
+   *        the active queries.
+   *
+   * @param work_order_messages The work order messages to be dispatched.
+   **/
+  void getWorkOrderMessages(
+      std::vector<std::unique_ptr<serialization::WorkOrderMessage>> *work_order_messages);
+
+  /**
+   * @brief Process the initiate rebuild work order response message.
+   *
+   * @param tagged_message The message.
+   **/
+  void processInitiateRebuildResponseMessage(const tmb::TaggedMessage &tagged_message);
+
+ private:
+  void decrementNumQueuedWorkOrders(const std::size_t shiftboss_index) override {
+    shiftboss_directory_->decrementNumQueuedWorkOrders(shiftboss_index);
+  }
+
+  void onQueryCompletion(QueryManagerBase *query_manager) override;
+
+  void initiateQueryInShiftboss(QueryHandle *query_handle);
+
+  const tmb::client_id foreman_client_id_;
+
+  ShiftbossDirectory *shiftboss_directory_;
+
+  tmb::MessageBus *bus_;
+
+  DISALLOW_COPY_AND_ASSIGN(PolicyEnforcerDistributed);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_DISTRIBUTED_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index f680d35..20b684e 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -120,13 +120,25 @@ message InitiateRebuildResponseMessage {
   required uint64 shiftboss_index = 4;
 }
 
+message QueryTeardownMessage {
+  required uint64 query_id = 1;
+}
+
 message SaveQueryResultMessage {
-  required int32 relation_id = 1;
-  repeated fixed64 blocks = 2 [packed=true];
+  required uint64 query_id = 1;
+  required int32 relation_id = 2;
+  repeated fixed64 blocks = 3 [packed=true];
+
+  required uint32 cli_id = 4;  // tmb::client_id.
 }
 
 message SaveQueryResultResponseMessage {
   required int32 relation_id = 1;
+  required uint32 cli_id = 2;  // tmb::client_id.
+}
+
+message QueryExecutionSuccessMessage {
+  optional CatalogRelationSchema result_relation = 1;
 }
 
 // BlockLocator related messages.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index 4643096..d154d84 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -86,9 +86,14 @@ enum QueryExecutionMessageType : message_type_id {
   kInitiateRebuildMessage,  // From Foreman to Shiftboss.
   kInitiateRebuildResponseMessage,  // From Shiftboss to Foreman.
 
+  kQueryTeardownMessage,  // From Foreman to Shiftboss.
+
   kSaveQueryResultMessage,  // From Foreman to Shiftboss.
   kSaveQueryResultResponseMessage,  // From Shiftboss to Foreman.
 
+  // From Foreman to CLI.
+  kQueryExecutionSuccessMessage,
+
   // BlockLocator related messages, sorted in a life cycle of StorageManager
   // with a unique block domain.
   kBlockDomainRegistrationMessage,  // From Worker to BlockLocator.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryManagerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.cpp b/query_execution/QueryManagerBase.cpp
index c60e323..8e37da8 100644
--- a/query_execution/QueryManagerBase.cpp
+++ b/query_execution/QueryManagerBase.cpp
@@ -37,7 +37,8 @@ using std::pair;
 namespace quickstep {
 
 QueryManagerBase::QueryManagerBase(QueryHandle *query_handle)
-    : query_id_(DCHECK_NOTNULL(query_handle)->query_id()),
+    : query_handle_(DCHECK_NOTNULL(query_handle)),
+      query_id_(query_handle->query_id()),
       query_dag_(DCHECK_NOTNULL(
           DCHECK_NOTNULL(query_handle->getQueryPlanMutable())->getQueryPlanDAGMutable())),
       num_operators_in_dag_(query_dag_->size()),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c2749ea/query_execution/QueryManagerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.hpp b/query_execution/QueryManagerBase.hpp
index 782b8ed..a274742 100644
--- a/query_execution/QueryManagerBase.hpp
+++ b/query_execution/QueryManagerBase.hpp
@@ -74,6 +74,13 @@ class QueryManagerBase {
   virtual ~QueryManagerBase() {}
 
   /**
+   * @brief Get the query handle.
+   **/
+  const QueryHandle* query_handle() const {
+    return query_handle_;
+  }
+
+  /**
    * @brief Get the QueryExecutionState for this query.
    **/
   inline const QueryExecutionState& getQueryExecutionState() const {
@@ -252,9 +259,11 @@ class QueryManagerBase {
     return query_exec_state_->hasRebuildInitiated(index);
   }
 
+  const QueryHandle *query_handle_;
+
   const std::size_t query_id_;
 
-  DAG<RelationalOperator, bool> *query_dag_;
+  DAG<RelationalOperator, bool> *query_dag_;  // Owned by 'query_handle_'.
   const dag_node_index num_operators_in_dag_;
 
   // For all nodes, store their receiving dependents.


[03/16] incubator-quickstep git commit: Added guards for gnu_source definition in CMake.

Posted by ji...@apache.org.
Added guards for gnu_source definition in CMake.


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

Branch: refs/heads/LIP-for-tpch
Commit: 658cb61760ab6bc98ead1faf231b7b7f27a0fa1e
Parents: 3c2749e
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Mon Aug 8 12:19:00 2016 -0500
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Mon Aug 8 10:42:51 2016 -0700

----------------------------------------------------------------------
 storage/CMakeLists.txt | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/658cb617/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index c4b8f70..65a7975 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -50,7 +50,9 @@ endif()
 # See if mmap can be used to allocate Linux hugetlb pages.
 include(CheckCXXSourceCompiles)
 CHECK_CXX_SOURCE_COMPILES("
+  #ifndef _GNU_SOURCE
   #define _GNU_SOURCE
+  #endif
   #include <sys/mman.h>
 
   int main() {