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/07/07 18:57:51 UTC

[1/8] incubator-quickstep git commit: Fixed the time measurement from milli to microseconds. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/adaptive-bloom-filters ccf55f50c -> 8904ba121 (forced update)


Fixed the time measurement from milli to microseconds.


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

Branch: refs/heads/adaptive-bloom-filters
Commit: 040a511aad35cb958d9d532fabc002313952cb11
Parents: b258821
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Sat Jul 2 15:58:52 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Sat Jul 2 15:58:52 2016 -0500

----------------------------------------------------------------------
 query_execution/Worker.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/040a511a/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index ae889c7..6ba27f1 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -121,7 +121,7 @@ void Worker::executeWorkOrderHelper(const TaggedMessage &tagged_message,
   end = std::chrono::steady_clock::now();
   delete worker_message.getWorkOrder();
   const uint64_t execution_time_microseconds =
-      std::chrono::duration_cast<std::chrono::milliseconds>(end - start)
+      std::chrono::duration_cast<std::chrono::microseconds>(end - start)
           .count();
   // Construct the proto message.
   proto->set_operator_index(worker_message.getRelationalOpIndex());


[2/8] incubator-quickstep git commit: Disallow negative number of worker threads.

Posted by ji...@apache.org.
Disallow negative number of worker threads.

- Fixed a bug thereby Quickstep command line now disallows negative
  number of worker threads.
- If the user provides zero or fewer worker threads, we switch to the
  default number of worker threasd, instead of terminating the process.


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

Branch: refs/heads/adaptive-bloom-filters
Commit: 31f1bbb1c71d9a18af27ee540c83f513125b656f
Parents: 040a511
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Thu Jun 30 11:04:29 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Sun Jul 3 23:20:00 2016 -0500

----------------------------------------------------------------------
 cli/QuickstepCli.cpp | 23 +++++++++++------------
 1 file changed, 11 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/31f1bbb1/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 3f99130..02a55a0 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -200,19 +200,18 @@ int main(int argc, char* argv[]) {
   // that we computed above, provided it did return a valid value.
   // TODO(jmp): May need to change this at some point to keep one thread
   //            available for the OS if the hardware concurrency level is high.
-  const unsigned int real_num_workers = quickstep::FLAGS_num_workers != 0
-                                      ? quickstep::FLAGS_num_workers
-                                      : (num_hw_threads != 0 ?
-                                         num_hw_threads
-                                         : 1);
-
-  if (real_num_workers > 0) {
-    printf("Starting Quickstep with %d worker thread(s) and a %.2f GB buffer pool\n",
-           real_num_workers,
-           (static_cast<double>(quickstep::FLAGS_buffer_pool_slots) * quickstep::kSlotSizeBytes)/quickstep::kAGigaByte);
-  } else {
-    LOG(FATAL) << "Quickstep needs at least one worker thread to run";
+  if (quickstep::FLAGS_num_workers <= 0) {
+    LOG(INFO) << "Quickstep expects at least one worker thread, switching to "
+                 "the default number of worker threads";
   }
+  const int real_num_workers = quickstep::FLAGS_num_workers > 0
+                                   ? quickstep::FLAGS_num_workers
+                                   : (num_hw_threads != 0 ? num_hw_threads : 1);
+
+  DCHECK_GT(real_num_workers, 0);
+  printf("Starting Quickstep with %d worker thread(s) and a %.2f GB buffer pool\n",
+         real_num_workers,
+         (static_cast<double>(quickstep::FLAGS_buffer_pool_slots) * quickstep::kSlotSizeBytes)/quickstep::kAGigaByte);
 
 #ifdef QUICKSTEP_HAVE_FILE_MANAGER_HDFS
   if (quickstep::FLAGS_use_hdfs) {


[6/8] 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/10b25333
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/10b25333
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/10b25333

Branch: refs/heads/adaptive-bloom-filters
Commit: 10b25333b3916621acebe15e89f0407225fcc41d
Parents: 04c8224
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 Jul 7 12:54:04 2016 -0500

----------------------------------------------------------------------
 CMakeLists.txt                                  |   2 +
 catalog/CMakeLists.txt                          |   9 +
 catalog/Catalog.proto                           |   5 +
 catalog/CatalogRelation.cpp                     |  16 +-
 catalog/CatalogRelationConstraints.cpp          |  55 ++++++
 catalog/CatalogRelationConstraints.hpp          |  97 ++++++++++
 catalog/CatalogRelationSchema.cpp               |  15 ++
 catalog/CatalogRelationSchema.hpp               |  16 +-
 cli/CommandExecutor.cpp                         |  25 ++-
 cli/QuickstepCli.cpp                            |  49 ++++++
 compression/CompressionDictionaryLite.hpp       |  42 +++++
 query_execution/CMakeLists.txt                  |   1 +
 query_execution/Worker.cpp                      |   5 +
 query_execution/tests/QueryManager_unittest.cpp |   4 +
 query_optimizer/ExecutionHeuristics.cpp         |  17 +-
 query_optimizer/ExecutionHeuristics.hpp         |   2 +-
 query_optimizer/PhysicalGenerator.cpp           |   2 +-
 query_optimizer/cost_model/SimpleCostModel.cpp  |   4 +-
 .../StarSchemaHashJoinOrderOptimization.cpp     |   1 +
 relational_operators/AggregationOperator.hpp    |   4 +
 relational_operators/BuildHashOperator.hpp      |   4 +
 relational_operators/CreateIndexOperator.hpp    |   4 +
 relational_operators/CreateTableOperator.hpp    |   4 +
 relational_operators/DeleteOperator.hpp         |   4 +
 relational_operators/DestroyHashOperator.hpp    |   4 +
 relational_operators/DropTableOperator.hpp      |   4 +
 .../FinalizeAggregationOperator.hpp             |   4 +
 relational_operators/HashJoinOperator.hpp       |  35 +++-
 relational_operators/InsertOperator.hpp         |   4 +
 .../NestedLoopsJoinOperator.hpp                 |   4 +
 relational_operators/RelationalOperator.hpp     |  16 ++
 relational_operators/SampleOperator.hpp         |   4 +
 relational_operators/SaveBlocksOperator.hpp     |   4 +
 relational_operators/SelectOperator.hpp         |   4 +
 relational_operators/SortMergeRunOperator.hpp   |   4 +
 .../SortRunGenerationOperator.hpp               |   4 +
 relational_operators/TableGeneratorOperator.hpp |   4 +
 relational_operators/TextScanOperator.hpp       |   4 +
 relational_operators/UpdateOperator.hpp         |   4 +
 relational_operators/WorkOrder.hpp              |  11 +-
 storage/BasicColumnStoreValueAccessor.hpp       |  26 ++-
 storage/CMakeLists.txt                          |   2 +
 storage/CompressedColumnStoreValueAccessor.hpp  |  22 +++
 .../CompressedPackedRowStoreValueAccessor.hpp   |  22 +++
 storage/PackedRowStoreValueAccessor.hpp         |  25 ++-
 storage/SplitRowStoreValueAccessor.hpp          |  45 +++++
 storage/ValueAccessor.hpp                       |  36 ++++
 types/containers/ColumnVector.hpp               |  35 ++++
 types/containers/ColumnVectorsValueAccessor.hpp |  17 ++
 utility/BloomFilterAdapter.hpp                  | 128 ++++++++++++++
 utility/CMakeLists.txt                          |  17 ++
 utility/DAGVisualizer.cpp                       | 167 ++++++++++++++++++
 utility/DAGVisualizer.hpp                       |  85 +++++++++
 utility/EventProfiler.cpp                       |  29 +++
 utility/EventProfiler.hpp                       | 176 +++++++++++++++++++
 55 files changed, 1279 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 20e1fb9..ae85b75 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -760,6 +760,8 @@ target_link_libraries(quickstep_cli_shell
                       quickstep_queryoptimizer_QueryProcessor
                       quickstep_storage_PreloaderThread
                       quickstep_threading_ThreadIDBasedMap
+                      quickstep_utility_DAGVisualizer
+                      quickstep_utility_EventProfiler
                       quickstep_utility_Macros
                       quickstep_utility_PtrVector
                       quickstep_utility_SqlError

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index 64b4f16..0f50706 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/10b25333/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index ce4bc2e..a51172f 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/10b25333/catalog/CatalogRelation.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelation.cpp b/catalog/CatalogRelation.cpp
index 01aebb5..682b6be 100644
--- a/catalog/CatalogRelation.cpp
+++ b/catalog/CatalogRelation.cpp
@@ -143,21 +143,7 @@ CatalogRelation::CatalogRelation(const serialization::CatalogRelationSchema &pro
 }
 
 serialization::CatalogRelationSchema CatalogRelation::getProto() const {
-  serialization::CatalogRelationSchema proto;
-
-  proto.set_relation_id(id_);
-  proto.set_name(name_);
-  proto.set_temporary(temporary_);
-
-  for (PtrVector<CatalogAttribute, true>::const_iterator it = attr_vec_.begin();
-       it != attr_vec_.end();
-       ++it) {
-    if (it.isNull()) {
-      proto.add_attributes();
-    } else {
-      proto.add_attributes()->MergeFrom(it->getProto());
-    }
-  }
+  serialization::CatalogRelationSchema proto = CatalogRelationSchema::getProto();
 
   proto.MutableExtension(serialization::CatalogRelation::default_layout)
       ->MergeFrom(getDefaultStorageBlockLayout().getDescription());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/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/10b25333/catalog/CatalogRelationConstraints.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationConstraints.hpp b/catalog/CatalogRelationConstraints.hpp
new file mode 100644
index 0000000..135ccb9
--- /dev/null
+++ b/catalog/CatalogRelationConstraints.hpp
@@ -0,0 +1,97 @@
+/**
+ *   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 <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();
+  }
+
+ 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/10b25333/catalog/CatalogRelationSchema.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationSchema.cpp b/catalog/CatalogRelationSchema.cpp
index 97c834f..bf8217d 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/10b25333/catalog/CatalogRelationSchema.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationSchema.hpp b/catalog/CatalogRelationSchema.hpp
index d773bc7..0c6c207 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/10b25333/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 7083ef5..bff8c11 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -252,7 +252,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,
@@ -261,8 +262,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) {
@@ -342,8 +354,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/10b25333/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 02a55a0..24f053b 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,8 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 
 #include "storage/PreloaderThread.hpp"
 #include "threading/ThreadIDBasedMap.hpp"
+#include "utility/DAGVisualizer.hpp"
+#include "utility/EventProfiler.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
 #include "utility/SqlError.hpp"
@@ -89,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;
 }
@@ -185,9 +192,33 @@ DEFINE_string(profile_file_name, "",
               // To put things in perspective, the first run is, in my experiments, about 5-10
               // times more expensive than the average run. That means the query needs to be
               // run at least a hundred times to make the impact of the first run small (< 5 %).
+DEFINE_string(profile_output, "",
+              "Output file name for writing the profiled events.");
+DEFINE_bool(visualize_dag, false,
+            "If true, visualize the execution plan DAG into a graph in DOT format.");
 
 }  // 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);
+  }
+}
+
 int main(int argc, char* argv[]) {
   google::InitGoogleLogging(argv[0]);
   gflags::ParseCommandLineFlags(&argc, &argv, true);
@@ -295,6 +326,12 @@ int main(int argc, char* argv[]) {
     LOG(FATAL) << "NON-STANDARD EXCEPTION DURING STARTUP";
   }
 
+//  addPrimaryKeyInfoForTPCHTables(query_processor->getDefaultDatabase());
+//  std::string proto_str;
+//  google::protobuf::TextFormat::PrintToString(
+//      query_processor->getDefaultDatabase()->getProto(), &proto_str);
+//  std::cerr << proto_str << "\n";
+
   // Parse the CPU affinities for workers and the preloader thread, if enabled
   // to warm up the buffer pool.
   const vector<int> worker_cpu_affinities =
@@ -433,6 +470,8 @@ int main(int argc, char* argv[]) {
         }
 
         DCHECK(query_handle->getQueryPlanMutable() != nullptr);
+        quickstep::simple_profiler.clear();
+        quickstep::relop_profiler.clear();
         start = std::chrono::steady_clock::now();
         QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
             main_thread_client_id,
@@ -445,6 +484,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,
@@ -470,6 +514,11 @@ int main(int argc, char* argv[]) {
             foreman.printWorkOrderProfilingResults(query_handle->query_id(),
                                                    stdout);
           }
+          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/10b25333/compression/CompressionDictionaryLite.hpp
----------------------------------------------------------------------
diff --git a/compression/CompressionDictionaryLite.hpp b/compression/CompressionDictionaryLite.hpp
index 45019c0..8c7741f 100644
--- a/compression/CompressionDictionaryLite.hpp
+++ b/compression/CompressionDictionaryLite.hpp
@@ -174,6 +174,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
@@ -255,6 +264,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/10b25333/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index b031a44..5facbb0 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -186,6 +186,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/10b25333/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index 6ba27f1..f94089f 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -29,6 +29,7 @@
 #include "relational_operators/WorkOrder.hpp"
 #include "threading/ThreadIDBasedMap.hpp"
 #include "threading/ThreadUtil.hpp"
+#include "utility/EventProfiler.hpp"
 
 #include "glog/logging.h"
 
@@ -116,8 +117,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();
   const uint64_t execution_time_microseconds =

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManager_unittest.cpp b/query_execution/tests/QueryManager_unittest.cpp
index 37e2cdd..dd3196b 100644
--- a/query_execution/tests/QueryManager_unittest.cpp
+++ b/query_execution/tests/QueryManager_unittest.cpp
@@ -105,6 +105,10 @@ class MockOperator: public RelationalOperator {
         num_calls_donefeedingblocks_(0) {
   }
 
+  std::string getName() const override {
+    return "MockOperator";
+  }
+
 #define MOCK_OP_LOG(x) VLOG(x) << "Op[" << op_index_ << "]: " << __func__ << ": "
 
   // The methods below are used to check whether QueryManager calls the Relational

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/query_optimizer/ExecutionHeuristics.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.cpp b/query_optimizer/ExecutionHeuristics.cpp
index fc31c53..1f2163e 100644
--- a/query_optimizer/ExecutionHeuristics.cpp
+++ b/query_optimizer/ExecutionHeuristics.cpp
@@ -107,20 +107,9 @@ void ExecutionHeuristics::optimizeExecutionPlan(QueryPlan *query_plan,
 
 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 = relation->getStatistics().getNumTuples();
+  bloom_filter_proto->set_bloom_filter_size(cardinality);
+  bloom_filter_proto->set_number_of_hashes(3);
 }
 
 }  // namespace optimizer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/query_optimizer/ExecutionHeuristics.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionHeuristics.hpp b/query_optimizer/ExecutionHeuristics.hpp
index 92a7fe8..c43d591 100644
--- a/query_optimizer/ExecutionHeuristics.hpp
+++ b/query_optimizer/ExecutionHeuristics.hpp
@@ -49,7 +49,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;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 75a7bc9..ee133b5 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -108,7 +108,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/10b25333/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index 48f76fa..8f31265 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -84,7 +84,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(
@@ -115,7 +115,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()));
 }
 
 }  // namespace cost

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index 9770606..9357590 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -281,6 +281,7 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
       // 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->estimated_selectivity = cost_model_->estimateSelectivity(output);
 
       second_table_info->join_attribute_pairs.insert(first_table_info->join_attribute_pairs.begin(),
                                                      first_table_info->join_attribute_pairs.end());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index 4bcbcf6..c46ba2c 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -77,6 +77,10 @@ class AggregationOperator : public RelationalOperator {
 
   ~AggregationOperator() override {}
 
+  std::string getName() const override {
+    return "AggregationOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 464bbf8..952c7ac 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -93,6 +93,10 @@ class BuildHashOperator : public RelationalOperator {
 
   ~BuildHashOperator() override {}
 
+  std::string getName() const override {
+    return "BuildHashOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index 18ca656..4e05448 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -69,6 +69,10 @@ class CreateIndexOperator : public RelationalOperator {
 
   ~CreateIndexOperator() override {}
 
+  std::string getName() const override {
+    return "CreateIndexOperator";
+  }
+
   /**
    * @note No WorkOrder generated for this operator.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 6d91142..b7b707b 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -66,6 +66,10 @@ class CreateTableOperator : public RelationalOperator {
 
   ~CreateTableOperator() override {}
 
+  std::string getName() const override {
+    return "CreateTableOperator";
+  }
+
   /**
    * @note No WorkOrder generated for this operator.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index 74da8c1..abfe4a9 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -81,6 +81,10 @@ class DeleteOperator : public RelationalOperator {
 
   ~DeleteOperator() override {}
 
+  std::string getName() const override {
+    return "DeleteOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 181386f..ae65de5 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -58,6 +58,10 @@ class DestroyHashOperator : public RelationalOperator {
 
   ~DestroyHashOperator() override {}
 
+  std::string getName() const override {
+    return "DestroyHashOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index 6c7fca3..f854b4f 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -74,6 +74,10 @@ class DropTableOperator : public RelationalOperator {
 
   ~DropTableOperator() override {}
 
+  std::string getName() const override {
+    return "DropTableOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index 158a637..0dcfc9e 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -74,6 +74,10 @@ class FinalizeAggregationOperator : public RelationalOperator {
 
   ~FinalizeAggregationOperator() override {}
 
+  std::string getName() const override {
+    return "FinalizeAggregationOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 5d3d7da..0b9e72b 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -157,6 +157,21 @@ class HashJoinOperator : public RelationalOperator {
 
   ~HashJoinOperator() override {}
 
+  std::string getName() const override {
+    switch (join_type_) {
+      case JoinType::kInnerJoin:
+        return "HashJoinOperator";
+      case JoinType::kLeftSemiJoin:
+        return "HashJoinOperator(LeftSemi)";
+      case JoinType::kLeftAntiJoin:
+        return "HashJoinOperator(LeftAnti)";
+      case JoinType::kLeftOuterJoin:
+        return "HashJoinOperator(LeftOuter)";
+      default: break;
+    }
+    LOG(FATAL) << "Unknown join type in HashJoinOperator::getName()";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,
@@ -283,8 +298,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),
@@ -411,8 +427,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),
@@ -458,8 +475,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)),
@@ -535,8 +553,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/10b25333/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index 78f5199..2c6aca7 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -73,6 +73,10 @@ class InsertOperator : public RelationalOperator {
 
   ~InsertOperator() override {}
 
+  std::string getName() const override {
+    return "InsertOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index 992e76d..cf190fe 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -116,6 +116,10 @@ class NestedLoopsJoinOperator : public RelationalOperator {
 
   ~NestedLoopsJoinOperator() override {}
 
+  std::string getName() const override {
+    return "NestedLoopsJoinOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index 116727b..65cd213 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -55,6 +55,13 @@ class RelationalOperator {
   virtual ~RelationalOperator() {}
 
   /**
+   * @brief Get the name of this relational operator.
+   *
+   * @return The name of this relational operator.
+   */
+  virtual std::string getName() const = 0;
+
+  /**
    * @brief Generate all the next WorkOrders for this RelationalOperator.
    *
    * @note If a RelationalOperator has blocking dependencies, it should not
@@ -226,6 +233,15 @@ class RelationalOperator {
     op_index_ = operator_index;
   }
 
+  /**
+   * @brief Get the index of this operator in the query plan DAG.
+   *
+   * @return The index of this operator in the query plan DAG.
+   */
+  std::size_t getOperatorIndex() const {
+    return op_index_;
+  }
+
  protected:
   /**
    * @brief Constructor

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index f8fe5f6..08f08c8 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -93,6 +93,10 @@ class SampleOperator : public RelationalOperator {
 
   ~SampleOperator() override {}
 
+  std::string getName() const override {
+    return "SampleOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 50032b6..ebc5ffc 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -64,6 +64,10 @@ class SaveBlocksOperator : public RelationalOperator {
 
   ~SaveBlocksOperator() override {}
 
+  std::string getName() const override {
+    return "SaveBlocksOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 0c10686..ee25886 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -189,6 +189,10 @@ class SelectOperator : public RelationalOperator {
 
   ~SelectOperator() override {}
 
+  std::string getName() const override {
+    return "SelectOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index 177836f..9b07ad6 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -129,6 +129,10 @@ class SortMergeRunOperator : public RelationalOperator {
    **/
   ~SortMergeRunOperator() {}
 
+  std::string getName() const override {
+    return "SortMergeRunOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index 96a3ce1..54c7feb 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -109,6 +109,10 @@ class SortRunGenerationOperator : public RelationalOperator {
 
   ~SortRunGenerationOperator() {}
 
+  std::string getName() const override {
+    return "SortRunGenerationOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index 1b791a6..15e7052 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -76,6 +76,10 @@ class TableGeneratorOperator : public RelationalOperator {
 
   ~TableGeneratorOperator() override {}
 
+  std::string getName() const override {
+    return "TableGeneratorOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index 1a62ded..6890d7d 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -134,6 +134,10 @@ class TextScanOperator : public RelationalOperator {
 
   ~TextScanOperator() override {}
 
+  std::string getName() const override {
+    return "TextScanOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index 4471a17..d021844 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -94,6 +94,10 @@ class UpdateOperator : public RelationalOperator {
 
   ~UpdateOperator() override {}
 
+  std::string getName() const override {
+    return "UpdateOperator";
+  }
+
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,
                         StorageManager *storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index df195cc..4eb6b3a 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -299,16 +299,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/10b25333/storage/BasicColumnStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/BasicColumnStoreValueAccessor.hpp b/storage/BasicColumnStoreValueAccessor.hpp
index 759e187..7907fd5 100644
--- a/storage/BasicColumnStoreValueAccessor.hpp
+++ b/storage/BasicColumnStoreValueAccessor.hpp
@@ -18,6 +18,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"
@@ -43,7 +45,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 {
@@ -61,9 +64,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,
@@ -80,6 +97,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/10b25333/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index b536411..38bc507 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -669,6 +669,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/10b25333/storage/CompressedColumnStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedColumnStoreValueAccessor.hpp b/storage/CompressedColumnStoreValueAccessor.hpp
index 64eb315..984dea3 100644
--- a/storage/CompressedColumnStoreValueAccessor.hpp
+++ b/storage/CompressedColumnStoreValueAccessor.hpp
@@ -52,6 +52,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),
@@ -84,6 +85,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]) {
@@ -138,6 +159,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/10b25333/storage/CompressedPackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/CompressedPackedRowStoreValueAccessor.hpp b/storage/CompressedPackedRowStoreValueAccessor.hpp
index 024b0ec..7058aec 100644
--- a/storage/CompressedPackedRowStoreValueAccessor.hpp
+++ b/storage/CompressedPackedRowStoreValueAccessor.hpp
@@ -58,6 +58,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),
@@ -92,6 +93,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]) {
@@ -150,6 +171,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/10b25333/storage/PackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreValueAccessor.hpp b/storage/PackedRowStoreValueAccessor.hpp
index 03a975e..cbd273e 100644
--- a/storage/PackedRowStoreValueAccessor.hpp
+++ b/storage/PackedRowStoreValueAccessor.hpp
@@ -18,6 +18,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"
@@ -40,7 +42,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 {
@@ -65,6 +68,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();
@@ -79,6 +101,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/10b25333/storage/SplitRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreValueAccessor.hpp b/storage/SplitRowStoreValueAccessor.hpp
index 9ea1a3a..19937f2 100644
--- a/storage/SplitRowStoreValueAccessor.hpp
+++ b/storage/SplitRowStoreValueAccessor.hpp
@@ -100,6 +100,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_);
   }
@@ -140,6 +145,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));
@@ -317,6 +360,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()) {
   }
 
@@ -327,6 +371,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/10b25333/storage/ValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessor.hpp b/storage/ValueAccessor.hpp
index e2a898e..e9370cc 100644
--- a/storage/ValueAccessor.hpp
+++ b/storage/ValueAccessor.hpp
@@ -375,6 +375,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_);
   }
@@ -387,6 +392,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);
@@ -560,6 +572,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_]);
   }
