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 2017/01/28 03:01:36 UTC

[01/12] incubator-quickstep git commit: Refactored SelectOperator::getAllWorkOrders. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/reorder-attrs 0dce7b245 -> 26c3db420 (forced update)


Refactored SelectOperator::getAllWorkOrders.


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

Branch: refs/heads/reorder-attrs
Commit: 222942320c6f441ca3b1b974de8a7e80babc081f
Parents: c8c5d45
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Jan 11 18:27:59 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Jan 12 13:48:26 2017 -0800

----------------------------------------------------------------------
 relational_operators/SelectOperator.cpp | 192 +++++++++------------------
 relational_operators/SelectOperator.hpp |  14 --
 2 files changed, 61 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/22294232/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index 236ee7c..f5c9ee9 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -22,6 +22,7 @@
 #include <memory>
 #include <vector>
 
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
@@ -43,104 +44,6 @@ namespace quickstep {
 
 class Predicate;
 
-void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
-                                   QueryContext *query_context,
-                                   StorageManager *storage_manager,
-                                   const Predicate *predicate,
-                                   const std::vector<std::unique_ptr<const Scalar>> *selection,
-                                   InsertDestination *output_destination) {
-  if (input_relation_is_stored_) {
-    for (const block_id input_block_id : input_relation_block_ids_) {
-      container->addNormalWorkOrder(
-          new SelectWorkOrder(
-              query_id_,
-              input_relation_,
-              input_block_id,
-              predicate,
-              simple_projection_,
-              simple_selection_,
-              selection,
-              output_destination,
-              storage_manager,
-              CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
-          op_index_);
-    }
-  } else {
-    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
-      container->addNormalWorkOrder(
-          new SelectWorkOrder(
-              query_id_,
-              input_relation_,
-              input_relation_block_ids_[num_workorders_generated_],
-              predicate,
-              simple_projection_,
-              simple_selection_,
-              selection,
-              output_destination,
-              storage_manager,
-              CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
-          op_index_);
-      ++num_workorders_generated_;
-    }
-  }
-}
-
-#ifdef QUICKSTEP_HAVE_LIBNUMA
-void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
-                                                 QueryContext *query_context,
-                                                 StorageManager *storage_manager,
-                                                 const Predicate *predicate,
-                                                 const std::vector<std::unique_ptr<const Scalar>> *selection,
-                                                 InsertDestination *output_destination) {
-  DCHECK(placement_scheme_ != nullptr);
-  const std::size_t num_partitions = input_relation_.getPartitionScheme().getPartitionSchemeHeader().getNumPartitions();
-  if (input_relation_is_stored_) {
-    for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
-      for (const block_id input_block_id :
-           input_relation_block_ids_in_partition_[part_id]) {
-        container->addNormalWorkOrder(
-            new SelectWorkOrder(
-                query_id_,
-                input_relation_,
-                input_block_id,
-                predicate,
-                simple_projection_,
-                simple_selection_,
-                selection,
-                output_destination,
-                storage_manager,
-                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context),
-                placement_scheme_->getNUMANodeForBlock(input_block_id)),
-            op_index_);
-      }
-    }
-  } else {
-    for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
-      while (num_workorders_generated_in_partition_[part_id] <
-             input_relation_block_ids_in_partition_[part_id].size()) {
-        block_id block_in_partition
-            = input_relation_block_ids_in_partition_[part_id][num_workorders_generated_in_partition_[part_id]];
-        container->addNormalWorkOrder(
-            new SelectWorkOrder(
-                query_id_,
-                input_relation_,
-                block_in_partition,
-                predicate,
-                simple_projection_,
-                simple_selection_,
-                selection,
-                output_destination,
-                storage_manager,
-                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context),
-                placement_scheme_->getNUMANodeForBlock(block_in_partition)),
-            op_index_);
-        ++num_workorders_generated_in_partition_[part_id];
-      }
-    }
-  }
-}
-#endif
-
 bool SelectOperator::getAllWorkOrders(
     WorkOrdersContainer *container,
     QueryContext *query_context,
@@ -159,48 +62,75 @@ bool SelectOperator::getAllWorkOrders(
       query_context->getInsertDestination(output_destination_index_);
 
   if (input_relation_is_stored_) {
-    if (!started_) {
-      if (input_relation_.hasPartitionScheme()) {
+    if (started_) {
+      return true;
+    }
+
+    if (input_relation_.hasPartitionScheme()) {
+      const std::size_t num_partitions =
+          input_relation_.getPartitionScheme().getPartitionSchemeHeader().getNumPartitions();
+
+      for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
+        for (const block_id input_block_id : input_relation_block_ids_in_partition_[part_id]) {
+          numa_node_id numa_node = 0;
 #ifdef QUICKSTEP_HAVE_LIBNUMA
-        if (input_relation_.hasNUMAPlacementScheme()) {
-          addPartitionAwareWorkOrders(container,
-                                      query_context,
-                                      storage_manager,
-                                      predicate,
-                                      selection,
-                                      output_destination);
+          if (input_relation_.hasNUMAPlacementScheme()) {
+            numa_node = placement_scheme_->getNUMANodeForBlock(input_block_id);
+          }
+#endif  // QUICKSTEP_HAVE_LIBNUMA
+          container->addNormalWorkOrder(
+              new SelectWorkOrder(query_id_, input_relation_, input_block_id, predicate, simple_projection_,
+                                  simple_selection_, selection, output_destination, storage_manager,
+                                  CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context), numa_node),
+              op_index_);
         }
-#endif
-      } else {
-        addWorkOrders(container,
-                      query_context,
-                      storage_manager,
-                      predicate,
-                      selection,
-                      output_destination);
       }
-      started_ = true;
+    } else {
+      for (const block_id input_block_id : input_relation_block_ids_) {
+        container->addNormalWorkOrder(
+            new SelectWorkOrder(query_id_, input_relation_, input_block_id, predicate, simple_projection_,
+                                simple_selection_, selection, output_destination, storage_manager,
+                                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
+            op_index_);
+      }
     }
-    return started_;
+    started_ = true;
+    return true;
   } else {
     if (input_relation_.hasPartitionScheme()) {
+      const std::size_t num_partitions =
+          input_relation_.getPartitionScheme().getPartitionSchemeHeader().getNumPartitions();
+
+      for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
+        while (num_workorders_generated_in_partition_[part_id] <
+               input_relation_block_ids_in_partition_[part_id].size()) {
+          const block_id block_in_partition
+              = input_relation_block_ids_in_partition_[part_id][num_workorders_generated_in_partition_[part_id]];
+
+          numa_node_id numa_node = 0;
 #ifdef QUICKSTEP_HAVE_LIBNUMA
-        if (input_relation_.hasNUMAPlacementScheme()) {
-          addPartitionAwareWorkOrders(container,
-                                      query_context,
-                                      storage_manager,
-                                      predicate,
-                                      selection,
-                                      output_destination);
+          if (input_relation_.hasNUMAPlacementScheme()) {
+            numa_node = placement_scheme_->getNUMANodeForBlock(block_in_partition);
+          }
+#endif  // QUICKSTEP_HAVE_LIBNUMA
+          container->addNormalWorkOrder(
+              new SelectWorkOrder(query_id_, input_relation_, block_in_partition, predicate, simple_projection_,
+                                  simple_selection_, selection, output_destination, storage_manager,
+                                  CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context), numa_node),
+              op_index_);
+          ++num_workorders_generated_in_partition_[part_id];
         }
-#endif
+      }
     } else {
-        addWorkOrders(container,
-                      query_context,
-                      storage_manager,
-                      predicate,
-                      selection,
-                      output_destination);
+      while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+        container->addNormalWorkOrder(
+            new SelectWorkOrder(query_id_, input_relation_, input_relation_block_ids_[num_workorders_generated_],
+                                predicate, simple_projection_, simple_selection_, selection, output_destination,
+                                storage_manager,
+                                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
+            op_index_);
+        ++num_workorders_generated_;
+      }
     }
     return done_feeding_input_relation_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/22294232/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 2ace458..6741f45 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -248,20 +248,6 @@ class SelectOperator : public RelationalOperator {
     return output_relation_.getID();
   }
 
-  void addWorkOrders(WorkOrdersContainer *container,
-                     QueryContext *query_context,
-                     StorageManager *storage_manager,
-                     const Predicate *predicate,
-                     const std::vector<std::unique_ptr<const Scalar>> *selection,
-                     InsertDestination *output_destination);
-
-  void addPartitionAwareWorkOrders(WorkOrdersContainer *container,
-                                   QueryContext *query_context,
-                                   StorageManager *storage_manager,
-                                   const Predicate *predicate,
-                                   const std::vector<std::unique_ptr<const Scalar>> *selection,
-                                   InsertDestination *output_destination);
-
  private:
   /**
    * @brief Create Work Order proto.


[04/12] incubator-quickstep git commit: Refactored SelectorOperator regarding partitions.

Posted by ji...@apache.org.
Refactored SelectorOperator regarding partitions.


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

Branch: refs/heads/reorder-attrs
Commit: 60e34060a8a9e82330442fa32b3f7030fa3a615d
Parents: e26cc1e
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 17 18:38:25 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jan 17 18:38:25 2017 -0800

----------------------------------------------------------------------
 relational_operators/SelectOperator.cpp | 10 +----
 relational_operators/SelectOperator.hpp | 62 +++++++++++++---------------
 2 files changed, 31 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/60e34060/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index d2759c4..5419cf8 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -67,10 +67,7 @@ bool SelectOperator::getAllWorkOrders(
     }
 
     if (input_relation_.hasPartitionScheme()) {
-      const std::size_t num_partitions =
-          input_relation_.getPartitionScheme()->getPartitionSchemeHeader().getNumPartitions();
-
-      for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
+      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
         for (const block_id input_block_id : input_relation_block_ids_in_partition_[part_id]) {
           numa_node_id numa_node = 0;
 #ifdef QUICKSTEP_HAVE_LIBNUMA
@@ -98,10 +95,7 @@ bool SelectOperator::getAllWorkOrders(
     return true;
   } else {
     if (input_relation_.hasPartitionScheme()) {
-      const std::size_t num_partitions =
-          input_relation_.getPartitionScheme()->getPartitionSchemeHeader().getNumPartitions();
-
-      for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
+      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
         while (num_workorders_generated_in_partition_[part_id] <
                input_relation_block_ids_in_partition_[part_id].size()) {
           const block_id block_in_partition

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/60e34060/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 989eaac..9c0767b 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -100,9 +100,6 @@ class SelectOperator : public RelationalOperator {
         output_destination_index_(output_destination_index),
         predicate_index_(predicate_index),
         selection_index_(selection_index),
-        input_relation_block_ids_(input_relation_is_stored
-                                      ? input_relation.getBlocksSnapshot()
-                                      : std::vector<block_id>()),
         num_workorders_generated_(0),
         simple_projection_(false),
         input_relation_is_stored_(input_relation_is_stored),
@@ -112,20 +109,20 @@ class SelectOperator : public RelationalOperator {
 #endif
     if (input_relation.hasPartitionScheme()) {
       const PartitionScheme &part_scheme = *input_relation.getPartitionScheme();
-      const PartitionSchemeHeader &part_scheme_header = part_scheme.getPartitionSchemeHeader();
-      const std::size_t num_partitions = part_scheme_header.getNumPartitions();
-      input_relation_block_ids_in_partition_.resize(num_partitions);
-      num_workorders_generated_in_partition_.resize(num_partitions);
-      num_workorders_generated_in_partition_.assign(num_partitions, 0);
-      for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
-        if (input_relation_is_stored) {
-          input_relation_block_ids_in_partition_[part_id] =
-              part_scheme.getBlocksInPartition(part_id);
-        } else {
-          input_relation_block_ids_in_partition_[part_id] =
-              std::vector<block_id>();
+      num_partitions_ = part_scheme.getPartitionSchemeHeader().getNumPartitions();
+
+      num_workorders_generated_in_partition_.resize(num_partitions_);
+
+      if (input_relation_is_stored) {
+        for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+          input_relation_block_ids_in_partition_.push_back(
+              part_scheme.getBlocksInPartition(part_id));
         }
+      } else {
+        input_relation_block_ids_in_partition_.resize(num_partitions_);
       }
+    } else if (input_relation_is_stored) {
+      input_relation_block_ids_ = input_relation.getBlocksSnapshot();
     }
   }
 
@@ -163,9 +160,6 @@ class SelectOperator : public RelationalOperator {
         predicate_index_(predicate_index),
         selection_index_(QueryContext::kInvalidScalarGroupId),
         simple_selection_(std::move(selection)),
-        input_relation_block_ids_(input_relation_is_stored
-                                      ? input_relation.getBlocksSnapshot()
-                                      : std::vector<block_id>()),
         num_workorders_generated_(0),
         simple_projection_(true),
         input_relation_is_stored_(input_relation_is_stored),
@@ -175,20 +169,20 @@ class SelectOperator : public RelationalOperator {
 #endif
     if (input_relation.hasPartitionScheme()) {
       const PartitionScheme &part_scheme = *input_relation.getPartitionScheme();
-      const PartitionSchemeHeader &part_scheme_header = part_scheme.getPartitionSchemeHeader();
-      const std::size_t num_partitions = part_scheme_header.getNumPartitions();
-      input_relation_block_ids_in_partition_.resize(num_partitions);
-      num_workorders_generated_in_partition_.resize(num_partitions);
-      num_workorders_generated_in_partition_.assign(num_partitions, 0);
-      for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
-        if (input_relation_is_stored) {
-          input_relation_block_ids_in_partition_[part_id] =
-              part_scheme.getBlocksInPartition(part_id);
-        } else {
-          input_relation_block_ids_in_partition_[part_id] =
-              std::vector<block_id>();
+      num_partitions_ = part_scheme.getPartitionSchemeHeader().getNumPartitions();
+
+      num_workorders_generated_in_partition_.resize(num_partitions_);
+
+      if (input_relation_is_stored) {
+        for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+          input_relation_block_ids_in_partition_.push_back(
+              part_scheme.getBlocksInPartition(part_id));
         }
+      } else {
+        input_relation_block_ids_in_partition_.resize(num_partitions_);
       }
+    } else if (input_relation_is_stored) {
+      input_relation_block_ids_ = input_relation.getBlocksSnapshot();
     }
   }
 
@@ -245,14 +239,16 @@ class SelectOperator : public RelationalOperator {
   const std::vector<attribute_id> simple_selection_;
 
   std::vector<block_id> input_relation_block_ids_;
+  // A single workorder is generated for each block of input relation.
+  std::vector<block_id>::size_type num_workorders_generated_;
+
+  // Used for the partition case only.
   // A vector of vectors V where V[i] indicates the list of block IDs of the
   // input relation that belong to the partition i.
   std::vector<std::vector<block_id>> input_relation_block_ids_in_partition_;
-
-  // A single workorder is generated for each block of input relation.
-  std::vector<block_id>::size_type num_workorders_generated_;
   // A single workorder is generated for each block in each partition of input relation.
   std::vector<std::size_t> num_workorders_generated_in_partition_;
+  std::size_t num_partitions_;
 
   const bool simple_projection_;
   const bool input_relation_is_stored_;


[03/12] incubator-quickstep git commit: CatalogRelation::getPartitionScheme returns a pointer instead of ref.

Posted by ji...@apache.org.
CatalogRelation::getPartitionScheme returns a pointer instead of ref.


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

Branch: refs/heads/reorder-attrs
Commit: e26cc1e037a7a5d07664b99ac2c178b0e8e8bc66
Parents: b0e5968
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 17 13:55:24 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jan 17 13:55:24 2017 -0800

----------------------------------------------------------------------
 catalog/CatalogRelation.hpp             | 7 +++----
 relational_operators/SelectOperator.cpp | 4 ++--
 relational_operators/SelectOperator.hpp | 6 +++---
 storage/PreloaderThread.cpp             | 2 +-
 4 files changed, 9 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e26cc1e0/catalog/CatalogRelation.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelation.hpp b/catalog/CatalogRelation.hpp
index c38e526..e1fd79a 100644
--- a/catalog/CatalogRelation.hpp
+++ b/catalog/CatalogRelation.hpp
@@ -111,12 +111,11 @@ class CatalogRelation : public CatalogRelationSchema {
 
   /**
    * @brief Get the partition scheme of the catalog relation.
-   * @warning This is only safe if hasPartitionScheme() is true.
    *
-   * @return A const reference to the partition scheme of the relation.
+   * @return A const pointer to the partition scheme of the relation.
    **/
-  const PartitionScheme& getPartitionScheme() const {
-    return *partition_scheme_;
+  const PartitionScheme* getPartitionScheme() const {
+    return partition_scheme_.get();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e26cc1e0/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index f5c9ee9..d2759c4 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -68,7 +68,7 @@ bool SelectOperator::getAllWorkOrders(
 
     if (input_relation_.hasPartitionScheme()) {
       const std::size_t num_partitions =
-          input_relation_.getPartitionScheme().getPartitionSchemeHeader().getNumPartitions();
+          input_relation_.getPartitionScheme()->getPartitionSchemeHeader().getNumPartitions();
 
       for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
         for (const block_id input_block_id : input_relation_block_ids_in_partition_[part_id]) {
@@ -99,7 +99,7 @@ bool SelectOperator::getAllWorkOrders(
   } else {
     if (input_relation_.hasPartitionScheme()) {
       const std::size_t num_partitions =
-          input_relation_.getPartitionScheme().getPartitionSchemeHeader().getNumPartitions();
+          input_relation_.getPartitionScheme()->getPartitionSchemeHeader().getNumPartitions();
 
       for (std::size_t part_id = 0; part_id < num_partitions; ++part_id) {
         while (num_workorders_generated_in_partition_[part_id] <

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e26cc1e0/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 5846eda..989eaac 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -111,7 +111,7 @@ class SelectOperator : public RelationalOperator {
     placement_scheme_ = input_relation.getNUMAPlacementSchemePtr();
 #endif
     if (input_relation.hasPartitionScheme()) {
-      const PartitionScheme &part_scheme = input_relation.getPartitionScheme();
+      const PartitionScheme &part_scheme = *input_relation.getPartitionScheme();
       const PartitionSchemeHeader &part_scheme_header = part_scheme.getPartitionSchemeHeader();
       const std::size_t num_partitions = part_scheme_header.getNumPartitions();
       input_relation_block_ids_in_partition_.resize(num_partitions);
@@ -174,7 +174,7 @@ class SelectOperator : public RelationalOperator {
     placement_scheme_ = input_relation.getNUMAPlacementSchemePtr();
 #endif
     if (input_relation.hasPartitionScheme()) {
-      const PartitionScheme &part_scheme = input_relation.getPartitionScheme();
+      const PartitionScheme &part_scheme = *input_relation.getPartitionScheme();
       const PartitionSchemeHeader &part_scheme_header = part_scheme.getPartitionSchemeHeader();
       const std::size_t num_partitions = part_scheme_header.getNumPartitions();
       input_relation_block_ids_in_partition_.resize(num_partitions);
@@ -213,7 +213,7 @@ class SelectOperator : public RelationalOperator {
   void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
     if (input_relation_.hasPartitionScheme()) {
       const partition_id part_id =
-          input_relation_.getPartitionScheme().getPartitionForBlock(input_block_id);
+          input_relation_.getPartitionScheme()->getPartitionForBlock(input_block_id);
       input_relation_block_ids_in_partition_[part_id].push_back(input_block_id);
     } else {
       input_relation_block_ids_.push_back(input_block_id);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e26cc1e0/storage/PreloaderThread.cpp
----------------------------------------------------------------------
diff --git a/storage/PreloaderThread.cpp b/storage/PreloaderThread.cpp
index 7f97a05..edaa143 100644
--- a/storage/PreloaderThread.cpp
+++ b/storage/PreloaderThread.cpp
@@ -93,7 +93,7 @@ std::size_t PreloaderThread::preloadNUMAAware(
       relation.getNUMAPlacementSchemePtr();
   DCHECK(placement_scheme != nullptr);
   DCHECK(relation.hasPartitionScheme());
-  const PartitionScheme &part_scheme = relation.getPartitionScheme();
+  const PartitionScheme &part_scheme = *relation.getPartitionScheme();
   const PartitionSchemeHeader &part_scheme_header =
       part_scheme.getPartitionSchemeHeader();
   const std::size_t num_partitions = part_scheme_header.getNumPartitions();


[05/12] incubator-quickstep git commit: Marked LIP as a non-default argument.

Posted by ji...@apache.org.
Marked LIP as a non-default argument.


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

Branch: refs/heads/reorder-attrs
Commit: bed1e2616061f1841fd106f4f54463f7ebfe888b
Parents: 60e3406
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 17 17:38:50 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jan 17 18:40:22 2017 -0800

----------------------------------------------------------------------
 relational_operators/AggregationOperator.hpp |  2 +-
 relational_operators/BuildHashOperator.hpp   |  4 ++--
 relational_operators/HashJoinOperator.hpp    | 12 ++++++------
 relational_operators/SelectOperator.hpp      |  4 ++--
 storage/AggregationOperationState.hpp        |  2 +-
 5 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bed1e261/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index ce6015d..c4e887d 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -137,7 +137,7 @@ class AggregationWorkOrder : public WorkOrder {
   AggregationWorkOrder(const std::size_t query_id,
                        const block_id input_block_id,
                        AggregationOperationState *state,
-                       LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr)
+                       LIPFilterAdaptiveProber *lip_filter_adaptive_prober)
       : WorkOrder(query_id),
         input_block_id_(input_block_id),
         state_(DCHECK_NOTNULL(state)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bed1e261/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index f0f42e3..074c1e1 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -166,7 +166,7 @@ class BuildHashWorkOrder : public WorkOrder {
                      const block_id build_block_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager,
-                     LIPFilterBuilder *lip_filter_builder = nullptr)
+                     LIPFilterBuilder *lip_filter_builder)
       : WorkOrder(query_id),
         input_relation_(input_relation),
         join_key_attributes_(join_key_attributes),
@@ -196,7 +196,7 @@ class BuildHashWorkOrder : public WorkOrder {
                      const block_id build_block_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager,
-                     LIPFilterBuilder *lip_filter_builder = nullptr)
+                     LIPFilterBuilder *lip_filter_builder)
       : WorkOrder(query_id),
         input_relation_(input_relation),
         join_key_attributes_(std::move(join_key_attributes)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bed1e261/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 8829d1f..190f9d2 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -302,7 +302,7 @@ class HashInnerJoinWorkOrder : public WorkOrder {
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
       StorageManager *storage_manager,
-      LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr)
+      LIPFilterAdaptiveProber *lip_filter_adaptive_prober)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -352,7 +352,7 @@ class HashInnerJoinWorkOrder : public WorkOrder {
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
       StorageManager *storage_manager,
-      LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr)
+      LIPFilterAdaptiveProber *lip_filter_adaptive_prober)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -438,7 +438,7 @@ class HashSemiJoinWorkOrder : public WorkOrder {
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
       StorageManager *storage_manager,
-      LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr)
+      LIPFilterAdaptiveProber *lip_filter_adaptive_prober)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -488,7 +488,7 @@ class HashSemiJoinWorkOrder : public WorkOrder {
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
       StorageManager *storage_manager,
-      LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr)
+      LIPFilterAdaptiveProber *lip_filter_adaptive_prober)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -570,7 +570,7 @@ class HashAntiJoinWorkOrder : public WorkOrder {
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
       StorageManager *storage_manager,
-      LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr)
+      LIPFilterAdaptiveProber *lip_filter_adaptive_prober)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -620,7 +620,7 @@ class HashAntiJoinWorkOrder : public WorkOrder {
       const JoinHashTable &hash_table,
       InsertDestination *output_destination,
       StorageManager *storage_manager,
-      LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr)
+      LIPFilterAdaptiveProber *lip_filter_adaptive_prober)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bed1e261/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 9c0767b..e58ff81 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -295,7 +295,7 @@ class SelectWorkOrder : public WorkOrder {
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager,
-                  LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr,
+                  LIPFilterAdaptiveProber *lip_filter_adaptive_prober,
                   const numa_node_id numa_node = 0)
       : WorkOrder(query_id),
         input_relation_(input_relation),
@@ -340,7 +340,7 @@ class SelectWorkOrder : public WorkOrder {
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager,
-                  LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr,
+                  LIPFilterAdaptiveProber *lip_filter_adaptive_prober,
                   const numa_node_id numa_node = 0)
       : WorkOrder(query_id),
         input_relation_(input_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bed1e261/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index e0826b0..591e3a1 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -166,7 +166,7 @@ class AggregationOperationState {
    *        the block.
    **/
   void aggregateBlock(const block_id input_block,
-                      LIPFilterAdaptiveProber *lip_filter_adaptive_prober = nullptr);
+                      LIPFilterAdaptiveProber *lip_filter_adaptive_prober);
 
   /**
    * @brief Generate the final results for the aggregates managed by this


[09/12] incubator-quickstep git commit: Added Operator support for Partitioned HashJoin.

Posted by ji...@apache.org.
Added Operator support for Partitioned HashJoin.


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

Branch: refs/heads/reorder-attrs
Commit: 300284c2c5d84fbe3800dc2aab5f9d7f60f4ff5e
Parents: ee3b7f0
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 17 13:18:08 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Fri Jan 20 15:26:35 2017 -0800

----------------------------------------------------------------------
 query_execution/QueryContext.cpp                |  14 +-
 query_execution/QueryContext.hpp                |  25 ++-
 query_execution/QueryContext.proto              |   7 +-
 query_optimizer/ExecutionGenerator.cpp          |  30 ++-
 relational_operators/BuildHashOperator.cpp      |  76 ++++----
 relational_operators/BuildHashOperator.hpp      |  81 ++++++--
 relational_operators/CMakeLists.txt             |   9 +-
 relational_operators/DestroyHashOperator.cpp    |  25 ++-
 relational_operators/DestroyHashOperator.hpp    |   9 +
 relational_operators/HashJoinOperator.cpp       | 192 ++++++++++---------
 relational_operators/HashJoinOperator.hpp       | 174 ++++++++++++++++-
 relational_operators/WorkOrder.proto            |   7 +
 relational_operators/WorkOrderFactory.cpp       |  38 +++-
 .../tests/HashJoinOperator_unittest.cpp         |  42 ++--
 storage/StorageBlockInfo.hpp                    |   3 +
 15 files changed, 531 insertions(+), 201 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/query_execution/QueryContext.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.cpp b/query_execution/QueryContext.cpp
index 0e6636d..71839a7 100644
--- a/query_execution/QueryContext.cpp
+++ b/query_execution/QueryContext.cpp
@@ -79,9 +79,15 @@ QueryContext::QueryContext(const serialization::QueryContext &proto,
   }
 
   for (int i = 0; i < proto.join_hash_tables_size(); ++i) {
-    join_hash_tables_.emplace_back(
-        JoinHashTableFactory::CreateResizableFromProto(proto.join_hash_tables(i),
-                                                       storage_manager));
+    PartitionedJoinHashTables partitioned_join_hash_tables;
+
+    const serialization::QueryContext::HashTableContext &hash_table_context_proto = proto.join_hash_tables(i);
+    for (std::uint64_t j = 0; j < hash_table_context_proto.num_partitions(); ++j) {
+      partitioned_join_hash_tables.emplace_back(
+          JoinHashTableFactory::CreateResizableFromProto(hash_table_context_proto.join_hash_table(), storage_manager));
+    }
+
+    join_hash_tables_.push_back(move(partitioned_join_hash_tables));
   }
 
   for (int i = 0; i < proto.insert_destinations_size(); ++i) {
@@ -178,7 +184,7 @@ bool QueryContext::ProtoIsValid(const serialization::QueryContext &proto,
   }
 
   for (int i = 0; i < proto.join_hash_tables_size(); ++i) {
-    if (!JoinHashTableFactory::ProtoIsValid(proto.join_hash_tables(i))) {
+    if (!JoinHashTableFactory::ProtoIsValid(proto.join_hash_tables(i).join_hash_table())) {
       return false;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/query_execution/QueryContext.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.hpp b/query_execution/QueryContext.hpp
index 7ad8fa1..895c2ea 100644
--- a/query_execution/QueryContext.hpp
+++ b/query_execution/QueryContext.hpp
@@ -278,33 +278,37 @@ class QueryContext {
    * @brief Whether the given JoinHashTable id is valid.
    *
    * @param id The JoinHashTable id.
+   * @param part_id The partition id.
    *
    * @return True if valid, otherwise false.
    **/
-  bool isValidJoinHashTableId(const join_hash_table_id id) const {
-    return id < join_hash_tables_.size();
+  bool isValidJoinHashTableId(const join_hash_table_id id, const partition_id part_id) const {
+    return id < join_hash_tables_.size() &&
+           part_id < join_hash_tables_[id].size();
   }
 
   /**
    * @brief Get the JoinHashTable.
    *
    * @param id The JoinHashTable id in the query.
+   * @param part_id The partition id.
    *
    * @return The JoinHashTable, already created in the constructor.
    **/
-  inline JoinHashTable* getJoinHashTable(const join_hash_table_id id) {
-    DCHECK_LT(id, join_hash_tables_.size());
-    return join_hash_tables_[id].get();
+  inline JoinHashTable* getJoinHashTable(const join_hash_table_id id, const partition_id part_id) {
+    DCHECK(isValidJoinHashTableId(id, part_id));
+    return join_hash_tables_[id][part_id].get();
   }
 
   /**
    * @brief Destory the given JoinHashTable.
    *
    * @param id The id of the JoinHashTable to destroy.
+   * @param part_id The partition id.
    **/
-  inline void destroyJoinHashTable(const join_hash_table_id id) {
-    DCHECK_LT(id, join_hash_tables_.size());
-    join_hash_tables_[id].reset();
+  inline void destroyJoinHashTable(const join_hash_table_id id, const partition_id part_id) {
+    DCHECK(isValidJoinHashTableId(id, part_id));
+    join_hash_tables_[id][part_id].reset();
   }
 
   /**
@@ -562,10 +566,13 @@ class QueryContext {
   }
 
  private:
+  // Per hash join, the index is the partition id.
+  typedef std::vector<std::unique_ptr<JoinHashTable>> PartitionedJoinHashTables;
+
   std::vector<std::unique_ptr<AggregationOperationState>> aggregation_states_;
   std::vector<std::unique_ptr<const GeneratorFunctionHandle>> generator_functions_;
   std::vector<std::unique_ptr<InsertDestination>> insert_destinations_;
-  std::vector<std::unique_ptr<JoinHashTable>> join_hash_tables_;
+  std::vector<PartitionedJoinHashTables> join_hash_tables_;
   std::vector<std::unique_ptr<LIPFilterDeployment>> lip_deployments_;
   std::vector<std::unique_ptr<LIPFilter>> lip_filters_;
   std::vector<std::unique_ptr<const Predicate>> predicates_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/query_execution/QueryContext.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.proto b/query_execution/QueryContext.proto
index ab0f520..6dce6b8 100644
--- a/query_execution/QueryContext.proto
+++ b/query_execution/QueryContext.proto
@@ -30,6 +30,11 @@ import "utility/SortConfiguration.proto";
 import "utility/lip_filter/LIPFilter.proto";
 
 message QueryContext {
+  message HashTableContext {
+    required HashTable join_hash_table = 1;
+    optional uint64 num_partitions = 2 [default = 1];
+  }
+
   message ScalarGroup {
     repeated Scalar scalars = 1;
   }
@@ -47,7 +52,7 @@ message QueryContext {
 
   repeated AggregationOperationState aggregation_states = 1;
   repeated GeneratorFunctionHandle generator_functions = 2;
-  repeated HashTable join_hash_tables = 3;
+  repeated HashTableContext join_hash_tables = 3;
   repeated InsertDestination insert_destinations = 4;
   repeated LIPFilter lip_filters = 5;
   repeated LIPFilterDeployment lip_filter_deployments = 6;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index e0bfb3b..e25b8ad 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -694,15 +694,32 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
                 build_physical->getOutputAttributes())));
   }
 
+  const CatalogRelation *build_relation = build_relation_info->relation;
+
   // FIXME(quickstep-team): Add support for self-join.
-  if (build_relation_info->relation == probe_operator_info->relation) {
+  if (build_relation == probe_operator_info->relation) {
     THROW_SQL_ERROR() << "Self-join is not supported";
   }
 
   // Create join hash table proto.
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto_->join_hash_tables_size();
-  S::HashTable *hash_table_proto = query_context_proto_->add_join_hash_tables();
+  S::QueryContext::HashTableContext *hash_table_context_proto =
+      query_context_proto_->add_join_hash_tables();
+
+  // No partition.
+  std::size_t num_partitions = 1;
+  if (build_relation->hasPartitionScheme() &&
+      build_attribute_ids.size() == 1) {
+    const PartitionSchemeHeader &partition_scheme_header =
+        build_relation->getPartitionScheme()->getPartitionSchemeHeader();
+    if (build_attribute_ids[0] == partition_scheme_header.getPartitionAttributeId()) {
+      // TODO(zuyu): add optimizer support for partitioned hash joins.
+      hash_table_context_proto->set_num_partitions(num_partitions);
+    }
+  }
+
+  S::HashTable *hash_table_proto = hash_table_context_proto->mutable_join_hash_table();
 
   // SimplifyHashTableImplTypeProto() switches the hash table implementation
   // from SeparateChaining to SimpleScalarSeparateChaining when there is a
@@ -712,7 +729,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
           HashTableImplTypeProtoFromString(FLAGS_join_hashtable_type),
           key_types));
 
-  const CatalogRelationSchema *build_relation = build_relation_info->relation;
   for (const attribute_id build_attribute : build_attribute_ids) {
     hash_table_proto->add_key_types()->CopyFrom(
         build_relation->getAttributeById(build_attribute)->getType().getProto());
@@ -725,10 +741,11 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
       execution_plan_->addRelationalOperator(
           new BuildHashOperator(
               query_handle_->query_id(),
-              *build_relation_info->relation,
+              *build_relation,
               build_relation_info->isStoredRelation(),
               build_attribute_ids,
               any_build_attributes_nullable,
+              num_partitions,
               join_hash_table_index));
 
   // Create InsertDestination proto.
@@ -766,11 +783,12 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
       execution_plan_->addRelationalOperator(
           new HashJoinOperator(
               query_handle_->query_id(),
-              *build_relation_info->relation,
+              *build_relation,
               *probe_operator_info->relation,
               probe_operator_info->isStoredRelation(),
               probe_attribute_ids,
               any_probe_attributes_nullable,
+              num_partitions,
               *output_relation,
               insert_destination_index,
               join_hash_table_index,
@@ -782,7 +800,7 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
 
   const QueryPlan::DAGNodeIndex destroy_operator_index =
       execution_plan_->addRelationalOperator(new DestroyHashOperator(
-          query_handle_->query_id(), join_hash_table_index));
+          query_handle_->query_id(), num_partitions, join_hash_table_index));
 
   if (!build_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(build_operator_index,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index 60e091f..14ec204 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -70,38 +70,36 @@ bool BuildHashOperator::getAllWorkOrders(
     tmb::MessageBus *bus) {
   DCHECK(query_context != nullptr);
 
-  JoinHashTable *hash_table = query_context->getJoinHashTable(hash_table_index_);
   if (input_relation_is_stored_) {
-    if (!started_) {
-      for (const block_id input_block_id : input_relation_block_ids_) {
+    if (started_) {
+      return true;
+    }
+
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      JoinHashTable *hash_table = query_context->getJoinHashTable(hash_table_index_, part_id);
+      for (const block_id block : input_relation_block_ids_[part_id]) {
         container->addNormalWorkOrder(
-            new BuildHashWorkOrder(query_id_,
-                                   input_relation_,
-                                   join_key_attributes_,
-                                   any_join_key_attributes_nullable_,
-                                   input_block_id,
-                                   hash_table,
-                                   storage_manager,
+            new BuildHashWorkOrder(query_id_, input_relation_, join_key_attributes_, any_join_key_attributes_nullable_,
+                                   num_partitions_, part_id, block, hash_table, storage_manager,
                                    CreateLIPFilterBuilderHelper(lip_deployment_index_, query_context)),
             op_index_);
       }
-      started_ = true;
     }
-    return started_;
+    started_ = true;
+    return true;
   } else {
-    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
-      container->addNormalWorkOrder(
-          new BuildHashWorkOrder(
-              query_id_,
-              input_relation_,
-              join_key_attributes_,
-              any_join_key_attributes_nullable_,
-              input_relation_block_ids_[num_workorders_generated_],
-              hash_table,
-              storage_manager,
-              CreateLIPFilterBuilderHelper(lip_deployment_index_, query_context)),
-          op_index_);
-      ++num_workorders_generated_;
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      JoinHashTable *hash_table = query_context->getJoinHashTable(hash_table_index_, part_id);
+      while (num_workorders_generated_[part_id] <
+             input_relation_block_ids_[part_id].size()) {
+        container->addNormalWorkOrder(
+            new BuildHashWorkOrder(query_id_, input_relation_, join_key_attributes_, any_join_key_attributes_nullable_,
+                                   num_partitions_, part_id,
+                                   input_relation_block_ids_[part_id][num_workorders_generated_[part_id]], hash_table,
+                                   storage_manager, CreateLIPFilterBuilderHelper(lip_deployment_index_, query_context)),
+            op_index_);
+        ++num_workorders_generated_[part_id];
+      }
     }
     return done_feeding_input_relation_;
   }
@@ -109,25 +107,31 @@ bool BuildHashOperator::getAllWorkOrders(
 
 bool BuildHashOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
   if (input_relation_is_stored_) {
-    if (!started_) {
-      for (const block_id input_block_id : input_relation_block_ids_) {
-        container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+    if (started_) {
+      return true;
+    }
+
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      for (const block_id block : input_relation_block_ids_[part_id]) {
+        container->addWorkOrderProto(createWorkOrderProto(block, part_id), op_index_);
       }
-      started_ = true;
     }
+    started_ = true;
     return true;
   } else {
-    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
-      container->addWorkOrderProto(
-          createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
-          op_index_);
-      ++num_workorders_generated_;
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      while (num_workorders_generated_[part_id] < input_relation_block_ids_[part_id].size()) {
+        container->addWorkOrderProto(
+            createWorkOrderProto(input_relation_block_ids_[part_id][num_workorders_generated_[part_id]], part_id),
+            op_index_);
+        ++num_workorders_generated_[part_id];
+      }
     }
     return done_feeding_input_relation_;
   }
 }
 
-serialization::WorkOrder* BuildHashOperator::createWorkOrderProto(const block_id block) {
+serialization::WorkOrder* BuildHashOperator::createWorkOrderProto(const block_id block, const partition_id part_id) {
   serialization::WorkOrder *proto = new serialization::WorkOrder;
   proto->set_work_order_type(serialization::BUILD_HASH);
   proto->set_query_id(query_id_);
@@ -138,7 +142,9 @@ serialization::WorkOrder* BuildHashOperator::createWorkOrderProto(const block_id
   }
   proto->SetExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable,
                       any_join_key_attributes_nullable_);
+  proto->SetExtension(serialization::BuildHashWorkOrder::num_partitions, num_partitions_);
   proto->SetExtension(serialization::BuildHashWorkOrder::join_hash_table_index, hash_table_index_);
+  proto->SetExtension(serialization::BuildHashWorkOrder::partition_id, part_id);
   proto->SetExtension(serialization::BuildHashWorkOrder::block_id, block);
   proto->SetExtension(serialization::BuildHashWorkOrder::lip_deployment_index, lip_deployment_index_);
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index dec121c..c9f29cb 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -27,6 +27,7 @@
 
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogTypedefs.hpp"
+#include "catalog/PartitionScheme.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.hpp"
@@ -75,6 +76,8 @@ class BuildHashOperator : public RelationalOperator {
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'input_relation'. If no
+   *        partitions, it is one.
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
    *        The HashTable's key Type(s) should be the Type(s) of the
    *        join_key_attributes in input_relation.
@@ -84,17 +87,30 @@ class BuildHashOperator : public RelationalOperator {
                     const bool input_relation_is_stored,
                     const std::vector<attribute_id> &join_key_attributes,
                     const bool any_join_key_attributes_nullable,
+                    const std::size_t num_partitions,
                     const QueryContext::join_hash_table_id hash_table_index)
-    : RelationalOperator(query_id),
-      input_relation_(input_relation),
-      input_relation_is_stored_(input_relation_is_stored),
-      join_key_attributes_(join_key_attributes),
-      any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-      hash_table_index_(hash_table_index),
-      input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
-                                                         : std::vector<block_id>()),
-      num_workorders_generated_(0),
-      started_(false) {}
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
+        input_relation_is_stored_(input_relation_is_stored),
+        join_key_attributes_(join_key_attributes),
+        any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        hash_table_index_(hash_table_index),
+        input_relation_block_ids_(num_partitions),
+        num_workorders_generated_(num_partitions),
+        started_(false) {
+    if (input_relation_is_stored) {
+      if (input_relation.hasPartitionScheme()) {
+        const PartitionScheme &part_scheme = *input_relation.getPartitionScheme();
+        for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+          input_relation_block_ids_[part_id] = part_scheme.getBlocksInPartition(part_id);
+        }
+      } else {
+        // No partition.
+        input_relation_block_ids_[0] = input_relation.getBlocksSnapshot();
+      }
+    }
+  }
 
   ~BuildHashOperator() override {}
 
@@ -116,7 +132,7 @@ class BuildHashOperator : public RelationalOperator {
 
   void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
                       const partition_id part_id) override {
-    input_relation_block_ids_.push_back(input_block_id);
+    input_relation_block_ids_[part_id].push_back(input_block_id);
   }
 
  private:
@@ -124,17 +140,20 @@ class BuildHashOperator : public RelationalOperator {
    * @brief Create Work Order proto.
    *
    * @param block The block id used in the Work Order.
+   * @param part_id The partition id of 'input_relation_'.
    **/
-  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+  serialization::WorkOrder* createWorkOrderProto(const block_id block, const partition_id part_id);
 
   const CatalogRelation &input_relation_;
   const bool input_relation_is_stored_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
+  const std::size_t num_partitions_;
   const QueryContext::join_hash_table_id hash_table_index_;
 
-  std::vector<block_id> input_relation_block_ids_;
-  std::vector<block_id>::size_type num_workorders_generated_;
+  // The index is the partition id.
+  std::vector<BlocksInPartition> input_relation_block_ids_;
+  std::vector<std::size_t> num_workorders_generated_;
 
   bool started_;
 
@@ -154,6 +173,9 @@ class BuildHashWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'input_relation'. If no
+   *        partitions, it is one.
+   * @param part_id The partition id of 'input_relation'.
    * @param build_block_id The block id.
    * @param hash_table The JoinHashTable to use.
    * @param storage_manager The StorageManager to use.
@@ -163,6 +185,8 @@ class BuildHashWorkOrder : public WorkOrder {
                      const CatalogRelationSchema &input_relation,
                      const std::vector<attribute_id> &join_key_attributes,
                      const bool any_join_key_attributes_nullable,
+                     const std::size_t num_partitions,
+                     const partition_id part_id,
                      const block_id build_block_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager,
@@ -171,6 +195,8 @@ class BuildHashWorkOrder : public WorkOrder {
         input_relation_(input_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        part_id_(part_id),
         build_block_id_(build_block_id),
         hash_table_(DCHECK_NOTNULL(hash_table)),
         storage_manager_(DCHECK_NOTNULL(storage_manager)),
@@ -184,6 +210,9 @@ class BuildHashWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'input_relation'. If no
+   *        partitions, it is one.
+   * @param part_id The partition id of 'input_relation'.
    * @param build_block_id The block id.
    * @param hash_table The JoinHashTable to use.
    * @param storage_manager The StorageManager to use.
@@ -193,6 +222,8 @@ class BuildHashWorkOrder : public WorkOrder {
                      const CatalogRelationSchema &input_relation,
                      std::vector<attribute_id> &&join_key_attributes,
                      const bool any_join_key_attributes_nullable,
+                     const std::size_t num_partitions,
+                     const partition_id part_id,
                      const block_id build_block_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager,
@@ -201,6 +232,8 @@ class BuildHashWorkOrder : public WorkOrder {
         input_relation_(input_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        part_id_(part_id),
         build_block_id_(build_block_id),
         hash_table_(DCHECK_NOTNULL(hash_table)),
         storage_manager_(DCHECK_NOTNULL(storage_manager)),
@@ -214,10 +247,30 @@ class BuildHashWorkOrder : public WorkOrder {
 
   void execute() override;
 
+  /**
+   * @brief Get the number of partitions.
+   *
+   * @return The number of partitions.
+   */
+  std::size_t num_partitions() const {
+    return num_partitions_;
+  }
+
+  /**
+   * @brief Get the partition id.
+   *
+   * @return The partition id.
+   */
+  partition_id getPartitionId() const {
+    return part_id_;
+  }
+
  private:
   const CatalogRelationSchema &input_relation_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
+  const std::size_t num_partitions_;
+  const partition_id part_id_;
   const block_id build_block_id_;
 
   JoinHashTable *hash_table_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 526820e..66ea2d1 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -40,8 +40,8 @@ add_library(quickstep_relationaloperators_AggregationOperator AggregationOperato
 add_library(quickstep_relationaloperators_BuildHashOperator BuildHashOperator.cpp BuildHashOperator.hpp)
 add_library(quickstep_relationaloperators_CreateIndexOperator CreateIndexOperator.cpp CreateIndexOperator.hpp)
 add_library(quickstep_relationaloperators_CreateTableOperator CreateTableOperator.cpp CreateTableOperator.hpp)
-add_library(quickstep_relationaloperators_DestroyAggregationStateOperator 
-            DestroyAggregationStateOperator.cpp 
+add_library(quickstep_relationaloperators_DestroyAggregationStateOperator
+            DestroyAggregationStateOperator.cpp
             DestroyAggregationStateOperator.hpp)
 add_library(quickstep_relationaloperators_DeleteOperator DeleteOperator.cpp DeleteOperator.hpp)
 add_library(quickstep_relationaloperators_DestroyHashOperator DestroyHashOperator.cpp DestroyHashOperator.hpp)
@@ -99,6 +99,7 @@ target_link_libraries(quickstep_relationaloperators_BuildHashOperator
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_catalog_PartitionScheme
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
@@ -162,6 +163,7 @@ target_link_libraries(quickstep_relationaloperators_DestroyAggregationStateOpera
                       tmb)
 target_link_libraries(quickstep_relationaloperators_DestroyHashOperator
                       glog
+                      quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
@@ -204,6 +206,7 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_catalog_PartitionScheme
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_queryexecution_QueryContext
@@ -796,4 +799,4 @@ add_test(TextScanOperator_faulty_unittest
          TextScanOperator_unittest
          ${TEXT_SCAN_FAULTY_INPUT_FILE}
          ${TEXT_SCAN_FAULTY_GOLDEN_OUTPUT_FILE}
-         ${TEXT_SCAN_FAULTY_FAILURE_OUTPUT_FILE})
\ No newline at end of file
+         ${TEXT_SCAN_FAULTY_FAILURE_OUTPUT_FILE})

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/relational_operators/DestroyHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.cpp b/relational_operators/DestroyHashOperator.cpp
index 96d2561..4827ef5 100644
--- a/relational_operators/DestroyHashOperator.cpp
+++ b/relational_operators/DestroyHashOperator.cpp
@@ -35,31 +35,36 @@ bool DestroyHashOperator::getAllWorkOrders(
     const tmb::client_id scheduler_client_id,
     tmb::MessageBus *bus) {
   if (blocking_dependencies_met_ && !work_generated_) {
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      container->addNormalWorkOrder(
+          new DestroyHashWorkOrder(query_id_, hash_table_index_, part_id, query_context),
+          op_index_);
+    }
     work_generated_ = true;
-    container->addNormalWorkOrder(
-        new DestroyHashWorkOrder(query_id_, hash_table_index_, query_context),
-        op_index_);
   }
   return work_generated_;
 }
 
 bool DestroyHashOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
   if (blocking_dependencies_met_ && !work_generated_) {
-    work_generated_ = true;
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      serialization::WorkOrder *proto = new serialization::WorkOrder;
+      proto->set_work_order_type(serialization::DESTROY_HASH);
+      proto->set_query_id(query_id_);
+      proto->SetExtension(serialization::DestroyHashWorkOrder::join_hash_table_index, hash_table_index_);
+      proto->SetExtension(serialization::DestroyHashWorkOrder::partition_id, part_id);
 
-    serialization::WorkOrder *proto = new serialization::WorkOrder;
-    proto->set_work_order_type(serialization::DESTROY_HASH);
-    proto->set_query_id(query_id_);
-    proto->SetExtension(serialization::DestroyHashWorkOrder::join_hash_table_index, hash_table_index_);
+      container->addWorkOrderProto(proto, op_index_);
+    }
 
-    container->addWorkOrderProto(proto, op_index_);
+    work_generated_ = true;
   }
   return work_generated_;
 }
 
 
 void DestroyHashWorkOrder::execute() {
-  query_context_->destroyJoinHashTable(hash_table_index_);
+  query_context_->destroyJoinHashTable(hash_table_index_, part_id_);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index d402766..8a1fabd 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -22,6 +22,7 @@
 
 #include <string>
 
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.hpp"
@@ -52,11 +53,14 @@ class DestroyHashOperator : public RelationalOperator {
    * @brief Constructor.
    *
    * @param query_id The ID of the query to which this operator belongs.
+   * @param num_partitions The number of partitions.
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
    **/
   DestroyHashOperator(const std::size_t query_id,
+                      const std::size_t num_partitions,
                       const QueryContext::join_hash_table_id hash_table_index)
       : RelationalOperator(query_id),
+        num_partitions_(num_partitions),
         hash_table_index_(hash_table_index),
         work_generated_(false) {}
 
@@ -75,6 +79,7 @@ class DestroyHashOperator : public RelationalOperator {
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
  private:
+  const std::size_t num_partitions_;
   const QueryContext::join_hash_table_id hash_table_index_;
   bool work_generated_;
 
@@ -91,13 +96,16 @@ class DestroyHashWorkOrder : public WorkOrder {
    *
    * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
+   * @param part_id The partition id.
    * @param query_context The QueryContext to use.
    **/
   DestroyHashWorkOrder(const std::size_t query_id,
                        const QueryContext::join_hash_table_id hash_table_index,
+                       const partition_id part_id,
                        QueryContext *query_context)
       : WorkOrder(query_id),
         hash_table_index_(hash_table_index),
+        part_id_(part_id),
         query_context_(DCHECK_NOTNULL(query_context)) {}
 
   ~DestroyHashWorkOrder() override {}
@@ -106,6 +114,7 @@ class DestroyHashWorkOrder : public WorkOrder {
 
  private:
   const QueryContext::join_hash_table_id hash_table_index_;
+  const partition_id part_id_;
   QueryContext *query_context_;
 
   DISALLOW_COPY_AND_ASSIGN(DestroyHashWorkOrder);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 2028046..0062b93 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -207,49 +207,43 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
         query_context->getScalarGroup(selection_index_);
     InsertDestination *output_destination =
         query_context->getInsertDestination(output_destination_index_);
-    const JoinHashTable &hash_table =
-        *(query_context->getJoinHashTable(hash_table_index_));
 
     if (probe_relation_is_stored_) {
-      if (!started_) {
-        for (const block_id probe_block_id : probe_relation_block_ids_) {
+      if (started_) {
+        return true;
+      }
+
+      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+        const JoinHashTable &hash_table =
+            *(query_context->getJoinHashTable(hash_table_index_, part_id));
+
+        for (const block_id probe_block_id : probe_relation_block_ids_[part_id]) {
           container->addNormalWorkOrder(
-              new JoinWorkOrderClass(query_id_,
-                                     build_relation_,
-                                     probe_relation_,
-                                     join_key_attributes_,
-                                     any_join_key_attributes_nullable_,
-                                     probe_block_id,
-                                     residual_predicate,
-                                     selection,
-                                     hash_table,
-                                     output_destination,
-                                     storage_manager,
+              new JoinWorkOrderClass(query_id_, build_relation_, probe_relation_, join_key_attributes_,
+                                     any_join_key_attributes_nullable_, num_partitions_, part_id, probe_block_id,
+                                     residual_predicate, selection, hash_table, output_destination, storage_manager,
                                      CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
               op_index_);
         }
-        started_ = true;
       }
-      return started_;
+      started_ = true;
+      return true;
     } else {
-      while (num_workorders_generated_ < probe_relation_block_ids_.size()) {
-        container->addNormalWorkOrder(
-            new JoinWorkOrderClass(
-                query_id_,
-                build_relation_,
-                probe_relation_,
-                join_key_attributes_,
-                any_join_key_attributes_nullable_,
-                probe_relation_block_ids_[num_workorders_generated_],
-                residual_predicate,
-                selection,
-                hash_table,
-                output_destination,
-                storage_manager,
-                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
-            op_index_);
-        ++num_workorders_generated_;
-      }  // end while
+      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+        const JoinHashTable &hash_table =
+            *(query_context->getJoinHashTable(hash_table_index_, part_id));
+
+        while (num_workorders_generated_[part_id] < probe_relation_block_ids_[part_id].size()) {
+          container->addNormalWorkOrder(
+              new JoinWorkOrderClass(query_id_, build_relation_, probe_relation_, join_key_attributes_,
+                                     any_join_key_attributes_nullable_, num_partitions_, part_id,
+                                     probe_relation_block_ids_[part_id][num_workorders_generated_[part_id]],
+                                     residual_predicate, selection, hash_table, output_destination, storage_manager,
+                                     CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
+              op_index_);
+          ++num_workorders_generated_[part_id];
+        }  // end while
+      }  // end for
       return done_feeding_input_relation_;
     }  // end else (probe_relation_is_stored_)
   }  // end if (blocking_dependencies_met_)
@@ -269,49 +263,44 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
 
     InsertDestination *output_destination =
         query_context->getInsertDestination(output_destination_index_);
-    const JoinHashTable &hash_table =
-        *(query_context->getJoinHashTable(hash_table_index_));
 
     if (probe_relation_is_stored_) {
-      if (!started_) {
-        for (const block_id probe_block_id : probe_relation_block_ids_) {
+      if (started_) {
+        return true;
+      }
+
+      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+        const JoinHashTable &hash_table =
+            *(query_context->getJoinHashTable(hash_table_index_, part_id));
+
+        for (const block_id probe_block_id : probe_relation_block_ids_[part_id]) {
           container->addNormalWorkOrder(
-              new HashOuterJoinWorkOrder(
-                  query_id_,
-                  build_relation_,
-                  probe_relation_,
-                  join_key_attributes_,
-                  any_join_key_attributes_nullable_,
-                  probe_block_id,
-                  selection,
-                  is_selection_on_build_,
-                  hash_table,
-                  output_destination,
-                  storage_manager,
-                  CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
+              new HashOuterJoinWorkOrder(query_id_, build_relation_, probe_relation_, join_key_attributes_,
+                                         any_join_key_attributes_nullable_, num_partitions_, part_id, probe_block_id,
+                                         selection, is_selection_on_build_, hash_table, output_destination,
+                                         storage_manager,
+                                         CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
               op_index_);
         }
-        started_ = true;
       }
-      return started_;
+      started_ = true;
+      return true;
     } else {
-      while (num_workorders_generated_ < probe_relation_block_ids_.size()) {
-        container->addNormalWorkOrder(
-            new HashOuterJoinWorkOrder(
-                query_id_,
-                build_relation_,
-                probe_relation_,
-                join_key_attributes_,
-                any_join_key_attributes_nullable_,
-                probe_relation_block_ids_[num_workorders_generated_],
-                selection,
-                is_selection_on_build_,
-                hash_table,
-                output_destination,
-                storage_manager,
-                CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
-            op_index_);
-        ++num_workorders_generated_;
+      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+        const JoinHashTable &hash_table =
+            *(query_context->getJoinHashTable(hash_table_index_, part_id));
+
+        while (num_workorders_generated_[part_id] < probe_relation_block_ids_[part_id].size()) {
+          container->addNormalWorkOrder(
+              new HashOuterJoinWorkOrder(query_id_, build_relation_, probe_relation_, join_key_attributes_,
+                                         any_join_key_attributes_nullable_, num_partitions_, part_id,
+                                         probe_relation_block_ids_[part_id][num_workorders_generated_[part_id]],
+                                         selection, is_selection_on_build_, hash_table, output_destination,
+                                         storage_manager,
+                                         CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
+              op_index_);
+          ++num_workorders_generated_[part_id];
+        }
       }
       return done_feeding_input_relation_;
     }  // end else (probe_relation_is_stored_)
@@ -343,22 +332,29 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrderProtos(
   }
 
   if (probe_relation_is_stored_) {
-    if (!started_) {
-      for (const block_id probe_block_id : probe_relation_block_ids_) {
+    if (started_) {
+      return true;
+    }
+
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      for (const block_id probe_block_id : probe_relation_block_ids_[part_id]) {
         container->addWorkOrderProto(
-            createNonOuterJoinWorkOrderProto(hash_join_type, probe_block_id),
+            createNonOuterJoinWorkOrderProto(hash_join_type, probe_block_id, part_id),
             op_index_);
       }
-      started_ = true;
     }
+    started_ = true;
     return true;
   } else {
-    while (num_workorders_generated_ < probe_relation_block_ids_.size()) {
-      container->addWorkOrderProto(
-          createNonOuterJoinWorkOrderProto(hash_join_type,
-                                           probe_relation_block_ids_[num_workorders_generated_]),
-          op_index_);
-      ++num_workorders_generated_;
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      while (num_workorders_generated_[part_id] < probe_relation_block_ids_[part_id].size()) {
+        container->addWorkOrderProto(
+            createNonOuterJoinWorkOrderProto(hash_join_type,
+                                             probe_relation_block_ids_[part_id][num_workorders_generated_[part_id]],
+                                             part_id),
+            op_index_);
+        ++num_workorders_generated_[part_id];
+      }
     }
 
     return done_feeding_input_relation_;
@@ -367,7 +363,7 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrderProtos(
 
 serialization::WorkOrder* HashJoinOperator::createNonOuterJoinWorkOrderProto(
     const serialization::HashJoinWorkOrder::HashJoinWorkOrderType hash_join_type,
-    const block_id block) {
+    const block_id block, const partition_id part_id) {
   serialization::WorkOrder *proto = new serialization::WorkOrder;
   proto->set_work_order_type(serialization::HASH_JOIN);
   proto->set_query_id(query_id_);
@@ -380,8 +376,10 @@ serialization::WorkOrder* HashJoinOperator::createNonOuterJoinWorkOrderProto(
   }
   proto->SetExtension(serialization::HashJoinWorkOrder::any_join_key_attributes_nullable,
                       any_join_key_attributes_nullable_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::num_partitions, num_partitions_);
   proto->SetExtension(serialization::HashJoinWorkOrder::insert_destination_index, output_destination_index_);
   proto->SetExtension(serialization::HashJoinWorkOrder::join_hash_table_index, hash_table_index_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::partition_id, part_id);
   proto->SetExtension(serialization::HashJoinWorkOrder::selection_index, selection_index_);
   proto->SetExtension(serialization::HashJoinWorkOrder::block_id, block);
   proto->SetExtension(serialization::HashJoinWorkOrder::residual_predicate_index, residual_predicate_index_);
@@ -397,26 +395,34 @@ bool HashJoinOperator::getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *
   }
 
   if (probe_relation_is_stored_) {
-    if (!started_) {
-      for (const block_id probe_block_id : probe_relation_block_ids_) {
-        container->addWorkOrderProto(createOuterJoinWorkOrderProto(probe_block_id), op_index_);
+    if (started_) {
+      return true;
+    }
+
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      for (const block_id probe_block_id : probe_relation_block_ids_[part_id]) {
+        container->addWorkOrderProto(createOuterJoinWorkOrderProto(probe_block_id, part_id), op_index_);
       }
-      started_ = true;
     }
+    started_ = true;
     return true;
   } else {
-    while (num_workorders_generated_ < probe_relation_block_ids_.size()) {
-      container->addWorkOrderProto(
-          createOuterJoinWorkOrderProto(probe_relation_block_ids_[num_workorders_generated_]),
-          op_index_);
-      ++num_workorders_generated_;
+    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      while (num_workorders_generated_[part_id] < probe_relation_block_ids_[part_id].size()) {
+        container->addWorkOrderProto(
+            createOuterJoinWorkOrderProto(probe_relation_block_ids_[part_id][num_workorders_generated_[part_id]],
+                                          part_id),
+            op_index_);
+        ++num_workorders_generated_[part_id];
+      }
     }
 
     return done_feeding_input_relation_;
   }
 }
 
-serialization::WorkOrder* HashJoinOperator::createOuterJoinWorkOrderProto(const block_id block) {
+serialization::WorkOrder* HashJoinOperator::createOuterJoinWorkOrderProto(const block_id block,
+                                                                          const partition_id part_id) {
   serialization::WorkOrder *proto = new serialization::WorkOrder;
   proto->set_work_order_type(serialization::HASH_JOIN);
   proto->set_query_id(query_id_);
@@ -430,10 +436,12 @@ serialization::WorkOrder* HashJoinOperator::createOuterJoinWorkOrderProto(const
   }
   proto->SetExtension(serialization::HashJoinWorkOrder::any_join_key_attributes_nullable,
                       any_join_key_attributes_nullable_);
+  proto->SetExtension(serialization::HashJoinWorkOrder::num_partitions, num_partitions_);
   proto->SetExtension(serialization::HashJoinWorkOrder::insert_destination_index, output_destination_index_);
   proto->SetExtension(serialization::HashJoinWorkOrder::join_hash_table_index, hash_table_index_);
   proto->SetExtension(serialization::HashJoinWorkOrder::selection_index, selection_index_);
   proto->SetExtension(serialization::HashJoinWorkOrder::block_id, block);
+  proto->SetExtension(serialization::HashJoinWorkOrder::partition_id, part_id);
   proto->SetExtension(serialization::HashJoinWorkOrder::lip_deployment_index, lip_deployment_index_);
 
   for (const bool is_attribute_on_build : is_selection_on_build_) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 98c87bf..e655f70 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -28,6 +28,7 @@
 
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogTypedefs.hpp"
+#include "catalog/PartitionScheme.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.hpp"
@@ -100,6 +101,8 @@ class HashJoinOperator : public RelationalOperator {
    * @param join_key_attributes The IDs of equijoin attributes in
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'input_relation'. If no
+   *        partitions, it is one.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert the join results.
@@ -125,6 +128,7 @@ class HashJoinOperator : public RelationalOperator {
       const bool probe_relation_is_stored,
       const std::vector<attribute_id> &join_key_attributes,
       const bool any_join_key_attributes_nullable,
+      const std::size_t num_partitions,
       const CatalogRelation &output_relation,
       const QueryContext::insert_destination_id output_destination_index,
       const QueryContext::join_hash_table_id hash_table_index,
@@ -138,6 +142,7 @@ class HashJoinOperator : public RelationalOperator {
         probe_relation_is_stored_(probe_relation_is_stored),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         hash_table_index_(hash_table_index),
@@ -147,14 +152,24 @@ class HashJoinOperator : public RelationalOperator {
                                    ? std::vector<bool>()
                                    : *is_selection_on_build),
         join_type_(join_type),
-        probe_relation_block_ids_(probe_relation_is_stored
-                                      ? probe_relation.getBlocksSnapshot()
-                                      : std::vector<block_id>()),
-        num_workorders_generated_(0),
+        probe_relation_block_ids_(num_partitions),
+        num_workorders_generated_(num_partitions),
         started_(false) {
     DCHECK(join_type != JoinType::kLeftOuterJoin ||
                (is_selection_on_build != nullptr &&
                 residual_predicate_index == QueryContext::kInvalidPredicateId));
+
+    if (probe_relation_is_stored) {
+      if (probe_relation.hasPartitionScheme()) {
+        const PartitionScheme &part_scheme = *probe_relation.getPartitionScheme();
+        for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+          probe_relation_block_ids_[part_id] = part_scheme.getBlocksInPartition(part_id);
+        }
+      } else {
+        // No partition.
+        probe_relation_block_ids_[0] = probe_relation.getBlocksSnapshot();
+      }
+    }
   }
 
   ~HashJoinOperator() override {}
@@ -192,8 +207,9 @@ class HashJoinOperator : public RelationalOperator {
 
   void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
                       const partition_id part_id) override {
-    DCHECK(input_relation_id == probe_relation_.getID());
-    probe_relation_block_ids_.push_back(input_block_id);
+    DCHECK_EQ(probe_relation_.getID(), input_relation_id);
+
+    probe_relation_block_ids_[part_id].push_back(input_block_id);
   }
 
   QueryContext::insert_destination_id getInsertDestinationID() const override {
@@ -229,7 +245,7 @@ class HashJoinOperator : public RelationalOperator {
 
   serialization::WorkOrder* createNonOuterJoinWorkOrderProto(
       const serialization::HashJoinWorkOrder::HashJoinWorkOrderType hash_join_type,
-      const block_id block);
+      const block_id block, const partition_id part_id);
 
   bool getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *container);
 
@@ -238,13 +254,14 @@ class HashJoinOperator : public RelationalOperator {
    *
    * @param block The block id used in the Work Order.
    **/
-  serialization::WorkOrder* createOuterJoinWorkOrderProto(const block_id block);
+  serialization::WorkOrder* createOuterJoinWorkOrderProto(const block_id block, const partition_id part_id);
 
   const CatalogRelation &build_relation_;
   const CatalogRelation &probe_relation_;
   const bool probe_relation_is_stored_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
+  const std::size_t num_partitions_;
   const CatalogRelation &output_relation_;
   const QueryContext::insert_destination_id output_destination_index_;
   const QueryContext::join_hash_table_id hash_table_index_;
@@ -253,8 +270,9 @@ class HashJoinOperator : public RelationalOperator {
   const std::vector<bool> is_selection_on_build_;
   const JoinType join_type_;
 
-  std::vector<block_id> probe_relation_block_ids_;
-  std::size_t num_workorders_generated_;
+  // The index is the partition id.
+  std::vector<BlocksInPartition> probe_relation_block_ids_;
+  std::vector<std::size_t> num_workorders_generated_;
 
   bool started_;
 
@@ -277,6 +295,9 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'probe_relation'. If no
+   *        partitions, it is one.
+   * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
    *        pairs of tuples that match the hash-join (i.e. key equality)
@@ -296,6 +317,8 @@ class HashInnerJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       const std::vector<attribute_id> &join_key_attributes,
       const bool any_join_key_attributes_nullable,
+      const std::size_t num_partitions,
+      const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
       const std::vector<std::unique_ptr<const Scalar>> &selection,
@@ -308,6 +331,8 @@ class HashInnerJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
         selection_(selection),
@@ -327,6 +352,9 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'probe_relation'. If no
+   *        partitions, it is one.
+   * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
    *        pairs of tuples that match the hash-join (i.e. key equality)
@@ -346,6 +374,8 @@ class HashInnerJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       std::vector<attribute_id> &&join_key_attributes,
       const bool any_join_key_attributes_nullable,
+      const std::size_t num_partitions,
+      const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
       const std::vector<std::unique_ptr<const Scalar>> &selection,
@@ -358,6 +388,8 @@ class HashInnerJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
         selection_(selection),
@@ -378,11 +410,31 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    **/
   void execute() override;
 
+  /**
+   * @brief Get the number of partitions.
+   *
+   * @return The number of partitions.
+   */
+  std::size_t num_partitions() const {
+    return num_partitions_;
+  }
+
+  /**
+   * @brief Get the partition id.
+   *
+   * @return The partition id.
+   */
+  partition_id getPartitionId() const {
+    return part_id_;
+  }
+
  private:
   const CatalogRelationSchema &build_relation_;
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
+  const std::size_t num_partitions_;
+  const partition_id part_id_;
   const block_id block_id_;
   const Predicate *residual_predicate_;
   const std::vector<std::unique_ptr<const Scalar>> &selection_;
@@ -413,6 +465,9 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'probe_relation'. If no
+   *        partitions, it is one.
+   * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
    *        pairs of tuples that match the hash-join (i.e. key equality)
@@ -432,6 +487,8 @@ class HashSemiJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       const std::vector<attribute_id> &join_key_attributes,
       const bool any_join_key_attributes_nullable,
+      const std::size_t num_partitions,
+      const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
       const std::vector<std::unique_ptr<const Scalar>> &selection,
@@ -444,6 +501,8 @@ class HashSemiJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
         selection_(selection),
@@ -463,6 +522,9 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'probe_relation'. If no
+   *        partitions, it is one.
+   * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
    *        pairs of tuples that match the hash-join (i.e. key equality)
@@ -482,6 +544,8 @@ class HashSemiJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       std::vector<attribute_id> &&join_key_attributes,
       const bool any_join_key_attributes_nullable,
+      const std::size_t num_partitions,
+      const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
       const std::vector<std::unique_ptr<const Scalar>> &selection,
@@ -494,6 +558,8 @@ class HashSemiJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
         selection_(selection),
@@ -506,6 +572,24 @@ class HashSemiJoinWorkOrder : public WorkOrder {
 
   void execute() override;
 
+  /**
+   * @brief Get the number of partitions.
+   *
+   * @return The number of partitions.
+   */
+  std::size_t num_partitions() const {
+    return num_partitions_;
+  }
+
+  /**
+   * @brief Get the partition id.
+   *
+   * @return The partition id.
+   */
+  partition_id getPartitionId() const {
+    return part_id_;
+  }
+
  private:
   void executeWithoutResidualPredicate();
 
@@ -515,6 +599,8 @@ class HashSemiJoinWorkOrder : public WorkOrder {
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
+  const std::size_t num_partitions_;
+  const partition_id part_id_;
   const block_id block_id_;
   const Predicate *residual_predicate_;
   const std::vector<std::unique_ptr<const Scalar>> &selection_;
@@ -545,6 +631,9 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'probe_relation'. If no
+   *        partitions, it is one.
+   * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
    *        pairs of tuples that match the hash-join (i.e. key equality)
@@ -564,6 +653,8 @@ class HashAntiJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       const std::vector<attribute_id> &join_key_attributes,
       const bool any_join_key_attributes_nullable,
+      const std::size_t num_partitions,
+      const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
       const std::vector<std::unique_ptr<const Scalar>> &selection,
@@ -576,6 +667,8 @@ class HashAntiJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
         selection_(selection),
@@ -595,6 +688,9 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'probe_relation'. If no
+   *        partitions, it is one.
+   * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
    *        pairs of tuples that match the hash-join (i.e. key equality)
@@ -614,6 +710,8 @@ class HashAntiJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       std::vector<attribute_id> &&join_key_attributes,
       const bool any_join_key_attributes_nullable,
+      const std::size_t num_partitions,
+      const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
       const std::vector<std::unique_ptr<const Scalar>> &selection,
@@ -626,6 +724,8 @@ class HashAntiJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
         selection_(selection),
@@ -644,6 +744,24 @@ class HashAntiJoinWorkOrder : public WorkOrder {
     }
   }
 
+  /**
+   * @brief Get the number of partitions.
+   *
+   * @return The number of partitions.
+   */
+  std::size_t num_partitions() const {
+    return num_partitions_;
+  }
+
+  /**
+   * @brief Get the partition id.
+   *
+   * @return The partition id.
+   */
+  partition_id getPartitionId() const {
+    return part_id_;
+  }
+
  private:
   void executeWithoutResidualPredicate();
 
@@ -653,6 +771,8 @@ class HashAntiJoinWorkOrder : public WorkOrder {
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
+  const std::size_t num_partitions_;
+  const partition_id part_id_;
   const block_id block_id_;
   const Predicate *residual_predicate_;
   const std::vector<std::unique_ptr<const Scalar>> &selection_;
@@ -682,6 +802,9 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'probe_relation'. If no
+   *        partitions, it is one.
+   * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param selection A list of Scalars corresponding to the relation attributes
    *        in \c output_destination. Each Scalar is evaluated for the joined
@@ -700,6 +823,8 @@ class HashOuterJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       const std::vector<attribute_id> &join_key_attributes,
       const bool any_join_key_attributes_nullable,
+      const std::size_t num_partitions,
+      const partition_id part_id,
       const block_id lookup_block_id,
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       const std::vector<bool> &is_selection_on_build,
@@ -712,6 +837,8 @@ class HashOuterJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        part_id_(part_id),
         block_id_(lookup_block_id),
         selection_(selection),
         is_selection_on_build_(is_selection_on_build),
@@ -731,6 +858,9 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param num_partitions The number of partitions in 'probe_relation'. If no
+   *        partitions, it is one.
+   * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param selection A list of Scalars corresponding to the relation attributes
    *        in \c output_destination. Each Scalar is evaluated for the joined
@@ -748,6 +878,8 @@ class HashOuterJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       std::vector<attribute_id> &&join_key_attributes,
       const bool any_join_key_attributes_nullable,
+      const std::size_t num_partitions,
+      const partition_id part_id,
       const block_id lookup_block_id,
       const std::vector<std::unique_ptr<const Scalar>> &selection,
       std::vector<bool> &&is_selection_on_build,
@@ -760,6 +892,8 @@ class HashOuterJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
+        num_partitions_(num_partitions),
+        part_id_(part_id),
         block_id_(lookup_block_id),
         selection_(selection),
         is_selection_on_build_(std::move(is_selection_on_build)),
@@ -772,11 +906,31 @@ class HashOuterJoinWorkOrder : public WorkOrder {
 
   void execute() override;
 
+  /**
+   * @brief Get the number of partitions.
+   *
+   * @return The number of partitions.
+   */
+  std::size_t num_partitions() const {
+    return num_partitions_;
+  }
+
+  /**
+   * @brief Get the partition id.
+   *
+   * @return The partition id.
+   */
+  partition_id getPartitionId() const {
+    return part_id_;
+  }
+
  private:
   const CatalogRelationSchema &build_relation_;
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
+  const std::size_t num_partitions_;
+  const partition_id part_id_;
   const block_id block_id_;
   const std::vector<std::unique_ptr<const Scalar>> &selection_;
   const std::vector<bool> is_selection_on_build_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 86f34b8..b914fce 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -63,13 +63,16 @@ message AggregationWorkOrder {
   }
 }
 
+// Next tag: 40.
 message BuildHashWorkOrder {
   extend WorkOrder {
     // All required.
     optional int32 relation_id = 32;
     repeated int32 join_key_attributes = 33;
     optional bool any_join_key_attributes_nullable = 34;
+    optional uint64 num_partitions = 38;
     optional uint32 join_hash_table_index = 35;
+    optional uint64 partition_id = 39;
     optional fixed64 block_id = 36;
     optional int32 lip_deployment_index = 37;
   }
@@ -89,6 +92,7 @@ message DestroyHashWorkOrder {
   extend WorkOrder {
     // All required.
     optional uint32 join_hash_table_index = 112;
+    optional uint64 partition_id = 113;
   }
 }
 
@@ -109,6 +113,7 @@ message FinalizeAggregationWorkOrder {
   }
 }
 
+// Next tag: 174.
 message HashJoinWorkOrder {
   enum HashJoinWorkOrderType {
     HASH_ANTI_JOIN = 0;
@@ -124,8 +129,10 @@ message HashJoinWorkOrder {
     optional int32 probe_relation_id = 162;
     repeated int32 join_key_attributes = 163;
     optional bool any_join_key_attributes_nullable = 164;
+    optional uint64 num_partitions = 172;
     optional int32 insert_destination_index = 165;
     optional uint32 join_hash_table_index = 166;
+    optional uint64 partition_id = 173;
     optional int32 selection_index = 167;
     optional fixed64 block_id = 168;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 871f74d..c09bcbe 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -98,15 +98,20 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
             proto.GetExtension(serialization::BuildHashWorkOrder::join_key_attributes, i));
       }
 
+      const partition_id part_id =
+          proto.GetExtension(serialization::BuildHashWorkOrder::partition_id);
+
       return new BuildHashWorkOrder(
           proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::BuildHashWorkOrder::relation_id)),
           move(join_key_attributes),
           proto.GetExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable),
+          proto.GetExtension(serialization::BuildHashWorkOrder::num_partitions),
+          part_id,
           proto.GetExtension(serialization::BuildHashWorkOrder::block_id),
           query_context->getJoinHashTable(
-              proto.GetExtension(serialization::BuildHashWorkOrder::join_hash_table_index)),
+              proto.GetExtension(serialization::BuildHashWorkOrder::join_hash_table_index), part_id),
           storage_manager,
           CreateLIPFilterBuilderHelper(
               proto.GetExtension(serialization::BuildHashWorkOrder::lip_deployment_index), query_context));
@@ -139,6 +144,8 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           proto.query_id(),
           proto.GetExtension(
               serialization::DestroyHashWorkOrder::join_hash_table_index),
+          proto.GetExtension(
+              serialization::DestroyHashWorkOrder::partition_id),
           query_context);
     }
     case serialization::DROP_TABLE: {
@@ -192,6 +199,9 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       const block_id lookup_block_id =
           proto.GetExtension(serialization::HashJoinWorkOrder::block_id);
 
+      const std::size_t num_partitions =
+          proto.GetExtension(serialization::HashJoinWorkOrder::num_partitions);
+
       const Predicate *residual_predicate = nullptr;
       if (hash_join_work_order_type != serialization::HashJoinWorkOrder::HASH_OUTER_JOIN) {
         residual_predicate =
@@ -202,9 +212,11 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       const std::vector<std::unique_ptr<const Scalar>> &selection =
           query_context->getScalarGroup(
               proto.GetExtension(serialization::HashJoinWorkOrder::selection_index));
+      const partition_id part_id =
+          proto.GetExtension(serialization::HashJoinWorkOrder::partition_id);
       const JoinHashTable &hash_table =
           *query_context->getJoinHashTable(
-              proto.GetExtension(serialization::HashJoinWorkOrder::join_hash_table_index));
+              proto.GetExtension(serialization::HashJoinWorkOrder::join_hash_table_index), part_id);
       InsertDestination *output_destination =
           query_context->getInsertDestination(
               proto.GetExtension(serialization::HashJoinWorkOrder::insert_destination_index));
@@ -221,6 +233,8 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               probe_relation,
               move(join_key_attributes),
               any_join_key_attributes_nullable,
+              num_partitions,
+              part_id,
               lookup_block_id,
               residual_predicate,
               selection,
@@ -237,6 +251,8 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               probe_relation,
               move(join_key_attributes),
               any_join_key_attributes_nullable,
+              num_partitions,
+              part_id,
               lookup_block_id,
               residual_predicate,
               selection,
@@ -261,6 +277,8 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               probe_relation,
               move(join_key_attributes),
               any_join_key_attributes_nullable,
+              num_partitions,
+              part_id,
               lookup_block_id,
               selection,
               move(is_selection_on_build),
@@ -277,6 +295,8 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               probe_relation,
               move(join_key_attributes),
               any_join_key_attributes_nullable,
+              num_partitions,
+              part_id,
               lookup_block_id,
               residual_predicate,
               selection,
@@ -524,8 +544,11 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
       return proto.HasExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable) &&
              proto.HasExtension(serialization::BuildHashWorkOrder::block_id) &&
              proto.HasExtension(serialization::BuildHashWorkOrder::join_hash_table_index) &&
+             proto.HasExtension(serialization::BuildHashWorkOrder::num_partitions) &&
+             proto.HasExtension(serialization::BuildHashWorkOrder::partition_id) &&
              query_context.isValidJoinHashTableId(
-                 proto.GetExtension(serialization::BuildHashWorkOrder::join_hash_table_index));
+                 proto.GetExtension(serialization::BuildHashWorkOrder::join_hash_table_index),
+                 proto.GetExtension(serialization::BuildHashWorkOrder::partition_id));
     }
     case serialization::DELETE: {
       return proto.HasExtension(serialization::DeleteWorkOrder::relation_id) &&
@@ -544,8 +567,10 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
     }
     case serialization::DESTROY_HASH: {
       return proto.HasExtension(serialization::DestroyHashWorkOrder::join_hash_table_index) &&
+             proto.HasExtension(serialization::DestroyHashWorkOrder::partition_id) &&
              query_context.isValidJoinHashTableId(
-                 proto.GetExtension(serialization::DestroyHashWorkOrder::join_hash_table_index));
+                 proto.GetExtension(serialization::DestroyHashWorkOrder::join_hash_table_index),
+                 proto.GetExtension(serialization::DestroyHashWorkOrder::partition_id));
     }
     case serialization::DROP_TABLE: {
       return true;
@@ -623,8 +648,11 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
              query_context.isValidInsertDestinationId(
                  proto.GetExtension(serialization::HashJoinWorkOrder::insert_destination_index)) &&
              proto.HasExtension(serialization::HashJoinWorkOrder::join_hash_table_index) &&
+             proto.HasExtension(serialization::HashJoinWorkOrder::num_partitions) &&
+             proto.HasExtension(serialization::HashJoinWorkOrder::partition_id) &&
              query_context.isValidJoinHashTableId(
-                 proto.GetExtension(serialization::HashJoinWorkOrder::join_hash_table_index)) &&
+                 proto.GetExtension(serialization::HashJoinWorkOrder::join_hash_table_index),
+                 proto.GetExtension(serialization::HashJoinWorkOrder::partition_id)) &&
              proto.HasExtension(serialization::HashJoinWorkOrder::selection_index) &&
              query_context.isValidScalarGroupId(
                  proto.GetExtension(serialization::HashJoinWorkOrder::selection_index)) &&


[10/12] incubator-quickstep git commit: Minor refactors in HashJoin unit test.

Posted by ji...@apache.org.
Minor refactors in HashJoin unit test.


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

Branch: refs/heads/reorder-attrs
Commit: 7bce0b8516be8285fdba4efc85c348a8c6d17a27
Parents: 300284c
Author: Zuyu Zhang <zu...@apache.org>
Authored: Mon Jan 23 14:40:18 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jan 23 14:40:18 2017 -0800

----------------------------------------------------------------------
 .../tests/HashJoinOperator_unittest.cpp         | 92 ++++++++++----------
 1 file changed, 44 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/7bce0b85/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 2370c92..60f05ea 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -97,7 +97,7 @@ constexpr tuple_id kBlockSize = 10;
 constexpr std::size_t kQueryId = 0;
 constexpr int kOpIndex = 0;
 
-constexpr std::size_t kNumPartitions = 1;
+constexpr std::size_t kSinglePartition = 1;
 
 }  // namespace
 
@@ -194,8 +194,6 @@ class HashJoinOperatorTest : public ::testing::TestWithParam<HashTableImplType>
       }
       storage_block->rebuild();
     }
-
-    num_partitions_ = kNumPartitions;
   }
 
   virtual void TearDown() {
@@ -295,14 +293,12 @@ class HashJoinOperatorTest : public ::testing::TestWithParam<HashTableImplType>
   unique_ptr<CatalogDatabase> db_;
   // The following CatalogRelations are owned by db_.
   CatalogRelation *dim_table_, *fact_table_;
-
-  std::size_t num_partitions_;
 };
 
 TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
-  query_context_proto.set_query_id(0);  // dummy query ID.
+  query_context_proto.set_query_id(kQueryId);
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -334,7 +330,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
 
   const Type &long_type = LongType::InstanceNonNullable();
 
-  hash_table_proto->add_key_types()->CopyFrom(long_type.getProto());
+  hash_table_proto->add_key_types()->MergeFrom(long_type.getProto());
   hash_table_proto->set_estimated_num_entries(kNumDimTuples);
 
   const CatalogAttribute &dim_col_long = *dim_table_->getAttributeByName("long");
@@ -347,13 +343,13 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_long.getID()),
                             dim_col_long.getType().isNullable(),
-                            num_partitions_,
+                            kSinglePartition,
                             join_hash_table_index));
 
   // Create the prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
   ScalarAttribute scalar_attr(dim_col_long);
-  query_context_proto.add_scalar_groups()->add_scalars()->CopyFrom(scalar_attr.getProto());
+  query_context_proto.add_scalar_groups()->add_scalars()->MergeFrom(scalar_attr.getProto());
 
   // Create result_table, owned by db_.
   CatalogRelation *result_table = new CatalogRelation(NULL, "result_table", 102);
@@ -377,7 +373,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
       true /* is_stored */,
       std::vector<attribute_id>(1, fact_col_long.getID()),
       fact_col_long.getType().isNullable(),
-      num_partitions_,
+      kSinglePartition,
       *result_table,
       output_destination_index,
       join_hash_table_index,
@@ -435,7 +431,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, kSinglePartition, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -445,7 +441,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
 TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
-  query_context_proto.set_query_id(0);  // dummy query ID.
+  query_context_proto.set_query_id(kQueryId);
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -478,7 +474,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   const Type &long_type = LongType::InstanceNonNullable();
   const Type &int_type = IntType::InstanceNonNullable();
 
-  hash_table_proto->add_key_types()->CopyFrom(int_type.getProto());
+  hash_table_proto->add_key_types()->MergeFrom(int_type.getProto());
   hash_table_proto->set_estimated_num_entries(kNumDimTuples);
 
   const CatalogAttribute &dim_col_long = *dim_table_->getAttributeByName("long");
@@ -493,7 +489,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_int.getID()),
                             dim_col_int.getType().isNullable(),
-                            num_partitions_,
+                            kSinglePartition,
                             join_hash_table_index));
 
   // Create the prober operator with two selection attributes.
@@ -501,9 +497,9 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   serialization::QueryContext::ScalarGroup *scalar_group_proto = query_context_proto.add_scalar_groups();
 
   ScalarAttribute scalar_attr_dim(dim_col_long);
-  scalar_group_proto->add_scalars()->CopyFrom(scalar_attr_dim.getProto());
+  scalar_group_proto->add_scalars()->MergeFrom(scalar_attr_dim.getProto());
   ScalarAttribute scalar_attr_fact(fact_col_long);
-  scalar_group_proto->add_scalars()->CopyFrom(scalar_attr_fact.getProto());
+  scalar_group_proto->add_scalars()->MergeFrom(scalar_attr_fact.getProto());
 
   // Create result_table, owned by db_.
   CatalogRelation *result_table = new CatalogRelation(NULL, "result_table", 102);
@@ -528,7 +524,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
       true /* is_stored */,
       std::vector<attribute_id>(1, fact_col_int.getID()),
       fact_col_int.getType().isNullable(),
-      num_partitions_,
+      kSinglePartition,
       *result_table,
       output_destination_index,
       join_hash_table_index,
@@ -608,7 +604,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, kSinglePartition, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -618,7 +614,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
 TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
-  query_context_proto.set_query_id(0);  // dummy query ID.
+  query_context_proto.set_query_id(kQueryId);
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -644,7 +640,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   const Type &long_type = LongType::InstanceNonNullable();
   const Type &char_type = CharType::InstanceNonNullable(kCharLength);
 
-  hash_table_proto->add_key_types()->CopyFrom(char_type.getProto());
+  hash_table_proto->add_key_types()->MergeFrom(char_type.getProto());
   hash_table_proto->set_estimated_num_entries(kNumDimTuples);
 
   const CatalogAttribute &dim_col_long = *dim_table_->getAttributeByName("long");
@@ -658,13 +654,13 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_char.getID()),
                             dim_col_char.getType().isNullable(),
-                            num_partitions_,
+                            kSinglePartition,
                             join_hash_table_index));
 
   // Create prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
   ScalarAttribute scalar_attr(dim_col_long);
-  query_context_proto.add_scalar_groups()->add_scalars()->CopyFrom(scalar_attr.getProto());
+  query_context_proto.add_scalar_groups()->add_scalars()->MergeFrom(scalar_attr.getProto());
 
   // Create result_table, owned by db_.
   CatalogRelation *result_table = new CatalogRelation(NULL, "result_table", 102);
@@ -688,7 +684,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
       true /* is_stored */,
       std::vector<attribute_id>(1, fact_col_char.getID()),
       fact_col_char.getType().isNullable(),
-      num_partitions_,
+      kSinglePartition,
       *result_table,
       output_destination_index,
       join_hash_table_index,
@@ -746,7 +742,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, kSinglePartition, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -756,7 +752,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
 TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
-  query_context_proto.set_query_id(0);  // dummy query ID.
+  query_context_proto.set_query_id(kQueryId);
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -782,7 +778,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   const Type &long_type = LongType::InstanceNonNullable();
   const Type &varchar_type = VarCharType::InstanceNonNullable(kCharLength);
 
-  hash_table_proto->add_key_types()->CopyFrom(varchar_type.getProto());
+  hash_table_proto->add_key_types()->MergeFrom(varchar_type.getProto());
   hash_table_proto->set_estimated_num_entries(kNumDimTuples);
 
   const CatalogAttribute &dim_col_long = *dim_table_->getAttributeByName("long");
@@ -797,7 +793,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_varchar.getID()),
                             dim_col_varchar.getType().isNullable(),
-                            num_partitions_,
+                            kSinglePartition,
                             join_hash_table_index));
 
   // Create prober operator with two selection attributes.
@@ -805,9 +801,9 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   serialization::QueryContext::ScalarGroup *scalar_group_proto = query_context_proto.add_scalar_groups();
 
   ScalarAttribute scalar_attr_dim(dim_col_long);
-  scalar_group_proto->add_scalars()->CopyFrom(scalar_attr_dim.getProto());
+  scalar_group_proto->add_scalars()->MergeFrom(scalar_attr_dim.getProto());
   ScalarAttribute scalar_attr_fact(fact_col_long);
-  scalar_group_proto->add_scalars()->CopyFrom(scalar_attr_fact.getProto());
+  scalar_group_proto->add_scalars()->MergeFrom(scalar_attr_fact.getProto());
 
   // Create result_table, owned by db_.
   CatalogRelation *result_table = new CatalogRelation(NULL, "result_table", 102);
@@ -832,7 +828,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
       true /* is_stored */,
       std::vector<attribute_id>(1, fact_col_varchar.getID()),
       fact_col_varchar.getType().isNullable(),
-      num_partitions_,
+      kSinglePartition,
       *result_table,
       output_destination_index,
       join_hash_table_index,
@@ -916,7 +912,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   }
 
   // Create the cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, kSinglePartition, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -926,7 +922,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
 TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
-  query_context_proto.set_query_id(0);  // dummy query ID.
+  query_context_proto.set_query_id(kQueryId);
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -952,8 +948,8 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   const Type &long_type = LongType::InstanceNonNullable();
   const Type &varchar_type = VarCharType::InstanceNonNullable(kCharLength);
 
-  hash_table_proto->add_key_types()->CopyFrom(long_type.getProto());
-  hash_table_proto->add_key_types()->CopyFrom(varchar_type.getProto());
+  hash_table_proto->add_key_types()->MergeFrom(long_type.getProto());
+  hash_table_proto->add_key_types()->MergeFrom(varchar_type.getProto());
   hash_table_proto->set_estimated_num_entries(kNumDimTuples);
 
   const CatalogAttribute &dim_col_long = *dim_table_->getAttributeByName("long");
@@ -972,7 +968,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            num_partitions_,
+                            kSinglePartition,
                             join_hash_table_index));
 
   // Create the prober operator with two selection attributes.
@@ -980,9 +976,9 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   serialization::QueryContext::ScalarGroup *scalar_group_proto = query_context_proto.add_scalar_groups();
 
   ScalarAttribute scalar_attr_dim(dim_col_long);
-  scalar_group_proto->add_scalars()->CopyFrom(scalar_attr_dim.getProto());
+  scalar_group_proto->add_scalars()->MergeFrom(scalar_attr_dim.getProto());
   ScalarAttribute scalar_attr_fact(fact_col_long);
-  scalar_group_proto->add_scalars()->CopyFrom(scalar_attr_fact.getProto());
+  scalar_group_proto->add_scalars()->MergeFrom(scalar_attr_fact.getProto());
 
   // Create result_table, owned by db_.
   CatalogRelation *result_table = new CatalogRelation(NULL, "result_table", 102);
@@ -1012,7 +1008,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
       fact_key_attrs,
       fact_col_long.getType().isNullable() ||
           fact_col_varchar.getType().isNullable(),
-      num_partitions_,
+      kSinglePartition,
       *result_table,
       output_destination_index,
       join_hash_table_index,
@@ -1095,7 +1091,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, kSinglePartition, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -1106,7 +1102,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
 TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   // Setup the hash table proto in the query context proto.
   serialization::QueryContext query_context_proto;
-  query_context_proto.set_query_id(0);  // dummy query ID.
+  query_context_proto.set_query_id(kQueryId);
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -1132,8 +1128,8 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   const Type &long_type = LongType::InstanceNonNullable();
   const Type &varchar_type = VarCharType::InstanceNonNullable(kCharLength);
 
-  hash_table_proto->add_key_types()->CopyFrom(long_type.getProto());
-  hash_table_proto->add_key_types()->CopyFrom(varchar_type.getProto());
+  hash_table_proto->add_key_types()->MergeFrom(long_type.getProto());
+  hash_table_proto->add_key_types()->MergeFrom(varchar_type.getProto());
   hash_table_proto->set_estimated_num_entries(kNumDimTuples);
 
   const CatalogAttribute &dim_col_long = *dim_table_->getAttributeByName("long");
@@ -1152,7 +1148,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            num_partitions_,
+                            kSinglePartition,
                             join_hash_table_index));
 
   // Create prober operator with two selection attributes.
@@ -1160,9 +1156,9 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   serialization::QueryContext::ScalarGroup *scalar_group_proto = query_context_proto.add_scalar_groups();
 
   ScalarAttribute scalar_attr_dim(dim_col_long);
-  scalar_group_proto->add_scalars()->CopyFrom(scalar_attr_dim.getProto());
+  scalar_group_proto->add_scalars()->MergeFrom(scalar_attr_dim.getProto());
   ScalarAttribute scalar_attr_fact(fact_col_long);
-  scalar_group_proto->add_scalars()->CopyFrom(scalar_attr_fact.getProto());
+  scalar_group_proto->add_scalars()->MergeFrom(scalar_attr_fact.getProto());
 
   // Create result_table, owned by db_.
   CatalogRelation *result_table = new CatalogRelation(NULL, "result_table", 102);
@@ -1192,7 +1188,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   fact_key_attrs.push_back(fact_col_varchar.getID());
 
   const QueryContext::predicate_id residual_pred_index = query_context_proto.predicates_size();
-  query_context_proto.add_predicates()->CopyFrom(residual_pred->getProto());
+  query_context_proto.add_predicates()->MergeFrom(residual_pred->getProto());
 
   unique_ptr<HashJoinOperator> prober(
       new HashJoinOperator(kQueryId,
@@ -1202,7 +1198,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
                            fact_key_attrs,
                            fact_col_long.getType().isNullable() ||
                                fact_col_varchar.getType().isNullable(),
-                           num_partitions_,
+                           kSinglePartition,
                            *result_table,
                            output_destination_index,
                            join_hash_table_index,
@@ -1285,7 +1281,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, kSinglePartition, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 


[12/12] incubator-quickstep git commit: Reorder output attribute order to improve copy performance.

Posted by ji...@apache.org.
Reorder output attribute order to improve copy performance.


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

Branch: refs/heads/reorder-attrs
Commit: 26c3db420cfba1917e8384cbbf33cfc40bbb28be
Parents: 968ce3f
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Jan 12 18:41:17 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Fri Jan 27 21:01:22 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/PhysicalGenerator.cpp           |   2 +
 query_optimizer/rules/CMakeLists.txt            |  12 +
 query_optimizer/rules/ReorderColumns.cpp        | 217 +++++++++++++++++++
 query_optimizer/rules/ReorderColumns.hpp        |  64 ++++++
 relational_operators/HashJoinOperator.cpp       | 149 +++----------
 storage/SplitRowStoreValueAccessor.hpp          |   5 +
 storage/ValueAccessor.hpp                       |  30 +++
 types/containers/ColumnVectorsValueAccessor.hpp |   5 +
 9 files changed, 365 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/26c3db42/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 1a15271..38dadc2 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -211,6 +211,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_AttachLIPFilters
                       quickstep_queryoptimizer_rules_PruneColumns
+                      quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_strategy_Aggregate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/26c3db42/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 7cb97dc..d1e4c65 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -28,6 +28,7 @@
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
+#include "query_optimizer/rules/ReorderColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
 #include "query_optimizer/strategy/Aggregate.hpp"
@@ -109,6 +110,7 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   } else {
     rules.emplace_back(new SwapProbeBuild());
   }
+  rules.emplace_back(new ReorderColumns());
   if (FLAGS_use_lip_filters) {
     rules.emplace_back(new AttachLIPFilters());
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/26c3db42/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 7fffadc..0ffa061 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -25,6 +25,7 @@ add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp Gener
 add_library(quickstep_queryoptimizer_rules_PruneColumns PruneColumns.cpp PruneColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownFilter PushDownFilter.cpp PushDownFilter.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin PushDownSemiAntiJoin.cpp PushDownSemiAntiJoin.hpp)
+add_library(quickstep_queryoptimizer_rules_ReorderColumns ReorderColumns.cpp ReorderColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_Rule ../../empty_src.cpp Rule.hpp)
 add_library(quickstep_queryoptimizer_rules_RuleHelper RuleHelper.cpp RuleHelper.hpp)
 add_library(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
@@ -118,6 +119,16 @@ target_link_libraries(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_logical_PatternMatcher
                       quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_ReorderColumns
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_Rule
                       glog
                       quickstep_utility_Macros)
@@ -213,6 +224,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
+                      quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/26c3db42/query_optimizer/rules/ReorderColumns.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReorderColumns.cpp b/query_optimizer/rules/ReorderColumns.cpp
new file mode 100644
index 0000000..e04b810
--- /dev/null
+++ b/query_optimizer/rules/ReorderColumns.cpp
@@ -0,0 +1,217 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "query_optimizer/rules/ReorderColumns.hpp"
+
+#include <algorithm>
+#include <limits>
+#include <vector>
+
+#include "catalog/CatalogRelation.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/Selection.hpp"
+#include "query_optimizer/physical/TableReference.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr ReorderColumns::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input, true);
+}
+
+P::PhysicalPtr ReorderColumns::applyInternal(const P::PhysicalPtr &input,
+                                             bool lock_ordering) {
+  // We have to guarantee that the top level ordering of the columns remain
+  // unchanged so that the output columns are ordered as specified by the user.
+  // So here we use the flag "lock_ordering" to skip the first transformable
+  // node (i.e. the first Selection or HashJoin).
+  bool skip_transform;
+  if (IsTransformable(input)) {
+    if (lock_ordering) {
+      skip_transform = true;
+      lock_ordering = false;
+    } else {
+      skip_transform = false;
+    }
+  } else {
+    skip_transform = true;
+  }
+
+  if (skip_transform) {
+    std::vector<P::PhysicalPtr> new_children;
+    for (const P::PhysicalPtr &child : input->children()) {
+      new_children.emplace_back(applyInternal(child, lock_ordering));
+    }
+
+    if (new_children != input->children()) {
+      return input->copyWithNewChildren(new_children);
+    } else {
+      return input;
+    }
+  }
+
+  // Collect the maximal chain of transformable nodes.
+  std::vector<P::PhysicalPtr> nodes;
+  for (P::PhysicalPtr node = input; IsTransformable(node); node = node->children().front()) {
+    nodes.emplace_back(node);
+  }
+  std::reverse(nodes.begin(), nodes.end());
+
+  // Analyze the attributes in the nodes.
+  std::unordered_map<E::ExprId, std::size_t> base, gen, kill;
+
+  const P::PhysicalPtr base_node =
+      applyInternal(nodes.front()->children().front(), false);
+  P::TableReferencePtr base_table;
+  if (P::SomeTableReference::MatchesWithConditionalCast(base_node, &base_table)) {
+  }
+
+  const std::vector<E::AttributeReferencePtr> base_attrs =
+      nodes.front()->children().front()->getOutputAttributes();
+  for (std::size_t i = 0; i < base_attrs.size(); ++i) {
+    base.emplace(base_attrs[i]->id(), i);
+  }
+
+  for (std::size_t i = 0; i < nodes.size(); ++i) {
+    for (const auto &attr : nodes[i]->getOutputAttributes()) {
+      const E::ExprId attr_id = attr->id();
+      if (gen.find(attr_id) == gen.end()) {
+        gen.emplace(attr_id, i);
+      }
+      kill[attr_id] = i;
+    }
+  }
+
+//  std::cout << "gen: \n";
+//  for (const auto &pair : gen) {
+//    std::cout << pair.first << ": " << pair.second << "\n";
+//  }
+//
+//  std::cout << "kill: \n";
+//  for (const auto &pair : kill) {
+//    std::cout << pair.first << ": " << pair.second << "\n";
+//  }
+
+  const auto comparator = [&gen, &kill, &base](const E::NamedExpressionPtr &lhs,
+                                               const E::NamedExpressionPtr &rhs) -> bool {
+    const E::ExprId lhs_id = lhs->id();
+    const E::ExprId rhs_id = rhs->id();
+
+    const std::size_t lhs_gen = gen.at(lhs_id);
+    const std::size_t rhs_gen = gen.at(rhs_id);
+    if (lhs_gen != rhs_gen) {
+      return lhs_gen < rhs_gen;
+    }
+
+    const std::size_t lhs_kill = kill.at(lhs_id);
+    const std::size_t rhs_kill = kill.at(rhs_id);
+    if (lhs_kill != rhs_kill) {
+      return lhs_kill < rhs_kill;
+    }
+
+    const auto lhs_base_it = base.find(lhs_id);
+    const auto rhs_base_it = base.find(rhs_id);
+    const std::size_t lhs_base =
+        lhs_base_it == base.end() ? std::numeric_limits<std::size_t>::max()
+                                  : lhs_base_it->second;
+    const std::size_t rhs_base =
+        rhs_base_it == base.end() ? std::numeric_limits<std::size_t>::max()
+                                  : rhs_base_it->second;
+    if (lhs_base != rhs_base) {
+      return lhs_base < rhs_base;
+    }
+
+    return lhs_id < rhs_id;
+  };
+
+  P::PhysicalPtr output = base_node;
+
+  for (const auto &node : nodes) {
+    std::vector<E::NamedExpressionPtr> project_expressions;
+    switch (node->getPhysicalType()) {
+      case P::PhysicalType::kHashJoin: {
+        project_expressions =
+            std::static_pointer_cast<const P::HashJoin>(node)->project_expressions();
+        break;
+      }
+      case P::PhysicalType::kSelection: {
+        project_expressions =
+            std::static_pointer_cast<const P::Selection>(node)->project_expressions();
+        break;
+      }
+      default:
+        LOG(FATAL) << "Unsupported physical type";
+    }
+
+    std::sort(project_expressions.begin(), project_expressions.end(), comparator);
+
+    switch (node->getPhysicalType()) {
+      case P::PhysicalType::kHashJoin: {
+        const P::HashJoinPtr old_node =
+            std::static_pointer_cast<const P::HashJoin>(node);
+        output = P::HashJoin::Create(output,
+                                     applyInternal(old_node->right(), false),
+                                     old_node->left_join_attributes(),
+                                     old_node->right_join_attributes(),
+                                     old_node->residual_predicate(),
+                                     project_expressions,
+                                     old_node->join_type());
+        break;
+      }
+      case P::PhysicalType::kSelection: {
+        const P::SelectionPtr old_node =
+            std::static_pointer_cast<const P::Selection>(node);
+        output = P::Selection::Create(output,
+                                      project_expressions,
+                                      old_node->filter_predicate());
+        break;
+      }
+      default:
+        LOG(FATAL) << "Unsupported physical type";
+    }
+  }
+
+  return output;
+}
+
+bool ReorderColumns::IsTransformable(const physical::PhysicalPtr &input) {
+  switch (input->getPhysicalType()) {
+    case P::PhysicalType::kHashJoin:  // Fall through
+    case P::PhysicalType::kSelection:
+      return true;
+    default:
+      return false;
+  }
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/26c3db42/query_optimizer/rules/ReorderColumns.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReorderColumns.hpp b/query_optimizer/rules/ReorderColumns.hpp
new file mode 100644
index 0000000..ad699aa
--- /dev/null
+++ b/query_optimizer/rules/ReorderColumns.hpp
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_
+
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+class ReorderColumns : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  ReorderColumns() {}
+
+  ~ReorderColumns() override {}
+
+  std::string getName() const override {
+    return "ReorderColumns";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input,
+                                      bool lock_ordering);
+
+  inline static bool IsTransformable(const physical::PhysicalPtr &input);
+
+  DISALLOW_COPY_AND_ASSIGN(ReorderColumns);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif /* QUICKSTEP_QUERY_OPTIMIZER_RULES_REORDER_COLUMNS_HPP_ */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/26c3db42/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index fd3841f..cda1465 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -20,6 +20,7 @@
 #include "relational_operators/HashJoinOperator.hpp"
 
 #include <algorithm>
+#include <map>
 #include <memory>
 #include <unordered_map>
 #include <utility>
@@ -30,6 +31,7 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
+#include "expressions/scalar/ScalarAttribute.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
@@ -63,6 +65,9 @@ namespace quickstep {
 
 namespace {
 
+typedef std::vector<std::pair<tuple_id, tuple_id>> VectorOfPairs;
+typedef std::pair<std::vector<tuple_id>, std::vector<tuple_id>> PairOfVectors;
+
 // Functor passed to HashTable::getAllFromValueAccessor() to collect matching
 // tuples from the inner relation. It stores matching tuple ID pairs
 // in an unordered_map keyed by inner block ID and a vector of
@@ -82,8 +87,7 @@ class VectorsOfPairsJoinedTuplesCollector {
   // key is inner block_id, values are vectors of joined tuple ID pairs with
   // tuple ID from the inner block on the left and the outer block on the
   // right.
-  inline std::unordered_map<block_id, std::vector<std::pair<tuple_id, tuple_id>>>*
-      getJoinedTuples() {
+  inline std::unordered_map<block_id, VectorOfPairs>* getJoinedTuples() {
     return &joined_tuples_;
   }
 
@@ -93,7 +97,7 @@ class VectorsOfPairsJoinedTuplesCollector {
   // cross-product of all tuples from both blocks, but simply using pairs of
   // tuple-IDs is expected to be more space efficient if the result set is less
   // than 1/64 the cardinality of the cross-product.
-  std::unordered_map<block_id, std::vector<std::pair<tuple_id, tuple_id>>> joined_tuples_;
+  std::unordered_map<block_id, VectorOfPairs> joined_tuples_;
 };
 
 // Another collector using an unordered_map keyed on inner block just like above,
@@ -106,15 +110,15 @@ class PairsOfVectorsJoinedTuplesCollector {
   template <typename ValueAccessorT>
   inline void operator()(const ValueAccessorT &accessor,
                          const TupleReference &tref) {
-    joined_tuples_[tref.block].first.push_back(tref.tuple);
-    joined_tuples_[tref.block].second.push_back(accessor.getCurrentPosition());
+    auto &entry = joined_tuples_[tref.block];
+    entry.first.emplace_back(tref.tuple);
+    entry.second.emplace_back(accessor.getCurrentPosition());
   }
 
   // Get a mutable pointer to the collected map of joined tuple ID pairs. The
   // key is inner block_id, value is a pair consisting of
   // inner block tuple IDs (first) and outer block tuple IDs (second).
-  inline std::unordered_map< block_id, std::pair<std::vector<tuple_id>, std::vector<tuple_id>>>*
-      getJoinedTuples() {
+  inline std::unordered_map<block_id, PairOfVectors>* getJoinedTuples() {
     return &joined_tuples_;
   }
 
@@ -466,7 +470,7 @@ void HashInnerJoinWorkOrder::execute() {
         base_accessor->createSharedTupleIdSequenceAdapterVirtual(*existence_map));
   }
 
-  PairsOfVectorsJoinedTuplesCollector collector;
+  VectorsOfPairsJoinedTuplesCollector collector;
   if (join_key_attributes_.size() == 1) {
     hash_table_.getAllFromValueAccessor(
         probe_accessor.get(),
@@ -484,14 +488,12 @@ void HashInnerJoinWorkOrder::execute() {
   const relation_id build_relation_id = build_relation_.getID();
   const relation_id probe_relation_id = probe_relation_.getID();
 
-  for (std::pair<const block_id, std::pair<std::vector<tuple_id>, std::vector<tuple_id>>>
+  for (std::pair<const block_id, VectorOfPairs>
            &build_block_entry : *collector.getJoinedTuples()) {
     BlockReference build_block =
         storage_manager_->getBlock(build_block_entry.first, build_relation_);
     const TupleStorageSubBlock &build_store = build_block->getTupleStorageSubBlock();
     std::unique_ptr<ValueAccessor> build_accessor(build_store.createValueAccessor());
-    const std::vector<tuple_id> &build_tids = build_block_entry.second.first;
-    const std::vector<tuple_id> &probe_tids = build_block_entry.second.second;
 
     // Evaluate '*residual_predicate_', if any.
     //
@@ -504,128 +506,35 @@ void HashInnerJoinWorkOrder::execute() {
     // hash join is below a reasonable threshold so that we don't blow up
     // temporary memory requirements to an unreasonable degree.
     if (residual_predicate_ != nullptr) {
-      std::pair<std::vector<tuple_id>, std::vector<tuple_id>> filtered_matches;
-      for (std::size_t i = 0; i < build_tids.size(); ++i) {
+      VectorOfPairs filtered_matches;
+
+      for (const std::pair<tuple_id, tuple_id> &hash_match
+           : build_block_entry.second) {
         if (residual_predicate_->matchesForJoinedTuples(*build_accessor,
                                                         build_relation_id,
-                                                        build_tids[i],
+                                                        hash_match.first,
                                                         *probe_accessor,
                                                         probe_relation_id,
-                                                        probe_tids[i])) {
-          filtered_matches.first.push_back(build_tids[i]);
-          filtered_matches.second.push_back(probe_tids[i]);
+                                                        hash_match.second)) {
+          filtered_matches.emplace_back(hash_match);
         }
       }
 
       build_block_entry.second = std::move(filtered_matches);
     }
 
-    // TODO(chasseur): If all the output expressions are ScalarAttributes,
-    // we could implement a similar fast-path to StorageBlock::selectSimple()
-    // that avoids a copy.
-    //
-    // TODO(chasseur): See TODO in NestedLoopsJoinOperator.cpp about limiting
-    // the size of materialized temporary results. In common usage, this
-    // probably won't be an issue for hash-joins, but in the worst case a hash
-    // join can still devolve into a cross-product.
-    //
-    // NOTE(chasseur): We could also create one big ColumnVectorsValueAccessor
-    // and accumulate all the results across multiple block pairs into it
-    // before inserting anything into output blocks, but this would require
-    // some significant API extensions to the expressions system for a dubious
-    // benefit (probably only a real performance win when there are very few
-    // matching tuples in each individual inner block but very many inner
-    // blocks with at least one match).
-
-    // We now create ordered value accessors for both build and probe side,
-    // using the joined tuple TIDs. Note that we have to use this Lambda-based
-    // invocation method here because the accessors don't have a virtual
-    // function that creates such an OrderedTupleIdSequenceAdapterValueAccessor.
-    std::unique_ptr<ValueAccessor> ordered_build_accessor, ordered_probe_accessor;
-    InvokeOnValueAccessorNotAdapter(
-        build_accessor.get(),
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-          ordered_build_accessor.reset(
-              accessor->createSharedOrderedTupleIdSequenceAdapter(build_tids));
-        });
-
-    if (probe_accessor->isTupleIdSequenceAdapter()) {
-      InvokeOnTupleIdSequenceAdapterValueAccessor(
-        probe_accessor.get(),
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-          ordered_probe_accessor.reset(
-            accessor->createSharedOrderedTupleIdSequenceAdapter(probe_tids));
-        });
-    } else {
-      InvokeOnValueAccessorNotAdapter(
-        probe_accessor.get(),
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-          ordered_probe_accessor.reset(
-            accessor->createSharedOrderedTupleIdSequenceAdapter(probe_tids));
-        });
-    }
-
-
-    // We also need a temp value accessor to store results of any scalar expressions.
     ColumnVectorsValueAccessor temp_result;
-
-    // Create a map of ValueAccessors and what attributes we want to pick from them
-    std::vector<std::pair<ValueAccessor *, std::vector<attribute_id>>> accessor_attribute_map;
-    const std::vector<ValueAccessor *> accessors{
-        ordered_build_accessor.get(), ordered_probe_accessor.get(), &temp_result};
-    const unsigned int build_index = 0, probe_index = 1, temp_index = 2;
-    for (auto &accessor : accessors) {
-      accessor_attribute_map.push_back(std::make_pair(
-          accessor,
-          std::vector<attribute_id>(selection_.size(), kInvalidCatalogId)));
+    for (auto selection_cit = selection_.begin();
+         selection_cit != selection_.end();
+         ++selection_cit) {
+      temp_result.addColumn((*selection_cit)->getAllValuesForJoin(build_relation_id,
+                                                                  build_accessor.get(),
+                                                                  probe_relation_id,
+                                                                  probe_accessor.get(),
+                                                                  build_block_entry.second));
     }
 
-    attribute_id dest_attr = 0;
-    std::vector<std::pair<tuple_id, tuple_id>> zipped_joined_tuple_ids;
-
-    for (auto &selection_cit : selection_) {
-      // If the Scalar (column) is not an attribute in build/probe blocks, then
-      // insert it into a ColumnVectorsValueAccessor.
-      if (selection_cit->getDataSource() != Scalar::ScalarDataSource::kAttribute) {
-        // Current destination attribute maps to the column we'll create now.
-        accessor_attribute_map[temp_index].second[dest_attr] = temp_result.getNumColumns();
-
-        if (temp_result.getNumColumns() == 0) {
-          // The getAllValuesForJoin function below needs joined tuple IDs as
-          // a vector of pair of (build-tuple-ID, probe-tuple-ID), and we have
-          // a pair of (build-tuple-IDs-vector, probe-tuple-IDs-vector). So
-          // we'll have to zip our two vectors together. We do this inside
-          // the loop because most queries don't exercise this code since
-          // they don't have scalar expressions with attributes from both
-          // build and probe relations (other expressions would have been
-          // pushed down to before the join).
-          zipped_joined_tuple_ids.reserve(build_tids.size());
-          for (std::size_t i = 0; i < build_tids.size(); ++i) {
-            zipped_joined_tuple_ids.push_back(std::make_pair(build_tids[i], probe_tids[i]));
-          }
-        }
-        temp_result.addColumn(
-            selection_cit
-                ->getAllValuesForJoin(build_relation_id, build_accessor.get(),
-                                      probe_relation_id, probe_accessor.get(),
-                                      zipped_joined_tuple_ids));
-      } else {
-        auto scalar_attr = static_cast<const ScalarAttribute *>(selection_cit.get());
-        const attribute_id attr_id = scalar_attr->getAttribute().getID();
-        if (scalar_attr->getAttribute().getParent().getID() == build_relation_id) {
-          accessor_attribute_map[build_index].second[dest_attr] = attr_id;
-        } else {
-          accessor_attribute_map[probe_index].second[dest_attr] = attr_id;
-        }
-      }
-      ++dest_attr;
-    }
-
-    // NOTE(chasseur): calling the bulk-insert method of InsertDestination once
-    // for each pair of joined blocks incurs some extra overhead that could be
-    // avoided by keeping checked-out MutableBlockReferences across iterations
-    // of this loop, but that would get messy when combined with partitioning.
-    output_destination_->bulkInsertTuplesFromValueAccessors(accessor_attribute_map);
+    output_destination_->bulkInsertTuples(&temp_result);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/26c3db42/storage/SplitRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreValueAccessor.hpp b/storage/SplitRowStoreValueAccessor.hpp
index 951a20a..46367b3 100644
--- a/storage/SplitRowStoreValueAccessor.hpp
+++ b/storage/SplitRowStoreValueAccessor.hpp
@@ -318,6 +318,11 @@ class SplitRowStoreValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/26c3db42/storage/ValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessor.hpp b/storage/ValueAccessor.hpp
index 654bbf9..f183efe 100644
--- a/storage/ValueAccessor.hpp
+++ b/storage/ValueAccessor.hpp
@@ -305,6 +305,21 @@ class ValueAccessor {
       const TupleIdSequence &id_sequence) = 0;
 
   /**
+   * @brief Create a new OrderedTupleIdSequenceAdapterValueAccessor that wraps
+   *        this ValueAccessor.
+   * @warning The newly-created adapter does NOT take ownership of this
+   *          ValueAccessor nor the provided OrderedTupleIdSequence. Both must
+   *          remain valid so long as the adapter will be used.
+   *
+   * @param id_sequence An OrderedTupleIdSequence specifying some subset of the
+   *        tuples for this ValueAccessor that the adapter will iterate over.
+   * @return A new OrderedTupleIdSequenceAdapterValueAccessor that will iterate
+   *         over only the tuples specified in id_sequence.
+   **/
+  virtual ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) = 0;
+
+  /**
    * @brief Get a TupleIdSequence indicating which positions this ValueAccessor
    *        is iterating over.
    *
@@ -512,6 +527,11 @@ class TupleIdSequenceAdapterValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }
@@ -718,6 +738,11 @@ class OrderedTupleIdSequenceAdapterValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }
@@ -944,6 +969,11 @@ class PackedTupleStorageSubBlockValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/26c3db42/types/containers/ColumnVectorsValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVectorsValueAccessor.hpp b/types/containers/ColumnVectorsValueAccessor.hpp
index fbbdc1b..6dc1124 100644
--- a/types/containers/ColumnVectorsValueAccessor.hpp
+++ b/types/containers/ColumnVectorsValueAccessor.hpp
@@ -290,6 +290,11 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
     return createSharedTupleIdSequenceAdapter(id_sequence);
   }
 
+  ValueAccessor* createSharedOrderedTupleIdSequenceAdapterVirtual(
+      const OrderedTupleIdSequence &id_sequence) override {
+    return createSharedOrderedTupleIdSequenceAdapter(id_sequence);
+  }
+
   const TupleIdSequence* getTupleIdSequenceVirtual() const override {
     return getTupleIdSequence();
   }


[06/12] incubator-quickstep git commit: Added optimizer support regarding hash partitions.

Posted by ji...@apache.org.
Added optimizer support regarding hash partitions.

  - CreateTable
  - Insert / InsertSelection
  - UpdateTable


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

Branch: refs/heads/reorder-attrs
Commit: 6c10e99f11f424926ab26fff1d1fd0e5c76e4fa7
Parents: bed1e26
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 10 17:00:08 2017 -0800
Committer: jianqiao <ji...@node-2.jianqiao.quickstep-pg0.wisc.cloudlab.us>
Committed: Wed Jan 18 14:31:01 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   2 +
 query_optimizer/ExecutionGenerator.cpp          |  46 ++++++--
 query_optimizer/logical/CMakeLists.txt          |   1 +
 query_optimizer/logical/CreateTable.hpp         |  27 ++++-
 query_optimizer/physical/CMakeLists.txt         |   1 +
 query_optimizer/physical/CreateTable.hpp        |  27 ++++-
 query_optimizer/resolver/CMakeLists.txt         |   3 +
 query_optimizer/resolver/Resolver.cpp           | 106 ++++++++++++++-----
 query_optimizer/resolver/Resolver.hpp           |  13 ++-
 query_optimizer/strategy/OneToOne.cpp           |   3 +-
 .../tests/execution_generator/CMakeLists.txt    |   6 ++
 .../tests/execution_generator/Create.test       |   9 ++
 .../tests/execution_generator/Partition.test    |  54 ++++++++++
 .../tests/logical_generator/Create.test         |  10 ++
 .../tests/physical_generator/Create.test        |  18 ++++
 query_optimizer/tests/resolver/Create.test      |  10 ++
 utility/PtrList.hpp                             |  16 +++
 17 files changed, 304 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 10c52a1..1a15271 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -63,6 +63,8 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_catalog_PartitionScheme
+                      quickstep_catalog_PartitionSchemeHeader
                       quickstep_expressions_Expressions_proto
                       quickstep_expressions_aggregation_AggregateFunction
                       quickstep_expressions_aggregation_AggregateFunction_proto

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 29e67f7..e0bfb3b 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -44,6 +44,8 @@
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationSchema.hpp"
 #include "catalog/CatalogTypedefs.hpp"
+#include "catalog/PartitionScheme.hpp"
+#include "catalog/PartitionSchemeHeader.hpp"
 #include "expressions/Expressions.pb.h"
 #include "expressions/aggregation/AggregateFunction.hpp"
 #include "expressions/aggregation/AggregateFunction.pb.h"
@@ -1017,6 +1019,11 @@ void ExecutionGenerator::convertCreateTable(
     catalog_relation->setDefaultStorageBlockLayout(layout.release());
   }
 
+  if (physical_plan->partition_scheme_header_proto()) {
+    catalog_relation->setPartitionScheme(new PartitionScheme(
+        PartitionSchemeHeader::ReconstructFromProto(*physical_plan->partition_scheme_header_proto())));
+  }
+
   execution_plan_->addRelationalOperator(
       new CreateTableOperator(query_handle_->query_id(),
                               catalog_relation.release(),
@@ -1139,14 +1146,21 @@ void ExecutionGenerator::convertInsertTuple(
       query_context_proto_->insert_destinations_size();
   S::InsertDestination *insert_destination_proto = query_context_proto_->add_insert_destinations();
 
-  insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
   insert_destination_proto->set_relation_id(input_relation.getID());
   insert_destination_proto->mutable_layout()->MergeFrom(
       input_relation.getDefaultStorageBlockLayout().getDescription());
 
-  const vector<block_id> blocks(input_relation.getBlocksSnapshot());
-  for (const block_id block : blocks) {
-    insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+  if (input_relation.hasPartitionScheme()) {
+    insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::PARTITION_AWARE);
+    insert_destination_proto->MutableExtension(S::PartitionAwareInsertDestination::partition_scheme)
+        ->MergeFrom(input_relation.getPartitionScheme()->getProto());
+  } else {
+    insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
+
+    const vector<block_id> blocks(input_relation.getBlocksSnapshot());
+    for (const block_id block : blocks) {
+      insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+    }
   }
 
   const QueryPlan::DAGNodeIndex insert_operator_index =
@@ -1197,15 +1211,21 @@ void ExecutionGenerator::convertInsertSelection(
   const QueryContext::insert_destination_id insert_destination_index =
       query_context_proto_->insert_destinations_size();
   S::InsertDestination *insert_destination_proto = query_context_proto_->add_insert_destinations();
-
-  insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
   insert_destination_proto->set_relation_id(destination_relation.getID());
   insert_destination_proto->mutable_layout()->MergeFrom(
       destination_relation.getDefaultStorageBlockLayout().getDescription());
 
-  const vector<block_id> blocks(destination_relation.getBlocksSnapshot());
-  for (const block_id block : blocks) {
-    insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+  if (destination_relation.hasPartitionScheme()) {
+    insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::PARTITION_AWARE);
+    insert_destination_proto->MutableExtension(S::PartitionAwareInsertDestination::partition_scheme)
+        ->MergeFrom(destination_relation.getPartitionScheme()->getProto());
+  } else {
+    insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
+
+    const vector<block_id> blocks(destination_relation.getBlocksSnapshot());
+    for (const block_id block : blocks) {
+      insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+    }
   }
 
   const CatalogRelationInfo *selection_relation_info =
@@ -1274,7 +1294,13 @@ void ExecutionGenerator::convertUpdateTable(
       query_context_proto_->insert_destinations_size();
   S::InsertDestination *relocation_destination_proto = query_context_proto_->add_insert_destinations();
 
-  relocation_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
+  if (input_relation->hasPartitionScheme()) {
+    relocation_destination_proto->set_insert_destination_type(S::InsertDestinationType::PARTITION_AWARE);
+    relocation_destination_proto->MutableExtension(S::PartitionAwareInsertDestination::partition_scheme)
+        ->MergeFrom(input_relation->getPartitionScheme()->getProto());
+  } else {
+    relocation_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
+  }
   relocation_destination_proto->set_relation_id(input_rel_id);
 
   // Convert the predicate proto.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/logical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CMakeLists.txt b/query_optimizer/logical/CMakeLists.txt
index c67f96f..8aca550 100644
--- a/query_optimizer/logical/CMakeLists.txt
+++ b/query_optimizer/logical/CMakeLists.txt
@@ -86,6 +86,7 @@ target_link_libraries(quickstep_queryoptimizer_logical_CreateIndex
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_logical_CreateTable
                       glog
+                      quickstep_catalog_Catalog_proto
                       quickstep_queryoptimizer_OptimizerTree
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_logical_Logical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/logical/CreateTable.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CreateTable.hpp b/query_optimizer/logical/CreateTable.hpp
index cc7c6d7..da4325d 100644
--- a/query_optimizer/logical/CreateTable.hpp
+++ b/query_optimizer/logical/CreateTable.hpp
@@ -24,6 +24,7 @@
 #include <string>
 #include <vector>
 
+#include "catalog/Catalog.pb.h"
 #include "query_optimizer/OptimizerTree.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/logical/Logical.hpp"
@@ -72,10 +73,17 @@ class CreateTable : public Logical {
     return block_properties_;
   }
 
+  /**
+   * @return Shared pointer to the serialized partition scheme header.
+   */
+  const std::shared_ptr<const serialization::PartitionSchemeHeader> partition_scheme_header_proto() const {
+    return partition_scheme_header_proto_;
+  }
+
   LogicalPtr copyWithNewChildren(
       const std::vector<LogicalPtr> &new_children) const override {
     DCHECK_EQ(getNumChildren(), new_children.size());
-    return Create(relation_name_, attributes_, block_properties_);
+    return Create(relation_name_, attributes_, block_properties_, partition_scheme_header_proto_);
   }
 
   std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override {
@@ -95,13 +103,19 @@ class CreateTable : public Logical {
    * @param block_properties The physical layout description of this block.
    *        Note that the pointer's ownership is assumed and shared by
    *        Logical::CreateTable and Physical::CreateTable.
+   * @param partition_scheme_header_proto The serialized partition scheme
+   *        header. It is 'nullptr' if no partitions specified. Note that the
+   *        pointer's ownership is shared by Logical::CreateTable and
+   *        Physical::CreateTable.
+   *
    * @return An immutable CreateTable node.
    */
   static CreateTablePtr Create(
       const std::string &relation_name,
       const std::vector<expressions::AttributeReferencePtr> &attributes,
-      const std::shared_ptr<const StorageBlockLayoutDescription> &block_properties) {
-    return CreateTablePtr(new CreateTable(relation_name, attributes, block_properties));
+      const std::shared_ptr<const StorageBlockLayoutDescription> &block_properties,
+      const std::shared_ptr<const serialization::PartitionSchemeHeader> &partition_scheme_header_proto) {
+    return CreateTablePtr(new CreateTable(relation_name, attributes, block_properties, partition_scheme_header_proto));
   }
 
  protected:
@@ -117,17 +131,20 @@ class CreateTable : public Logical {
   CreateTable(
       const std::string &relation_name,
       const std::vector<expressions::AttributeReferencePtr> &attributes,
-      const std::shared_ptr<const StorageBlockLayoutDescription> &block_properties)
+      const std::shared_ptr<const StorageBlockLayoutDescription> &block_properties,
+      const std::shared_ptr<const serialization::PartitionSchemeHeader> &partition_scheme_header_proto)
       : relation_name_(relation_name),
         attributes_(attributes),
         block_properties_(block_properties),
         block_properties_representation_(
-            getOptimizerRepresentationForProto<OptimizerTreeBaseNodePtr>(block_properties_.get())) {}
+            getOptimizerRepresentationForProto<OptimizerTreeBaseNodePtr>(block_properties_.get())),
+        partition_scheme_header_proto_(partition_scheme_header_proto) {}
 
   std::string relation_name_;
   std::vector<expressions::AttributeReferencePtr> attributes_;
   std::shared_ptr<const StorageBlockLayoutDescription> block_properties_;
   std::shared_ptr<const OptimizerProtoRepresentation<OptimizerTreeBaseNodePtr> > block_properties_representation_;
+  std::shared_ptr<const serialization::PartitionSchemeHeader> partition_scheme_header_proto_;
 
   DISALLOW_COPY_AND_ASSIGN(CreateTable);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/physical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CMakeLists.txt b/query_optimizer/physical/CMakeLists.txt
index 5c2cd0b..7f26943 100644
--- a/query_optimizer/physical/CMakeLists.txt
+++ b/query_optimizer/physical/CMakeLists.txt
@@ -85,6 +85,7 @@ target_link_libraries(quickstep_queryoptimizer_physical_CreateIndex
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_physical_CreateTable
                       glog
+                      quickstep_catalog_Catalog_proto
                       quickstep_queryoptimizer_OptimizerTree
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ExpressionUtil

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/physical/CreateTable.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CreateTable.hpp b/query_optimizer/physical/CreateTable.hpp
index 8e3bbd4..05eab0d 100644
--- a/query_optimizer/physical/CreateTable.hpp
+++ b/query_optimizer/physical/CreateTable.hpp
@@ -24,6 +24,7 @@
 #include <string>
 #include <vector>
 
+#include "catalog/Catalog.pb.h"
 #include "query_optimizer/OptimizerTree.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ExpressionUtil.hpp"
@@ -75,10 +76,17 @@ class CreateTable : public Physical {
     return block_properties_;
   }
 
+  /**
+   * @return Shared pointer to the serialized partition scheme header.
+   */
+  const std::shared_ptr<const serialization::PartitionSchemeHeader> partition_scheme_header_proto() const {
+    return partition_scheme_header_proto_;
+  }
+
   PhysicalPtr copyWithNewChildren(
       const std::vector<PhysicalPtr> &new_children) const override {
     DCHECK_EQ(getNumChildren(), new_children.size());
-    return Create(relation_name_, attributes_, block_properties_);
+    return Create(relation_name_, attributes_, block_properties_, partition_scheme_header_proto_);
   }
 
   std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override {
@@ -102,13 +110,19 @@ class CreateTable : public Physical {
    * @param relation_name The name of the relation to be inserted.
    * @param attributes Schema of the relation.
    * @param block_properties The optional proto message describing the block.
+   * @param partition_scheme_header_proto The serialized partition scheme
+   *        header. It is 'nullptr' if no partitions specified. Note that the
+   *        pointer's ownership is shared by Logical::CreateTable and
+   *        Physical::CreateTable.
+   *
    * @return An immutable CreateTable node.
    */
   static CreateTablePtr Create(
       const std::string &relation_name,
       const std::vector<expressions::AttributeReferencePtr> &attributes,
-      const std::shared_ptr<const StorageBlockLayoutDescription> &block_properties) {
-    return CreateTablePtr(new CreateTable(relation_name, attributes, block_properties));
+      const std::shared_ptr<const StorageBlockLayoutDescription> &block_properties,
+      const std::shared_ptr<const serialization::PartitionSchemeHeader> &partition_scheme_header_proto) {
+    return CreateTablePtr(new CreateTable(relation_name, attributes, block_properties, partition_scheme_header_proto));
   }
 
  protected:
@@ -124,17 +138,20 @@ class CreateTable : public Physical {
   CreateTable(
       const std::string &relation_name,
       const std::vector<expressions::AttributeReferencePtr> &attributes,
-      const std::shared_ptr<const StorageBlockLayoutDescription> &block_properties)
+      const std::shared_ptr<const StorageBlockLayoutDescription> &block_properties,
+      const std::shared_ptr<const serialization::PartitionSchemeHeader> &partition_scheme_header_proto)
       : relation_name_(relation_name),
         attributes_(attributes),
         block_properties_(block_properties),
         block_properties_representation_(
-            getOptimizerRepresentationForProto<OptimizerTreeBaseNodePtr>(block_properties_.get())) {}
+            getOptimizerRepresentationForProto<OptimizerTreeBaseNodePtr>(block_properties_.get())),
+        partition_scheme_header_proto_(partition_scheme_header_proto) {}
 
   std::string relation_name_;
   std::vector<expressions::AttributeReferencePtr> attributes_;
   std::shared_ptr<const StorageBlockLayoutDescription> block_properties_;
   std::shared_ptr<const OptimizerProtoRepresentation<OptimizerTreeBaseNodePtr> > block_properties_representation_;
+  std::shared_ptr<const serialization::PartitionSchemeHeader> partition_scheme_header_proto_;
 
   DISALLOW_COPY_AND_ASSIGN(CreateTable);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index 5251ccc..a34273e 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -34,6 +34,8 @@ target_link_libraries(quickstep_queryoptimizer_resolver_NameResolver
 target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       glog
                       quickstep_catalog_CatalogDatabase
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_catalog_Catalog_proto
                       quickstep_expressions_aggregation_AggregateFunction
                       quickstep_expressions_aggregation_AggregateFunctionFactory
                       quickstep_expressions_tablegenerator_GeneratorFunction
@@ -53,6 +55,7 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_parser_ParseLimit
                       quickstep_parser_ParseLiteralValue
                       quickstep_parser_ParseOrderBy
+                      quickstep_parser_ParsePartitionClause
                       quickstep_parser_ParsePredicate
                       quickstep_parser_ParsePredicateExists
                       quickstep_parser_ParsePredicateInTableQuery

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 2580342..df589fd 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -29,7 +29,9 @@
 #include <vector>
 #include <utility>
 
+#include "catalog/Catalog.pb.h"
 #include "catalog/CatalogDatabase.hpp"
+#include "catalog/CatalogTypedefs.hpp"
 #include "expressions/aggregation/AggregateFunction.hpp"
 #include "expressions/aggregation/AggregateFunctionFactory.hpp"
 #include "expressions/table_generator/GeneratorFunction.hpp"
@@ -49,6 +51,7 @@
 #include "parser/ParseLimit.hpp"
 #include "parser/ParseLiteralValue.hpp"
 #include "parser/ParseOrderBy.hpp"
+#include "parser/ParsePartitionClause.hpp"
 #include "parser/ParsePredicate.hpp"
 #include "parser/ParsePredicateExists.hpp"
 #include "parser/ParsePredicateInTableQuery.hpp"
@@ -129,12 +132,15 @@
 
 #include "glog/logging.h"
 
+using std::make_unique;
+
 namespace quickstep {
 namespace optimizer {
 namespace resolver {
 
 namespace E = ::quickstep::optimizer::expressions;
 namespace L = ::quickstep::optimizer::logical;
+namespace S = ::quickstep::serialization;
 
 struct Resolver::ExpressionResolutionInfo {
   /**
@@ -478,9 +484,32 @@ L::LogicalPtr Resolver::resolveCreateTable(
   std::shared_ptr<const StorageBlockLayoutDescription>
       block_properties(resolveBlockProperties(create_table_statement));
 
-  return L::CreateTable::Create(relation_name, attributes, block_properties);
+  std::shared_ptr<const S::PartitionSchemeHeader> partition_scheme_header_proto(
+      resolvePartitionClause(create_table_statement));
+
+  return L::CreateTable::Create(relation_name, attributes, block_properties, partition_scheme_header_proto);
+}
+
+namespace {
+
+attribute_id GetAttributeIdFromName(const PtrList<ParseAttributeDefinition> &attribute_definition_list,
+                                    const std::string &attribute_name) {
+  const std::string lower_attribute_name = ToLower(attribute_name);
+
+  attribute_id attr_id = 0;
+  for (const ParseAttributeDefinition &attribute_definition : attribute_definition_list) {
+    if (lower_attribute_name == ToLower(attribute_definition.name()->value())) {
+      return attr_id;
+    }
+
+    ++attr_id;
+  }
+
+  return kInvalidAttributeID;
 }
 
+}  // namespace
+
 StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
     const ParseStatementCreateTable &create_table_statement) {
   const ParseBlockProperties *block_properties
@@ -540,25 +569,6 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
     THROW_SQL_ERROR_AT(type_parse_string) << "Unrecognized storage type.";
   }
 
-  // Helper lambda function which will be used in COMPRESS and SORT resolution.
-  // Returns the column id from the name of the given attribute. Returns -1 if
-  // the attribute is not found.
-  auto columnIdFromAttributeName = [&create_table_statement](
-      const std::string& attribute_name) -> int {
-    const std::string search_name = ToLower(attribute_name);
-    int i = 0;
-    for (const ParseAttributeDefinition &attribute_definition :
-     create_table_statement.attribute_definition_list()) {
-      const std::string lower_attribute_name =
-        ToLower(attribute_definition.name()->value());
-      if (lower_attribute_name.compare(search_name) == 0) {
-        return i;
-      }
-      i++;
-    }
-    return -1;
-  };
-
   // Resolve the SORT property.
   const ParseString *sort_parse_string = block_properties->getSort();
   if (block_requires_sort) {
@@ -566,10 +576,10 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
       THROW_SQL_ERROR_AT(type_parse_string)
           << "The SORT property must be specified as an attribute name.";
     } else {
-      const std::string &sort_name = sort_parse_string->value();
       // Lookup the name and map to a column id.
-      int sort_id = columnIdFromAttributeName(sort_name);
-      if (sort_id == -1) {
+      const attribute_id sort_id = GetAttributeIdFromName(create_table_statement.attribute_definition_list(),
+                                                          sort_parse_string->value());
+      if (sort_id == kInvalidAttributeID) {
         THROW_SQL_ERROR_AT(sort_parse_string)
           << "The SORT property did not match any attribute name.";
       } else {
@@ -609,8 +619,9 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
           << "The COMPRESS property must be specified as ALL or a list of attributes.";
       }
       for (const ParseString &compressed_attribute_name : *compress_parse_strings) {
-        int column_id = columnIdFromAttributeName(compressed_attribute_name.value());
-        if (column_id == -1) {
+        const attribute_id column_id = GetAttributeIdFromName(create_table_statement.attribute_definition_list(),
+                                                              compressed_attribute_name.value());
+        if (column_id == kInvalidAttributeID) {
           THROW_SQL_ERROR_AT(&compressed_attribute_name)
               << "The given attribute was not found.";
         } else {
@@ -671,6 +682,51 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
   return storage_block_description.release();
 }
 
+const S::PartitionSchemeHeader* Resolver::resolvePartitionClause(
+    const ParseStatementCreateTable &create_table_statement) {
+  const ParsePartitionClause *partition_clause = create_table_statement.opt_partition_clause();
+  if (partition_clause == nullptr) {
+    return nullptr;
+  }
+
+  const ParseString *partition_type_string = partition_clause->partition_type();
+  if (partition_type_string == nullptr) {
+    THROW_SQL_ERROR_AT(partition_clause)
+        << "Partition type must be specified and be a string.";
+  }
+
+  const PtrList<ParseString> &attribute_name_list = partition_clause->attribute_name_list();
+  if (attribute_name_list.size() != 1) {
+    THROW_SQL_ERROR_AT(partition_clause)
+        << "Partition is supported on only one attribute.";
+  }
+
+  const ParseString &partition_attribute_name = attribute_name_list.front();
+  const attribute_id attr_id = GetAttributeIdFromName(create_table_statement.attribute_definition_list(),
+                                                      partition_attribute_name.value());
+  if (attr_id == kInvalidAttributeID) {
+    THROW_SQL_ERROR_AT(&partition_attribute_name)
+        << "The given attribute was not found.";
+  }
+
+  auto proto = make_unique<S::PartitionSchemeHeader>();
+  proto->set_num_partitions(partition_clause->num_partitions()->long_value());
+  proto->set_partition_attribute_id(attr_id);
+
+  const std::string partition_type = ToLower(partition_type_string->value());
+  if (partition_type == kHashPartitionType) {
+    proto->set_partition_type(S::PartitionSchemeHeader::HASH);
+  } else if (partition_type == kRangePartitionType) {
+    proto->set_partition_type(S::PartitionSchemeHeader::RANGE);
+    THROW_SQL_ERROR_AT(partition_clause)
+        << "Range partition is not supported.";
+  } else {
+    THROW_SQL_ERROR_AT(partition_type_string) << "Unrecognized partition type: " << partition_type;
+  }
+
+  return proto.release();
+}
+
 L::LogicalPtr Resolver::resolveCreateIndex(
     const ParseStatementCreateIndex &create_index_statement) {
   // Resolve relation reference.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/resolver/Resolver.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.hpp b/query_optimizer/resolver/Resolver.hpp
index 855e6ba..a3d0833 100644
--- a/query_optimizer/resolver/Resolver.hpp
+++ b/query_optimizer/resolver/Resolver.hpp
@@ -74,9 +74,8 @@ class PtrList;
 class StorageBlockLayoutDescription;
 class Type;
 
-}  // namespace quickstep
+namespace serialization { class PartitionSchemeHeader; }
 
-namespace quickstep {
 namespace optimizer {
 
 class OptimizerContext;
@@ -213,6 +212,16 @@ class Resolver {
       const ParseStatementCreateTable &create_table_statement);
 
   /**
+   * @brief Resolves the PARTITION clause of a CREATE TABLE statement to a
+   *        the serialized PartitionSchemeHeader describing the user input.
+   *
+   * @param create_table_statement The create table statement.
+   * @return A pointer to a user-owned serialized PartitionSchemeHeader.
+   */
+  const serialization::PartitionSchemeHeader* resolvePartitionClause(
+      const ParseStatementCreateTable &create_table_statement);
+
+  /**
    * @brief Resolves a DELETE query and returns a logical plan.
    *
    * @param delete_statement The DELETE parse tree.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/strategy/OneToOne.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/OneToOne.cpp b/query_optimizer/strategy/OneToOne.cpp
index 78003f4..7d0c4cb 100644
--- a/query_optimizer/strategy/OneToOne.cpp
+++ b/query_optimizer/strategy/OneToOne.cpp
@@ -121,7 +121,8 @@ bool OneToOne::generatePlan(const L::LogicalPtr &logical_input,
           std::static_pointer_cast<const L::CreateTable>(logical_input);
       *physical_output = P::CreateTable::Create(create_table->relation_name(),
                                                 create_table->attributes(),
-                                                create_table->block_properties());
+                                                create_table->block_properties(),
+                                                create_table->partition_scheme_header_proto());
       return true;
     }
     case L::LogicalType::kDeleteTuples: {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/tests/execution_generator/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/CMakeLists.txt b/query_optimizer/tests/execution_generator/CMakeLists.txt
index 1ea6a17..2705130 100644
--- a/query_optimizer/tests/execution_generator/CMakeLists.txt
+++ b/query_optimizer/tests/execution_generator/CMakeLists.txt
@@ -107,6 +107,11 @@ add_test(quickstep_queryoptimizer_tests_executiongenerator_join
          "${CMAKE_CURRENT_SOURCE_DIR}/Join.test"
          "${CMAKE_CURRENT_BINARY_DIR}/Join.test"
          "${CMAKE_CURRENT_BINARY_DIR}/Join/")
+add_test(quickstep_queryoptimizer_tests_executiongenerator_partition
+         "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"
+         "${CMAKE_CURRENT_SOURCE_DIR}/Partition.test"
+         "${CMAKE_CURRENT_BINARY_DIR}/Partition.test"
+         "${CMAKE_CURRENT_BINARY_DIR}/Partition/")
 add_test(quickstep_queryoptimizer_tests_executiongenerator_select
          "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"
          "${CMAKE_CURRENT_SOURCE_DIR}/Select.test"
@@ -150,6 +155,7 @@ file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Drop)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Index)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Insert)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Join)
+file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Partition)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Select)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/StringPatternMatching)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/TableGenerator)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/tests/execution_generator/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Create.test b/query_optimizer/tests/execution_generator/Create.test
index 5bd0e76..4ffa665 100644
--- a/query_optimizer/tests/execution_generator/Create.test
+++ b/query_optimizer/tests/execution_generator/Create.test
@@ -40,3 +40,12 @@ CREATE TABLE foo2 (col1 INT, col2 VARCHAR(80))
   WITH BLOCKPROPERTIES (TYPE columnstore, SORT col2);
 --
 ERROR: BLOCKPROPERTIES is invalid.
+==
+
+CREATE TABLE foo3 (attr INT) PARTITION BY HASH(attr) PARTITIONS 4;
+SELECT * FROM foo3;
+--
++-----------+
+|attr       |
++-----------+
++-----------+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/tests/execution_generator/Partition.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/Partition.test b/query_optimizer/tests/execution_generator/Partition.test
new file mode 100644
index 0000000..ab05391
--- /dev/null
+++ b/query_optimizer/tests/execution_generator/Partition.test
@@ -0,0 +1,54 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+CREATE TABLE foo (id INT NULL,
+                  name CHAR(20))
+PARTITION BY HASH(id) PARTITIONS 4;
+
+INSERT INTO foo
+SELECT int_col, char_col
+FROM test
+WHERE int_col > 0 OR int_col < 0;
+
+SELECT * FROM foo;
+--
++-----------+--------------------+
+|id         |name                |
++-----------+--------------------+
+|          4|          4 2.000000|
+|          8|          8 2.828427|
+|         12|         12 3.464102|
+|         16|         16 4.000000|
+|         24|         24 4.898979|
+|         -3|         -3 1.732051|
+|         -7|         -7 2.645751|
+|        -11|        -11 3.316625|
+|        -15|        -15 3.872983|
+|        -19|        -19 4.358899|
+|        -23|        -23 4.795832|
+|          2|          2 1.414214|
+|          6|          6 2.449490|
+|         14|         14 3.741657|
+|         18|         18 4.242641|
+|         22|         22 4.690416|
+|         -1|         -1 1.000000|
+|         -5|         -5 2.236068|
+|         -9|         -9 3.000000|
+|        -13|        -13 3.605551|
+|        -17|        -17 4.123106|
+|        -21|        -21 4.582576|
++-----------+--------------------+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/tests/logical_generator/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Create.test b/query_optimizer/tests/logical_generator/Create.test
index aeff9ec..04134f9 100644
--- a/query_optimizer/tests/logical_generator/Create.test
+++ b/query_optimizer/tests/logical_generator/Create.test
@@ -46,3 +46,13 @@ TopLevelPlan
 +-output_attributes=
   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
   +-AttributeReference[id=1,name=attr2,relation=foo,type=Int]
+==
+
+CREATE TABLE foo (attr INT) PARTITION BY HASH(attr) PARTITIONS 4;
+--
+TopLevelPlan
++-plan=CreateTable[relation=foo]
+| +-attributes=
+|   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=attr,relation=foo,type=Int]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/tests/physical_generator/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Create.test b/query_optimizer/tests/physical_generator/Create.test
index 54af3fa..c555371 100644
--- a/query_optimizer/tests/physical_generator/Create.test
+++ b/query_optimizer/tests/physical_generator/Create.test
@@ -115,3 +115,21 @@ TopLevelPlan
 |   +-AttributeReference[id=0,name=col1,relation=foo,type=Int]
 +-output_attributes=
   +-AttributeReference[id=0,name=col1,relation=foo,type=Int]
+==
+
+CREATE TABLE foo (attr INT) PARTITION BY HASH(attr) PARTITIONS 4;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=CreateTable[relation=foo]
+| +-attributes=
+|   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
+[Physical Plan]
+TopLevelPlan
++-plan=CreateTable[relation=foo]
+| +-attributes=
+|   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=attr,relation=foo,type=Int]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/query_optimizer/tests/resolver/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Create.test b/query_optimizer/tests/resolver/Create.test
index b04d785..28bd4f5 100644
--- a/query_optimizer/tests/resolver/Create.test
+++ b/query_optimizer/tests/resolver/Create.test
@@ -229,3 +229,13 @@ CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
 ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 23)
 (TYPE split_rowstore, BLOCKSIZEMB 2000);
                       ^
+==
+
+CREATE TABLE foo (attr INT) PARTITION BY HASH(attr) PARTITIONS 4;
+--
+TopLevelPlan
++-plan=CreateTable[relation=foo]
+| +-attributes=
+|   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]
++-output_attributes=
+  +-AttributeReference[id=0,name=attr,relation=foo,type=Int]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6c10e99f/utility/PtrList.hpp
----------------------------------------------------------------------
diff --git a/utility/PtrList.hpp b/utility/PtrList.hpp
index 5e34ec8..4707c26 100644
--- a/utility/PtrList.hpp
+++ b/utility/PtrList.hpp
@@ -218,6 +218,14 @@ class PtrList {
     return PtrListIterator(internal_list_.end());
   }
 
+  T& front() {
+    return *(internal_list_.front());
+  }
+
+  T& back() {
+    return *(internal_list_.back());
+  }
+
   const_iterator begin() const {
     return PtrListConstIterator(internal_list_.begin());
   }
@@ -226,6 +234,14 @@ class PtrList {
     return PtrListConstIterator(internal_list_.end());
   }
 
+  const T& front() const {
+    return *(internal_list_.front());
+  }
+
+  const T& back() const {
+    return *(internal_list_.back());
+  }
+
   void splice(iterator position, PtrList<T> &source) {  // NOLINT(runtime/references) - STL-style interface
     internal_list_.splice(position.internal_iterator_, source.internal_list_);
   }



[02/12] incubator-quickstep git commit: Removed the unused feedInputBlocks.

Posted by ji...@apache.org.
Removed the unused feedInputBlocks.


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

Branch: refs/heads/reorder-attrs
Commit: b0e5968158602f76ef037b748ed841aa153f23d5
Parents: 2229423
Author: Zuyu Zhang <zu...@apache.org>
Authored: Sun Jan 15 18:09:03 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jan 16 09:23:21 2017 -0800

----------------------------------------------------------------------
 .../tests/QueryManagerSingleNode_unittest.cpp   | 22 +-------------------
 relational_operators/AggregationOperator.hpp    |  6 ------
 relational_operators/BuildHashOperator.hpp      |  7 -------
 relational_operators/DeleteOperator.hpp         |  7 -------
 relational_operators/HashJoinOperator.hpp       |  8 -------
 .../NestedLoopsJoinOperator.cpp                 | 20 ------------------
 .../NestedLoopsJoinOperator.hpp                 | 13 +++++++++---
 relational_operators/RelationalOperator.hpp     | 14 -------------
 relational_operators/SampleOperator.hpp         |  6 ------
 relational_operators/SaveBlocksOperator.cpp     |  4 ----
 relational_operators/SaveBlocksOperator.hpp     | 10 ++-------
 relational_operators/SelectOperator.hpp         | 20 ------------------
 relational_operators/SortMergeRunOperator.hpp   | 11 ----------
 .../SortRunGenerationOperator.hpp               |  7 -------
 relational_operators/TableGeneratorOperator.hpp |  6 ------
 .../tests/SortMergeRunOperator_unittest.cpp     |  4 +++-
 16 files changed, 16 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/query_execution/tests/QueryManagerSingleNode_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManagerSingleNode_unittest.cpp b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
index 6ec6521..87b8934 100644
--- a/query_execution/tests/QueryManagerSingleNode_unittest.cpp
+++ b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
@@ -86,7 +86,6 @@ class MockOperator: public RelationalOperator {
  public:
   enum function_name {
     kFeedInputBlock = 0,
-    kFeedInputBlocks,
     kDoneFeedingInputBlocks,
     kGetAllWorkOrders
   };
@@ -103,7 +102,6 @@ class MockOperator: public RelationalOperator {
         num_calls_get_workorders_(0),
         num_workorders_generated_(0),
         num_calls_feedblock_(0),
-        num_calls_feedblocks_(0),
         num_calls_donefeedingblocks_(0) {
   }
 
@@ -123,8 +121,6 @@ class MockOperator: public RelationalOperator {
     switch (fname) {
       case kFeedInputBlock:
         return num_calls_feedblock_;
-      case kFeedInputBlocks:
-        return num_calls_feedblocks_;
       case kDoneFeedingInputBlocks:
         return num_calls_donefeedingblocks_;
       case kGetAllWorkOrders:
@@ -159,7 +155,7 @@ class MockOperator: public RelationalOperator {
     ++num_calls_get_workorders_;
     if (produce_workorders_) {
       if (has_streaming_input_) {
-        if ((num_calls_feedblock_ > 0 || num_calls_feedblocks_ > 0) && (num_workorders_generated_ < max_workorders_)) {
+        if (num_calls_feedblock_ > 0 && (num_workorders_generated_ < max_workorders_)) {
           MOCK_OP_LOG(3) << "[stream] generate WorkOrder";
           container->addNormalWorkOrder(new MockWorkOrder(op_index_), op_index_);
           ++num_workorders_generated_;
@@ -187,12 +183,6 @@ class MockOperator: public RelationalOperator {
     MOCK_OP_LOG(3) << "count(" << num_calls_feedblock_ << ")";
   }
 
-  void feedInputBlocks(const relation_id rel_id,
-                       std::vector<block_id> *partially_filled_blocks) override {
-    ++num_calls_feedblocks_;
-    MOCK_OP_LOG(3) << "count(" << num_calls_feedblocks_ << ")";
-  }
-
   void doneFeedingInputBlocks(const relation_id rel_id) override {
     ++num_calls_donefeedingblocks_;
     MOCK_OP_LOG(3) << "count(" << num_calls_donefeedingblocks_ << ")";
@@ -215,7 +205,6 @@ class MockOperator: public RelationalOperator {
   int num_calls_get_workorders_;
   int num_workorders_generated_;
   int num_calls_feedblock_;
-  int num_calls_feedblocks_;
   int num_calls_donefeedingblocks_;
 
   QueryContext::insert_destination_id insert_destination_index_ = QueryContext::kInvalidInsertDestinationId;
@@ -317,7 +306,6 @@ TEST_F(QueryManagerTest, SingleNodeDAGNoWorkOrdersTest) {
   // We expect one call for op's getAllWorkOrders().
   EXPECT_EQ(1, op.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlocks));
 }
 
 TEST_F(QueryManagerTest, SingleNodeDAGStaticWorkOrdersTest) {
@@ -336,7 +324,6 @@ TEST_F(QueryManagerTest, SingleNodeDAGStaticWorkOrdersTest) {
   // We expect one call for op's getAllWorkOrders().
   EXPECT_EQ(1, op.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlocks));
 
   // One workorder is generated.
   EXPECT_EQ(1, op.getNumWorkOrders());
@@ -422,7 +409,6 @@ TEST_F(QueryManagerTest, SingleNodeDAGDynamicWorkOrdersTest) {
 
   // We place this check in the end, since it's true throughout the test.
   EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op.getNumCalls(MockOperator::kFeedInputBlocks));
 }
 
 TEST_F(QueryManagerTest, TwoNodesDAGBlockingLinkTest) {
@@ -453,11 +439,9 @@ TEST_F(QueryManagerTest, TwoNodesDAGBlockingLinkTest) {
   // Only op1 should receive a call to getAllWorkOrders initially.
   EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlocks));
 
   EXPECT_EQ(0, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(0, op2.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op2.getNumCalls(MockOperator::kFeedInputBlocks));
 
   // Only op1 should produce a workorder.
   EXPECT_EQ(1, op1.getNumWorkOrders());
@@ -561,13 +545,11 @@ TEST_F(QueryManagerTest, TwoNodesDAGPipeLinkTest) {
   EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(1, op1.getNumWorkOrders());
   EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlocks));
 
   EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
   // op2 will generate workorder only after receiving a streaming input.
   EXPECT_EQ(0, op2.getNumWorkOrders());
   EXPECT_EQ(0, op2.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op2.getNumCalls(MockOperator::kFeedInputBlocks));
 
   unique_ptr<WorkerMessage> worker_message;
   worker_message.reset(query_manager_->getNextWorkerMessage(id1, -1));
@@ -590,11 +572,9 @@ TEST_F(QueryManagerTest, TwoNodesDAGPipeLinkTest) {
   // No additional call to op1's getAllWorkOrders.
   EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
   EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op1.getNumCalls(MockOperator::kFeedInputBlocks));
 
   // Output from op1 should be fed to op2.
   EXPECT_EQ(1, op2.getNumCalls(MockOperator::kFeedInputBlock));
-  EXPECT_EQ(0, op2.getNumCalls(MockOperator::kFeedInputBlocks));
 
   // A call to op2's getAllWorkOrders because of the streamed input.
   EXPECT_EQ(2, op2.getNumCalls(MockOperator::kGetAllWorkOrders));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index b5ed977..ce6015d 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -102,12 +102,6 @@ class AggregationOperator : public RelationalOperator {
     input_relation_block_ids_.push_back(input_block_id);
   }
 
-  void feedInputBlocks(const relation_id rel_id, std::vector<block_id> *partially_filled_blocks) override {
-    input_relation_block_ids_.insert(input_relation_block_ids_.end(),
-                                     partially_filled_blocks->begin(),
-                                     partially_filled_blocks->end());
-  }
-
  private:
   /**
    * @brief Create Work Order proto.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 0f96ef2..f0f42e3 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -119,13 +119,6 @@ class BuildHashOperator : public RelationalOperator {
     input_relation_block_ids_.push_back(input_block_id);
   }
 
-  void feedInputBlocks(const relation_id rel_id,
-                       std::vector<block_id> *partially_filled_blocks) override {
-    input_relation_block_ids_.insert(input_relation_block_ids_.end(),
-                                     partially_filled_blocks->begin(),
-                                     partially_filled_blocks->end());
-  }
-
  private:
   /**
    * @brief Create Work Order proto.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index 835434a..9c3f357 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -105,13 +105,6 @@ class DeleteOperator : public RelationalOperator {
     relation_block_ids_.push_back(input_block_id);
   }
 
-  void feedInputBlocks(const relation_id rel_id, std::vector<block_id> *partially_filled_blocks) override {
-    DCHECK(!relation_is_stored_);
-    relation_block_ids_.insert(relation_block_ids_.end(),
-                               partially_filled_blocks->begin(),
-                               partially_filled_blocks->end());
-  }
-
  private:
   /**
    * @brief Create Work Order proto.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 0ed1eeb..8829d1f 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -196,14 +196,6 @@ class HashJoinOperator : public RelationalOperator {
     probe_relation_block_ids_.push_back(input_block_id);
   }
 
-  void feedInputBlocks(const relation_id rel_id,
-                       std::vector<block_id> *partially_filled_blocks) override {
-    DCHECK(rel_id == probe_relation_.getID());
-    probe_relation_block_ids_.insert(probe_relation_block_ids_.end(),
-                                     partially_filled_blocks->begin(),
-                                     partially_filled_blocks->end());
-  }
-
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return output_destination_index_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/NestedLoopsJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.cpp b/relational_operators/NestedLoopsJoinOperator.cpp
index 9780c91..f17402f 100644
--- a/relational_operators/NestedLoopsJoinOperator.cpp
+++ b/relational_operators/NestedLoopsJoinOperator.cpp
@@ -48,26 +48,6 @@ using std::vector;
 
 namespace quickstep {
 
-void NestedLoopsJoinOperator::feedInputBlocks(const relation_id rel_id,
-                                              std::vector<block_id> *partially_filled_blocks) {
-  for (std::vector<block_id>::const_iterator it = partially_filled_blocks->begin();
-       it != partially_filled_blocks->end();
-       ++it) {
-    feedInputBlock(*it, rel_id);
-  }
-}
-
-void NestedLoopsJoinOperator::feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) {
-  if (input_relation_id == left_input_relation_.getID()) {
-    left_relation_block_ids_.push_back(input_block_id);
-  } else if (input_relation_id == right_input_relation_.getID()) {
-    right_relation_block_ids_.push_back(input_block_id);
-  } else {
-    FATAL_ERROR("The input block sent to the NestedLoopsJoinOperator belongs "
-                "to a different relation than the left and right relations");
-  }
-}
-
 bool NestedLoopsJoinOperator::getAllWorkOrders(
     WorkOrdersContainer *container,
     QueryContext *query_context,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index 3e3b049..951851d 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -141,9 +141,16 @@ class NestedLoopsJoinOperator : public RelationalOperator {
     }
   }
 
-  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override;
-
-  void feedInputBlocks(const relation_id rel_id, std::vector<block_id> *partially_filled_blocks) override;
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
+    if (input_relation_id == left_input_relation_.getID()) {
+      left_relation_block_ids_.push_back(input_block_id);
+    } else if (input_relation_id == right_input_relation_.getID()) {
+      right_relation_block_ids_.push_back(input_block_id);
+    } else {
+      LOG(FATAL) << "The input block sent to the NestedLoopsJoinOperator belongs "
+                 << "to a different relation than the left and right relations";
+    }
+  }
 
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return output_destination_index_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index 3eea189..a7f4177 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -145,20 +145,6 @@ class RelationalOperator {
                               const relation_id input_relation_id) {}
 
   /**
-   * @brief Receive partially filled input blocks for this RelationalOperator.
-   *
-   * @note The blocks in partially_filled_blocks may not be fully filled.
-   *
-   * @param rel_id ID of the relation to which all the partially filled blocks
-   *               belong to.
-   * @param partially_filled_blocks A pointer to the vector of block IDs of the
-   *                                partially filled input blocks.
-   **/
-  virtual void feedInputBlocks(
-      const relation_id rel_id,
-      std::vector<block_id> *partially_filled_blocks) {}
-
-  /**
    * @brief Signal the end of feeding of input blocks for this
    * RelationalOperator.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index d18b9a8..ccf6595 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -112,12 +112,6 @@ class SampleOperator : public RelationalOperator {
     input_relation_block_ids_.push_back(input_block_id);
   }
 
-  void feedInputBlocks(const relation_id rel_id, std::vector<block_id> *partially_filled_blocks) override {
-    input_relation_block_ids_.insert(input_relation_block_ids_.end(),
-                                     partially_filled_blocks->begin(),
-                                     partially_filled_blocks->end());
-  }
-
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return output_destination_index_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index a5a96c4..9d6c3f6 100644
--- a/relational_operators/SaveBlocksOperator.cpp
+++ b/relational_operators/SaveBlocksOperator.cpp
@@ -67,10 +67,6 @@ bool SaveBlocksOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *contain
   return done_feeding_input_relation_;
 }
 
-void SaveBlocksOperator::feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) {
-  destination_block_ids_.push_back(input_block_id);
-}
-
 void SaveBlocksOperator::updateCatalogOnCompletion() {
   // Note(jianqiao): We need to reset the exactness flag whenever a stored
   // relation gets changed. Given the pre-condition that all the data manipulation

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 573d81e..27fd911 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -83,14 +83,8 @@ class SaveBlocksOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override;
-
-  void feedInputBlocks(const relation_id rel_id, std::vector<block_id> *partially_filled_blocks) override {
-    for (std::vector<block_id>::const_iterator it = partially_filled_blocks->begin();
-         it != partially_filled_blocks->end();
-         ++it) {
-      feedInputBlock(*it, rel_id);
-    }
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
+    destination_block_ids_.push_back(input_block_id);
   }
 
   void updateCatalogOnCompletion() override;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 6741f45..5846eda 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -220,26 +220,6 @@ class SelectOperator : public RelationalOperator {
     }
   }
 
-  // TODO(gerald): Each call to getPartitionForBlock() involves grabbing shared
-  // locks on each partition's mutex, checking if the block belongs to the
-  // partition. Instead, we can provide a method getPartitionsForBlocks() which
-  // accepts a list of blocks and returns corresponding list of their partition IDs.
-  // Therefore, once we grab a lock for a partition, we search for all the blocks
-  // and then release the lock.
-  void feedInputBlocks(const relation_id rel_id, std::vector<block_id> *partially_filled_blocks) override {
-    if (input_relation_.hasPartitionScheme()) {
-      for (auto it = partially_filled_blocks->begin(); it != partially_filled_blocks->end(); ++it) {
-        const partition_id part_id = input_relation_.getPartitionScheme().getPartitionForBlock((*it));
-        input_relation_block_ids_in_partition_[part_id].insert(input_relation_block_ids_in_partition_[part_id].end(),
-                                                               *it);
-      }
-    } else {
-      input_relation_block_ids_.insert(input_relation_block_ids_.end(),
-                                       partially_filled_blocks->begin(),
-                                       partially_filled_blocks->end());
-    }
-  }
-
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return output_destination_index_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index 0839320..aff7d8d 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -152,17 +152,6 @@ class SortMergeRunOperator : public RelationalOperator {
     }
   }
 
-  void feedInputBlocks(
-      const relation_id rel_id,
-      std::vector<block_id> *partially_filled_blocks) override {
-    input_relation_block_ids_.insert(input_relation_block_ids_.end(),
-                                     partially_filled_blocks->begin(),
-                                     partially_filled_blocks->end());
-    if (started_) {
-      initializeInputRuns();
-    }
-  }
-
   void doneFeedingInputBlocks(const relation_id input_relation_id) override;
 
   void receiveFeedbackMessage(const WorkOrder::FeedbackMessage &msg) override;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index 125878c..a2ffb2b 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -129,13 +129,6 @@ class SortRunGenerationOperator : public RelationalOperator {
     input_relation_block_ids_.push_back(input_block_id);
   }
 
-  void feedInputBlocks(const relation_id rel_id, std::vector<block_id> *partially_filled_blocks) override {
-    DCHECK(rel_id == input_relation_.getID());
-    input_relation_block_ids_.insert(input_relation_block_ids_.end(),
-                                     partially_filled_blocks->begin(),
-                                     partially_filled_blocks->end());
-  }
-
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return output_destination_index_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index 7639966..c0930a4 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -90,12 +90,6 @@ class TableGeneratorOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
-  }
-
-  void feedInputBlocks(const relation_id rel_id, std::vector<block_id> *partially_filled_blocks) override {
-  }
-
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return output_destination_index_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b0e59681/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index a9128b1..74fecec 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -1601,7 +1601,9 @@ class SortMergeRunOperatorTest : public ::testing::Test {
 
     // Feed blocks.
     DVLOG(1) << "Feeding " << to_feed.size() << " blocks.";
-    merge_op_->feedInputBlocks(input_table_->getID(), &to_feed);
+    for (const block_id block : to_feed) {
+      merge_op_->feedInputBlock(block, input_table_->getID());
+    }
 
     // Remove fed blocks.
     blocks->erase(blocks->begin() + blocks->size() - count, blocks->end());


[11/12] incubator-quickstep git commit: Added the partitioned hash join in the distributed version.

Posted by ji...@apache.org.
Added the partitioned hash join in the distributed version.


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

Branch: refs/heads/reorder-attrs
Commit: 968ce3f75969817b3dcc16c0c1c3218817cfe9b8
Parents: 7bce0b8
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Jan 25 15:53:39 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Jan 25 16:41:35 2017 -0800

----------------------------------------------------------------------
 query_execution/CMakeLists.txt                  |   3 +
 query_execution/ForemanDistributed.cpp          |   7 +-
 query_execution/PolicyEnforcerDistributed.cpp   |   2 +
 query_execution/PolicyEnforcerDistributed.hpp   |  11 +-
 query_execution/QueryExecutionTypedefs.hpp      |   6 ++
 query_execution/QueryManagerDistributed.cpp     |  11 ++
 query_execution/QueryManagerDistributed.hpp     |  40 ++++---
 .../tests/execution_generator/CMakeLists.txt    |   6 ++
 relational_operators/BuildHashOperator.cpp      |   9 +-
 relational_operators/BuildHashOperator.hpp      |  18 ----
 relational_operators/CMakeLists.txt             |   1 +
 relational_operators/DestroyHashOperator.cpp    |   4 +-
 relational_operators/DestroyHashOperator.hpp    |   8 +-
 relational_operators/HashJoinOperator.cpp       |  31 +++---
 relational_operators/HashJoinOperator.hpp       | 105 ++++---------------
 relational_operators/WorkOrder.proto            |  10 +-
 relational_operators/WorkOrderFactory.cpp       |  10 --
 17 files changed, 115 insertions(+), 167 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 0f74384..c4c4079 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -156,6 +156,7 @@ if (ENABLE_DISTRIBUTED)
   target_link_libraries(quickstep_queryexecution_PolicyEnforcerDistributed
                         glog
                         quickstep_catalog_CatalogRelation
+                        quickstep_catalog_CatalogTypedefs
                         quickstep_catalog_Catalog_proto
                         quickstep_queryexecution_PolicyEnforcerBase
                         quickstep_queryexecution_QueryContext
@@ -252,7 +253,9 @@ target_link_libraries(quickstep_queryexecution_QueryManagerBase
                       quickstep_utility_Macros)
 if (ENABLE_DISTRIBUTED)
   target_link_libraries(quickstep_queryexecution_QueryManagerDistributed
+                        quickstep_catalog_CatalogTypedefs
                         quickstep_queryexecution_QueryContext
+                        quickstep_queryexecution_QueryContext_proto
                         quickstep_queryexecution_QueryExecutionMessages_proto
                         quickstep_queryexecution_QueryExecutionState
                         quickstep_queryexecution_QueryExecutionTypedefs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/query_execution/ForemanDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/ForemanDistributed.cpp b/query_execution/ForemanDistributed.cpp
index 0fa701d..fe4c483 100644
--- a/query_execution/ForemanDistributed.cpp
+++ b/query_execution/ForemanDistributed.cpp
@@ -278,23 +278,28 @@ bool ForemanDistributed::isHashJoinRelatedWorkOrder(const S::WorkOrderMessage &p
                                                     size_t *shiftboss_index_for_hash_join) {
   const S::WorkOrder &work_order_proto = proto.work_order();
   QueryContext::join_hash_table_id join_hash_table_index;
+  partition_id part_id;
 
   switch (work_order_proto.work_order_type()) {
     case S::BUILD_HASH:
       join_hash_table_index = work_order_proto.GetExtension(S::BuildHashWorkOrder::join_hash_table_index);
+      part_id = work_order_proto.GetExtension(S::BuildHashWorkOrder::partition_id);
       break;
     case S::HASH_JOIN:
       join_hash_table_index = work_order_proto.GetExtension(S::HashJoinWorkOrder::join_hash_table_index);
+      part_id = work_order_proto.GetExtension(S::HashJoinWorkOrder::partition_id);
       break;
     case S::DESTROY_HASH:
       join_hash_table_index = work_order_proto.GetExtension(S::DestroyHashWorkOrder::join_hash_table_index);
+      part_id = work_order_proto.GetExtension(S::DestroyHashWorkOrder::partition_id);
       break;
     default:
       return false;
   }
 
   static_cast<PolicyEnforcerDistributed*>(policy_enforcer_.get())->getShiftbossIndexForHashJoin(
-      proto.query_id(), join_hash_table_index, next_shiftboss_index_to_schedule, shiftboss_index_for_hash_join);
+      proto.query_id(), join_hash_table_index, part_id, next_shiftboss_index_to_schedule,
+      shiftboss_index_for_hash_join);
 
   return true;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/query_execution/PolicyEnforcerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.cpp b/query_execution/PolicyEnforcerDistributed.cpp
index c5642bc..e9f57d3 100644
--- a/query_execution/PolicyEnforcerDistributed.cpp
+++ b/query_execution/PolicyEnforcerDistributed.cpp
@@ -173,11 +173,13 @@ void PolicyEnforcerDistributed::getShiftbossIndexForAggregation(
 void PolicyEnforcerDistributed::getShiftbossIndexForHashJoin(
     const std::size_t query_id,
     const QueryContext::join_hash_table_id join_hash_table_index,
+    const partition_id part_id,
     const std::size_t next_shiftboss_index_to_schedule,
     std::size_t *shiftboss_index) {
   DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
   QueryManagerDistributed *query_manager = static_cast<QueryManagerDistributed*>(admitted_queries_[query_id].get());
   query_manager->getShiftbossIndexForHashJoin(join_hash_table_index,
+                                              part_id,
                                               next_shiftboss_index_to_schedule,
                                               shiftboss_index);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/query_execution/PolicyEnforcerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerDistributed.hpp b/query_execution/PolicyEnforcerDistributed.hpp
index 2c00a6b..248948a 100644
--- a/query_execution/PolicyEnforcerDistributed.hpp
+++ b/query_execution/PolicyEnforcerDistributed.hpp
@@ -21,6 +21,7 @@
 #include <utility>
 #include <vector>
 
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/PolicyEnforcerBase.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryExecutionMessages.pb.h"
@@ -124,19 +125,21 @@ class PolicyEnforcerDistributed final : public PolicyEnforcerBase {
 
   /**
    * @brief Get or set the index of Shiftboss for a HashJoin related WorkOrder.
-   * If it is the first BuildHash on <join_hash_table_index>, <shiftboss_index>
-   * will be set to <next_shiftboss_index_to_schedule>. Otherwise,
-   * <shiftboss_index> will be set to the index of the Shiftboss that has
-   * executed the first BuildHash.
+   * If it is the first BuildHash on <join_hash_table_index, part_id>,
+   * <shiftboss_index> will be set to <next_shiftboss_index_to_schedule>.
+   * Otherwise, <shiftboss_index> will be set to the index of the Shiftboss that
+   * has executed the first BuildHash.
    *
    * @param query_id The query id.
    * @param join_hash_table_index The Hash Table for the Join.
+   * @param part_id The partition ID.
    * @param next_shiftboss_index The index of Shiftboss to schedule a next WorkOrder.
    * @param shiftboss_index The index of Shiftboss to schedule the WorkOrder.
    **/
   void getShiftbossIndexForHashJoin(
       const std::size_t query_id,
       const QueryContext::join_hash_table_id join_hash_table_index,
+      const partition_id part_id,
       const std::size_t next_shiftboss_index_to_schedule,
       std::size_t *shiftboss_index);
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index faf2132..9f78302 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -62,6 +62,12 @@ using ClientIDMap = ThreadIDBasedMap<client_id,
                                      'a',
                                      'p'>;
 
+#ifdef QUICKSTEP_DISTRIBUTED
+
+constexpr std::size_t kInvalidShiftbossIndex = static_cast<std::size_t>(-1);
+
+#endif  // QUICKSTEP_DISTRIBUTED
+
 // We sort the following message types in the order of a life cycle of a query.
 enum QueryExecutionMessageType : message_type_id {
   kAdmitRequestMessage = 0,  // Requesting a query (or queries) to be admitted, from

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/query_execution/QueryManagerDistributed.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.cpp b/query_execution/QueryManagerDistributed.cpp
index 5c7e0d8..6ac96ab 100644
--- a/query_execution/QueryManagerDistributed.cpp
+++ b/query_execution/QueryManagerDistributed.cpp
@@ -23,8 +23,10 @@
 #include <cstdlib>
 #include <memory>
 #include <utility>
+#include <vector>
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/QueryContext.pb.h"
 #include "query_execution/QueryExecutionMessages.pb.h"
 #include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/QueryExecutionUtil.hpp"
@@ -45,6 +47,7 @@ using std::malloc;
 using std::move;
 using std::size_t;
 using std::unique_ptr;
+using std::vector;
 
 namespace quickstep {
 
@@ -65,6 +68,14 @@ QueryManagerDistributed::QueryManagerDistributed(QueryHandle *query_handle,
       processOperator(index, false);
     }
   }
+
+  const serialization::QueryContext &query_context_proto = query_handle->getQueryContextProto();
+  shiftboss_indexes_for_aggrs_.resize(query_context_proto.aggregation_states_size(), kInvalidShiftbossIndex);
+
+  for (int i = 0; i < query_context_proto.join_hash_tables_size(); ++i) {
+    shiftboss_indexes_for_hash_joins_.push_back(
+        vector<size_t>(query_context_proto.join_hash_tables(i).num_partitions(), kInvalidShiftbossIndex));
+  }
 }
 
 serialization::WorkOrderMessage* QueryManagerDistributed::getNextWorkOrderMessage(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/query_execution/QueryManagerDistributed.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerDistributed.hpp b/query_execution/QueryManagerDistributed.hpp
index 7a07fcb..631b15a 100644
--- a/query_execution/QueryManagerDistributed.hpp
+++ b/query_execution/QueryManagerDistributed.hpp
@@ -22,10 +22,12 @@
 
 #include <cstddef>
 #include <memory>
-#include <unordered_map>
+#include <vector>
 
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryExecutionState.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/QueryManagerBase.hpp"
 #include "query_execution/WorkOrderProtosContainer.hpp"
 #include "utility/Macros.hpp"
@@ -105,13 +107,12 @@ class QueryManagerDistributed final : public QueryManagerBase {
   void getShiftbossIndexForAggregation(const QueryContext::aggregation_state_id aggr_state_index,
                                        const std::size_t next_shiftboss_index_to_schedule,
                                        std::size_t *shiftboss_index) {
-    const auto cit = shiftboss_indexes_for_aggrs_.find(aggr_state_index);
-    if (cit != shiftboss_indexes_for_aggrs_.end()) {
-      *shiftboss_index = cit->second;
-    } else {
-      shiftboss_indexes_for_aggrs_.emplace(aggr_state_index, next_shiftboss_index_to_schedule);
-      *shiftboss_index = next_shiftboss_index_to_schedule;
+    DCHECK_LT(aggr_state_index, shiftboss_indexes_for_aggrs_.size());
+    if (shiftboss_indexes_for_aggrs_[aggr_state_index] == kInvalidShiftbossIndex) {
+      shiftboss_indexes_for_aggrs_[aggr_state_index] = next_shiftboss_index_to_schedule;
     }
+
+    *shiftboss_index = shiftboss_indexes_for_aggrs_[aggr_state_index];
   }
 
   /**
@@ -119,19 +120,22 @@ class QueryManagerDistributed final : public QueryManagerBase {
    * Shiftboss index is not found, set using <next_shiftboss_index_to_schedule>.
    *
    * @param join_hash_table_index The Hash Table for the Join.
+   * @param part_id The partition ID.
    * @param next_shiftboss_index The index of Shiftboss to schedule a next WorkOrder.
    * @param shiftboss_index The index of Shiftboss to schedule the WorkOrder.
    **/
   void getShiftbossIndexForHashJoin(const QueryContext::join_hash_table_id join_hash_table_index,
+                                    const partition_id part_id,
                                     const std::size_t next_shiftboss_index_to_schedule,
                                     std::size_t *shiftboss_index) {
-    const auto cit = shiftboss_indexes_for_hash_joins_.find(join_hash_table_index);
-    if (cit != shiftboss_indexes_for_hash_joins_.end()) {
-      *shiftboss_index = cit->second;
-    } else {
-      shiftboss_indexes_for_hash_joins_.emplace(join_hash_table_index, next_shiftboss_index_to_schedule);
-      *shiftboss_index = next_shiftboss_index_to_schedule;
+    DCHECK_LT(join_hash_table_index, shiftboss_indexes_for_hash_joins_.size());
+    DCHECK_LT(part_id, shiftboss_indexes_for_hash_joins_[join_hash_table_index].size());
+
+    if (shiftboss_indexes_for_hash_joins_[join_hash_table_index][part_id] == kInvalidShiftbossIndex) {
+      shiftboss_indexes_for_hash_joins_[join_hash_table_index][part_id] = next_shiftboss_index_to_schedule;
     }
+
+    *shiftboss_index = shiftboss_indexes_for_hash_joins_[join_hash_table_index][part_id];
   }
 
  private:
@@ -156,11 +160,13 @@ class QueryManagerDistributed final : public QueryManagerBase {
 
   std::unique_ptr<WorkOrderProtosContainer> normal_workorder_protos_container_;
 
-  // A map from an aggregation id to its scheduled Shiftboss index.
-  std::unordered_map<QueryContext::aggregation_state_id, std::size_t> shiftboss_indexes_for_aggrs_;
+  // From an aggregation id (QueryContext::aggregation_state_id) to its
+  // scheduled Shiftboss index.
+  std::vector<std::size_t> shiftboss_indexes_for_aggrs_;
 
-  // A map from a join hash table to its scheduled Shiftboss index.
-  std::unordered_map<QueryContext::join_hash_table_id, std::size_t> shiftboss_indexes_for_hash_joins_;
+  // Get the scheduled Shiftboss index given
+  // [QueryContext::join_hash_table_id][partition_id].
+  std::vector<std::vector<std::size_t>> shiftboss_indexes_for_hash_joins_;
 
   DISALLOW_COPY_AND_ASSIGN(QueryManagerDistributed);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/query_optimizer/tests/execution_generator/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/CMakeLists.txt b/query_optimizer/tests/execution_generator/CMakeLists.txt
index 2705130..d38f4aa 100644
--- a/query_optimizer/tests/execution_generator/CMakeLists.txt
+++ b/query_optimizer/tests/execution_generator/CMakeLists.txt
@@ -51,6 +51,11 @@ if (ENABLE_DISTRIBUTED)
            "${CMAKE_CURRENT_SOURCE_DIR}/Join.test"
            "${CMAKE_CURRENT_BINARY_DIR}/DistributedJoin.test"
            "${CMAKE_CURRENT_BINARY_DIR}/DistributedJoin/")
+  add_test(quickstep_queryoptimizer_tests_distributed_executiongenerator_partition
+           "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"
+           "${CMAKE_CURRENT_SOURCE_DIR}/Partition.test"
+           "${CMAKE_CURRENT_BINARY_DIR}/DistributedPartition.test"
+           "${CMAKE_CURRENT_BINARY_DIR}/DistributedPartition/")
   add_test(quickstep_queryoptimizer_tests_distributed_executiongenerator_select
            "../quickstep_queryoptimizer_tests_DistributedExecutionGeneratorTest"
            "${CMAKE_CURRENT_SOURCE_DIR}/Select.test"
@@ -146,6 +151,7 @@ if (ENABLE_DISTRIBUTED)
   file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/DistributedIndex)
   file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/DistributedInsert)
   file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/DistributedJoin)
+  file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/DistributedPartition)
   file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/DistributedSelect)
   file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/DistributedStringPatternMatching)
   file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/DistributedTableGenerator)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index 14ec204..8f40fbb 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -80,7 +80,7 @@ bool BuildHashOperator::getAllWorkOrders(
       for (const block_id block : input_relation_block_ids_[part_id]) {
         container->addNormalWorkOrder(
             new BuildHashWorkOrder(query_id_, input_relation_, join_key_attributes_, any_join_key_attributes_nullable_,
-                                   num_partitions_, part_id, block, hash_table, storage_manager,
+                                   part_id, block, hash_table, storage_manager,
                                    CreateLIPFilterBuilderHelper(lip_deployment_index_, query_context)),
             op_index_);
       }
@@ -94,9 +94,9 @@ bool BuildHashOperator::getAllWorkOrders(
              input_relation_block_ids_[part_id].size()) {
         container->addNormalWorkOrder(
             new BuildHashWorkOrder(query_id_, input_relation_, join_key_attributes_, any_join_key_attributes_nullable_,
-                                   num_partitions_, part_id,
-                                   input_relation_block_ids_[part_id][num_workorders_generated_[part_id]], hash_table,
-                                   storage_manager, CreateLIPFilterBuilderHelper(lip_deployment_index_, query_context)),
+                                   part_id, input_relation_block_ids_[part_id][num_workorders_generated_[part_id]],
+                                   hash_table, storage_manager,
+                                   CreateLIPFilterBuilderHelper(lip_deployment_index_, query_context)),
             op_index_);
         ++num_workorders_generated_[part_id];
       }
@@ -142,7 +142,6 @@ serialization::WorkOrder* BuildHashOperator::createWorkOrderProto(const block_id
   }
   proto->SetExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable,
                       any_join_key_attributes_nullable_);
-  proto->SetExtension(serialization::BuildHashWorkOrder::num_partitions, num_partitions_);
   proto->SetExtension(serialization::BuildHashWorkOrder::join_hash_table_index, hash_table_index_);
   proto->SetExtension(serialization::BuildHashWorkOrder::partition_id, part_id);
   proto->SetExtension(serialization::BuildHashWorkOrder::block_id, block);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index c9f29cb..273e3b1 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -173,8 +173,6 @@ class BuildHashWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'input_relation'. If no
-   *        partitions, it is one.
    * @param part_id The partition id of 'input_relation'.
    * @param build_block_id The block id.
    * @param hash_table The JoinHashTable to use.
@@ -185,7 +183,6 @@ class BuildHashWorkOrder : public WorkOrder {
                      const CatalogRelationSchema &input_relation,
                      const std::vector<attribute_id> &join_key_attributes,
                      const bool any_join_key_attributes_nullable,
-                     const std::size_t num_partitions,
                      const partition_id part_id,
                      const block_id build_block_id,
                      JoinHashTable *hash_table,
@@ -195,7 +192,6 @@ class BuildHashWorkOrder : public WorkOrder {
         input_relation_(input_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
         part_id_(part_id),
         build_block_id_(build_block_id),
         hash_table_(DCHECK_NOTNULL(hash_table)),
@@ -210,8 +206,6 @@ class BuildHashWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'input_relation'. If no
-   *        partitions, it is one.
    * @param part_id The partition id of 'input_relation'.
    * @param build_block_id The block id.
    * @param hash_table The JoinHashTable to use.
@@ -222,7 +216,6 @@ class BuildHashWorkOrder : public WorkOrder {
                      const CatalogRelationSchema &input_relation,
                      std::vector<attribute_id> &&join_key_attributes,
                      const bool any_join_key_attributes_nullable,
-                     const std::size_t num_partitions,
                      const partition_id part_id,
                      const block_id build_block_id,
                      JoinHashTable *hash_table,
@@ -232,7 +225,6 @@ class BuildHashWorkOrder : public WorkOrder {
         input_relation_(input_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
         part_id_(part_id),
         build_block_id_(build_block_id),
         hash_table_(DCHECK_NOTNULL(hash_table)),
@@ -248,15 +240,6 @@ class BuildHashWorkOrder : public WorkOrder {
   void execute() override;
 
   /**
-   * @brief Get the number of partitions.
-   *
-   * @return The number of partitions.
-   */
-  std::size_t num_partitions() const {
-    return num_partitions_;
-  }
-
-  /**
    * @brief Get the partition id.
    *
    * @return The partition id.
@@ -269,7 +252,6 @@ class BuildHashWorkOrder : public WorkOrder {
   const CatalogRelationSchema &input_relation_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
-  const std::size_t num_partitions_;
   const partition_id part_id_;
   const block_id build_block_id_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 66ea2d1..78da7b8 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -207,6 +207,7 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_catalog_CatalogRelationSchema
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_PartitionScheme
+                      quickstep_catalog_PartitionSchemeHeader
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_queryexecution_QueryContext

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/relational_operators/DestroyHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.cpp b/relational_operators/DestroyHashOperator.cpp
index 4827ef5..5b84bba 100644
--- a/relational_operators/DestroyHashOperator.cpp
+++ b/relational_operators/DestroyHashOperator.cpp
@@ -35,7 +35,7 @@ bool DestroyHashOperator::getAllWorkOrders(
     const tmb::client_id scheduler_client_id,
     tmb::MessageBus *bus) {
   if (blocking_dependencies_met_ && !work_generated_) {
-    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+    for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
       container->addNormalWorkOrder(
           new DestroyHashWorkOrder(query_id_, hash_table_index_, part_id, query_context),
           op_index_);
@@ -47,7 +47,7 @@ bool DestroyHashOperator::getAllWorkOrders(
 
 bool DestroyHashOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
   if (blocking_dependencies_met_ && !work_generated_) {
-    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+    for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
       serialization::WorkOrder *proto = new serialization::WorkOrder;
       proto->set_work_order_type(serialization::DESTROY_HASH);
       proto->set_query_id(query_id_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 8a1fabd..b707999 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -53,14 +53,14 @@ class DestroyHashOperator : public RelationalOperator {
    * @brief Constructor.
    *
    * @param query_id The ID of the query to which this operator belongs.
-   * @param num_partitions The number of partitions.
+   * @param build_num_partitions The number of partitions in 'build_relation'.
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
    **/
   DestroyHashOperator(const std::size_t query_id,
-                      const std::size_t num_partitions,
+                      const std::size_t build_num_partitions,
                       const QueryContext::join_hash_table_id hash_table_index)
       : RelationalOperator(query_id),
-        num_partitions_(num_partitions),
+        build_num_partitions_(build_num_partitions),
         hash_table_index_(hash_table_index),
         work_generated_(false) {}
 
@@ -79,7 +79,7 @@ class DestroyHashOperator : public RelationalOperator {
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
  private:
-  const std::size_t num_partitions_;
+  const std::size_t build_num_partitions_;
   const QueryContext::join_hash_table_id hash_table_index_;
   bool work_generated_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 0062b93..fd3841f 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -213,15 +213,15 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
         return true;
       }
 
-      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
         const JoinHashTable &hash_table =
             *(query_context->getJoinHashTable(hash_table_index_, part_id));
 
         for (const block_id probe_block_id : probe_relation_block_ids_[part_id]) {
           container->addNormalWorkOrder(
               new JoinWorkOrderClass(query_id_, build_relation_, probe_relation_, join_key_attributes_,
-                                     any_join_key_attributes_nullable_, num_partitions_, part_id, probe_block_id,
-                                     residual_predicate, selection, hash_table, output_destination, storage_manager,
+                                     any_join_key_attributes_nullable_, part_id, probe_block_id, residual_predicate,
+                                     selection, hash_table, output_destination, storage_manager,
                                      CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
               op_index_);
         }
@@ -229,14 +229,14 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
       started_ = true;
       return true;
     } else {
-      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
         const JoinHashTable &hash_table =
             *(query_context->getJoinHashTable(hash_table_index_, part_id));
 
         while (num_workorders_generated_[part_id] < probe_relation_block_ids_[part_id].size()) {
           container->addNormalWorkOrder(
               new JoinWorkOrderClass(query_id_, build_relation_, probe_relation_, join_key_attributes_,
-                                     any_join_key_attributes_nullable_, num_partitions_, part_id,
+                                     any_join_key_attributes_nullable_, part_id,
                                      probe_relation_block_ids_[part_id][num_workorders_generated_[part_id]],
                                      residual_predicate, selection, hash_table, output_destination, storage_manager,
                                      CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
@@ -269,16 +269,15 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
         return true;
       }
 
-      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
         const JoinHashTable &hash_table =
             *(query_context->getJoinHashTable(hash_table_index_, part_id));
 
         for (const block_id probe_block_id : probe_relation_block_ids_[part_id]) {
           container->addNormalWorkOrder(
               new HashOuterJoinWorkOrder(query_id_, build_relation_, probe_relation_, join_key_attributes_,
-                                         any_join_key_attributes_nullable_, num_partitions_, part_id, probe_block_id,
-                                         selection, is_selection_on_build_, hash_table, output_destination,
-                                         storage_manager,
+                                         any_join_key_attributes_nullable_, part_id, probe_block_id, selection,
+                                         is_selection_on_build_, hash_table, output_destination, storage_manager,
                                          CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context)),
               op_index_);
         }
@@ -286,14 +285,14 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
       started_ = true;
       return true;
     } else {
-      for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+      for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
         const JoinHashTable &hash_table =
             *(query_context->getJoinHashTable(hash_table_index_, part_id));
 
         while (num_workorders_generated_[part_id] < probe_relation_block_ids_[part_id].size()) {
           container->addNormalWorkOrder(
               new HashOuterJoinWorkOrder(query_id_, build_relation_, probe_relation_, join_key_attributes_,
-                                         any_join_key_attributes_nullable_, num_partitions_, part_id,
+                                         any_join_key_attributes_nullable_, part_id,
                                          probe_relation_block_ids_[part_id][num_workorders_generated_[part_id]],
                                          selection, is_selection_on_build_, hash_table, output_destination,
                                          storage_manager,
@@ -336,7 +335,7 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrderProtos(
       return true;
     }
 
-    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+    for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
       for (const block_id probe_block_id : probe_relation_block_ids_[part_id]) {
         container->addWorkOrderProto(
             createNonOuterJoinWorkOrderProto(hash_join_type, probe_block_id, part_id),
@@ -346,7 +345,7 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrderProtos(
     started_ = true;
     return true;
   } else {
-    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+    for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
       while (num_workorders_generated_[part_id] < probe_relation_block_ids_[part_id].size()) {
         container->addWorkOrderProto(
             createNonOuterJoinWorkOrderProto(hash_join_type,
@@ -376,7 +375,6 @@ serialization::WorkOrder* HashJoinOperator::createNonOuterJoinWorkOrderProto(
   }
   proto->SetExtension(serialization::HashJoinWorkOrder::any_join_key_attributes_nullable,
                       any_join_key_attributes_nullable_);
-  proto->SetExtension(serialization::HashJoinWorkOrder::num_partitions, num_partitions_);
   proto->SetExtension(serialization::HashJoinWorkOrder::insert_destination_index, output_destination_index_);
   proto->SetExtension(serialization::HashJoinWorkOrder::join_hash_table_index, hash_table_index_);
   proto->SetExtension(serialization::HashJoinWorkOrder::partition_id, part_id);
@@ -399,7 +397,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *
       return true;
     }
 
-    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+    for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
       for (const block_id probe_block_id : probe_relation_block_ids_[part_id]) {
         container->addWorkOrderProto(createOuterJoinWorkOrderProto(probe_block_id, part_id), op_index_);
       }
@@ -407,7 +405,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *
     started_ = true;
     return true;
   } else {
-    for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+    for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
       while (num_workorders_generated_[part_id] < probe_relation_block_ids_[part_id].size()) {
         container->addWorkOrderProto(
             createOuterJoinWorkOrderProto(probe_relation_block_ids_[part_id][num_workorders_generated_[part_id]],
@@ -436,7 +434,6 @@ serialization::WorkOrder* HashJoinOperator::createOuterJoinWorkOrderProto(const
   }
   proto->SetExtension(serialization::HashJoinWorkOrder::any_join_key_attributes_nullable,
                       any_join_key_attributes_nullable_);
-  proto->SetExtension(serialization::HashJoinWorkOrder::num_partitions, num_partitions_);
   proto->SetExtension(serialization::HashJoinWorkOrder::insert_destination_index, output_destination_index_);
   proto->SetExtension(serialization::HashJoinWorkOrder::join_hash_table_index, hash_table_index_);
   proto->SetExtension(serialization::HashJoinWorkOrder::selection_index, selection_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index e655f70..acfe3d2 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -29,6 +29,7 @@
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogTypedefs.hpp"
 #include "catalog/PartitionScheme.hpp"
+#include "catalog/PartitionSchemeHeader.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.hpp"
@@ -101,8 +102,8 @@ class HashJoinOperator : public RelationalOperator {
    * @param join_key_attributes The IDs of equijoin attributes in
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'input_relation'. If no
-   *        partitions, it is one.
+   * @param build_num_partitions The number of partitions in 'build_relation'.
+   *        If no partitions, it is one.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert the join results.
@@ -128,7 +129,7 @@ class HashJoinOperator : public RelationalOperator {
       const bool probe_relation_is_stored,
       const std::vector<attribute_id> &join_key_attributes,
       const bool any_join_key_attributes_nullable,
-      const std::size_t num_partitions,
+      const std::size_t build_num_partitions,
       const CatalogRelation &output_relation,
       const QueryContext::insert_destination_id output_destination_index,
       const QueryContext::join_hash_table_id hash_table_index,
@@ -142,7 +143,7 @@ class HashJoinOperator : public RelationalOperator {
         probe_relation_is_stored_(probe_relation_is_stored),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
+        build_num_partitions_(build_num_partitions),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         hash_table_index_(hash_table_index),
@@ -152,8 +153,8 @@ class HashJoinOperator : public RelationalOperator {
                                    ? std::vector<bool>()
                                    : *is_selection_on_build),
         join_type_(join_type),
-        probe_relation_block_ids_(num_partitions),
-        num_workorders_generated_(num_partitions),
+        probe_relation_block_ids_(build_num_partitions),
+        num_workorders_generated_(build_num_partitions),
         started_(false) {
     DCHECK(join_type != JoinType::kLeftOuterJoin ||
                (is_selection_on_build != nullptr &&
@@ -162,12 +163,15 @@ class HashJoinOperator : public RelationalOperator {
     if (probe_relation_is_stored) {
       if (probe_relation.hasPartitionScheme()) {
         const PartitionScheme &part_scheme = *probe_relation.getPartitionScheme();
-        for (std::size_t part_id = 0; part_id < num_partitions_; ++part_id) {
+        DCHECK_EQ(build_num_partitions_, part_scheme.getPartitionSchemeHeader().getNumPartitions());
+        for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
           probe_relation_block_ids_[part_id] = part_scheme.getBlocksInPartition(part_id);
         }
       } else {
-        // No partition.
-        probe_relation_block_ids_[0] = probe_relation.getBlocksSnapshot();
+        // Broadcast hash join if probe has no partitions.
+        for (std::size_t part_id = 0; part_id < build_num_partitions_; ++part_id) {
+          probe_relation_block_ids_[part_id] = probe_relation.getBlocksSnapshot();
+        }
       }
     }
   }
@@ -209,7 +213,14 @@ class HashJoinOperator : public RelationalOperator {
                       const partition_id part_id) override {
     DCHECK_EQ(probe_relation_.getID(), input_relation_id);
 
-    probe_relation_block_ids_[part_id].push_back(input_block_id);
+    if (probe_relation_.hasPartitionScheme()) {
+      probe_relation_block_ids_[part_id].push_back(input_block_id);
+    } else {
+      // Broadcast hash join if probe has no partitions.
+      for (std::size_t build_part_id = 0; build_part_id < build_num_partitions_; ++build_part_id) {
+        probe_relation_block_ids_[build_part_id].push_back(input_block_id);
+      }
+    }
   }
 
   QueryContext::insert_destination_id getInsertDestinationID() const override {
@@ -261,7 +272,7 @@ class HashJoinOperator : public RelationalOperator {
   const bool probe_relation_is_stored_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
-  const std::size_t num_partitions_;
+  const std::size_t build_num_partitions_;
   const CatalogRelation &output_relation_;
   const QueryContext::insert_destination_id output_destination_index_;
   const QueryContext::join_hash_table_id hash_table_index_;
@@ -295,8 +306,6 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'probe_relation'. If no
-   *        partitions, it is one.
    * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
@@ -317,7 +326,6 @@ class HashInnerJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       const std::vector<attribute_id> &join_key_attributes,
       const bool any_join_key_attributes_nullable,
-      const std::size_t num_partitions,
       const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
@@ -331,7 +339,6 @@ class HashInnerJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
         part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
@@ -352,8 +359,6 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'probe_relation'. If no
-   *        partitions, it is one.
    * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
@@ -374,7 +379,6 @@ class HashInnerJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       std::vector<attribute_id> &&join_key_attributes,
       const bool any_join_key_attributes_nullable,
-      const std::size_t num_partitions,
       const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
@@ -388,7 +392,6 @@ class HashInnerJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
         part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
@@ -411,15 +414,6 @@ class HashInnerJoinWorkOrder : public WorkOrder {
   void execute() override;
 
   /**
-   * @brief Get the number of partitions.
-   *
-   * @return The number of partitions.
-   */
-  std::size_t num_partitions() const {
-    return num_partitions_;
-  }
-
-  /**
    * @brief Get the partition id.
    *
    * @return The partition id.
@@ -433,7 +427,6 @@ class HashInnerJoinWorkOrder : public WorkOrder {
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
-  const std::size_t num_partitions_;
   const partition_id part_id_;
   const block_id block_id_;
   const Predicate *residual_predicate_;
@@ -465,8 +458,6 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'probe_relation'. If no
-   *        partitions, it is one.
    * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
@@ -487,7 +478,6 @@ class HashSemiJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       const std::vector<attribute_id> &join_key_attributes,
       const bool any_join_key_attributes_nullable,
-      const std::size_t num_partitions,
       const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
@@ -501,7 +491,6 @@ class HashSemiJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
         part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
@@ -522,8 +511,6 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'probe_relation'. If no
-   *        partitions, it is one.
    * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
@@ -544,7 +531,6 @@ class HashSemiJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       std::vector<attribute_id> &&join_key_attributes,
       const bool any_join_key_attributes_nullable,
-      const std::size_t num_partitions,
       const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
@@ -558,7 +544,6 @@ class HashSemiJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
         part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
@@ -573,15 +558,6 @@ class HashSemiJoinWorkOrder : public WorkOrder {
   void execute() override;
 
   /**
-   * @brief Get the number of partitions.
-   *
-   * @return The number of partitions.
-   */
-  std::size_t num_partitions() const {
-    return num_partitions_;
-  }
-
-  /**
    * @brief Get the partition id.
    *
    * @return The partition id.
@@ -599,7 +575,6 @@ class HashSemiJoinWorkOrder : public WorkOrder {
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
-  const std::size_t num_partitions_;
   const partition_id part_id_;
   const block_id block_id_;
   const Predicate *residual_predicate_;
@@ -631,8 +606,6 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'probe_relation'. If no
-   *        partitions, it is one.
    * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
@@ -653,7 +626,6 @@ class HashAntiJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       const std::vector<attribute_id> &join_key_attributes,
       const bool any_join_key_attributes_nullable,
-      const std::size_t num_partitions,
       const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
@@ -667,7 +639,6 @@ class HashAntiJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
         part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
@@ -688,8 +659,6 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'probe_relation'. If no
-   *        partitions, it is one.
    * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param residual_predicate If non-null, apply as an additional filter to
@@ -710,7 +679,6 @@ class HashAntiJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       std::vector<attribute_id> &&join_key_attributes,
       const bool any_join_key_attributes_nullable,
-      const std::size_t num_partitions,
       const partition_id part_id,
       const block_id lookup_block_id,
       const Predicate *residual_predicate,
@@ -724,7 +692,6 @@ class HashAntiJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
         part_id_(part_id),
         block_id_(lookup_block_id),
         residual_predicate_(residual_predicate),
@@ -745,15 +712,6 @@ class HashAntiJoinWorkOrder : public WorkOrder {
   }
 
   /**
-   * @brief Get the number of partitions.
-   *
-   * @return The number of partitions.
-   */
-  std::size_t num_partitions() const {
-    return num_partitions_;
-  }
-
-  /**
    * @brief Get the partition id.
    *
    * @return The partition id.
@@ -771,7 +729,6 @@ class HashAntiJoinWorkOrder : public WorkOrder {
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
-  const std::size_t num_partitions_;
   const partition_id part_id_;
   const block_id block_id_;
   const Predicate *residual_predicate_;
@@ -802,8 +759,6 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'probe_relation'. If no
-   *        partitions, it is one.
    * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param selection A list of Scalars corresponding to the relation attributes
@@ -823,7 +778,6 @@ class HashOuterJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       const std::vector<attribute_id> &join_key_attributes,
       const bool any_join_key_attributes_nullable,
-      const std::size_t num_partitions,
       const partition_id part_id,
       const block_id lookup_block_id,
       const std::vector<std::unique_ptr<const Scalar>> &selection,
@@ -837,7 +791,6 @@ class HashOuterJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
         part_id_(part_id),
         block_id_(lookup_block_id),
         selection_(selection),
@@ -858,8 +811,6 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param num_partitions The number of partitions in 'probe_relation'. If no
-   *        partitions, it is one.
    * @param part_id The partition id of 'probe_relation'.
    * @param lookup_block_id The block id of the probe_relation.
    * @param selection A list of Scalars corresponding to the relation attributes
@@ -878,7 +829,6 @@ class HashOuterJoinWorkOrder : public WorkOrder {
       const CatalogRelationSchema &probe_relation,
       std::vector<attribute_id> &&join_key_attributes,
       const bool any_join_key_attributes_nullable,
-      const std::size_t num_partitions,
       const partition_id part_id,
       const block_id lookup_block_id,
       const std::vector<std::unique_ptr<const Scalar>> &selection,
@@ -892,7 +842,6 @@ class HashOuterJoinWorkOrder : public WorkOrder {
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
-        num_partitions_(num_partitions),
         part_id_(part_id),
         block_id_(lookup_block_id),
         selection_(selection),
@@ -907,15 +856,6 @@ class HashOuterJoinWorkOrder : public WorkOrder {
   void execute() override;
 
   /**
-   * @brief Get the number of partitions.
-   *
-   * @return The number of partitions.
-   */
-  std::size_t num_partitions() const {
-    return num_partitions_;
-  }
-
-  /**
    * @brief Get the partition id.
    *
    * @return The partition id.
@@ -929,7 +869,6 @@ class HashOuterJoinWorkOrder : public WorkOrder {
   const CatalogRelationSchema &probe_relation_;
   const std::vector<attribute_id> join_key_attributes_;
   const bool any_join_key_attributes_nullable_;
-  const std::size_t num_partitions_;
   const partition_id part_id_;
   const block_id block_id_;
   const std::vector<std::unique_ptr<const Scalar>> &selection_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index b914fce..f8d9246 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -63,16 +63,15 @@ message AggregationWorkOrder {
   }
 }
 
-// Next tag: 40.
+// Next tag: 39.
 message BuildHashWorkOrder {
   extend WorkOrder {
     // All required.
     optional int32 relation_id = 32;
     repeated int32 join_key_attributes = 33;
     optional bool any_join_key_attributes_nullable = 34;
-    optional uint64 num_partitions = 38;
     optional uint32 join_hash_table_index = 35;
-    optional uint64 partition_id = 39;
+    optional uint64 partition_id = 38;
     optional fixed64 block_id = 36;
     optional int32 lip_deployment_index = 37;
   }
@@ -113,7 +112,7 @@ message FinalizeAggregationWorkOrder {
   }
 }
 
-// Next tag: 174.
+// Next tag: 173.
 message HashJoinWorkOrder {
   enum HashJoinWorkOrderType {
     HASH_ANTI_JOIN = 0;
@@ -129,10 +128,9 @@ message HashJoinWorkOrder {
     optional int32 probe_relation_id = 162;
     repeated int32 join_key_attributes = 163;
     optional bool any_join_key_attributes_nullable = 164;
-    optional uint64 num_partitions = 172;
     optional int32 insert_destination_index = 165;
     optional uint32 join_hash_table_index = 166;
-    optional uint64 partition_id = 173;
+    optional uint64 partition_id = 172;
     optional int32 selection_index = 167;
     optional fixed64 block_id = 168;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/968ce3f7/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index c09bcbe..a6cba02 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -107,7 +107,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::BuildHashWorkOrder::relation_id)),
           move(join_key_attributes),
           proto.GetExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable),
-          proto.GetExtension(serialization::BuildHashWorkOrder::num_partitions),
           part_id,
           proto.GetExtension(serialization::BuildHashWorkOrder::block_id),
           query_context->getJoinHashTable(
@@ -199,9 +198,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       const block_id lookup_block_id =
           proto.GetExtension(serialization::HashJoinWorkOrder::block_id);
 
-      const std::size_t num_partitions =
-          proto.GetExtension(serialization::HashJoinWorkOrder::num_partitions);
-
       const Predicate *residual_predicate = nullptr;
       if (hash_join_work_order_type != serialization::HashJoinWorkOrder::HASH_OUTER_JOIN) {
         residual_predicate =
@@ -233,7 +229,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               probe_relation,
               move(join_key_attributes),
               any_join_key_attributes_nullable,
-              num_partitions,
               part_id,
               lookup_block_id,
               residual_predicate,
@@ -251,7 +246,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               probe_relation,
               move(join_key_attributes),
               any_join_key_attributes_nullable,
-              num_partitions,
               part_id,
               lookup_block_id,
               residual_predicate,
@@ -277,7 +271,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               probe_relation,
               move(join_key_attributes),
               any_join_key_attributes_nullable,
-              num_partitions,
               part_id,
               lookup_block_id,
               selection,
@@ -295,7 +288,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               probe_relation,
               move(join_key_attributes),
               any_join_key_attributes_nullable,
-              num_partitions,
               part_id,
               lookup_block_id,
               residual_predicate,
@@ -544,7 +536,6 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
       return proto.HasExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable) &&
              proto.HasExtension(serialization::BuildHashWorkOrder::block_id) &&
              proto.HasExtension(serialization::BuildHashWorkOrder::join_hash_table_index) &&
-             proto.HasExtension(serialization::BuildHashWorkOrder::num_partitions) &&
              proto.HasExtension(serialization::BuildHashWorkOrder::partition_id) &&
              query_context.isValidJoinHashTableId(
                  proto.GetExtension(serialization::BuildHashWorkOrder::join_hash_table_index),
@@ -648,7 +639,6 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
              query_context.isValidInsertDestinationId(
                  proto.GetExtension(serialization::HashJoinWorkOrder::insert_destination_index)) &&
              proto.HasExtension(serialization::HashJoinWorkOrder::join_hash_table_index) &&
-             proto.HasExtension(serialization::HashJoinWorkOrder::num_partitions) &&
              proto.HasExtension(serialization::HashJoinWorkOrder::partition_id) &&
              query_context.isValidJoinHashTableId(
                  proto.GetExtension(serialization::HashJoinWorkOrder::join_hash_table_index),


[07/12] incubator-quickstep git commit: Added partition_id in feedInputBlock.

Posted by ji...@apache.org.
Added partition_id in feedInputBlock.


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

Branch: refs/heads/reorder-attrs
Commit: ee3b7f0e976523e0421d3003cd7e8cb972fccbbe
Parents: 6c10e99
Author: Zuyu Zhang <zu...@apache.org>
Authored: Sun Jan 15 19:53:54 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Jan 18 12:45:15 2017 -0800

----------------------------------------------------------------------
 query_execution/PolicyEnforcerBase.cpp                       | 2 +-
 query_execution/QueryExecutionMessages.proto                 | 3 +++
 query_execution/QueryManagerBase.cpp                         | 5 +++--
 query_execution/QueryManagerBase.hpp                         | 5 ++++-
 query_execution/tests/QueryManagerSingleNode_unittest.cpp    | 4 ++--
 relational_operators/AggregationOperator.hpp                 | 3 ++-
 relational_operators/BuildHashOperator.hpp                   | 4 ++--
 relational_operators/DeleteOperator.hpp                      | 3 ++-
 relational_operators/HashJoinOperator.hpp                    | 4 ++--
 relational_operators/NestedLoopsJoinOperator.hpp             | 3 ++-
 relational_operators/RelationalOperator.hpp                  | 6 +++---
 relational_operators/SampleOperator.hpp                      | 3 ++-
 relational_operators/SaveBlocksOperator.hpp                  | 3 ++-
 relational_operators/SelectOperator.hpp                      | 5 ++---
 relational_operators/SortMergeRunOperator.hpp                | 4 ++--
 relational_operators/SortRunGenerationOperator.hpp           | 3 ++-
 relational_operators/tests/SortMergeRunOperator_unittest.cpp | 2 +-
 storage/InsertDestination.cpp                                | 2 +-
 storage/InsertDestination.hpp                                | 4 +++-
 19 files changed, 41 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index 1a2ab46..a26b84e 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -118,7 +118,7 @@ void PolicyEnforcerBase::processMessage(const TaggedMessage &tagged_message) {
 
       op_index = proto.operator_index();
       admitted_queries_[query_id]->processDataPipelineMessage(
-          op_index, proto.block_id(), proto.relation_id());
+          op_index, proto.block_id(), proto.relation_id(), proto.partition_id());
       break;
     }
     case kWorkOrderFeedbackMessage: {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 28b5ebd..115a9a3 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -65,6 +65,9 @@ message DataPipelineMessage {
   required fixed64 block_id = 2;
   required int32 relation_id = 3;
   required uint64 query_id = 4;
+
+  // Used by PartitionAwareInsertDestination.
+  optional uint64 partition_id = 5 [default = 0];
 }
 
 // Distributed version related messages.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/query_execution/QueryManagerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.cpp b/query_execution/QueryManagerBase.cpp
index 8e37da8..5f8c6a3 100644
--- a/query_execution/QueryManagerBase.cpp
+++ b/query_execution/QueryManagerBase.cpp
@@ -192,13 +192,14 @@ void QueryManagerBase::processOperator(const dag_node_index index,
 
 void QueryManagerBase::processDataPipelineMessage(const dag_node_index op_index,
                                                   const block_id block,
-                                                  const relation_id rel_id) {
+                                                  const relation_id rel_id,
+                                                  const partition_id part_id) {
   for (const dag_node_index consumer_index :
        output_consumers_[op_index]) {
     // Feed the streamed block to the consumer. Note that 'output_consumers_'
     // only contain those dependents of operator with index = op_index which are
     // eligible to receive streamed input.
-    query_dag_->getNodePayloadMutable(consumer_index)->feedInputBlock(block, rel_id);
+    query_dag_->getNodePayloadMutable(consumer_index)->feedInputBlock(block, rel_id, part_id);
     // Because of the streamed input just fed, check if there are any new
     // WorkOrders available and if so, fetch them.
     fetchNormalWorkOrders(consumer_index);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/query_execution/QueryManagerBase.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerBase.hpp b/query_execution/QueryManagerBase.hpp
index ddb76d5..d0bb0ea 100644
--- a/query_execution/QueryManagerBase.hpp
+++ b/query_execution/QueryManagerBase.hpp
@@ -109,10 +109,13 @@ class QueryManagerBase {
    *        for the pipelining block.
    * @param block The block id.
    * @param rel_id The ID of the relation that produced 'block'.
+   * @param part_id The partition ID of 'block', if any. By default, a block
+   *        blongs to the only partition (aka, no partition).
    **/
   void processDataPipelineMessage(const dag_node_index op_index,
                                   const block_id block,
-                                  const relation_id rel_id);
+                                  const relation_id rel_id,
+                                  const partition_id part_id = 0);
 
   /**
    * @brief Fetch all work orders currently available in relational operator and

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/query_execution/tests/QueryManagerSingleNode_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManagerSingleNode_unittest.cpp b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
index 87b8934..28ab388 100644
--- a/query_execution/tests/QueryManagerSingleNode_unittest.cpp
+++ b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
@@ -177,8 +177,8 @@ class MockOperator: public RelationalOperator {
     return true;
   }
 
-  void feedInputBlock(const block_id input_block_id,
-                      const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     ++num_calls_feedblock_;
     MOCK_OP_LOG(3) << "count(" << num_calls_feedblock_ << ")";
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index c4e887d..2bd69f3 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -98,7 +98,8 @@ class AggregationOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     input_relation_block_ids_.push_back(input_block_id);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 074c1e1..dec121c 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -114,8 +114,8 @@ class BuildHashOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void feedInputBlock(const block_id input_block_id,
-                      const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     input_relation_block_ids_.push_back(input_block_id);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index 9c3f357..7b69d9c 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -100,7 +100,8 @@ class DeleteOperator : public RelationalOperator {
     return relation_.getID();
   }
 
-  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     DCHECK(!relation_is_stored_);
     relation_block_ids_.push_back(input_block_id);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 190f9d2..98c87bf 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -190,8 +190,8 @@ class HashJoinOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void feedInputBlock(const block_id input_block_id,
-                      const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     DCHECK(input_relation_id == probe_relation_.getID());
     probe_relation_block_ids_.push_back(input_block_id);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index 951851d..f8eb080 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -141,7 +141,8 @@ class NestedLoopsJoinOperator : public RelationalOperator {
     }
   }
 
-  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     if (input_relation_id == left_input_relation_.getID()) {
       left_relation_block_ids_.push_back(input_block_id);
     } else if (input_relation_id == right_input_relation_.getID()) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index a7f4177..fdea307 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -138,11 +138,11 @@ class RelationalOperator {
    * @brief Receive input blocks for this RelationalOperator.
    *
    * @param input_block_id The ID of the input block.
-   *
    * @param relation_id The ID of the relation that produced this input_block.
+   * @param part_id The partition ID of 'input_block_id'.
    **/
-  virtual void feedInputBlock(const block_id input_block_id,
-                              const relation_id input_relation_id) {}
+  virtual void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                              const partition_id part_id) {}
 
   /**
    * @brief Signal the end of feeding of input blocks for this

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index ccf6595..e56201a 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -108,7 +108,8 @@ class SampleOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     input_relation_block_ids_.push_back(input_block_id);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 27fd911..cd79733 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -83,7 +83,8 @@ class SaveBlocksOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     destination_block_ids_.push_back(input_block_id);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index e58ff81..79ab37f 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -204,10 +204,9 @@ class SelectOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     if (input_relation_.hasPartitionScheme()) {
-      const partition_id part_id =
-          input_relation_.getPartitionScheme()->getPartitionForBlock(input_block_id);
       input_relation_block_ids_in_partition_[part_id].push_back(input_block_id);
     } else {
       input_relation_block_ids_.push_back(input_block_id);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index aff7d8d..d2d9a2a 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -144,8 +144,8 @@ class SortMergeRunOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void feedInputBlock(const block_id input_block_id,
-                      const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     input_relation_block_ids_.push_back(input_block_id);
     if (started_) {
       initializeInputRuns();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index a2ffb2b..25a1273 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -124,7 +124,8 @@ class SortRunGenerationOperator : public RelationalOperator {
 
   bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
 
-  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
+  void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id,
+                      const partition_id part_id) override {
     DCHECK(input_relation_id == input_relation_.getID());
     input_relation_block_ids_.push_back(input_block_id);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index 74fecec..7a46e6e 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -1602,7 +1602,7 @@ class SortMergeRunOperatorTest : public ::testing::Test {
     // Feed blocks.
     DVLOG(1) << "Feeding " << to_feed.size() << " blocks.";
     for (const block_id block : to_feed) {
-      merge_op_->feedInputBlock(block, input_table_->getID());
+      merge_op_->feedInputBlock(block, input_table_->getID(), 0 /* partition_id */);
     }
 
     // Remove fed blocks.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 19bb356..944998f 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -789,7 +789,7 @@ void PartitionAwareInsertDestination::returnBlockInPartition(MutableBlockReferen
                                                        << "invalidated one or more IndexSubBlocks.");
   }
   // Note that the block will only be sent if it's full (true).
-  sendBlockFilledMessage(block->getID());
+  sendBlockFilledMessage(block->getID(), part_id);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ee3b7f0e/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index 3487638..c3c40bd 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -216,13 +216,15 @@ class InsertDestination : public InsertDestinationInterface {
    *        scheduler.
    *
    * @param id The id of the StorageBlock to be pipelined.
+   * @param part_id The partition id of Block <id>, if any.
    **/
-  void sendBlockFilledMessage(const block_id id) const {
+  void sendBlockFilledMessage(const block_id id, const partition_id part_id = 0) const {
     serialization::DataPipelineMessage proto;
     proto.set_operator_index(relational_op_index_);
     proto.set_block_id(id);
     proto.set_relation_id(relation_.getID());
     proto.set_query_id(query_id_);
+    proto.set_partition_id(part_id);
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const std::size_t proto_length = proto.ByteSize();


[08/12] incubator-quickstep git commit: Added Operator support for Partitioned HashJoin.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 239547f..2370c92 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -97,6 +97,8 @@ constexpr tuple_id kBlockSize = 10;
 constexpr std::size_t kQueryId = 0;
 constexpr int kOpIndex = 0;
 
+constexpr std::size_t kNumPartitions = 1;
+
 }  // namespace
 
 class HashJoinOperatorTest : public ::testing::TestWithParam<HashTableImplType> {
@@ -192,6 +194,8 @@ class HashJoinOperatorTest : public ::testing::TestWithParam<HashTableImplType>
       }
       storage_block->rebuild();
     }
+
+    num_partitions_ = kNumPartitions;
   }
 
   virtual void TearDown() {
@@ -291,6 +295,8 @@ class HashJoinOperatorTest : public ::testing::TestWithParam<HashTableImplType>
   unique_ptr<CatalogDatabase> db_;
   // The following CatalogRelations are owned by db_.
   CatalogRelation *dim_table_, *fact_table_;
+
+  std::size_t num_partitions_;
 };
 
 TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
@@ -302,7 +308,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
       query_context_proto.join_hash_tables_size();
 
   serialization::HashTable *hash_table_proto =
-      query_context_proto.add_join_hash_tables();
+      query_context_proto.add_join_hash_tables()->mutable_join_hash_table();
   switch (GetParam()) {
     case HashTableImplType::kLinearOpenAddressing:
       hash_table_proto->set_hash_table_impl_type(
@@ -341,6 +347,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_long.getID()),
                             dim_col_long.getType().isNullable(),
+                            num_partitions_,
                             join_hash_table_index));
 
   // Create the prober operator with one selection attribute.
@@ -370,6 +377,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
       true /* is_stored */,
       std::vector<attribute_id>(1, fact_col_long.getID()),
       fact_col_long.getType().isNullable(),
+      num_partitions_,
       *result_table,
       output_destination_index,
       join_hash_table_index,
@@ -427,7 +435,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -443,7 +451,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
       query_context_proto.join_hash_tables_size();
 
   serialization::HashTable *hash_table_proto =
-      query_context_proto.add_join_hash_tables();
+      query_context_proto.add_join_hash_tables()->mutable_join_hash_table();
   switch (GetParam()) {
     case HashTableImplType::kLinearOpenAddressing:
       hash_table_proto->set_hash_table_impl_type(
@@ -485,6 +493,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_int.getID()),
                             dim_col_int.getType().isNullable(),
+                            num_partitions_,
                             join_hash_table_index));
 
   // Create the prober operator with two selection attributes.
@@ -519,6 +528,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
       true /* is_stored */,
       std::vector<attribute_id>(1, fact_col_int.getID()),
       fact_col_int.getType().isNullable(),
+      num_partitions_,
       *result_table,
       output_destination_index,
       join_hash_table_index,
@@ -598,7 +608,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -614,7 +624,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
       query_context_proto.join_hash_tables_size();
 
   serialization::HashTable *hash_table_proto =
-      query_context_proto.add_join_hash_tables();
+      query_context_proto.add_join_hash_tables()->mutable_join_hash_table();
   switch (GetParam()) {
     case HashTableImplType::kLinearOpenAddressing:
       hash_table_proto->set_hash_table_impl_type(
@@ -648,6 +658,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_char.getID()),
                             dim_col_char.getType().isNullable(),
+                            num_partitions_,
                             join_hash_table_index));
 
   // Create prober operator with one selection attribute.
@@ -677,6 +688,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
       true /* is_stored */,
       std::vector<attribute_id>(1, fact_col_char.getID()),
       fact_col_char.getType().isNullable(),
+      num_partitions_,
       *result_table,
       output_destination_index,
       join_hash_table_index,
@@ -734,7 +746,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -750,7 +762,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
       query_context_proto.join_hash_tables_size();
 
   serialization::HashTable *hash_table_proto =
-      query_context_proto.add_join_hash_tables();
+      query_context_proto.add_join_hash_tables()->mutable_join_hash_table();
   switch (GetParam()) {
     case HashTableImplType::kLinearOpenAddressing:
       hash_table_proto->set_hash_table_impl_type(
@@ -785,6 +797,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_varchar.getID()),
                             dim_col_varchar.getType().isNullable(),
+                            num_partitions_,
                             join_hash_table_index));
 
   // Create prober operator with two selection attributes.
@@ -819,6 +832,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
       true /* is_stored */,
       std::vector<attribute_id>(1, fact_col_varchar.getID()),
       fact_col_varchar.getType().isNullable(),
+      num_partitions_,
       *result_table,
       output_destination_index,
       join_hash_table_index,
@@ -902,7 +916,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   }
 
   // Create the cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -918,7 +932,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
       query_context_proto.join_hash_tables_size();
 
   serialization::HashTable *hash_table_proto =
-      query_context_proto.add_join_hash_tables();
+      query_context_proto.add_join_hash_tables()->mutable_join_hash_table();
   switch (GetParam()) {
     case HashTableImplType::kLinearOpenAddressing:
       hash_table_proto->set_hash_table_impl_type(
@@ -958,6 +972,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
+                            num_partitions_,
                             join_hash_table_index));
 
   // Create the prober operator with two selection attributes.
@@ -997,6 +1012,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
       fact_key_attrs,
       fact_col_long.getType().isNullable() ||
           fact_col_varchar.getType().isNullable(),
+      num_partitions_,
       *result_table,
       output_destination_index,
       join_hash_table_index,
@@ -1079,7 +1095,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -1096,7 +1112,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
       query_context_proto.join_hash_tables_size();
 
   serialization::HashTable *hash_table_proto =
-      query_context_proto.add_join_hash_tables();
+      query_context_proto.add_join_hash_tables()->mutable_join_hash_table();
   switch (GetParam()) {
     case HashTableImplType::kLinearOpenAddressing:
       hash_table_proto->set_hash_table_impl_type(
@@ -1136,6 +1152,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
+                            num_partitions_,
                             join_hash_table_index));
 
   // Create prober operator with two selection attributes.
@@ -1185,6 +1202,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
                            fact_key_attrs,
                            fact_col_long.getType().isNullable() ||
                                fact_col_varchar.getType().isNullable(),
+                           num_partitions_,
                            *result_table,
                            output_destination_index,
                            join_hash_table_index,
@@ -1267,7 +1285,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, num_partitions_, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/300284c2/storage/StorageBlockInfo.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockInfo.hpp b/storage/StorageBlockInfo.hpp
index 361648f..ad5795b 100644
--- a/storage/StorageBlockInfo.hpp
+++ b/storage/StorageBlockInfo.hpp
@@ -25,6 +25,7 @@
 #include <cstdint>
 #include <limits>
 #include <string>
+#include <vector>
 
 #include "utility/Macros.hpp"
 
@@ -49,6 +50,8 @@ static constexpr int kBlockIdDomainLengthInDigits = std::numeric_limits<block_id
 static constexpr int kBlockIdCounterLengthInDigits = std::numeric_limits<block_id_counter>::digits10;
 static constexpr block_id_domain kMaxDomain = static_cast<block_id_domain>(0xFFFF);
 
+typedef std::vector<block_id> BlocksInPartition;
+
 /**
  * @brief All-static object that provides access to block_id.
  **/