@@ -571,6 +589,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 "
@@ -737,6 +762,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_);
   }
@@ -747,6 +777,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/10b25333/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index 76968ba..a9349ee 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -193,6 +193,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.
@@ -453,6 +469,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/10b25333/types/containers/ColumnVectorsValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVectorsValueAccessor.hpp b/types/containers/ColumnVectorsValueAccessor.hpp
index f1d29a2..d69d1d8 100644
--- a/types/containers/ColumnVectorsValueAccessor.hpp
+++ b/types/containers/ColumnVectorsValueAccessor.hpp
@@ -124,6 +124,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_);
   }
@@ -140,6 +145,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/10b25333/utility/BloomFilterAdapter.hpp
----------------------------------------------------------------------
diff --git a/utility/BloomFilterAdapter.hpp b/utility/BloomFilterAdapter.hpp
new file mode 100644
index 0000000..5deb275
--- /dev/null
+++ b/utility/BloomFilterAdapter.hpp
@@ -0,0 +1,128 @@
+/**
+ *   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<std::vector<attribute_id>> &attribute_ids)
+      : num_bloom_filters_(bloom_filters.size()) {
+    DCHECK_EQ(bloom_filters.size(), attribute_ids.size());
+
+    bloom_filter_entries_.reserve(num_bloom_filters_);
+    bloom_filter_entry_indices_.reserve(num_bloom_filters_);
+
+    for (std::size_t i = 0; i < num_bloom_filters_; ++i) {
+      bloom_filter_entries_.emplace_back(bloom_filters[i], attribute_ids[i]);
+      bloom_filter_entry_indices_.emplace_back(i);
+    }
+  }
+
+  template <typename ValueAccessorT>
+  inline bool miss(const ValueAccessorT *accessor) {
+    return missImpl<ValueAccessorT, true>(accessor);
+  }
+
+  template <typename ValueAccessorT, bool adapt_filters>
+  inline bool missImpl(const ValueAccessorT *accessor) {
+    for (std::size_t i = 0; i < num_bloom_filters_; ++i) {
+      const std::size_t entry_idx = bloom_filter_entry_indices_[i];
+      BloomFilterEntry &entry = bloom_filter_entries_[entry_idx];
+      if (adapt_filters) {
+        ++entry.cnt;
+      }
+
+      const BloomFilter *bloom_filter = entry.bloom_filter;
+      for (const attribute_id &attr_id : entry.attribute_ids) {
+        const std::pair<const void*, std::size_t> value_and_byte_length =
+            accessor->getUntypedValueAndByteLength(attr_id);
+        if (!bloom_filter->contains(static_cast<const std::uint8_t*>(value_and_byte_length.first),
+                                    value_and_byte_length.second)) {
+          if (adapt_filters) {
+            // Record miss
+            ++entry.miss;
+
+            // Update entry order
+            if (i > 0) {
+              const std::size_t prev_entry_idx = bloom_filter_entry_indices_[i-1];
+              if (entry.isBetterThan(bloom_filter_entries_[prev_entry_idx])) {
+                bloom_filter_entry_indices_[i-1] = entry_idx;
+                bloom_filter_entry_indices_[i] = prev_entry_idx;
+              }
+            }
+          }
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+ private:
+  struct BloomFilterEntry {
+    BloomFilterEntry(const BloomFilter *in_bloom_filter,
+                     const std::vector<attribute_id> &in_attribute_ids)
+        : bloom_filter(in_bloom_filter),
+          attribute_ids(in_attribute_ids),
+          miss(0),
+          cnt(0) {
+    }
+
+    inline bool isBetterThan(const BloomFilterEntry& other) {
+      return static_cast<std::uint64_t>(miss) * other.cnt
+                 > static_cast<std::uint64_t>(cnt + 5) * (other.miss + 5);
+    }
+
+    const BloomFilter *bloom_filter;
+    const std::vector<attribute_id> &attribute_ids;
+    std::uint32_t miss;
+    std::uint32_t cnt;
+  };
+
+  const std::size_t num_bloom_filters_;
+  std::vector<BloomFilterEntry> bloom_filter_entries_;
+  std::vector<std::size_t> bloom_filter_entry_indices_;
+
+  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/10b25333/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 2d3db8f..133e2f3 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -159,6 +159,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})
@@ -166,6 +167,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_DAGVisualizer DAGVisualizer.cpp DAGVisualizer.hpp)
+add_library(quickstep_utility_EventProfiler EventProfiler.cpp EventProfiler.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)
 add_library(quickstep_utility_Glob Glob.cpp Glob.hpp)
 add_library(quickstep_utility_HashPair ../empty_src.cpp HashPair.hpp)
@@ -216,6 +219,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
@@ -225,6 +232,8 @@ target_link_libraries(quickstep_utility_CheckSnprintf
 target_link_libraries(quickstep_utility_DAG
                       glog
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_utility_EventProfiler
+                      quickstep_threading_Mutex)
 target_link_libraries(quickstep_utility_Glob
                       glog)
 target_link_libraries(quickstep_utility_MemStream
@@ -243,6 +252,11 @@ target_link_libraries(quickstep_utility_PlanVisualizer
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_utility_Macros
                       quickstep_utility_StringUtil)
+target_link_libraries(quickstep_utility_DAGVisualizer
+                      quickstep_queryoptimizer_QueryPlan
+                      quickstep_utility_EventProfiler
+                      quickstep_utility_Macros
+                      quickstep_utility_StringUtil)
 target_link_libraries(quickstep_utility_PtrList
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_PtrMap
@@ -297,11 +311,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_DAGVisualizer
+                      quickstep_utility_EventProfiler
                       quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Glob
                       quickstep_utility_HashPair



[3/8] incubator-quickstep git commit: Minor changes in profiling work order output.

Posted by ji...@apache.org.
Minor changes in profiling work order output.

- Now prints query ID along with each work order entry.
- Removed spaces between two columns.


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

Branch: refs/heads/adaptive-bloom-filters
Commit: 04c8224b1584b982412c7023d041c1060d5c0342
Parents: 31f1bbb
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Wed Jul 6 11:38:49 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Jul 6 22:28:28 2016 -0500

----------------------------------------------------------------------
 query_execution/Foreman.cpp | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/04c8224b/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index f9f2e7a..98146e2 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -238,16 +238,17 @@ void Foreman::printWorkOrderProfilingResults(const std::size_t query_id,
   const std::vector<
       std::tuple<std::size_t, std::size_t, std::size_t>>
       &recorded_times = policy_enforcer_->getProfilingResults(query_id);
-  fputs("Worker ID, NUMA Socket, Operator ID, Time (microseconds)\n", out);
+  fputs("Query ID,Worker ID,NUMA Socket,Operator ID,Time (microseconds)\n", out);
   for (auto workorder_entry : recorded_times) {
     // Note: Index of the "worker thread index" in the tuple is 0.
     const std::size_t worker_id = std::get<0>(workorder_entry);
     fprintf(out,
-            "%lu, %d, %lu, %lu\n",
+            "%lu,%lu,%d,%lu,%lu\n",
+            query_id,
             worker_id,
             worker_directory_->getNUMANode(worker_id),
-            std::get<1>(workorder_entry),
-            std::get<2>(workorder_entry));
+            std::get<1>(workorder_entry),  // Operator ID.
+            std::get<2>(workorder_entry));  // Time.
   }
 }
 


[5/8] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/utility/DAGVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/DAGVisualizer.cpp b/utility/DAGVisualizer.cpp
new file mode 100644
index 0000000..e62f948
--- /dev/null
+++ b/utility/DAGVisualizer.cpp
@@ -0,0 +1,167 @@
+/**
+ *   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/DAGVisualizer.hpp"
+#include "utility/EventProfiler.hpp"
+
+#include <cmath>
+#include <cstddef>
+#include <iomanip>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "query_optimizer/QueryPlan.hpp"
+#include "utility/EventProfiler.hpp"
+#include "utility/StringUtil.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+std::string DAGVisualizer::toDOT() {
+  std::set<std::string> no_display_op_names =
+      { "DestroyHashOperator", "DropTableOperator" };
+
+  const auto &dag = plan_.getQueryPlanDAG();
+  const std::size_t num_nodes = dag.size();
+
+  std::vector<double> time_elapsed(num_nodes, 0);
+  std::vector<double> time_percentage(num_nodes, 0);
+  std::vector<double> time_start(num_nodes, std::numeric_limits<double>::max());
+  std::vector<double> time_end(num_nodes, 0);
+  const auto &zero_time = relop_profiler.zero_time();
+  for (const auto &container : relop_profiler.containers()) {
+    for (const auto &line : container.second.events) {
+      const std::size_t relop_index = line.first;
+      for (const auto &event : line.second) {
+        time_elapsed[relop_index] +=
+            std::chrono::duration<double>(event.end_time - event.start_time).count();
+        time_start[relop_index] =
+            std::min(time_start[relop_index],
+                     std::chrono::duration<double>(event.start_time - zero_time).count());
+        time_end[relop_index] =
+            std::max(time_end[relop_index],
+                     std::chrono::duration<double>(event.end_time - zero_time).count());
+      }
+    }
+  }
+  const std::size_t num_threads = relop_profiler.containers().size();
+  double total_time_elapsed = 0;
+  double max_percentage = 0;
+  for (std::size_t i = 0; i < time_elapsed.size(); ++i) {
+    time_elapsed[i] /= num_threads;
+    total_time_elapsed += time_elapsed[i];
+  }
+  for (std::size_t i = 0; i < time_elapsed.size(); ++i) {
+    time_percentage[i] = time_elapsed[i] / total_time_elapsed;
+    max_percentage = std::max(max_percentage, time_percentage[i]);
+  }
+
+  std::vector<bool> display_ops(num_nodes, false);
+  for (std::size_t node_index = 0; node_index < num_nodes; ++node_index) {
+    const auto &node = dag.getNodePayload(node_index);
+    const std::string relop_name = node.getName();
+    if (no_display_op_names.find(relop_name) == no_display_op_names.end()) {
+      display_ops[node_index] = true;
+
+      nodes_.emplace_back();
+      NodeInfo &node_info = nodes_.back();
+      node_info.id = node_index;
+
+      std::string hue =
+          std::to_string(std::sqrt(time_percentage[node_index] / max_percentage));
+      node_info.color = hue + " " + hue + " 1.0";
+
+      node_info.labels.emplace_back(
+          "[" + std::to_string(node.getOperatorIndex()) + "] " + relop_name);
+      node_info.labels.emplace_back(
+          std::to_string(std::lround(time_elapsed[node_index] * 1000)) +
+          "ms (" + PercentageToString(time_percentage[node_index] * 100) + "%)");
+      node_info.labels.emplace_back(
+          "span: [" +
+          std::to_string(std::lround(time_start[node_index] * 1000)) + "ms, " +
+          std::to_string(std::lround(time_end[node_index] * 1000)) + "ms]");
+    }
+  }
+  for (std::size_t node_index = 0; node_index < num_nodes; ++node_index) {
+    if (display_ops[node_index]) {
+      for (const auto &link : dag.getDependents(node_index)) {
+        if (display_ops[link.first]) {
+          edges_.emplace_back();
+          EdgeInfo &edge_info = edges_.back();
+          edge_info.src_node_id = node_index;
+          edge_info.dst_node_id = link.first;
+          edge_info.is_pipeline_breaker = link.second;
+        }
+      }
+    }
+  }
+
+  // Format output graph
+  std::ostringstream graph_oss;
+  graph_oss << "digraph g {\n";
+  graph_oss << "  rankdir=BT\n";
+  graph_oss << "  node [penwidth=2]\n";
+  graph_oss << "  edge [fontsize=16 fontcolor=gray penwidth=2]\n\n";
+
+  // Format nodes
+  for (const NodeInfo &node_info : nodes_) {
+    graph_oss << "  " << node_info.id << " [ ";
+    if (!node_info.labels.empty()) {
+      graph_oss << "label=\""
+                << EscapeSpecialChars(JoinToString(node_info.labels, "&#10;"))
+                << "\" ";
+    }
+    if (!node_info.color.empty()) {
+      graph_oss << "style=filled fillcolor=\"" << node_info.color << "\" ";
+    }
+    graph_oss << "]\n";
+  }
+  graph_oss << "\n";
+
+  // Format edges
+  for (const EdgeInfo &edge_info : edges_) {
+    graph_oss << "  " << edge_info.src_node_id << " -> "
+              << edge_info.dst_node_id << " [ ";
+    if (edge_info.is_pipeline_breaker) {
+      graph_oss << "style=dashed ";
+    }
+    if (!edge_info.labels.empty()) {
+      graph_oss << "label=\""
+                << EscapeSpecialChars(JoinToString(edge_info.labels, "&#10;"))
+                << "\" ";
+    }
+    graph_oss << "]\n";
+  }
+
+  graph_oss << "}\n";
+
+  return graph_oss.str();
+}
+
+std::string DAGVisualizer::PercentageToString(double percentage) {
+  std::ostringstream oss;
+  oss << static_cast<std::uint32_t>(percentage) << ".";
+  int digits = std::lround(percentage * 10000) % 100;
+  oss << digits / 10 << digits % 10;
+  return oss.str();
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/utility/DAGVisualizer.hpp
----------------------------------------------------------------------
diff --git a/utility/DAGVisualizer.hpp b/utility/DAGVisualizer.hpp
new file mode 100644
index 0000000..5c81d22
--- /dev/null
+++ b/utility/DAGVisualizer.hpp
@@ -0,0 +1,85 @@
+/**
+ *   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_DAG_VISUALIZER_HPP_
+#define QUICKSTEP_UTILITY_DAG_VISUALIZER_HPP_
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class QueryPlan;
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief A visualizer that converts an execution plan DAG into a graph in
+ *        DOT format. Note that DOT is a plain text graph description language.
+ *
+ * @note This utility tool can be further extended to be more generic.
+ */
+class DAGVisualizer {
+ public:
+  DAGVisualizer(const QueryPlan &plan)
+      : plan_(plan) {}
+
+  ~DAGVisualizer() {}
+
+  std::string toDOT();
+
+ private:
+  static std::string PercentageToString(double percentage);
+
+  /**
+   * @brief Information of a graph node.
+   */
+  struct NodeInfo {
+    std::size_t id;
+    std::vector<std::string> labels;
+    std::string color;
+  };
+
+  /**
+   * @brief Information of a graph edge.
+   */
+  struct EdgeInfo {
+    std::size_t src_node_id;
+    std::size_t dst_node_id;
+    std::vector<std::string> labels;
+    bool is_pipeline_breaker;
+  };
+
+  const QueryPlan &plan_;
+
+  std::vector<NodeInfo> nodes_;
+  std::vector<EdgeInfo> edges_;
+
+  DISALLOW_COPY_AND_ASSIGN(DAGVisualizer);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_UTILITY_DAG_VISUALIZER_HPP_ */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/utility/EventProfiler.cpp
----------------------------------------------------------------------
diff --git a/utility/EventProfiler.cpp b/utility/EventProfiler.cpp
new file mode 100644
index 0000000..cac6737
--- /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<std::string, int> simple_profiler;
+EventProfiler<std::size_t> relop_profiler;
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/10b25333/utility/EventProfiler.hpp
----------------------------------------------------------------------
diff --git a/utility/EventProfiler.hpp b/utility/EventProfiler.hpp
new file mode 100644
index 0000000..ca91593
--- /dev/null
+++ b/utility/EventProfiler.hpp
@@ -0,0 +1,176 @@
+/**
+ *   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 {
+    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];
+    }
+
+    std::map<TagT, std::vector<EventInfo>> events;
+  };
+
+  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<std::string, int> simple_profiler;
+extern EventProfiler<std::size_t> relop_profiler;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_EVENT_PROFILER_HPP_


[7/8] 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/a3ba59f9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/a3ba59f9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/a3ba59f9

Branch: refs/heads/adaptive-bloom-filters
Commit: a3ba59f9a5bd02620ed3e9101899c2c885343d57
Parents: 0e685e0
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Tue Jul 5 15:50:45 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Jul 7 12:54:04 2016 -0500

----------------------------------------------------------------------
 query_optimizer/PhysicalGenerator.cpp           |  1 +
 .../StarSchemaHashJoinOrderOptimization.cpp     | 29 ++++++++++++-
 .../StarSchemaHashJoinOrderOptimization.hpp     | 43 +++++++++++++++-----
 3 files changed, 61 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a3ba59f9/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index ee133b5..5f97d6f 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -95,6 +95,7 @@ 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());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a3ba59f9/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index c9bd7d2..2e0b6c6 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -73,6 +73,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;
@@ -145,7 +148,9 @@ 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()));
   }
 
   // Auxiliary mapping info.
@@ -237,6 +242,11 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
     CHECK(selected_probe_table_info != nullptr);
     CHECK(selected_build_table_info != nullptr);
 
+    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);
 
@@ -283,6 +293,10 @@ physical::PhysicalPtr StarSchemaHashJoinOrderOptimization::generatePlan(
       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());
+
       remaining_tables.emplace(selected_probe_table_info);
 
       // Update join attribute groups.
@@ -307,5 +321,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/a3ba59f9/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
index 9ff89fd..6ad300c 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
@@ -62,6 +62,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;
   };
@@ -70,20 +71,23 @@ 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)
+        : 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) {
     }
 
     const std::size_t table_info_id;
     physical::PhysicalPtr table;
     std::size_t estimated_cardinality;
     double estimated_selectivity;
+    std::size_t estimated_num_output_attributes;
   };
 
   struct JoinPair {
@@ -91,13 +95,26 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
         : probe(probe_in), build(build_in) {
     }
 
-    inline bool isBetterThan (const JoinPair &rhs) const {
+    inline bool isBetterThan(const JoinPair &rhs) const {
       const auto &lhs = *this;
-      const bool lhs_has_small_build = lhs.build->estimated_cardinality < 0x1000;
-      const bool rhs_has_small_build = rhs.build->estimated_cardinality < 0x1000;
+      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) {
+        return rhs_has_large_output;
+      }
+
+      const bool lhs_has_small_build =
+          !lhs_has_large_output && lhs.build->estimated_cardinality < 0x1000;
+      const bool rhs_has_small_build =
+          !rhs_has_large_output && rhs.build->estimated_cardinality < 0x1000;
       if (lhs_has_small_build != rhs_has_small_build) {
         return lhs_has_small_build;
       }
+
       if (lhs.probe->estimated_cardinality != rhs.probe->estimated_cardinality) {
         return lhs.probe->estimated_cardinality < rhs.probe->estimated_cardinality;
       }
@@ -126,6 +143,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);


[4/8] incubator-quickstep git commit: new plan

Posted by ji...@apache.org.
new plan


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

Branch: refs/heads/adaptive-bloom-filters
Commit: 0e685e0a03b3803ac502415531d33df830789f29
Parents: 10b2533
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Fri Jul 1 13:12:32 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Jul 7 12:54:04 2016 -0500

----------------------------------------------------------------------
 catalog/CatalogRelationConstraints.hpp          |  13 ++
 cli/QuickstepCli.cpp                            |   2 +
 query_optimizer/ExecutionGenerator.cpp          |  18 +-
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  26 ++-
 query_optimizer/expressions/ExpressionUtil.hpp  |   8 +-
 query_optimizer/physical/Aggregate.cpp          |   5 +
 query_optimizer/physical/Aggregate.hpp          |   3 +
 query_optimizer/physical/HashJoin.cpp           |   9 +
 query_optimizer/physical/HashJoin.hpp           |   3 +
 query_optimizer/physical/Physical.hpp           |   5 +
 query_optimizer/physical/Selection.cpp          |   6 +
 query_optimizer/physical/Selection.hpp          |   3 +
 query_optimizer/physical/TableReference.cpp     |  18 ++
 query_optimizer/physical/TableReference.hpp     |   3 +
 .../StarSchemaHashJoinOrderOptimization.cpp     | 207 ++++++++++---------
 .../StarSchemaHashJoinOrderOptimization.hpp     |  49 ++---
 utility/CMakeLists.txt                          |   2 +
 utility/DisjointTreeForest.hpp                  | 116 +++++++++++
 utility/PlanVisualizer.cpp                      |  18 +-
 19 files changed, 371 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/catalog/CatalogRelationConstraints.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationConstraints.hpp b/catalog/CatalogRelationConstraints.hpp
index 135ccb9..896c072 100644
--- a/catalog/CatalogRelationConstraints.hpp
+++ b/catalog/CatalogRelationConstraints.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_
 #define QUICKSTEP_CATALOG_CATALOG_RELATION_CONSTRAINTS_HPP_
 
+#include <algorithm>
 #include <cstddef>
 #include <memory>
 #include <set>
@@ -84,6 +85,18 @@ class CatalogRelationConstraints {
     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_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 24f053b..1f389fe 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -331,6 +331,8 @@ int main(int argc, char* argv[]) {
 //  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.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 45f5f78..f425d46 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -671,9 +671,21 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   std::size_t build_cardinality = cost_model_->estimateCardinality(build_physical);
   // For inner join, we may swap the probe table and the build table.
   if (physical_plan->join_type() == P::HashJoin::JoinType::kInnerJoin)  {
-    // Choose the smaller table as the inner build table,
-    // and the other one as the outer probe table.
-    if (probe_cardinality < build_cardinality) {
+    const bool left_unique_join_attrs =
+        probe_physical->impliesUniqueAttributes(left_join_attributes);
+    const bool right_unique_join_attrs =
+        build_physical->impliesUniqueAttributes(right_join_attributes);
+
+    bool swap_probe_build;
+    if (left_unique_join_attrs != right_unique_join_attrs) {
+      swap_probe_build = left_unique_join_attrs;
+    } else {
+      // Choose the smaller table as the inner build table,
+      // and the other one as the outer probe table.
+      swap_probe_build = (probe_cardinality < build_cardinality);
+    }
+
+    if (swap_probe_build) {
       // Switch the probe and build physical nodes.
       std::swap(probe_physical, build_physical);
       std::swap(probe_cardinality, build_cardinality);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index eb9fcc1..4a85928 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -118,12 +118,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(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/query_optimizer/expressions/ExpressionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionUtil.hpp b/query_optimizer/expressions/ExpressionUtil.hpp
index 4c35719..5e9d29d 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/0e685e0a/query_optimizer/physical/Aggregate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Aggregate.cpp b/query_optimizer/physical/Aggregate.cpp
index c582bba..969daa7 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/0e685e0a/query_optimizer/physical/Aggregate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Aggregate.hpp b/query_optimizer/physical/Aggregate.hpp
index 2c2aee7..e40d894 100644
--- a/query_optimizer/physical/Aggregate.hpp
+++ b/query_optimizer/physical/Aggregate.hpp
@@ -98,6 +98,9 @@ class Aggregate : public Physical {
     return false;
   }
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
   /**
    * @brief Creates an Aggregate physical node.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/query_optimizer/physical/HashJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.cpp b/query_optimizer/physical/HashJoin.cpp
index 71c3692..7bea86a 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,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/query_optimizer/physical/HashJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.hpp b/query_optimizer/physical/HashJoin.hpp
index b904b5f..55c9588 100644
--- a/query_optimizer/physical/HashJoin.hpp
+++ b/query_optimizer/physical/HashJoin.hpp
@@ -124,6 +124,9 @@ class HashJoin : public BinaryJoin {
       const expressions::UnorderedNamedExpressionSet &referenced_expressions,
       PhysicalPtr *output) const override;
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
   /**
    * @brief Creates a physical HashJoin. The left/right operand does not correspond to
    *        probe/build operand.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/query_optimizer/physical/Physical.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Physical.hpp b/query_optimizer/physical/Physical.hpp
index 9fdbeb5..721b987 100644
--- a/query_optimizer/physical/Physical.hpp
+++ b/query_optimizer/physical/Physical.hpp
@@ -84,6 +84,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/0e685e0a/query_optimizer/physical/Selection.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.cpp b/query_optimizer/physical/Selection.cpp
index 5e1a03f..f4cdd1a 100644
--- a/query_optimizer/physical/Selection.cpp
+++ b/query_optimizer/physical/Selection.cpp
@@ -80,6 +80,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/0e685e0a/query_optimizer/physical/Selection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Selection.hpp b/query_optimizer/physical/Selection.hpp
index d8c1319..68cae65 100644
--- a/query_optimizer/physical/Selection.hpp
+++ b/query_optimizer/physical/Selection.hpp
@@ -84,6 +84,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/0e685e0a/query_optimizer/physical/TableReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableReference.cpp b/query_optimizer/physical/TableReference.cpp
index 4a66ddf..bc73046 100644
--- a/query_optimizer/physical/TableReference.cpp
+++ b/query_optimizer/physical/TableReference.cpp
@@ -18,6 +18,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 
 #include <string>
+#include <set>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
@@ -30,6 +31,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/0e685e0a/query_optimizer/physical/TableReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TableReference.hpp b/query_optimizer/physical/TableReference.hpp
index bde9b97..bc07043 100644
--- a/query_optimizer/physical/TableReference.hpp
+++ b/query_optimizer/physical/TableReference.hpp
@@ -88,6 +88,9 @@ class TableReference : public Physical {
     return false;
   }
 
+  bool impliesUniqueAttributes(
+      const std::vector<expressions::AttributeReferencePtr> &attributes) const override;
+
   /**
    * @brief Creates a TableReference.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
index 9357590..c9bd7d2 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.cpp
@@ -31,6 +31,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"
 
@@ -161,9 +162,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());
@@ -176,129 +187,119 @@ 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);
+    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);
   }
 
-  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]);
-  }
-
-  // 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;
+    // 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)) {
+              best_join.reset(new_join.release());
+            }
+          }
         }
       }
-      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;
-        }
-      }
-      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);
+
+    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(build_child,
+                              probe_child,
+                              build_attributes,
+                              probe_attributes,
                               nullptr,
                               output_attributes,
                               P::HashJoin::JoinType::kInnerJoin);
 
-      second_table_info->table = output;
+      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->estimated_selectivity = cost_model_->estimateSelectivity(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);
+
+      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(build_child,
+                                 probe_child,
+                                 build_attributes,
+                                 probe_attributes,
                                  residual_predicate,
                                  project_expressions,
                                  P::HashJoin::JoinType::kInnerJoin);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
index deddffd..9ff89fd 100644
--- a/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
+++ b/query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp
@@ -84,35 +84,38 @@ class StarSchemaHashJoinOrderOptimization : public Rule<physical::Physical> {
     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;
   };
 
-  /**
-   * @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) {
+    }
 
-      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;
+    inline bool isBetterThan (const JoinPair &rhs) const {
+      const auto &lhs = *this;
+      const bool lhs_has_small_build = lhs.build->estimated_cardinality < 0x1000;
+      const bool rhs_has_small_build = rhs.build->estimated_cardinality < 0x1000;
+      if (lhs_has_small_build != rhs_has_small_build) {
+        return lhs_has_small_build;
+      }
+      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,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 133e2f3..8582981 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -168,6 +168,7 @@ 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_DAGVisualizer DAGVisualizer.cpp DAGVisualizer.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_Glob Glob.cpp Glob.hpp)
@@ -257,6 +258,7 @@ target_link_libraries(quickstep_utility_DAGVisualizer
                       quickstep_utility_EventProfiler
                       quickstep_utility_Macros
                       quickstep_utility_StringUtil)
+target_link_libraries(quickstep_utility_DisjointTreeForest)
 target_link_libraries(quickstep_utility_PtrList
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_PtrMap

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0e685e0a/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/0e685e0a/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index 962d577..37fa790 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -19,6 +19,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <set>
 #include <sstream>
 #include <string>
 #include <unordered_map>
@@ -101,6 +102,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);
 
@@ -111,10 +116,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());
       }
     }
@@ -145,6 +148,13 @@ 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");
+      }
+
       break;
     }
     default: {



[8/8] 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/8904ba12
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/8904ba12
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/8904ba12

Branch: refs/heads/adaptive-bloom-filters
Commit: 8904ba1214553fdc73000b9061d931a93f417af4
Parents: a3ba59f
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Jul 7 13:57:21 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Jul 7 13:57:21 2016 -0500

----------------------------------------------------------------------
 query_optimizer/physical/Physical.hpp | 5 +++++
 utility/PlanVisualizer.cpp            | 4 +++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8904ba12/query_optimizer/physical/Physical.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/Physical.hpp b/query_optimizer/physical/Physical.hpp
index 721b987..4603efc 100644
--- a/query_optimizer/physical/Physical.hpp
+++ b/query_optimizer/physical/Physical.hpp
@@ -89,6 +89,11 @@ class Physical : public OptimizerTree<Physical> {
     return false;
   }
 
+  virtual double estimateSelectivity(
+      const expressions::AttributeReferencePtr &attribute) const {
+    return 1.0;
+  }
+
  protected:
   /**
    * @brief Constructor.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8904ba12/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index 37fa790..26c555e 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -118,7 +118,9 @@ 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::ostringstream oss;
+        oss << attr->attribute_alias() << " : " << child->estimateSelectivity(attr);
+        edge_info.labels.emplace_back(oss.str());
       }
     }
   }